From 60bbf0ac1c7b89beaa0081278287a60c9ebec693 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Sat, 24 Nov 2018 18:58:32 -0800 Subject: [PATCH] Split membership sync endpoints and remove PurgeTs endpoint (#2773) - Now that Zero followers cannot send proposals to leader, Alphas must send their membership updates to the Zero leader. - The membership updates can be received from any Zero server, leader or follower. - So, this PR splits the job of sending and receiving updates into two different endpoints: StreamMembership and UpdateMembership. - Purge Timestamps are now calculated using the Snapshot timestamp of the group. This is simpler, easier to predict and makes this system run like clockwork. That is, a snapshot calculation in Alpha leader would cause all Alpha followers to get that snapshot, which would then cause Zero to purge the timestamps below that snapshot ts (assuming other groups have caught up). - This removes the need for Zero to query all Alpha leaders, via PurgeTs endpoint. That is now removed. - Removed a lot of error variables, switching them with explanatory errors about what is going on in the system, making things easier to debug. Commits: * Working membership endpoint split. * Removed PurgeTs endpoint. Passing snapshot ts during membership update, so Zero can use that information to purge Oracle. * Block on Tablet call until we find a Zero leader. * More debugging via traces. * Simplify connToZeroLeader. Try to not add new bugs. * Update blockade test with restart option. * Self Reviews. --- conn/raft_server.go | 7 + contrib/blockade/main.go | 15 +- dgraph/cmd/zero/oracle.go | 42 +- dgraph/cmd/zero/raft.go | 233 ++++++----- dgraph/cmd/zero/zero.go | 111 +++-- edgraph/server.go | 4 + posting/oracle.go | 9 - protos/pb.proto | 8 +- protos/pb/pb.pb.go | 822 +++++++++++++++++++++++--------------- worker/draft.go | 53 ++- worker/groups.go | 267 +++++++------ worker/mutation.go | 14 +- 12 files changed, 888 insertions(+), 697 deletions(-) diff --git a/conn/raft_server.go b/conn/raft_server.go index 73e4ab212ce..cb54a27ae97 100644 --- a/conn/raft_server.go +++ b/conn/raft_server.go @@ -78,6 +78,13 @@ func (p *proposals) Store(key string, pctx *ProposalCtx) bool { return true } +func (p *proposals) Ctx(key string) context.Context { + if pctx := p.Get(key); pctx != nil { + return pctx.Ctx + } + return context.Background() +} + func (p *proposals) Get(key string) *ProposalCtx { p.RLock() defer p.RUnlock() diff --git a/contrib/blockade/main.go b/contrib/blockade/main.go index 937fe038cdc..242efd870ee 100644 --- a/contrib/blockade/main.go +++ b/contrib/blockade/main.go @@ -105,7 +105,10 @@ func testCommon(remove, join string, minAlphasUp int) error { if err := increment(minAlphasUp); err != nil { return err } - // Then join. + // Then join, if available. + if len(join) == 0 { + continue + } if err := run(ctxb, join); err != nil { return err } @@ -155,10 +158,16 @@ func runTests() error { fmt.Println("===> Slow TEST: OK") if err := testCommon("blockade stop", "blockade start --all", 2); err != nil { - fmt.Printf("Error testRestart: %v\n", err) + fmt.Printf("Error testRestart with stop: %v\n", err) + return err + } + fmt.Println("===> Restart TEST1: OK") + + if err := testCommon("blockade restart", "", 3); err != nil { + fmt.Printf("Error testRestart with restart: %v\n", err) return err } - fmt.Println("===> Restart TEST: OK") + fmt.Println("===> Restart TEST2: OK") if err := testCommon("blockade partition", "blockade join", 2); err != nil { fmt.Printf("Error testPartitions: %v\n", err) diff --git a/dgraph/cmd/zero/oracle.go b/dgraph/cmd/zero/oracle.go index 17e738f517b..64028cbb924 100644 --- a/dgraph/cmd/zero/oracle.go +++ b/dgraph/cmd/zero/oracle.go @@ -86,10 +86,6 @@ func (o *Oracle) purgeBelow(minTs uint64) { o.Lock() defer o.Unlock() - // TODO: HACK. Remove this later. - glog.Infof("Not purging below: %d", minTs) - return - // Dropping would be cheaper if abort/commits map is sharded for ts := range o.commits { if ts < minTs { @@ -203,7 +199,9 @@ func (o *Oracle) sendDeltasToSubscribers() { // Don't goto slurp_loop, because it would break from select immediately. } - glog.V(2).Infof("DoneUntil: %d. Sending delta: %+v\n", o.doneUntil.DoneUntil(), delta) + if glog.V(3) { + glog.Infof("DoneUntil: %d. Sending delta: %+v\n", o.doneUntil.DoneUntil(), delta) + } o.Lock() for id, ch := range o.subscribers { select { @@ -446,40 +444,6 @@ func (s *Server) SyncedUntil() uint64 { return syncUntil } -func (s *Server) purgeOracle() { - ticker := time.NewTicker(time.Second * 30) - defer ticker.Stop() - - var lastPurgeTs uint64 -OUTER: - for { - <-ticker.C - groups := s.KnownGroups() - var minTs uint64 - for _, group := range groups { - pl := s.Leader(group) - if pl == nil { - glog.Errorf("No healthy connection found to leader of group %d\n", group) - goto OUTER - } - c := pb.NewWorkerClient(pl.Get()) - num, err := c.PurgeTs(context.Background(), &api.Payload{}) - if err != nil { - glog.Errorf("Error while fetching minTs from group %d, err: %v\n", group, err) - goto OUTER - } - if minTs == 0 || num.Val < minTs { - minTs = num.Val - } - } - - if minTs > 0 && minTs != lastPurgeTs { - s.orc.purgeBelow(minTs) - lastPurgeTs = minTs - } - } -} - func (s *Server) TryAbort(ctx context.Context, txns *pb.TxnTimestamps) (*pb.OracleDelta, error) { delta := &pb.OracleDelta{} diff --git a/dgraph/cmd/zero/raft.go b/dgraph/cmd/zero/raft.go index cbc524f2efc..306b5e0147a 100644 --- a/dgraph/cmd/zero/raft.go +++ b/dgraph/cmd/zero/raft.go @@ -20,8 +20,10 @@ import ( "errors" "fmt" "log" + "math" "time" + otrace "go.opencensus.io/trace" "google.golang.org/grpc" "github.com/coreos/etcd/raft" @@ -142,6 +144,116 @@ func newGroup() *pb.Group { } } +func (n *node) handleMemberProposal(member *pb.Member) error { + n.server.AssertLock() + state := n.server.state + + m := n.server.member(member.Addr) + // Ensures that different nodes don't have same address. + if m != nil && (m.Id != member.Id || m.GroupId != member.GroupId) { + return x.Errorf("Found another member %d with same address: %v", m.Id, m.Addr) + } + if member.GroupId == 0 { + state.Zeros[member.Id] = member + if member.Leader { + // Unset leader flag for other nodes, there can be only one + // leader at a time. + for _, m := range state.Zeros { + if m.Id != member.Id { + m.Leader = false + } + } + } + return nil + } + group := state.Groups[member.GroupId] + if group == nil { + group = newGroup() + state.Groups[member.GroupId] = group + } + m, has := group.Members[member.Id] + if member.AmDead { + if has { + delete(group.Members, member.Id) + state.Removed = append(state.Removed, m) + conn.Get().Remove(m.Addr) + } + // else already removed. + return nil + } + if !has && len(group.Members) >= n.server.NumReplicas { + // We shouldn't allow more members than the number of replicas. + return x.Errorf("Group reached replication level. Can't add another member: %+v", member) + } + + // Create a connection to this server. + go conn.Get().Connect(member.Addr) + + group.Members[member.Id] = member + // Increment nextGroup when we have enough replicas + if member.GroupId == n.server.nextGroup && + len(group.Members) >= n.server.NumReplicas { + n.server.nextGroup++ + } + if member.Leader { + // Unset leader flag for other nodes, there can be only one + // leader at a time. + for _, m := range group.Members { + if m.Id != member.Id { + m.Leader = false + } + } + } + // On replay of logs on restart we need to set nextGroup. + if n.server.nextGroup <= member.GroupId { + n.server.nextGroup = member.GroupId + 1 + } + return nil +} + +func (n *node) handleTabletProposal(tablet *pb.Tablet) error { + n.server.AssertLock() + state := n.server.state + + if tablet.GroupId == 0 { + return x.Errorf("Tablet group id is zero: %+v", tablet) + } + group := state.Groups[tablet.GroupId] + if tablet.Remove { + glog.Infof("Removing tablet for attr: [%v], gid: [%v]\n", tablet.Predicate, tablet.GroupId) + if group != nil { + delete(group.Tablets, tablet.Predicate) + } + return nil + } + if group == nil { + group = newGroup() + state.Groups[tablet.GroupId] = group + } + + // There's a edge case that we're handling. + // Two servers ask to serve the same tablet, then we need to ensure that + // only the first one succeeds. + if prev := n.server.servingTablet(tablet.Predicate); prev != nil { + if tablet.Force { + // TODO: Try and remove this whole Force flag logic. + originalGroup := state.Groups[prev.GroupId] + delete(originalGroup.Tablets, tablet.Predicate) + } else { + if prev.GroupId != tablet.GroupId { + glog.Infof( + "Tablet for attr: [%s], gid: [%d] already served by group: [%d]\n", + prev.Predicate, tablet.GroupId, prev.GroupId) + return errTabletAlreadyServed + } + // This update can come from tablet size. + tablet.ReadOnly = prev.ReadOnly + } + } + group.Tablets[tablet.Predicate] = tablet + return nil +} + func (n *node) applyProposal(e raftpb.Entry) (string, error) { var p pb.ZeroProposal // Raft commits empty entry on becoming a leader. @@ -154,6 +266,7 @@ func (n *node) applyProposal(e raftpb.Entry) (string, error) { if len(p.Key) == 0 { return p.Key, errInvalidProposal } + span := otrace.FromContext(n.Proposals.Ctx(p.Key)) n.server.Lock() defer n.server.Unlock() @@ -172,103 +285,33 @@ func (n *node) applyProposal(e raftpb.Entry) (string, error) { } state.MaxRaftId = p.MaxRaftId } - if p.Member != nil { - m := n.server.member(p.Member.Addr) - // Ensures that different nodes don't have same address. - if m != nil && (m.Id != p.Member.Id || m.GroupId != p.Member.GroupId) { - return p.Key, errInvalidAddress - } - if p.Member.GroupId == 0 { - state.Zeros[p.Member.Id] = p.Member - if p.Member.Leader { - // Unset leader flag for other nodes, there can be only one - // leader at a time. - for _, m := range state.Zeros { - if m.Id != p.Member.Id { - m.Leader = false - } - } + if p.SnapshotTs != nil { + for gid, ts := range p.SnapshotTs { + if group, ok := state.Groups[gid]; ok { + group.SnapshotTs = x.Max(group.SnapshotTs, ts) } - return p.Key, nil } - group := state.Groups[p.Member.GroupId] - if group == nil { - group = newGroup() - state.Groups[p.Member.GroupId] = group + purgeTs := uint64(math.MaxUint64) + for _, group := range state.Groups { + purgeTs = x.Min(purgeTs, group.SnapshotTs) } - m, has := group.Members[p.Member.Id] - if p.Member.AmDead { - if has { - delete(group.Members, p.Member.Id) - state.Removed = append(state.Removed, m) - conn.Get().Remove(m.Addr) - } - // else already removed. - return p.Key, nil - } - if !has && len(group.Members) >= n.server.NumReplicas { - // We shouldn't allow more members than the number of replicas. - return p.Key, errInvalidProposal - } - - // Create a connection to this server. - go conn.Get().Connect(p.Member.Addr) - - group.Members[p.Member.Id] = p.Member - // Increment nextGroup when we have enough replicas - if p.Member.GroupId == n.server.nextGroup && - len(group.Members) >= n.server.NumReplicas { - n.server.nextGroup++ - } - if p.Member.Leader { - // Unset leader flag for other nodes, there can be only one - // leader at a time. - for _, m := range group.Members { - if m.Id != p.Member.Id { - m.Leader = false - } - } + if purgeTs < math.MaxUint64 { + n.server.orc.purgeBelow(purgeTs) } - // On replay of logs on restart we need to set nextGroup. - if n.server.nextGroup <= p.Member.GroupId { - n.server.nextGroup = p.Member.GroupId + 1 + } + if p.Member != nil { + if err := n.handleMemberProposal(p.Member); err != nil { + span.Annotatef(nil, "While applying membership proposal: %+v", err) + glog.Errorf("While applying membership proposal: %+v", err) + return p.Key, err } } if p.Tablet != nil { - if p.Tablet.GroupId == 0 { - return p.Key, errInvalidProposal - } - group := state.Groups[p.Tablet.GroupId] - if p.Tablet.Remove { - glog.Infof("Removing tablet for attr: [%v], gid: [%v]\n", p.Tablet.Predicate, p.Tablet.GroupId) - if group != nil { - delete(group.Tablets, p.Tablet.Predicate) - } - return p.Key, nil - } - if group == nil { - group = newGroup() - state.Groups[p.Tablet.GroupId] = group - } - - // There's a edge case that we're handling. - // Two servers ask to serve the same tablet, then we need to ensure that - // only the first one succeeds. - if tablet := n.server.servingTablet(p.Tablet.Predicate); tablet != nil { - if p.Tablet.Force { - originalGroup := state.Groups[tablet.GroupId] - delete(originalGroup.Tablets, p.Tablet.Predicate) - } else { - if tablet.GroupId != p.Tablet.GroupId { - glog.Infof("Tablet for attr: [%s], gid: [%d] is already being served by group: [%d]\n", - tablet.Predicate, p.Tablet.GroupId, tablet.GroupId) - return p.Key, errTabletAlreadyServed - } - // This update can come from tablet size. - p.Tablet.ReadOnly = tablet.ReadOnly - } + if err := n.handleTabletProposal(p.Tablet); err != nil { + span.Annotatef(nil, "While applying tablet proposal: %+v", err) + glog.Errorf("While applying tablet proposal: %+v", err) + return p.Key, err } - group.Tablets[p.Tablet.Predicate] = p.Tablet } if p.MaxLeaseId > state.MaxLeaseId { @@ -359,19 +402,18 @@ func (n *node) initAndStartNode() error { } else if len(opts.peer) > 0 { p := conn.Get().Connect(opts.peer) if p == nil { - return errInvalidAddress + return x.Errorf("Unhealthy connection to %v", opts.peer) } gconn := p.Get() c := pb.NewRaftClient(gconn) - err := errJoinCluster timeout := 8 * time.Second - for i := 0; err != nil; i++ { + for { ctx, cancel := context.WithTimeout(n.ctx, timeout) defer cancel() // JoinCluster can block indefinitely, raft ignores conf change proposal // if it has pending configuration. - _, err = c.JoinCluster(ctx, n.RaftContext) + _, err := c.JoinCluster(ctx, n.RaftContext) if err == nil { break } @@ -387,9 +429,6 @@ func (n *node) initAndStartNode() error { } time.Sleep(timeout) // This is useful because JoinCluster can exit immediately. } - if err != nil { - x.Fatalf("Max retries exceeded while trying to join cluster: %v\n", err) - } glog.Infof("[%d] Starting node\n", n.Id) n.SetRaft(raft.StartNode(n.Cfg, nil)) @@ -519,7 +558,7 @@ func (n *node) Run() { } else if entry.Type == raftpb.EntryNormal { key, err := n.applyProposal(entry) - if err != nil && err != errTabletAlreadyServed { + if err != nil { glog.Errorf("While applying proposal: %v\n", err) } n.Proposals.Done(key, err) diff --git a/dgraph/cmd/zero/zero.go b/dgraph/cmd/zero/zero.go index 5f38e3cc990..cbf16257ef2 100644 --- a/dgraph/cmd/zero/zero.go +++ b/dgraph/cmd/zero/zero.go @@ -22,8 +22,10 @@ import ( "sync" "time" + otrace "go.opencensus.io/trace" "golang.org/x/net/context" + "github.com/dgraph-io/dgo/protos/api" "github.com/dgraph-io/dgraph/conn" "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/x" @@ -34,13 +36,7 @@ import ( var ( emptyMembershipState pb.MembershipState emptyConnectionState pb.ConnectionState - errInvalidId = errors.New("Invalid server id") - errInvalidAddress = errors.New("Invalid address") - errEmptyPredicate = errors.New("Empty predicate") - errInvalidGroup = errors.New("Invalid group id") - errInvalidQuery = errors.New("Invalid query") errInternalError = errors.New("Internal server error") - errJoinCluster = errors.New("Unable to join cluster") errUnknownMember = errors.New("Unknown cluster member") errUpdatedMember = errors.New("Cluster member has updated credentials.") errServerShutDown = errors.New("Server is being shut down.") @@ -82,7 +78,6 @@ func (s *Server) Init() { s.leaderChangeCh = make(chan struct{}, 1) s.shutDownCh = make(chan struct{}, 1) go s.rebalanceTablets() - go s.purgeOracle() } func (s *Server) periodicallyPostTelemetry() { @@ -294,7 +289,7 @@ func (s *Server) servingTablet(tablet string) *pb.Tablet { func (s *Server) createProposals(dst *pb.Group) ([]*pb.ZeroProposal, error) { var res []*pb.ZeroProposal if len(dst.Members) > 1 { - return res, errInvalidQuery + return res, x.Errorf("Create Proposal: Invalid group: %+v", dst) } s.RLock() @@ -321,6 +316,11 @@ func (s *Server) createProposals(dst *pb.Group) ([]*pb.ZeroProposal, error) { // Don't continue to tablets if request is not from the leader. return res, nil } + if dst.SnapshotTs > group.SnapshotTs { + res = append(res, &pb.ZeroProposal{ + SnapshotTs: map[uint32]uint64{dstMember.GroupId: dst.SnapshotTs}, + }) + } } for key, dstTablet := range dst.Tablets { group, has := s.state.Groups[dstTablet.GroupId] @@ -386,7 +386,7 @@ func (s *Server) Connect(ctx context.Context, return cs, err } if len(m.Addr) == 0 { - return &emptyConnectionState, errInvalidAddress + return &emptyConnectionState, x.Errorf("No address provided: %+v", m) } for _, member := range s.membershipState().Removed { @@ -482,15 +482,19 @@ func (s *Server) Connect(ctx context.Context, func (s *Server) ShouldServe( ctx context.Context, tablet *pb.Tablet) (resp *pb.Tablet, err error) { + ctx, span := otrace.StartSpan(ctx, "Zero.ShouldServe") + defer span.End() + if len(tablet.Predicate) == 0 { - return resp, errEmptyPredicate + return resp, x.Errorf("Tablet predicate is empty in %+v", tablet) } if tablet.GroupId == 0 { - return resp, errInvalidGroup + return resp, x.Errorf("Group ID is Zero in %+v", tablet) } // Check who is serving this tablet. tab := s.ServingTablet(tablet.Predicate) + span.Annotatef(nil, "Tablet for %s: %+v", tablet.Predicate, tab) if tab != nil { // Someone is serving this tablet. Could be the caller as well. // The caller should compare the returned group against the group it holds to check who's @@ -504,74 +508,59 @@ func (s *Server) ShouldServe( tablet.Force = false proposal.Tablet = tablet if err := s.Node.proposeAndWait(ctx, &proposal); err != nil && err != errTabletAlreadyServed { + span.Annotatef(nil, "While proposing tablet: %v", err) return tablet, err } tab = s.ServingTablet(tablet.Predicate) x.AssertTrue(tab != nil) + span.Annotatef(nil, "Now serving tablet for %s: %+v", tablet.Predicate, tab) return tab, nil } -func (s *Server) receiveUpdates(stream pb.Zero_UpdateServer) error { - for { - group, err := stream.Recv() - // Due to closeSend on client Side - if group == nil { - return nil - } - // Could be EOF also, but we don't care about error type. - if err != nil { - return err - } - proposals, err := s.createProposals(group) - if err != nil { - // Sleep here so the caller doesn't keep on retrying indefinitely, creating a busy - // wait. - time.Sleep(time.Second) - glog.Errorf("Error while creating proposals in stream %v\n", err) - return err - } +func (s *Server) UpdateMembership(ctx context.Context, group *pb.Group) (*api.Payload, error) { + proposals, err := s.createProposals(group) + if err != nil { + // Sleep here so the caller doesn't keep on retrying indefinitely, creating a busy + // wait. + time.Sleep(time.Second) + glog.Errorf("Error while creating proposals in Update: %v\n", err) + return nil, err + } - errCh := make(chan error) - for _, pr := range proposals { - go func(pr *pb.ZeroProposal) { - errCh <- s.Node.proposeAndWait(context.Background(), pr) - }(pr) - } + ctx, cancel := context.WithCancel(ctx) + defer cancel() - for range proposals { - // We Don't care about these errors - // Ideally shouldn't error out. - if err := <-errCh; err != nil { - glog.Errorf("Error while applying proposal in update stream %v\n", err) - } + errCh := make(chan error) + for _, pr := range proposals { + go func(pr *pb.ZeroProposal) { + errCh <- s.Node.proposeAndWait(ctx, pr) + }(pr) + } + + for range proposals { + // We Don't care about these errors + // Ideally shouldn't error out. + if err := <-errCh; err != nil { + glog.Errorf("Error while applying proposal in Update stream: %v\n", err) + return nil, err } } + return &api.Payload{Data: []byte("OK")}, nil } -func (s *Server) Update(stream pb.Zero_UpdateServer) error { - che := make(chan error, 1) - // Server side cancellation can only be done by existing the handler - // since Recv is blocking we need to run it in a goroutine. - go func() { - che <- s.receiveUpdates(stream) - }() - +func (s *Server) StreamMembership(_ *api.Payload, stream pb.Zero_StreamMembershipServer) error { // Send MembershipState right away. So, the connection is correctly established. ctx := stream.Context() ms, err := s.latestMembershipState(ctx) if err != nil { return err } - if ms != nil { - // grpc will error out during marshalling if we send nil. - if err := stream.Send(ms); err != nil { - return err - } + if err := stream.Send(ms); err != nil { + return err } ticker := time.NewTicker(time.Second) defer ticker.Stop() - for { select { case <-ticker.C: @@ -580,13 +569,9 @@ func (s *Server) Update(stream pb.Zero_UpdateServer) error { if err != nil { return err } - // TODO: Don't send if only lease has changed. if err := stream.Send(ms); err != nil { return err } - case err := <-che: - // Error while receiving updates. - return err case <-ctx.Done(): return ctx.Err() case <-s.shutDownCh: @@ -599,5 +584,9 @@ func (s *Server) latestMembershipState(ctx context.Context) (*pb.MembershipState if err := s.Node.WaitLinearizableRead(ctx); err != nil { return nil, err } - return s.membershipState(), nil + ms := s.membershipState() + if ms == nil { + return &pb.MembershipState{}, nil + } + return ms, nil } diff --git a/edgraph/server.go b/edgraph/server.go index f9cccb2d4b1..c2ee5833684 100644 --- a/edgraph/server.go +++ b/edgraph/server.go @@ -264,6 +264,10 @@ func (s *ServerState) getTimestamp(readOnly bool) uint64 { } func (s *Server) Alter(ctx context.Context, op *api.Operation) (*api.Payload, error) { + ctx, span := otrace.StartSpan(ctx, "Server.Alter") + defer span.End() + span.Annotatef(nil, "Alter operation: %+v", op) + // Always print out Alter operations because they are important and rare. glog.Infof("Received ALTER op: %+v", op) diff --git a/posting/oracle.go b/posting/oracle.go index 1b7a6123775..a336c6a3673 100644 --- a/posting/oracle.go +++ b/posting/oracle.go @@ -120,15 +120,6 @@ func (o *oracle) MinPendingStartTs() uint64 { return min } -// PurgeTs gives a start ts, below which all entries can be purged by Zero, -// because their status has been successfully applied to Raft group. -func (o *oracle) PurgeTs() uint64 { - // o.MinPendingStartTs can be inf, but we don't want Zero to delete new - // records that haven't yet reached us. So, we also consider MaxAssigned - // that we have received so far, so only records below MaxAssigned are purged. - return x.Min(o.MinPendingStartTs()-1, o.MaxAssigned()) -} - func (o *oracle) TxnOlderThan(dur time.Duration) (res []uint64) { o.RLock() defer o.RUnlock() diff --git a/protos/pb.proto b/protos/pb.proto index 7b73a2dd957..e2773908766 100644 --- a/protos/pb.proto +++ b/protos/pb.proto @@ -131,9 +131,11 @@ message Member { message Group { map members = 1; // Raft ID is the key. map tablets = 2; // Predicate + others are key. + uint64 snapshot_ts = 3; // Stores Snapshot transaction ts. } message ZeroProposal { + map snapshot_ts = 1; // Group ID -> Snapshot Ts. Member member = 2; Tablet tablet = 3; uint64 maxLeaseId = 4; @@ -399,8 +401,11 @@ service Raft { } service Zero { + // These 3 endpoints are for handling membership. rpc Connect (Member) returns (ConnectionState) {} - rpc Update (stream Group) returns (stream MembershipState) {} + rpc UpdateMembership (Group) returns (api.Payload) {} + rpc StreamMembership (api.Payload) returns (stream MembershipState) {} + rpc Oracle (api.Payload) returns (stream OracleDelta) {} rpc ShouldServe (Tablet) returns (Tablet) {} rpc AssignUids (Num) returns (AssignedIds) {} @@ -416,7 +421,6 @@ service Worker { rpc StreamSnapshot (stream Snapshot) returns (stream KVS) {} rpc Sort (SortMessage) returns (SortResult) {} rpc Schema (SchemaRequest) returns (SchemaResult) {} - rpc PurgeTs (api.Payload) returns (Num) {} rpc Backup (BackupRequest) returns (Status) {} rpc Export (ExportRequest) returns (Status) {} rpc ReceivePredicate(stream KVS) returns (api.Payload) {} diff --git a/protos/pb/pb.pb.go b/protos/pb/pb.pb.go index 3a13e43104f..22085b5fe37 100644 --- a/protos/pb/pb.pb.go +++ b/protos/pb/pb.pb.go @@ -48,7 +48,7 @@ func (x DirectedEdge_Op) String() string { return proto.EnumName(DirectedEdge_Op_name, int32(x)) } func (DirectedEdge_Op) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{17, 0} + return fileDescriptor_pb_a9faf9bf79258c83, []int{17, 0} } type Posting_ValType int32 @@ -95,7 +95,7 @@ func (x Posting_ValType) String() string { return proto.EnumName(Posting_ValType_name, int32(x)) } func (Posting_ValType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{24, 0} + return fileDescriptor_pb_a9faf9bf79258c83, []int{24, 0} } type Posting_PostingType int32 @@ -121,7 +121,7 @@ func (x Posting_PostingType) String() string { return proto.EnumName(Posting_PostingType_name, int32(x)) } func (Posting_PostingType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{24, 1} + return fileDescriptor_pb_a9faf9bf79258c83, []int{24, 1} } type SchemaUpdate_Directive int32 @@ -150,7 +150,7 @@ func (x SchemaUpdate_Directive) String() string { return proto.EnumName(SchemaUpdate_Directive_name, int32(x)) } func (SchemaUpdate_Directive) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{36, 0} + return fileDescriptor_pb_a9faf9bf79258c83, []int{36, 0} } type List struct { @@ -164,7 +164,7 @@ func (m *List) Reset() { *m = List{} } func (m *List) String() string { return proto.CompactTextString(m) } func (*List) ProtoMessage() {} func (*List) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{0} + return fileDescriptor_pb_a9faf9bf79258c83, []int{0} } func (m *List) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -212,7 +212,7 @@ func (m *TaskValue) Reset() { *m = TaskValue{} } func (m *TaskValue) String() string { return proto.CompactTextString(m) } func (*TaskValue) ProtoMessage() {} func (*TaskValue) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{1} + return fileDescriptor_pb_a9faf9bf79258c83, []int{1} } func (m *TaskValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -268,7 +268,7 @@ func (m *SrcFunction) Reset() { *m = SrcFunction{} } func (m *SrcFunction) String() string { return proto.CompactTextString(m) } func (*SrcFunction) ProtoMessage() {} func (*SrcFunction) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{2} + return fileDescriptor_pb_a9faf9bf79258c83, []int{2} } func (m *SrcFunction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -341,7 +341,7 @@ func (m *Query) Reset() { *m = Query{} } func (m *Query) String() string { return proto.CompactTextString(m) } func (*Query) ProtoMessage() {} func (*Query) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{3} + return fileDescriptor_pb_a9faf9bf79258c83, []int{3} } func (m *Query) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -458,7 +458,7 @@ func (m *ValueList) Reset() { *m = ValueList{} } func (m *ValueList) String() string { return proto.CompactTextString(m) } func (*ValueList) ProtoMessage() {} func (*ValueList) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{4} + return fileDescriptor_pb_a9faf9bf79258c83, []int{4} } func (m *ValueList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -505,7 +505,7 @@ func (m *LangList) Reset() { *m = LangList{} } func (m *LangList) String() string { return proto.CompactTextString(m) } func (*LangList) ProtoMessage() {} func (*LangList) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{5} + return fileDescriptor_pb_a9faf9bf79258c83, []int{5} } func (m *LangList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -558,7 +558,7 @@ func (m *Result) Reset() { *m = Result{} } func (m *Result) String() string { return proto.CompactTextString(m) } func (*Result) ProtoMessage() {} func (*Result) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{6} + return fileDescriptor_pb_a9faf9bf79258c83, []int{6} } func (m *Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -649,7 +649,7 @@ func (m *Order) Reset() { *m = Order{} } func (m *Order) String() string { return proto.CompactTextString(m) } func (*Order) ProtoMessage() {} func (*Order) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{7} + return fileDescriptor_pb_a9faf9bf79258c83, []int{7} } func (m *Order) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -714,7 +714,7 @@ func (m *SortMessage) Reset() { *m = SortMessage{} } func (m *SortMessage) String() string { return proto.CompactTextString(m) } func (*SortMessage) ProtoMessage() {} func (*SortMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{8} + return fileDescriptor_pb_a9faf9bf79258c83, []int{8} } func (m *SortMessage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -789,7 +789,7 @@ func (m *SortResult) Reset() { *m = SortResult{} } func (m *SortResult) String() string { return proto.CompactTextString(m) } func (*SortResult) ProtoMessage() {} func (*SortResult) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{9} + return fileDescriptor_pb_a9faf9bf79258c83, []int{9} } func (m *SortResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -839,7 +839,7 @@ func (m *RaftContext) Reset() { *m = RaftContext{} } func (m *RaftContext) String() string { return proto.CompactTextString(m) } func (*RaftContext) ProtoMessage() {} func (*RaftContext) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{10} + return fileDescriptor_pb_a9faf9bf79258c83, []int{10} } func (m *RaftContext) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -916,7 +916,7 @@ func (m *Member) Reset() { *m = Member{} } func (m *Member) String() string { return proto.CompactTextString(m) } func (*Member) ProtoMessage() {} func (*Member) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{11} + return fileDescriptor_pb_a9faf9bf79258c83, []int{11} } func (m *Member) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -997,6 +997,7 @@ func (m *Member) GetClusterInfoOnly() bool { type Group struct { Members map[uint64]*Member `protobuf:"bytes,1,rep,name=members" json:"members,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` Tablets map[string]*Tablet `protobuf:"bytes,2,rep,name=tablets" json:"tablets,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + SnapshotTs uint64 `protobuf:"varint,3,opt,name=snapshot_ts,json=snapshotTs,proto3" json:"snapshot_ts,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1006,7 +1007,7 @@ func (m *Group) Reset() { *m = Group{} } func (m *Group) String() string { return proto.CompactTextString(m) } func (*Group) ProtoMessage() {} func (*Group) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{12} + return fileDescriptor_pb_a9faf9bf79258c83, []int{12} } func (m *Group) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1049,25 +1050,33 @@ func (m *Group) GetTablets() map[string]*Tablet { return nil } +func (m *Group) GetSnapshotTs() uint64 { + if m != nil { + return m.SnapshotTs + } + return 0 +} + type ZeroProposal struct { - Member *Member `protobuf:"bytes,2,opt,name=member" json:"member,omitempty"` - Tablet *Tablet `protobuf:"bytes,3,opt,name=tablet" json:"tablet,omitempty"` - MaxLeaseId uint64 `protobuf:"varint,4,opt,name=maxLeaseId,proto3" json:"maxLeaseId,omitempty"` - MaxTxnTs uint64 `protobuf:"varint,5,opt,name=maxTxnTs,proto3" json:"maxTxnTs,omitempty"` - MaxRaftId uint64 `protobuf:"varint,6,opt,name=maxRaftId,proto3" json:"maxRaftId,omitempty"` - Txn *api.TxnContext `protobuf:"bytes,7,opt,name=txn" json:"txn,omitempty"` - Key string `protobuf:"bytes,8,opt,name=key,proto3" json:"key,omitempty"` - Cid string `protobuf:"bytes,9,opt,name=cid,proto3" json:"cid,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + SnapshotTs map[uint32]uint64 `protobuf:"bytes,1,rep,name=snapshot_ts,json=snapshotTs" json:"snapshot_ts,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Member *Member `protobuf:"bytes,2,opt,name=member" json:"member,omitempty"` + Tablet *Tablet `protobuf:"bytes,3,opt,name=tablet" json:"tablet,omitempty"` + MaxLeaseId uint64 `protobuf:"varint,4,opt,name=maxLeaseId,proto3" json:"maxLeaseId,omitempty"` + MaxTxnTs uint64 `protobuf:"varint,5,opt,name=maxTxnTs,proto3" json:"maxTxnTs,omitempty"` + MaxRaftId uint64 `protobuf:"varint,6,opt,name=maxRaftId,proto3" json:"maxRaftId,omitempty"` + Txn *api.TxnContext `protobuf:"bytes,7,opt,name=txn" json:"txn,omitempty"` + Key string `protobuf:"bytes,8,opt,name=key,proto3" json:"key,omitempty"` + Cid string `protobuf:"bytes,9,opt,name=cid,proto3" json:"cid,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ZeroProposal) Reset() { *m = ZeroProposal{} } func (m *ZeroProposal) String() string { return proto.CompactTextString(m) } func (*ZeroProposal) ProtoMessage() {} func (*ZeroProposal) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{13} + return fileDescriptor_pb_a9faf9bf79258c83, []int{13} } func (m *ZeroProposal) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1096,6 +1105,13 @@ func (m *ZeroProposal) XXX_DiscardUnknown() { var xxx_messageInfo_ZeroProposal proto.InternalMessageInfo +func (m *ZeroProposal) GetSnapshotTs() map[uint32]uint64 { + if m != nil { + return m.SnapshotTs + } + return nil +} + func (m *ZeroProposal) GetMember() *Member { if m != nil { return m.Member @@ -1173,7 +1189,7 @@ func (m *MembershipState) Reset() { *m = MembershipState{} } func (m *MembershipState) String() string { return proto.CompactTextString(m) } func (*MembershipState) ProtoMessage() {} func (*MembershipState) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{14} + return fileDescriptor_pb_a9faf9bf79258c83, []int{14} } func (m *MembershipState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1271,7 +1287,7 @@ func (m *ConnectionState) Reset() { *m = ConnectionState{} } func (m *ConnectionState) String() string { return proto.CompactTextString(m) } func (*ConnectionState) ProtoMessage() {} func (*ConnectionState) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{15} + return fileDescriptor_pb_a9faf9bf79258c83, []int{15} } func (m *ConnectionState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1337,7 +1353,7 @@ func (m *Tablet) Reset() { *m = Tablet{} } func (m *Tablet) String() string { return proto.CompactTextString(m) } func (*Tablet) ProtoMessage() {} func (*Tablet) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{16} + return fileDescriptor_pb_a9faf9bf79258c83, []int{16} } func (m *Tablet) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1427,7 +1443,7 @@ func (m *DirectedEdge) Reset() { *m = DirectedEdge{} } func (m *DirectedEdge) String() string { return proto.CompactTextString(m) } func (*DirectedEdge) ProtoMessage() {} func (*DirectedEdge) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{17} + return fileDescriptor_pb_a9faf9bf79258c83, []int{17} } func (m *DirectedEdge) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1535,7 +1551,7 @@ func (m *Mutations) Reset() { *m = Mutations{} } func (m *Mutations) String() string { return proto.CompactTextString(m) } func (*Mutations) ProtoMessage() {} func (*Mutations) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{18} + return fileDescriptor_pb_a9faf9bf79258c83, []int{18} } func (m *Mutations) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1617,7 +1633,7 @@ func (m *KeyValues) Reset() { *m = KeyValues{} } func (m *KeyValues) String() string { return proto.CompactTextString(m) } func (*KeyValues) ProtoMessage() {} func (*KeyValues) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{19} + return fileDescriptor_pb_a9faf9bf79258c83, []int{19} } func (m *KeyValues) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1668,7 +1684,7 @@ func (m *Snapshot) Reset() { *m = Snapshot{} } func (m *Snapshot) String() string { return proto.CompactTextString(m) } func (*Snapshot) ProtoMessage() {} func (*Snapshot) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{20} + return fileDescriptor_pb_a9faf9bf79258c83, []int{20} } func (m *Snapshot) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1743,7 +1759,7 @@ func (m *Proposal) Reset() { *m = Proposal{} } func (m *Proposal) String() string { return proto.CompactTextString(m) } func (*Proposal) ProtoMessage() {} func (*Proposal) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{21} + return fileDescriptor_pb_a9faf9bf79258c83, []int{21} } func (m *Proposal) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1841,7 +1857,7 @@ func (m *KVS) Reset() { *m = KVS{} } func (m *KVS) String() string { return proto.CompactTextString(m) } func (*KVS) ProtoMessage() {} func (*KVS) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{22} + return fileDescriptor_pb_a9faf9bf79258c83, []int{22} } func (m *KVS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1898,7 +1914,7 @@ func (m *KV) Reset() { *m = KV{} } func (m *KV) String() string { return proto.CompactTextString(m) } func (*KV) ProtoMessage() {} func (*KV) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{23} + return fileDescriptor_pb_a9faf9bf79258c83, []int{23} } func (m *KV) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1977,7 +1993,7 @@ func (m *Posting) Reset() { *m = Posting{} } func (m *Posting) String() string { return proto.CompactTextString(m) } func (*Posting) ProtoMessage() {} func (*Posting) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{24} + return fileDescriptor_pb_a9faf9bf79258c83, []int{24} } func (m *Posting) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2091,7 +2107,7 @@ func (m *UidBlock) Reset() { *m = UidBlock{} } func (m *UidBlock) String() string { return proto.CompactTextString(m) } func (*UidBlock) ProtoMessage() {} func (*UidBlock) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{25} + return fileDescriptor_pb_a9faf9bf79258c83, []int{25} } func (m *UidBlock) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2146,7 +2162,7 @@ func (m *UidPack) Reset() { *m = UidPack{} } func (m *UidPack) String() string { return proto.CompactTextString(m) } func (*UidPack) ProtoMessage() {} func (*UidPack) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{26} + return fileDescriptor_pb_a9faf9bf79258c83, []int{26} } func (m *UidPack) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2202,7 +2218,7 @@ func (m *PostingList) Reset() { *m = PostingList{} } func (m *PostingList) String() string { return proto.CompactTextString(m) } func (*PostingList) ProtoMessage() {} func (*PostingList) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{27} + return fileDescriptor_pb_a9faf9bf79258c83, []int{27} } func (m *PostingList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2264,7 +2280,7 @@ func (m *FacetParam) Reset() { *m = FacetParam{} } func (m *FacetParam) String() string { return proto.CompactTextString(m) } func (*FacetParam) ProtoMessage() {} func (*FacetParam) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{28} + return fileDescriptor_pb_a9faf9bf79258c83, []int{28} } func (m *FacetParam) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2319,7 +2335,7 @@ func (m *FacetParams) Reset() { *m = FacetParams{} } func (m *FacetParams) String() string { return proto.CompactTextString(m) } func (*FacetParams) ProtoMessage() {} func (*FacetParams) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{29} + return fileDescriptor_pb_a9faf9bf79258c83, []int{29} } func (m *FacetParams) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2373,7 +2389,7 @@ func (m *Facets) Reset() { *m = Facets{} } func (m *Facets) String() string { return proto.CompactTextString(m) } func (*Facets) ProtoMessage() {} func (*Facets) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{30} + return fileDescriptor_pb_a9faf9bf79258c83, []int{30} } func (m *Facets) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2420,7 +2436,7 @@ func (m *FacetsList) Reset() { *m = FacetsList{} } func (m *FacetsList) String() string { return proto.CompactTextString(m) } func (*FacetsList) ProtoMessage() {} func (*FacetsList) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{31} + return fileDescriptor_pb_a9faf9bf79258c83, []int{31} } func (m *FacetsList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2469,7 +2485,7 @@ func (m *Function) Reset() { *m = Function{} } func (m *Function) String() string { return proto.CompactTextString(m) } func (*Function) ProtoMessage() {} func (*Function) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{32} + return fileDescriptor_pb_a9faf9bf79258c83, []int{32} } func (m *Function) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2533,7 +2549,7 @@ func (m *FilterTree) Reset() { *m = FilterTree{} } func (m *FilterTree) String() string { return proto.CompactTextString(m) } func (*FilterTree) ProtoMessage() {} func (*FilterTree) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{33} + return fileDescriptor_pb_a9faf9bf79258c83, []int{33} } func (m *FilterTree) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2598,7 +2614,7 @@ func (m *SchemaRequest) Reset() { *m = SchemaRequest{} } func (m *SchemaRequest) String() string { return proto.CompactTextString(m) } func (*SchemaRequest) ProtoMessage() {} func (*SchemaRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{34} + return fileDescriptor_pb_a9faf9bf79258c83, []int{34} } func (m *SchemaRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2659,7 +2675,7 @@ func (m *SchemaResult) Reset() { *m = SchemaResult{} } func (m *SchemaResult) String() string { return proto.CompactTextString(m) } func (*SchemaResult) ProtoMessage() {} func (*SchemaResult) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{35} + return fileDescriptor_pb_a9faf9bf79258c83, []int{35} } func (m *SchemaResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2713,7 +2729,7 @@ func (m *SchemaUpdate) Reset() { *m = SchemaUpdate{} } func (m *SchemaUpdate) String() string { return proto.CompactTextString(m) } func (*SchemaUpdate) ProtoMessage() {} func (*SchemaUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{36} + return fileDescriptor_pb_a9faf9bf79258c83, []int{36} } func (m *SchemaUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2813,7 +2829,7 @@ func (m *MapEntry) Reset() { *m = MapEntry{} } func (m *MapEntry) String() string { return proto.CompactTextString(m) } func (*MapEntry) ProtoMessage() {} func (*MapEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{37} + return fileDescriptor_pb_a9faf9bf79258c83, []int{37} } func (m *MapEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2877,7 +2893,7 @@ func (m *MovePredicatePayload) Reset() { *m = MovePredicatePayload{} } func (m *MovePredicatePayload) String() string { return proto.CompactTextString(m) } func (*MovePredicatePayload) ProtoMessage() {} func (*MovePredicatePayload) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{38} + return fileDescriptor_pb_a9faf9bf79258c83, []int{38} } func (m *MovePredicatePayload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2946,7 +2962,7 @@ func (m *TxnStatus) Reset() { *m = TxnStatus{} } func (m *TxnStatus) String() string { return proto.CompactTextString(m) } func (*TxnStatus) ProtoMessage() {} func (*TxnStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{39} + return fileDescriptor_pb_a9faf9bf79258c83, []int{39} } func (m *TxnStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3001,7 +3017,7 @@ func (m *OracleDelta) Reset() { *m = OracleDelta{} } func (m *OracleDelta) String() string { return proto.CompactTextString(m) } func (*OracleDelta) ProtoMessage() {} func (*OracleDelta) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{40} + return fileDescriptor_pb_a9faf9bf79258c83, []int{40} } func (m *OracleDelta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3055,7 +3071,7 @@ func (m *TxnTimestamps) Reset() { *m = TxnTimestamps{} } func (m *TxnTimestamps) String() string { return proto.CompactTextString(m) } func (*TxnTimestamps) ProtoMessage() {} func (*TxnTimestamps) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{41} + return fileDescriptor_pb_a9faf9bf79258c83, []int{41} } func (m *TxnTimestamps) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3102,7 +3118,7 @@ func (m *PeerResponse) Reset() { *m = PeerResponse{} } func (m *PeerResponse) String() string { return proto.CompactTextString(m) } func (*PeerResponse) ProtoMessage() {} func (*PeerResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{42} + return fileDescriptor_pb_a9faf9bf79258c83, []int{42} } func (m *PeerResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3150,7 +3166,7 @@ func (m *RaftBatch) Reset() { *m = RaftBatch{} } func (m *RaftBatch) String() string { return proto.CompactTextString(m) } func (*RaftBatch) ProtoMessage() {} func (*RaftBatch) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{43} + return fileDescriptor_pb_a9faf9bf79258c83, []int{43} } func (m *RaftBatch) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3205,7 +3221,7 @@ func (m *Num) Reset() { *m = Num{} } func (m *Num) String() string { return proto.CompactTextString(m) } func (*Num) ProtoMessage() {} func (*Num) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{44} + return fileDescriptor_pb_a9faf9bf79258c83, []int{44} } func (m *Num) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3262,7 +3278,7 @@ func (m *AssignedIds) Reset() { *m = AssignedIds{} } func (m *AssignedIds) String() string { return proto.CompactTextString(m) } func (*AssignedIds) ProtoMessage() {} func (*AssignedIds) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{45} + return fileDescriptor_pb_a9faf9bf79258c83, []int{45} } func (m *AssignedIds) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3324,7 +3340,7 @@ func (m *SnapshotMeta) Reset() { *m = SnapshotMeta{} } func (m *SnapshotMeta) String() string { return proto.CompactTextString(m) } func (*SnapshotMeta) ProtoMessage() {} func (*SnapshotMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{46} + return fileDescriptor_pb_a9faf9bf79258c83, []int{46} } func (m *SnapshotMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3368,7 +3384,7 @@ func (m *SnapshotMeta) GetGroupId() uint32 { } // Status describes a general status response. -// code: 0 = success, -1 = unknown failure +// code: 0 = success, 0 != failure. type Status struct { Code int32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` @@ -3381,7 +3397,7 @@ func (m *Status) Reset() { *m = Status{} } func (m *Status) String() string { return proto.CompactTextString(m) } func (*Status) ProtoMessage() {} func (*Status) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{47} + return fileDescriptor_pb_a9faf9bf79258c83, []int{47} } func (m *Status) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3438,7 +3454,7 @@ func (m *BackupRequest) Reset() { *m = BackupRequest{} } func (m *BackupRequest) String() string { return proto.CompactTextString(m) } func (*BackupRequest) ProtoMessage() {} func (*BackupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{48} + return fileDescriptor_pb_a9faf9bf79258c83, []int{48} } func (m *BackupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3508,7 +3524,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pb_1efee8619f75cb94, []int{49} + return fileDescriptor_pb_a9faf9bf79258c83, []int{49} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3575,6 +3591,7 @@ func init() { proto.RegisterMapType((map[uint64]*Member)(nil), "pb.Group.MembersEntry") proto.RegisterMapType((map[string]*Tablet)(nil), "pb.Group.TabletsEntry") proto.RegisterType((*ZeroProposal)(nil), "pb.ZeroProposal") + proto.RegisterMapType((map[uint32]uint64)(nil), "pb.ZeroProposal.SnapshotTsEntry") proto.RegisterType((*MembershipState)(nil), "pb.MembershipState") proto.RegisterMapType((map[uint32]*Group)(nil), "pb.MembershipState.GroupsEntry") proto.RegisterMapType((map[uint64]*Member)(nil), "pb.MembershipState.ZerosEntry") @@ -3794,8 +3811,10 @@ var _Raft_serviceDesc = grpc.ServiceDesc{ // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type ZeroClient interface { + // These 3 endpoints are for handling membership. Connect(ctx context.Context, in *Member, opts ...grpc.CallOption) (*ConnectionState, error) - Update(ctx context.Context, opts ...grpc.CallOption) (Zero_UpdateClient, error) + UpdateMembership(ctx context.Context, in *Group, opts ...grpc.CallOption) (*api.Payload, error) + StreamMembership(ctx context.Context, in *api.Payload, opts ...grpc.CallOption) (Zero_StreamMembershipClient, error) Oracle(ctx context.Context, in *api.Payload, opts ...grpc.CallOption) (Zero_OracleClient, error) ShouldServe(ctx context.Context, in *Tablet, opts ...grpc.CallOption) (*Tablet, error) AssignUids(ctx context.Context, in *Num, opts ...grpc.CallOption) (*AssignedIds, error) @@ -3821,30 +3840,40 @@ func (c *zeroClient) Connect(ctx context.Context, in *Member, opts ...grpc.CallO return out, nil } -func (c *zeroClient) Update(ctx context.Context, opts ...grpc.CallOption) (Zero_UpdateClient, error) { - stream, err := c.cc.NewStream(ctx, &_Zero_serviceDesc.Streams[0], "/pb.Zero/Update", opts...) +func (c *zeroClient) UpdateMembership(ctx context.Context, in *Group, opts ...grpc.CallOption) (*api.Payload, error) { + out := new(api.Payload) + err := c.cc.Invoke(ctx, "/pb.Zero/UpdateMembership", in, out, opts...) if err != nil { return nil, err } - x := &zeroUpdateClient{stream} + return out, nil +} + +func (c *zeroClient) StreamMembership(ctx context.Context, in *api.Payload, opts ...grpc.CallOption) (Zero_StreamMembershipClient, error) { + stream, err := c.cc.NewStream(ctx, &_Zero_serviceDesc.Streams[0], "/pb.Zero/StreamMembership", opts...) + if err != nil { + return nil, err + } + x := &zeroStreamMembershipClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } return x, nil } -type Zero_UpdateClient interface { - Send(*Group) error +type Zero_StreamMembershipClient interface { Recv() (*MembershipState, error) grpc.ClientStream } -type zeroUpdateClient struct { +type zeroStreamMembershipClient struct { grpc.ClientStream } -func (x *zeroUpdateClient) Send(m *Group) error { - return x.ClientStream.SendMsg(m) -} - -func (x *zeroUpdateClient) Recv() (*MembershipState, error) { +func (x *zeroStreamMembershipClient) Recv() (*MembershipState, error) { m := new(MembershipState) if err := x.ClientStream.RecvMsg(m); err != nil { return nil, err @@ -3931,8 +3960,10 @@ func (c *zeroClient) TryAbort(ctx context.Context, in *TxnTimestamps, opts ...gr // ZeroServer is the server API for Zero service. type ZeroServer interface { + // These 3 endpoints are for handling membership. Connect(context.Context, *Member) (*ConnectionState, error) - Update(Zero_UpdateServer) error + UpdateMembership(context.Context, *Group) (*api.Payload, error) + StreamMembership(*api.Payload, Zero_StreamMembershipServer) error Oracle(*api.Payload, Zero_OracleServer) error ShouldServe(context.Context, *Tablet) (*Tablet, error) AssignUids(context.Context, *Num) (*AssignedIds, error) @@ -3963,32 +3994,45 @@ func _Zero_Connect_Handler(srv interface{}, ctx context.Context, dec func(interf return interceptor(ctx, in, info, handler) } -func _Zero_Update_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(ZeroServer).Update(&zeroUpdateServer{stream}) +func _Zero_UpdateMembership_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(Group) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ZeroServer).UpdateMembership(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pb.Zero/UpdateMembership", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ZeroServer).UpdateMembership(ctx, req.(*Group)) + } + return interceptor(ctx, in, info, handler) } -type Zero_UpdateServer interface { +func _Zero_StreamMembership_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(api.Payload) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(ZeroServer).StreamMembership(m, &zeroStreamMembershipServer{stream}) +} + +type Zero_StreamMembershipServer interface { Send(*MembershipState) error - Recv() (*Group, error) grpc.ServerStream } -type zeroUpdateServer struct { +type zeroStreamMembershipServer struct { grpc.ServerStream } -func (x *zeroUpdateServer) Send(m *MembershipState) error { +func (x *zeroStreamMembershipServer) Send(m *MembershipState) error { return x.ServerStream.SendMsg(m) } -func (x *zeroUpdateServer) Recv() (*Group, error) { - m := new(Group) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - func _Zero_Oracle_Handler(srv interface{}, stream grpc.ServerStream) error { m := new(api.Payload) if err := stream.RecvMsg(m); err != nil { @@ -4108,6 +4152,10 @@ var _Zero_serviceDesc = grpc.ServiceDesc{ MethodName: "Connect", Handler: _Zero_Connect_Handler, }, + { + MethodName: "UpdateMembership", + Handler: _Zero_UpdateMembership_Handler, + }, { MethodName: "ShouldServe", Handler: _Zero_ShouldServe_Handler, @@ -4131,10 +4179,9 @@ var _Zero_serviceDesc = grpc.ServiceDesc{ }, Streams: []grpc.StreamDesc{ { - StreamName: "Update", - Handler: _Zero_Update_Handler, + StreamName: "StreamMembership", + Handler: _Zero_StreamMembership_Handler, ServerStreams: true, - ClientStreams: true, }, { StreamName: "Oracle", @@ -4155,7 +4202,6 @@ type WorkerClient interface { StreamSnapshot(ctx context.Context, opts ...grpc.CallOption) (Worker_StreamSnapshotClient, error) Sort(ctx context.Context, in *SortMessage, opts ...grpc.CallOption) (*SortResult, error) Schema(ctx context.Context, in *SchemaRequest, opts ...grpc.CallOption) (*SchemaResult, error) - PurgeTs(ctx context.Context, in *api.Payload, opts ...grpc.CallOption) (*Num, error) Backup(ctx context.Context, in *BackupRequest, opts ...grpc.CallOption) (*Status, error) Export(ctx context.Context, in *ExportRequest, opts ...grpc.CallOption) (*Status, error) ReceivePredicate(ctx context.Context, opts ...grpc.CallOption) (Worker_ReceivePredicateClient, error) @@ -4237,15 +4283,6 @@ func (c *workerClient) Schema(ctx context.Context, in *SchemaRequest, opts ...gr return out, nil } -func (c *workerClient) PurgeTs(ctx context.Context, in *api.Payload, opts ...grpc.CallOption) (*Num, error) { - out := new(Num) - err := c.cc.Invoke(ctx, "/pb.Worker/PurgeTs", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *workerClient) Backup(ctx context.Context, in *BackupRequest, opts ...grpc.CallOption) (*Status, error) { out := new(Status) err := c.cc.Invoke(ctx, "/pb.Worker/Backup", in, out, opts...) @@ -4315,7 +4352,6 @@ type WorkerServer interface { StreamSnapshot(Worker_StreamSnapshotServer) error Sort(context.Context, *SortMessage) (*SortResult, error) Schema(context.Context, *SchemaRequest) (*SchemaResult, error) - PurgeTs(context.Context, *api.Payload) (*Num, error) Backup(context.Context, *BackupRequest) (*Status, error) Export(context.Context, *ExportRequest) (*Status, error) ReceivePredicate(Worker_ReceivePredicateServer) error @@ -4424,24 +4460,6 @@ func _Worker_Schema_Handler(srv interface{}, ctx context.Context, dec func(inter return interceptor(ctx, in, info, handler) } -func _Worker_PurgeTs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(api.Payload) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(WorkerServer).PurgeTs(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/pb.Worker/PurgeTs", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(WorkerServer).PurgeTs(ctx, req.(*api.Payload)) - } - return interceptor(ctx, in, info, handler) -} - func _Worker_Backup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(BackupRequest) if err := dec(in); err != nil { @@ -4542,10 +4560,6 @@ var _Worker_serviceDesc = grpc.ServiceDesc{ MethodName: "Schema", Handler: _Worker_Schema_Handler, }, - { - MethodName: "PurgeTs", - Handler: _Worker_PurgeTs_Handler, - }, { MethodName: "Backup", Handler: _Worker_Backup_Handler, @@ -5318,6 +5332,11 @@ func (m *Group) MarshalTo(dAtA []byte) (int, error) { } } } + if m.SnapshotTs != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintPb(dAtA, i, uint64(m.SnapshotTs)) + } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) } @@ -5339,6 +5358,21 @@ func (m *ZeroProposal) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.SnapshotTs) > 0 { + for k, _ := range m.SnapshotTs { + dAtA[i] = 0xa + i++ + v := m.SnapshotTs[k] + mapSize := 1 + sovPb(uint64(k)) + 1 + sovPb(uint64(v)) + i = encodeVarintPb(dAtA, i, uint64(mapSize)) + dAtA[i] = 0x8 + i++ + i = encodeVarintPb(dAtA, i, uint64(k)) + dAtA[i] = 0x10 + i++ + i = encodeVarintPb(dAtA, i, uint64(v)) + } + } if m.Member != nil { dAtA[i] = 0x12 i++ @@ -7519,6 +7553,9 @@ func (m *Group) Size() (n int) { n += mapEntrySize + 1 + sovPb(uint64(mapEntrySize)) } } + if m.SnapshotTs != 0 { + n += 1 + sovPb(uint64(m.SnapshotTs)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -7531,6 +7568,14 @@ func (m *ZeroProposal) Size() (n int) { } var l int _ = l + if len(m.SnapshotTs) > 0 { + for k, v := range m.SnapshotTs { + _ = k + _ = v + mapEntrySize := 1 + sovPb(uint64(k)) + 1 + sovPb(uint64(v)) + n += mapEntrySize + 1 + sovPb(uint64(mapEntrySize)) + } + } if m.Member != nil { l = m.Member.Size() n += 1 + l + sovPb(uint64(l)) @@ -10574,6 +10619,25 @@ func (m *Group) Unmarshal(dAtA []byte) error { } m.Tablets[mapkey] = mapvalue iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SnapshotTs", wireType) + } + m.SnapshotTs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SnapshotTs |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipPb(dAtA[iNdEx:]) @@ -10625,6 +10689,102 @@ func (m *ZeroProposal) Unmarshal(dAtA []byte) error { return fmt.Errorf("proto: ZeroProposal: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SnapshotTs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SnapshotTs == nil { + m.SnapshotTs = make(map[uint32]uint64) + } + var mapkey uint32 + var mapvalue uint64 + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + } else if fieldNum == 2 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapvalue |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + } else { + iNdEx = entryPreIndex + skippy, err := skipPb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPb + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.SnapshotTs[mapkey] = mapvalue + iNdEx = postIndex case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Member", wireType) @@ -16047,205 +16207,207 @@ var ( ErrIntOverflowPb = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("pb.proto", fileDescriptor_pb_1efee8619f75cb94) } - -var fileDescriptor_pb_1efee8619f75cb94 = []byte{ - // 3140 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x39, 0xcb, 0x72, 0x1b, 0xc7, - 0xb5, 0x9c, 0x01, 0x30, 0x98, 0x39, 0x00, 0x28, 0xb8, 0xad, 0x2b, 0xc3, 0xb4, 0x2f, 0x45, 0x8f, - 0xf5, 0xa0, 0x64, 0x9b, 0x57, 0xa6, 0x7d, 0xaf, 0x1f, 0x3b, 0x4a, 0x84, 0x54, 0xb4, 0xf8, 0xba, - 0x0d, 0x50, 0xbe, 0xd7, 0x95, 0x32, 0xaa, 0x39, 0xd3, 0x04, 0x27, 0x1c, 0xcc, 0x4c, 0xa6, 0x07, - 0x2c, 0x50, 0xbb, 0xfc, 0x85, 0x17, 0xa9, 0x2c, 0xb2, 0x4c, 0x7e, 0x20, 0xf9, 0x80, 0x54, 0x25, - 0xbb, 0x6c, 0xb3, 0x4b, 0x29, 0xab, 0xac, 0x53, 0xa9, 0x2c, 0xb2, 0x49, 0xf5, 0xe9, 0x9e, 0x07, - 0x20, 0x52, 0xb2, 0x53, 0x95, 0x15, 0xe6, 0x3c, 0xfa, 0x75, 0xde, 0xe7, 0x00, 0xec, 0xe4, 0x78, - 0x23, 0x49, 0xe3, 0x2c, 0x26, 0x66, 0x72, 0xbc, 0xe2, 0xb0, 0x24, 0x50, 0xa0, 0xbb, 0x02, 0xf5, - 0xdd, 0x40, 0x64, 0x84, 0x40, 0x7d, 0x1a, 0xf8, 0xa2, 0x67, 0xac, 0xd5, 0xd6, 0x2d, 0x8a, 0xdf, - 0xee, 0x1e, 0x38, 0x43, 0x26, 0xce, 0x9e, 0xb1, 0x70, 0xca, 0x49, 0x17, 0x6a, 0xe7, 0x2c, 0xec, - 0x19, 0x6b, 0xc6, 0x7a, 0x9b, 0xca, 0x4f, 0xb2, 0x01, 0xf6, 0x39, 0x0b, 0x47, 0xd9, 0x45, 0xc2, - 0x7b, 0xe6, 0x9a, 0xb1, 0xbe, 0xbc, 0xf9, 0xe6, 0x46, 0x72, 0xbc, 0x71, 0x18, 0x8b, 0x2c, 0x88, - 0xc6, 0x1b, 0xcf, 0x58, 0x38, 0xbc, 0x48, 0x38, 0x6d, 0x9e, 0xab, 0x0f, 0xf7, 0x00, 0x5a, 0x83, - 0xd4, 0x7b, 0x3c, 0x8d, 0xbc, 0x2c, 0x88, 0x23, 0x79, 0x62, 0xc4, 0x26, 0x1c, 0x77, 0x74, 0x28, - 0x7e, 0x4b, 0x1c, 0x4b, 0xc7, 0xa2, 0x57, 0x5b, 0xab, 0x49, 0x9c, 0xfc, 0x26, 0x3d, 0x68, 0x06, - 0xe2, 0x51, 0x3c, 0x8d, 0xb2, 0x5e, 0x7d, 0xcd, 0x58, 0xb7, 0x69, 0x0e, 0xba, 0x7f, 0x35, 0xa1, - 0xf1, 0xbf, 0x53, 0x9e, 0x5e, 0xe0, 0xba, 0x2c, 0x4b, 0xf3, 0xbd, 0xe4, 0x37, 0xb9, 0x0e, 0x8d, - 0x90, 0x45, 0x63, 0xd1, 0x33, 0x71, 0x33, 0x05, 0x90, 0x77, 0xc0, 0x61, 0x27, 0x19, 0x4f, 0x47, - 0xd3, 0xc0, 0xef, 0xd5, 0xd6, 0x8c, 0x75, 0x8b, 0xda, 0x88, 0x38, 0x0a, 0x7c, 0xf2, 0x36, 0xd8, - 0x7e, 0x3c, 0xf2, 0xaa, 0x67, 0xf9, 0x31, 0x9e, 0x45, 0xde, 0x07, 0x7b, 0x1a, 0xf8, 0xa3, 0x30, - 0x10, 0x59, 0xaf, 0xb1, 0x66, 0xac, 0xb7, 0x36, 0x6d, 0xf9, 0x58, 0x29, 0x3b, 0xda, 0x9c, 0x06, - 0x3e, 0x0a, 0xf1, 0x3e, 0xd8, 0x22, 0xf5, 0x46, 0x27, 0xd3, 0xc8, 0xeb, 0x59, 0xc8, 0x74, 0x4d, - 0x32, 0x55, 0x5e, 0x4d, 0x9b, 0x42, 0x01, 0xf2, 0x59, 0x29, 0x3f, 0xe7, 0xa9, 0xe0, 0xbd, 0xa6, - 0x3a, 0x4a, 0x83, 0xe4, 0x01, 0xb4, 0x4e, 0x98, 0xc7, 0xb3, 0x51, 0xc2, 0x52, 0x36, 0xe9, 0xd9, - 0xe5, 0x46, 0x8f, 0x25, 0xfa, 0x50, 0x62, 0x05, 0x85, 0x93, 0x02, 0x20, 0x9f, 0x40, 0x07, 0x21, - 0x31, 0x3a, 0x09, 0xc2, 0x8c, 0xa7, 0x3d, 0x07, 0xd7, 0x2c, 0xe3, 0x1a, 0xc4, 0x0c, 0x53, 0xce, - 0x69, 0x5b, 0x31, 0x29, 0x0c, 0xf9, 0x4f, 0x00, 0x3e, 0x4b, 0x58, 0xe4, 0x8f, 0x58, 0x18, 0xf6, - 0x00, 0xef, 0xe0, 0x28, 0xcc, 0x56, 0x18, 0x92, 0xb7, 0xe4, 0xfd, 0x98, 0x3f, 0xca, 0x44, 0xaf, - 0xb3, 0x66, 0xac, 0xd7, 0xa9, 0x25, 0xc1, 0xa1, 0x70, 0x37, 0xc1, 0x41, 0x8b, 0xc0, 0x17, 0xdf, - 0x06, 0xeb, 0x5c, 0x02, 0xca, 0x70, 0x5a, 0x9b, 0x1d, 0x79, 0x64, 0x61, 0x34, 0x54, 0x13, 0xdd, - 0x55, 0xb0, 0x77, 0x59, 0x34, 0xce, 0x2d, 0x4d, 0xaa, 0x02, 0x17, 0x38, 0x14, 0xbf, 0xdd, 0xef, - 0x4c, 0xb0, 0x28, 0x17, 0xd3, 0x30, 0x23, 0x77, 0x01, 0xa4, 0xa0, 0x27, 0x2c, 0x4b, 0x83, 0x99, - 0xde, 0xb5, 0x14, 0xb5, 0x33, 0x0d, 0xfc, 0x3d, 0x24, 0x91, 0x07, 0xd0, 0xc6, 0xdd, 0x73, 0x56, - 0xb3, 0xbc, 0x40, 0x71, 0x3f, 0xda, 0x42, 0x16, 0xbd, 0xe2, 0x06, 0x58, 0xa8, 0x5b, 0x65, 0x5f, - 0x1d, 0xaa, 0x21, 0x72, 0x1b, 0x96, 0x83, 0x28, 0x93, 0xb2, 0xf7, 0xb2, 0x91, 0xcf, 0x45, 0xae, - 0xfc, 0x4e, 0x81, 0xdd, 0xe6, 0x22, 0x23, 0x1f, 0x83, 0x12, 0x60, 0x7e, 0x60, 0x03, 0x0f, 0x5c, - 0x2e, 0x14, 0x23, 0xd4, 0x89, 0xc8, 0xa3, 0x4f, 0xfc, 0x08, 0x5a, 0xf2, 0x7d, 0xf9, 0x0a, 0x0b, - 0x57, 0xb4, 0xf1, 0x35, 0x5a, 0x1c, 0x14, 0x24, 0x83, 0x66, 0x97, 0xa2, 0x91, 0x06, 0xa6, 0x0c, - 0x02, 0xbf, 0xdd, 0x3e, 0x34, 0x0e, 0x52, 0x9f, 0xa7, 0x97, 0xda, 0x38, 0x81, 0xba, 0xcf, 0x85, - 0x87, 0xee, 0x67, 0x53, 0xfc, 0x2e, 0xed, 0xbe, 0x56, 0xb1, 0x7b, 0xf7, 0xe7, 0x06, 0xb4, 0x06, - 0x71, 0x9a, 0xed, 0x71, 0x21, 0xd8, 0x98, 0x93, 0x9b, 0xd0, 0x88, 0xe5, 0xb6, 0x5a, 0xc2, 0x8e, - 0xbc, 0x13, 0x9e, 0x43, 0x15, 0x7e, 0x41, 0x0f, 0xe6, 0xd5, 0x7a, 0xb8, 0x0e, 0x0d, 0xe5, 0x31, - 0xd2, 0x9b, 0x1a, 0x54, 0x01, 0x52, 0xd6, 0xf1, 0xc9, 0x89, 0xe0, 0x4a, 0x96, 0x0d, 0xaa, 0xa1, - 0xab, 0xcd, 0xea, 0xbf, 0x01, 0xe4, 0xfd, 0x7e, 0xa0, 0x15, 0xb8, 0xa7, 0xd0, 0xa2, 0xec, 0x24, - 0x7b, 0x14, 0x47, 0x19, 0x9f, 0x65, 0x64, 0x19, 0xcc, 0xc0, 0x47, 0x11, 0x59, 0xd4, 0x0c, 0x7c, - 0x79, 0xb9, 0x71, 0x1a, 0x4f, 0x13, 0x94, 0x50, 0x87, 0x2a, 0x00, 0x45, 0xe9, 0xfb, 0x29, 0xde, - 0x58, 0x8a, 0xd2, 0xf7, 0x53, 0x72, 0x13, 0x5a, 0x22, 0x62, 0x89, 0x38, 0x8d, 0x33, 0x79, 0xb9, - 0x3a, 0x5e, 0x0e, 0x72, 0xd4, 0x50, 0xb8, 0xbf, 0x35, 0xc0, 0xda, 0xe3, 0x93, 0x63, 0x9e, 0xbe, - 0x74, 0xca, 0xdb, 0x60, 0xe3, 0xc6, 0xa3, 0xc0, 0xd7, 0x07, 0x35, 0x11, 0xde, 0xf1, 0x2f, 0x3d, - 0xea, 0x06, 0x58, 0x21, 0x67, 0x52, 0xf8, 0xca, 0xce, 0x34, 0x24, 0x65, 0xc3, 0x26, 0x23, 0x9f, - 0x33, 0x1f, 0x43, 0x8c, 0x4d, 0x2d, 0x36, 0xd9, 0xe6, 0xcc, 0x97, 0x77, 0x0b, 0x99, 0xc8, 0x46, - 0xd3, 0xc4, 0x67, 0x19, 0xc7, 0xd0, 0x52, 0x97, 0x86, 0x23, 0xb2, 0x23, 0xc4, 0x90, 0xfb, 0xf0, - 0x86, 0x17, 0x4e, 0x85, 0x8c, 0x6b, 0x41, 0x74, 0x12, 0x8f, 0xe2, 0x28, 0xbc, 0x40, 0xf9, 0xda, - 0xf4, 0x9a, 0x26, 0xec, 0x44, 0x27, 0xf1, 0x41, 0x14, 0x5e, 0xb8, 0xff, 0x30, 0xa0, 0xf1, 0x04, - 0xc5, 0xf0, 0x00, 0x9a, 0x13, 0x7c, 0x50, 0xee, 0xbd, 0x37, 0xa4, 0x84, 0x91, 0xb6, 0xa1, 0x5e, - 0x2a, 0xfa, 0x51, 0x96, 0x5e, 0xd0, 0x9c, 0x4d, 0xae, 0xc8, 0xd8, 0x71, 0xc8, 0x33, 0xa1, 0x2d, - 0xa2, 0xb2, 0x62, 0xa8, 0x08, 0x7a, 0x85, 0x66, 0x5b, 0x79, 0x0c, 0xed, 0xea, 0x56, 0x32, 0x8d, - 0x9c, 0xf1, 0x0b, 0x94, 0x5d, 0x9d, 0xca, 0x4f, 0xb2, 0x06, 0x0d, 0x74, 0x52, 0x94, 0x5c, 0x6b, - 0x13, 0xe4, 0x8e, 0x6a, 0x09, 0x55, 0x84, 0x2f, 0xcd, 0xcf, 0x0d, 0xb9, 0x4f, 0xf5, 0x80, 0xea, - 0x3e, 0xce, 0xd5, 0xfb, 0xa8, 0x25, 0x95, 0x7d, 0xdc, 0xbf, 0x1b, 0xd0, 0xfe, 0x86, 0xa7, 0xf1, - 0x61, 0x1a, 0x27, 0xb1, 0x60, 0x21, 0x71, 0xc1, 0x52, 0xaf, 0xbb, 0xe4, 0x7c, 0x4d, 0x91, 0x3c, - 0xea, 0x3d, 0xa8, 0xc6, 0xf9, 0xbd, 0x35, 0x85, 0xac, 0x02, 0x4c, 0xd8, 0x6c, 0x97, 0x33, 0xc1, - 0x77, 0xfc, 0xdc, 0x7c, 0x4a, 0x0c, 0x59, 0x01, 0x7b, 0xc2, 0x66, 0xc3, 0x59, 0x34, 0x14, 0xa8, - 0xdd, 0x3a, 0x2d, 0x60, 0xf2, 0x2e, 0x38, 0x13, 0x36, 0x93, 0x76, 0xbc, 0xe3, 0x6b, 0xed, 0x96, - 0x08, 0xf2, 0x1e, 0xd4, 0xb2, 0x59, 0x84, 0x41, 0x41, 0xe6, 0x01, 0x99, 0xbb, 0x87, 0xb3, 0x48, - 0x5b, 0x3c, 0x95, 0xb4, 0x5c, 0x1a, 0x76, 0x29, 0x8d, 0x2e, 0xd4, 0xbc, 0xc0, 0xc7, 0x44, 0xe0, - 0x50, 0xf9, 0xe9, 0xfe, 0xba, 0x06, 0xd7, 0xb4, 0x2a, 0x4e, 0x83, 0x64, 0x90, 0x49, 0xbb, 0xe9, - 0x41, 0x13, 0xdd, 0x95, 0xa7, 0x5a, 0x23, 0x39, 0x48, 0x3e, 0x03, 0x0b, 0x4d, 0x38, 0x57, 0xf4, - 0xcd, 0x52, 0x2c, 0xc5, 0x72, 0xa5, 0x78, 0xad, 0x71, 0xcd, 0x4e, 0x3e, 0x85, 0xc6, 0x73, 0x9e, - 0xc6, 0x2a, 0xfc, 0xb4, 0x36, 0x57, 0x2f, 0x5b, 0x27, 0x15, 0xa0, 0x97, 0x29, 0xe6, 0x7f, 0xa3, - 0xf4, 0x6e, 0xc9, 0x80, 0x33, 0x89, 0xcf, 0xb9, 0xdf, 0x6b, 0xe2, 0x8d, 0xaa, 0x0a, 0xce, 0x49, - 0xb9, 0xb8, 0xec, 0x42, 0x5c, 0x2b, 0xdb, 0xd0, 0xaa, 0x3c, 0xaf, 0x6a, 0x6f, 0x1d, 0x25, 0xe1, - 0x9b, 0xf3, 0xf6, 0xe6, 0x14, 0x9e, 0x50, 0x35, 0xdb, 0x6d, 0x80, 0xf2, 0xb1, 0xff, 0xaa, 0xf1, - 0xbb, 0x3f, 0x35, 0xe0, 0xda, 0xa3, 0x38, 0x8a, 0x38, 0xd6, 0x10, 0x4a, 0x75, 0xa5, 0xdd, 0x1a, - 0x57, 0xda, 0xed, 0x3d, 0x68, 0x08, 0xc9, 0xac, 0x77, 0x7f, 0xf3, 0x12, 0x5d, 0x50, 0xc5, 0x21, - 0x43, 0xcc, 0x84, 0xcd, 0x46, 0x09, 0x8f, 0xfc, 0x20, 0x1a, 0xa3, 0x9d, 0x2b, 0x0d, 0x1c, 0x2a, - 0x8c, 0xfb, 0x0b, 0x03, 0x2c, 0x65, 0xf2, 0x73, 0xe1, 0xce, 0x98, 0x0f, 0x77, 0xef, 0x82, 0x93, - 0xa4, 0xdc, 0x0f, 0xbc, 0xfc, 0x54, 0x87, 0x96, 0x08, 0x19, 0x8d, 0x4f, 0xe2, 0xd4, 0xe3, 0xb8, - 0xbd, 0x4d, 0x15, 0x20, 0x4b, 0x32, 0x4c, 0x09, 0x18, 0xb4, 0x54, 0x44, 0xb4, 0x25, 0x42, 0x46, - 0x2b, 0xb9, 0x44, 0x24, 0xcc, 0x53, 0x45, 0x52, 0x8d, 0x2a, 0x40, 0x46, 0x50, 0xa5, 0x39, 0xd4, - 0x98, 0x4d, 0x35, 0xe4, 0xfe, 0xd2, 0x84, 0xf6, 0x76, 0x90, 0x72, 0x2f, 0xe3, 0x7e, 0xdf, 0x1f, - 0x23, 0x23, 0x8f, 0xb2, 0x20, 0xbb, 0xd0, 0xd1, 0x5a, 0x43, 0x45, 0x32, 0x35, 0xe7, 0x0b, 0x46, - 0xa5, 0x8b, 0x1a, 0xd6, 0xb8, 0x0a, 0x20, 0x9b, 0x00, 0xaa, 0xcc, 0xc0, 0x3a, 0xb7, 0x7e, 0x75, - 0x9d, 0xeb, 0x20, 0x9b, 0xfc, 0x94, 0x02, 0x52, 0x6b, 0x02, 0x15, 0xc9, 0x2d, 0x2c, 0x82, 0xa7, - 0xd2, 0x90, 0x31, 0x3b, 0x1f, 0xf3, 0x10, 0x0d, 0x15, 0xb3, 0xf3, 0x31, 0x0f, 0x8b, 0x9a, 0xa8, - 0xa9, 0xae, 0x23, 0xbf, 0xc9, 0xfb, 0x60, 0xc6, 0x09, 0xbe, 0x4f, 0x1f, 0x58, 0x7d, 0xd8, 0xc6, - 0x41, 0x42, 0xcd, 0x38, 0x91, 0x56, 0xa0, 0x8a, 0xba, 0x9e, 0xa3, 0x8d, 0x5b, 0x86, 0x07, 0x2c, - 0x47, 0xa8, 0xa6, 0xb8, 0x37, 0xc0, 0x3c, 0x48, 0x48, 0x13, 0x6a, 0x83, 0xfe, 0xb0, 0xbb, 0x24, - 0x3f, 0xb6, 0xfb, 0xbb, 0x5d, 0xc3, 0x7d, 0x61, 0x80, 0xb3, 0x37, 0xcd, 0x98, 0xb4, 0x29, 0xf1, - 0x2a, 0xa5, 0xbe, 0x0d, 0xb6, 0xc8, 0x58, 0x8a, 0x79, 0xd1, 0x54, 0x61, 0x02, 0xe1, 0xa1, 0x20, - 0x77, 0xa0, 0xc1, 0xfd, 0x31, 0xcf, 0xbd, 0xbd, 0xbb, 0x78, 0x4f, 0xaa, 0xc8, 0x64, 0x1d, 0x2c, - 0xe1, 0x9d, 0xf2, 0x09, 0xeb, 0xd5, 0x4b, 0xc6, 0x01, 0x62, 0x54, 0x0a, 0xa3, 0x9a, 0x8e, 0x35, - 0x78, 0x1a, 0x27, 0x58, 0x94, 0x36, 0x74, 0x0d, 0x9e, 0xc6, 0x89, 0x2c, 0x49, 0x37, 0xe1, 0x3f, - 0x82, 0x71, 0x14, 0xa7, 0x7c, 0x14, 0x44, 0x3e, 0x9f, 0x8d, 0xbc, 0x38, 0x3a, 0x09, 0x03, 0x2f, - 0x43, 0x59, 0xda, 0xf4, 0x4d, 0x45, 0xdc, 0x91, 0xb4, 0x47, 0x9a, 0xe4, 0xbe, 0x0f, 0xce, 0x53, - 0x7e, 0x81, 0x05, 0xa1, 0x20, 0x37, 0xc0, 0x3c, 0x3b, 0xd7, 0xb9, 0xce, 0x92, 0x37, 0x78, 0xfa, - 0x8c, 0x9a, 0x67, 0xe7, 0xee, 0x0c, 0xec, 0x81, 0x4e, 0xf4, 0xe4, 0x9e, 0x0c, 0x89, 0x18, 0x5a, - 0xb5, 0x63, 0x61, 0xe5, 0x5d, 0xa9, 0x31, 0x68, 0x4e, 0x97, 0xba, 0xc4, 0x8b, 0x68, 0xa1, 0x28, - 0xa0, 0x5a, 0xe1, 0xd4, 0xaa, 0x15, 0x0e, 0x16, 0x6b, 0x71, 0xc4, 0xb5, 0x89, 0xe3, 0xb7, 0xfb, - 0x33, 0x13, 0xec, 0x22, 0x15, 0x7d, 0x00, 0xce, 0x24, 0xd7, 0x87, 0x76, 0x59, 0x2c, 0x67, 0x0b, - 0x25, 0xd1, 0x92, 0xae, 0xdf, 0x52, 0x5f, 0x7c, 0x4b, 0xe9, 0xf3, 0x8d, 0xd7, 0xfa, 0xfc, 0x5d, - 0xb8, 0xe6, 0x85, 0x9c, 0x45, 0xa3, 0xd2, 0x65, 0x95, 0x55, 0x2e, 0x23, 0xfa, 0xb0, 0xf0, 0x5b, - 0x1d, 0xb7, 0x9a, 0x65, 0x7a, 0xb9, 0x0d, 0x0d, 0x9f, 0x87, 0x19, 0xab, 0x76, 0x27, 0x07, 0x29, - 0xf3, 0x42, 0xbe, 0x2d, 0xd1, 0x54, 0x51, 0xc9, 0x3a, 0xd8, 0x79, 0x05, 0xa5, 0x7b, 0x12, 0x2c, - 0x7e, 0x73, 0x61, 0xd3, 0x82, 0x5a, 0xca, 0x12, 0x2a, 0xb2, 0x74, 0x3f, 0x86, 0xda, 0xd3, 0x67, - 0x83, 0xab, 0xf4, 0x56, 0x48, 0xd4, 0xac, 0x48, 0xf4, 0x5b, 0x30, 0x9f, 0x3e, 0xab, 0x46, 0xda, - 0x76, 0x91, 0x10, 0x65, 0xff, 0x6a, 0x96, 0xfd, 0xeb, 0x0a, 0xd8, 0x53, 0xc1, 0xd3, 0x3d, 0x9e, - 0x31, 0xed, 0xf2, 0x05, 0x2c, 0x13, 0xa3, 0x6c, 0xc6, 0x82, 0x38, 0xd2, 0xc9, 0x28, 0x07, 0xdd, - 0xbf, 0xd4, 0xa0, 0xa9, 0x5d, 0x5f, 0xee, 0x39, 0x2d, 0x0a, 0x41, 0xf9, 0x59, 0xc6, 0x10, 0xb3, - 0x1a, 0x43, 0xaa, 0x9d, 0x72, 0xed, 0xf5, 0x9d, 0x32, 0xf9, 0x12, 0xda, 0x89, 0xa2, 0x55, 0xa3, - 0xce, 0x5b, 0xd5, 0x35, 0xfa, 0x17, 0xd7, 0xb5, 0x92, 0x12, 0x90, 0xfe, 0x83, 0x2d, 0x47, 0xc6, - 0xc6, 0x68, 0x02, 0x6d, 0xda, 0x94, 0xf0, 0x90, 0x8d, 0xaf, 0x88, 0x3d, 0xdf, 0x23, 0x84, 0xc8, - 0x82, 0x37, 0x4e, 0x7a, 0x6d, 0x0c, 0x0b, 0x32, 0xec, 0x54, 0x23, 0x42, 0x67, 0x3e, 0x22, 0xbc, - 0x03, 0x8e, 0x17, 0x4f, 0x26, 0x01, 0xd2, 0x96, 0x55, 0xaa, 0x56, 0x88, 0xa1, 0x70, 0x9f, 0x43, - 0x53, 0x3f, 0x96, 0xb4, 0xa0, 0xb9, 0xdd, 0x7f, 0xbc, 0x75, 0xb4, 0x2b, 0x63, 0x12, 0x80, 0xf5, - 0x70, 0x67, 0x7f, 0x8b, 0xfe, 0x7f, 0xd7, 0x90, 0xf1, 0x69, 0x67, 0x7f, 0xd8, 0x35, 0x89, 0x03, - 0x8d, 0xc7, 0xbb, 0x07, 0x5b, 0xc3, 0x6e, 0x8d, 0xd8, 0x50, 0x7f, 0x78, 0x70, 0xb0, 0xdb, 0xad, - 0x93, 0x36, 0xd8, 0xdb, 0x5b, 0xc3, 0xfe, 0x70, 0x67, 0xaf, 0xdf, 0x6d, 0x48, 0xde, 0x27, 0xfd, - 0x83, 0xae, 0x25, 0x3f, 0x8e, 0x76, 0xb6, 0xbb, 0x4d, 0x49, 0x3f, 0xdc, 0x1a, 0x0c, 0xbe, 0x3e, - 0xa0, 0xdb, 0x5d, 0x5b, 0xee, 0x3b, 0x18, 0xd2, 0x9d, 0xfd, 0x27, 0x5d, 0xc7, 0xfd, 0x18, 0x5a, - 0x15, 0xa1, 0xc9, 0x15, 0xb4, 0xff, 0xb8, 0xbb, 0x24, 0x8f, 0x79, 0xb6, 0xb5, 0x7b, 0xd4, 0xef, - 0x1a, 0x64, 0x19, 0x00, 0x3f, 0x47, 0xbb, 0x5b, 0xfb, 0x4f, 0xba, 0xa6, 0xfb, 0x3f, 0x60, 0x1f, - 0x05, 0xfe, 0xc3, 0x30, 0xf6, 0xce, 0xa4, 0xad, 0x1d, 0x33, 0xc1, 0x75, 0xf2, 0xc6, 0x6f, 0x99, - 0x5d, 0xd0, 0xce, 0x85, 0x56, 0xb7, 0x86, 0xdc, 0x7d, 0x68, 0x1e, 0x05, 0xfe, 0x21, 0xf3, 0xce, - 0x64, 0x97, 0x7d, 0x2c, 0xd7, 0x8f, 0x44, 0xf0, 0x9c, 0xeb, 0xc0, 0xea, 0x20, 0x66, 0x10, 0x3c, - 0xe7, 0xe4, 0x16, 0x58, 0x08, 0xe4, 0x65, 0x16, 0xba, 0x47, 0x7e, 0x26, 0xd5, 0x34, 0x37, 0x2b, - 0xae, 0x8e, 0x1d, 0xf4, 0x4d, 0xa8, 0x27, 0xcc, 0x3b, 0xd3, 0xf1, 0xa9, 0xa5, 0x97, 0xc8, 0xe3, - 0x28, 0x12, 0xc8, 0x5d, 0xb0, 0xb5, 0x49, 0xe4, 0xfb, 0xb6, 0x2a, 0xb6, 0x43, 0x0b, 0xe2, 0xbc, - 0xb2, 0x6a, 0x0b, 0xca, 0xfa, 0x14, 0xa0, 0x1c, 0x38, 0x5c, 0x52, 0x70, 0x5f, 0x87, 0x06, 0x0b, - 0x03, 0xfd, 0x78, 0x87, 0x2a, 0xc0, 0xdd, 0x87, 0x56, 0x65, 0x4c, 0x21, 0x2d, 0x85, 0x85, 0xe1, - 0xe8, 0x8c, 0x5f, 0x08, 0x5c, 0x6b, 0xd3, 0x26, 0x0b, 0xc3, 0xa7, 0xfc, 0x42, 0x90, 0x5b, 0xd0, - 0x50, 0x13, 0x0e, 0x73, 0xa1, 0x91, 0xc6, 0xa5, 0x54, 0x11, 0xdd, 0x0f, 0xc1, 0x52, 0xdd, 0x75, - 0xc5, 0x50, 0x8d, 0x2b, 0x73, 0xdd, 0x17, 0xfa, 0xce, 0xd8, 0x8b, 0x93, 0x0f, 0xf4, 0x24, 0x45, - 0xa8, 0xb9, 0x8d, 0x51, 0xd6, 0x7f, 0x8a, 0x49, 0x0f, 0x51, 0x90, 0xd9, 0xdd, 0x06, 0xfb, 0x95, - 0xb3, 0x29, 0x2d, 0x00, 0xb3, 0x14, 0xc0, 0x25, 0xd3, 0x2a, 0xf7, 0xc7, 0x00, 0xe5, 0xc4, 0x45, - 0xfb, 0x8d, 0xda, 0x45, 0xfa, 0xcd, 0x7d, 0xb0, 0xbd, 0xd3, 0x20, 0xf4, 0x53, 0x1e, 0xcd, 0xbd, - 0xba, 0x9c, 0xd1, 0x14, 0x74, 0xb2, 0x06, 0x75, 0x1c, 0x24, 0xd5, 0xca, 0xb8, 0x59, 0x4c, 0x91, - 0x90, 0xe2, 0x1e, 0x43, 0x47, 0xa5, 0x50, 0xca, 0x7f, 0x32, 0xe5, 0xe2, 0x95, 0x85, 0xd9, 0x2a, - 0x40, 0x11, 0xe5, 0xf3, 0x91, 0x58, 0x05, 0x23, 0x4d, 0xf9, 0x24, 0xe0, 0xa1, 0x9f, 0xbf, 0x46, - 0x43, 0xee, 0x67, 0xd0, 0xce, 0xcf, 0xd0, 0x8d, 0x79, 0x9e, 0xc8, 0x95, 0x34, 0x55, 0x3f, 0xa2, - 0x58, 0xf6, 0x63, 0xbf, 0xc8, 0xe3, 0xee, 0x1f, 0xcd, 0x7c, 0xa5, 0xee, 0x51, 0xe7, 0x4a, 0x43, - 0x63, 0xb1, 0x34, 0x9c, 0x2f, 0xb3, 0xcc, 0xef, 0x55, 0x66, 0x7d, 0x0e, 0x8e, 0x8f, 0xb5, 0x46, - 0x70, 0x9e, 0xc7, 0xd5, 0x95, 0xc5, 0xba, 0x42, 0x57, 0x23, 0xc1, 0x39, 0xa7, 0x25, 0xb3, 0xbc, - 0x4b, 0x16, 0x9f, 0xf1, 0x28, 0x78, 0x8e, 0x4d, 0xb8, 0x7c, 0x70, 0x89, 0x28, 0x27, 0x1a, 0xaa, - 0xfe, 0xd0, 0x13, 0x8d, 0x7c, 0x38, 0x63, 0x95, 0xc3, 0x19, 0x29, 0xb5, 0x69, 0x22, 0x78, 0x9a, - 0xe5, 0x75, 0xa8, 0x82, 0x8a, 0x7a, 0xce, 0xd1, 0xbc, 0x2c, 0x1a, 0xbb, 0x5f, 0x80, 0x53, 0xdc, - 0x45, 0x06, 0xb4, 0xfd, 0x83, 0xfd, 0xbe, 0x0a, 0x3f, 0x3b, 0xfb, 0xdb, 0xfd, 0xff, 0xeb, 0x1a, - 0x32, 0x24, 0xd2, 0xfe, 0xb3, 0x3e, 0x1d, 0xf4, 0xbb, 0xa6, 0x0c, 0x5d, 0xdb, 0xfd, 0xdd, 0xfe, - 0xb0, 0xdf, 0xad, 0x7d, 0x55, 0xb7, 0x9b, 0x5d, 0x9b, 0xda, 0x7c, 0x96, 0x84, 0x81, 0x17, 0x64, - 0xee, 0x11, 0xd8, 0x7b, 0x2c, 0x79, 0xa9, 0xa7, 0x28, 0x33, 0xdd, 0x54, 0x0f, 0x22, 0x74, 0x56, - 0xba, 0x0d, 0x4d, 0xed, 0xf2, 0xda, 0x9a, 0xe6, 0xc2, 0x41, 0x4e, 0x73, 0x7f, 0x65, 0xc0, 0xf5, - 0xbd, 0xf8, 0x9c, 0x17, 0x89, 0xff, 0x90, 0x5d, 0x84, 0x31, 0xf3, 0x5f, 0xa3, 0xba, 0x3b, 0x70, - 0x4d, 0xc4, 0xd3, 0xd4, 0xe3, 0xa3, 0x85, 0x21, 0x48, 0x47, 0xa1, 0x9f, 0x68, 0x13, 0x74, 0xa1, - 0xe3, 0x73, 0x91, 0x95, 0x5c, 0x35, 0xe4, 0x6a, 0x49, 0x64, 0xce, 0x53, 0x54, 0x2f, 0xf5, 0xd7, - 0x55, 0x2f, 0xee, 0x23, 0x70, 0x86, 0x33, 0x6c, 0x86, 0xa6, 0x62, 0x2e, 0x21, 0x19, 0xaf, 0x48, - 0x48, 0xe6, 0x42, 0x8c, 0x1b, 0x40, 0xab, 0x52, 0xb6, 0x90, 0xf7, 0xa0, 0x9e, 0xcd, 0xa2, 0xf9, - 0x61, 0x66, 0x7e, 0x06, 0x45, 0x12, 0x79, 0x0f, 0xda, 0xb2, 0x51, 0x62, 0x42, 0x04, 0xe3, 0x88, - 0xfb, 0x7a, 0x47, 0xd9, 0x3c, 0x6d, 0x69, 0x94, 0x7b, 0x13, 0x3a, 0xb2, 0x33, 0x0d, 0x26, 0x5c, - 0x64, 0x6c, 0x92, 0x60, 0xfa, 0xd4, 0x51, 0xab, 0x4e, 0xcd, 0x4c, 0xb8, 0x77, 0xa0, 0x7d, 0xc8, - 0x79, 0x4a, 0xb9, 0x48, 0xe2, 0x48, 0xe5, 0x11, 0x81, 0x67, 0xe8, 0x10, 0xa9, 0x21, 0xf7, 0x5b, - 0x70, 0x64, 0xe1, 0xf9, 0x90, 0x65, 0xde, 0xe9, 0x0f, 0x29, 0x4c, 0xef, 0x40, 0x33, 0x51, 0xaa, - 0xd3, 0x65, 0x64, 0x1b, 0xbd, 0x54, 0xab, 0x93, 0xe6, 0x44, 0xf7, 0x53, 0xa8, 0xed, 0x4f, 0x27, - 0xd5, 0xd1, 0x7e, 0x5d, 0x95, 0x46, 0x73, 0x2d, 0x99, 0x39, 0xdf, 0x92, 0xb9, 0xdf, 0x40, 0x2b, - 0x7f, 0xea, 0x8e, 0x8f, 0xf3, 0x79, 0x14, 0xf5, 0x8e, 0x3f, 0x27, 0x79, 0xd5, 0xeb, 0xf0, 0xc8, - 0xdf, 0xc9, 0x65, 0xa4, 0x80, 0xf9, 0xbd, 0x75, 0x2f, 0x5f, 0xec, 0xfd, 0x18, 0xda, 0x79, 0x71, - 0x88, 0x75, 0x98, 0x54, 0x5e, 0x18, 0xf0, 0xa8, 0xa2, 0x58, 0x5b, 0x21, 0x86, 0xe2, 0x15, 0x63, - 0x37, 0x77, 0x03, 0x2c, 0x6d, 0x19, 0x04, 0xea, 0x5e, 0xec, 0x2b, 0xb3, 0x6d, 0x50, 0xfc, 0x96, - 0x0f, 0x9e, 0x88, 0x71, 0x1e, 0xca, 0x27, 0x62, 0xec, 0x66, 0xd0, 0x79, 0xc8, 0xbc, 0xb3, 0x69, - 0x92, 0x87, 0xd2, 0x4a, 0x15, 0x6f, 0xcc, 0x55, 0xf1, 0xaf, 0x98, 0xf5, 0xbd, 0x05, 0xcd, 0x69, - 0x14, 0xcc, 0xf2, 0x5c, 0xea, 0x50, 0x4b, 0x82, 0x43, 0x0c, 0xae, 0x19, 0x4b, 0xc7, 0x7a, 0x18, - 0xea, 0x50, 0x0d, 0xb9, 0x3f, 0x82, 0x4e, 0x7f, 0x96, 0xe0, 0xd4, 0xf3, 0xb5, 0x01, 0xbc, 0x72, - 0x21, 0x73, 0xee, 0x42, 0x0b, 0xa7, 0xd6, 0xf2, 0x53, 0x37, 0x7f, 0x63, 0x40, 0x5d, 0x9a, 0x07, - 0xb9, 0x05, 0xf5, 0xbe, 0x77, 0x1a, 0x93, 0x39, 0x2b, 0x58, 0x99, 0x83, 0xdc, 0x25, 0xf2, 0xa1, - 0x9a, 0xa4, 0xe6, 0x03, 0xe2, 0x4e, 0x6e, 0x5d, 0x68, 0x7d, 0x2f, 0x71, 0x6f, 0x40, 0xeb, 0xab, - 0x38, 0x88, 0x1e, 0xa9, 0xe1, 0x22, 0x59, 0xb4, 0xc5, 0x97, 0xf8, 0x3f, 0x02, 0x6b, 0x47, 0x48, - 0xa3, 0x7f, 0x99, 0x15, 0x7b, 0xc1, 0xaa, 0x3f, 0xb8, 0x4b, 0x9b, 0x7f, 0x33, 0xa1, 0xfe, 0x0d, - 0x4f, 0x63, 0xf2, 0x21, 0x34, 0xf5, 0xe4, 0x83, 0x54, 0x26, 0x1c, 0x2b, 0x18, 0x18, 0x16, 0x46, - 0x22, 0x78, 0x2b, 0x4b, 0x67, 0x9b, 0x72, 0x1c, 0xb3, 0x72, 0x59, 0x10, 0x71, 0x97, 0xd6, 0x8d, - 0x07, 0x06, 0xf9, 0x00, 0x2c, 0xe5, 0xfe, 0x0b, 0xb2, 0x59, 0xec, 0x67, 0xdc, 0xa5, 0x07, 0x06, - 0xb9, 0x0b, 0xad, 0xc1, 0x69, 0x3c, 0x0d, 0xfd, 0x01, 0x4f, 0xcf, 0x39, 0xa9, 0x0c, 0x01, 0x57, - 0x2a, 0xdf, 0xee, 0x12, 0x59, 0x07, 0x50, 0x0e, 0x72, 0x14, 0xf8, 0x82, 0x34, 0x25, 0x6d, 0x7f, - 0x3a, 0x51, 0x9b, 0x56, 0x3c, 0x47, 0x71, 0x56, 0xc2, 0xc4, 0xab, 0x38, 0x3f, 0x81, 0xce, 0x23, - 0x0c, 0x5a, 0x07, 0xe9, 0xd6, 0x71, 0x9c, 0x66, 0x64, 0x71, 0x10, 0xb8, 0xb2, 0x88, 0x70, 0x97, - 0xc8, 0x03, 0xb0, 0x87, 0xe9, 0x85, 0xe2, 0x7f, 0x43, 0x07, 0xb3, 0xf2, 0xbc, 0x4b, 0x5e, 0xb9, - 0xf9, 0xfb, 0x1a, 0x58, 0x5f, 0xc7, 0xe9, 0x19, 0x4f, 0xc9, 0x7d, 0xb0, 0xb0, 0xf1, 0xd4, 0xa6, - 0x50, 0x34, 0xa1, 0x97, 0x1d, 0x74, 0x0b, 0x1c, 0x14, 0xca, 0x90, 0x89, 0x33, 0x25, 0x7a, 0xfc, - 0x5f, 0x4e, 0xc9, 0x45, 0x55, 0x0e, 0x68, 0x03, 0xcb, 0x83, 0x2c, 0xe5, 0x6c, 0x52, 0x34, 0xdb, - 0x73, 0xdd, 0xe0, 0x4a, 0x53, 0xb5, 0x76, 0x03, 0xad, 0x9c, 0x7b, 0x50, 0x1f, 0xa8, 0x97, 0x4a, - 0xa6, 0xf2, 0xbf, 0x8b, 0x95, 0xe5, 0x1c, 0x51, 0xec, 0xfc, 0x5f, 0x60, 0xa9, 0xa4, 0xaf, 0x9e, - 0x39, 0x57, 0x15, 0xad, 0x74, 0xab, 0x28, 0xbd, 0xc0, 0x85, 0xe6, 0xe1, 0x34, 0x1d, 0xf3, 0xa1, - 0x58, 0xd0, 0x7c, 0xae, 0x03, 0x77, 0x89, 0xdc, 0x03, 0x4b, 0xc5, 0x04, 0xb5, 0xe9, 0x5c, 0x7c, - 0x50, 0x2f, 0x53, 0x21, 0x46, 0xb1, 0x2a, 0x47, 0x56, 0xac, 0x73, 0x4e, 0xbd, 0xc0, 0xfa, 0x11, - 0x74, 0x29, 0xf7, 0x78, 0x50, 0x49, 0xb3, 0x24, 0x7f, 0xf8, 0xa2, 0xd7, 0xac, 0x1b, 0xe4, 0x0b, - 0xe8, 0xcc, 0xa5, 0x64, 0xd2, 0x43, 0x65, 0x5c, 0x92, 0xa5, 0x17, 0x17, 0x3f, 0xec, 0xfe, 0xee, - 0xc5, 0xaa, 0xf1, 0x87, 0x17, 0xab, 0xc6, 0x9f, 0x5e, 0xac, 0x1a, 0xdf, 0xfd, 0x79, 0x75, 0xe9, - 0xd8, 0xc2, 0xff, 0x7c, 0x3f, 0xf9, 0x67, 0x00, 0x00, 0x00, 0xff, 0xff, 0x76, 0x8a, 0x0d, 0x0b, - 0x0e, 0x1e, 0x00, 0x00, +func init() { proto.RegisterFile("pb.proto", fileDescriptor_pb_a9faf9bf79258c83) } + +var fileDescriptor_pb_a9faf9bf79258c83 = []byte{ + // 3183 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0xcd, 0x73, 0x1b, 0xc7, + 0x95, 0xe7, 0x0c, 0x80, 0xc1, 0xcc, 0x03, 0x40, 0xc1, 0x6d, 0xad, 0x0c, 0xd3, 0x36, 0x45, 0x8f, + 0xf5, 0x41, 0xc9, 0x36, 0x57, 0xa6, 0xbd, 0x6b, 0xcb, 0x55, 0x7b, 0xa0, 0x44, 0x48, 0x45, 0x8b, + 0x5f, 0xdb, 0x00, 0xe5, 0x5d, 0xd7, 0x96, 0x51, 0xcd, 0x99, 0x26, 0x38, 0xcb, 0xc1, 0xcc, 0xec, + 0xf4, 0x80, 0x05, 0xea, 0xb6, 0xff, 0x85, 0x0f, 0x5b, 0x7b, 0x48, 0x55, 0x2e, 0xc9, 0x21, 0xd7, + 0xe4, 0x0f, 0x48, 0x55, 0x8e, 0xb9, 0xe6, 0x96, 0x52, 0x4e, 0x39, 0xe7, 0x94, 0x5b, 0xaa, 0x5f, + 0xf7, 0x7c, 0x00, 0x22, 0x25, 0x3b, 0x55, 0x39, 0x61, 0x5e, 0xbf, 0xd7, 0x5f, 0xaf, 0xdf, 0xfb, + 0xbd, 0x0f, 0x80, 0x9d, 0x1c, 0x6f, 0x24, 0x69, 0x9c, 0xc5, 0xc4, 0x4c, 0x8e, 0x57, 0x1c, 0x96, + 0x04, 0x8a, 0x74, 0x57, 0xa0, 0xbe, 0x1b, 0x88, 0x8c, 0x10, 0xa8, 0x4f, 0x03, 0x5f, 0xf4, 0x8c, + 0xb5, 0xda, 0xba, 0x45, 0xf1, 0xdb, 0xdd, 0x03, 0x67, 0xc8, 0xc4, 0xd9, 0x73, 0x16, 0x4e, 0x39, + 0xe9, 0x42, 0xed, 0x9c, 0x85, 0x3d, 0x63, 0xcd, 0x58, 0x6f, 0x53, 0xf9, 0x49, 0x36, 0xc0, 0x3e, + 0x67, 0xe1, 0x28, 0xbb, 0x48, 0x78, 0xcf, 0x5c, 0x33, 0xd6, 0x97, 0x37, 0xdf, 0xde, 0x48, 0x8e, + 0x37, 0x0e, 0x63, 0x91, 0x05, 0xd1, 0x78, 0xe3, 0x39, 0x0b, 0x87, 0x17, 0x09, 0xa7, 0xcd, 0x73, + 0xf5, 0xe1, 0x1e, 0x40, 0x6b, 0x90, 0x7a, 0x4f, 0xa6, 0x91, 0x97, 0x05, 0x71, 0x24, 0x77, 0x8c, + 0xd8, 0x84, 0xe3, 0x8a, 0x0e, 0xc5, 0x6f, 0x39, 0xc6, 0xd2, 0xb1, 0xe8, 0xd5, 0xd6, 0x6a, 0x72, + 0x4c, 0x7e, 0x93, 0x1e, 0x34, 0x03, 0xf1, 0x38, 0x9e, 0x46, 0x59, 0xaf, 0xbe, 0x66, 0xac, 0xdb, + 0x34, 0x27, 0xdd, 0xbf, 0x98, 0xd0, 0xf8, 0xf7, 0x29, 0x4f, 0x2f, 0x70, 0x5e, 0x96, 0xa5, 0xf9, + 0x5a, 0xf2, 0x9b, 0x5c, 0x87, 0x46, 0xc8, 0xa2, 0xb1, 0xe8, 0x99, 0xb8, 0x98, 0x22, 0xc8, 0x7b, + 0xe0, 0xb0, 0x93, 0x8c, 0xa7, 0xa3, 0x69, 0xe0, 0xf7, 0x6a, 0x6b, 0xc6, 0xba, 0x45, 0x6d, 0x1c, + 0x38, 0x0a, 0x7c, 0xf2, 0x2e, 0xd8, 0x7e, 0x3c, 0xf2, 0xaa, 0x7b, 0xf9, 0x31, 0xee, 0x45, 0x3e, + 0x02, 0x7b, 0x1a, 0xf8, 0xa3, 0x30, 0x10, 0x59, 0xaf, 0xb1, 0x66, 0xac, 0xb7, 0x36, 0x6d, 0x79, + 0x59, 0xa9, 0x3b, 0xda, 0x9c, 0x06, 0x3e, 0x2a, 0xf1, 0x3e, 0xd8, 0x22, 0xf5, 0x46, 0x27, 0xd3, + 0xc8, 0xeb, 0x59, 0x28, 0x74, 0x4d, 0x0a, 0x55, 0x6e, 0x4d, 0x9b, 0x42, 0x11, 0xf2, 0x5a, 0x29, + 0x3f, 0xe7, 0xa9, 0xe0, 0xbd, 0xa6, 0xda, 0x4a, 0x93, 0xe4, 0x01, 0xb4, 0x4e, 0x98, 0xc7, 0xb3, + 0x51, 0xc2, 0x52, 0x36, 0xe9, 0xd9, 0xe5, 0x42, 0x4f, 0xe4, 0xf0, 0xa1, 0x1c, 0x15, 0x14, 0x4e, + 0x0a, 0x82, 0x7c, 0x0e, 0x1d, 0xa4, 0xc4, 0xe8, 0x24, 0x08, 0x33, 0x9e, 0xf6, 0x1c, 0x9c, 0xb3, + 0x8c, 0x73, 0x70, 0x64, 0x98, 0x72, 0x4e, 0xdb, 0x4a, 0x48, 0x8d, 0x90, 0x0f, 0x00, 0xf8, 0x2c, + 0x61, 0x91, 0x3f, 0x62, 0x61, 0xd8, 0x03, 0x3c, 0x83, 0xa3, 0x46, 0xb6, 0xc2, 0x90, 0xbc, 0x23, + 0xcf, 0xc7, 0xfc, 0x51, 0x26, 0x7a, 0x9d, 0x35, 0x63, 0xbd, 0x4e, 0x2d, 0x49, 0x0e, 0x85, 0xbb, + 0x09, 0x0e, 0x5a, 0x04, 0xde, 0xf8, 0x36, 0x58, 0xe7, 0x92, 0x50, 0x86, 0xd3, 0xda, 0xec, 0xc8, + 0x2d, 0x0b, 0xa3, 0xa1, 0x9a, 0xe9, 0xae, 0x82, 0xbd, 0xcb, 0xa2, 0x71, 0x6e, 0x69, 0xf2, 0x29, + 0x70, 0x82, 0x43, 0xf1, 0xdb, 0xfd, 0xc1, 0x04, 0x8b, 0x72, 0x31, 0x0d, 0x33, 0x72, 0x17, 0x40, + 0x2a, 0x7a, 0xc2, 0xb2, 0x34, 0x98, 0xe9, 0x55, 0x4b, 0x55, 0x3b, 0xd3, 0xc0, 0xdf, 0x43, 0x16, + 0x79, 0x00, 0x6d, 0x5c, 0x3d, 0x17, 0x35, 0xcb, 0x03, 0x14, 0xe7, 0xa3, 0x2d, 0x14, 0xd1, 0x33, + 0x6e, 0x80, 0x85, 0x6f, 0xab, 0xec, 0xab, 0x43, 0x35, 0x45, 0x6e, 0xc3, 0x72, 0x10, 0x65, 0x52, + 0xf7, 0x5e, 0x36, 0xf2, 0xb9, 0xc8, 0x1f, 0xbf, 0x53, 0x8c, 0x6e, 0x73, 0x91, 0x91, 0xcf, 0x40, + 0x29, 0x30, 0xdf, 0xb0, 0x81, 0x1b, 0x2e, 0x17, 0x0f, 0x23, 0xd4, 0x8e, 0x28, 0xa3, 0x77, 0xfc, + 0x14, 0x5a, 0xf2, 0x7e, 0xf9, 0x0c, 0x0b, 0x67, 0xb4, 0xf1, 0x36, 0x5a, 0x1d, 0x14, 0xa4, 0x80, + 0x16, 0x97, 0xaa, 0x91, 0x06, 0xa6, 0x0c, 0x02, 0xbf, 0xdd, 0x3e, 0x34, 0x0e, 0x52, 0x9f, 0xa7, + 0x97, 0xda, 0x38, 0x81, 0xba, 0xcf, 0x85, 0x87, 0xee, 0x67, 0x53, 0xfc, 0x2e, 0xed, 0xbe, 0x56, + 0xb1, 0x7b, 0xf7, 0xff, 0x0d, 0x68, 0x0d, 0xe2, 0x34, 0xdb, 0xe3, 0x42, 0xb0, 0x31, 0x27, 0x37, + 0xa1, 0x11, 0xcb, 0x65, 0xb5, 0x86, 0x1d, 0x79, 0x26, 0xdc, 0x87, 0xaa, 0xf1, 0x85, 0x77, 0x30, + 0xaf, 0x7e, 0x87, 0xeb, 0xd0, 0x50, 0x1e, 0x23, 0xbd, 0xa9, 0x41, 0x15, 0x21, 0x75, 0x1d, 0x9f, + 0x9c, 0x08, 0xae, 0x74, 0xd9, 0xa0, 0x9a, 0xba, 0xda, 0xac, 0xfe, 0x05, 0x40, 0x9e, 0xef, 0x27, + 0x5a, 0x81, 0x7b, 0x0a, 0x2d, 0xca, 0x4e, 0xb2, 0xc7, 0x71, 0x94, 0xf1, 0x59, 0x46, 0x96, 0xc1, + 0x0c, 0x7c, 0x54, 0x91, 0x45, 0xcd, 0xc0, 0x97, 0x87, 0x1b, 0xa7, 0xf1, 0x34, 0x41, 0x0d, 0x75, + 0xa8, 0x22, 0x50, 0x95, 0xbe, 0x9f, 0xe2, 0x89, 0xa5, 0x2a, 0x7d, 0x3f, 0x25, 0x37, 0xa1, 0x25, + 0x22, 0x96, 0x88, 0xd3, 0x38, 0x93, 0x87, 0xab, 0xe3, 0xe1, 0x20, 0x1f, 0x1a, 0x0a, 0xf7, 0xb7, + 0x06, 0x58, 0x7b, 0x7c, 0x72, 0xcc, 0xd3, 0x57, 0x76, 0x79, 0x17, 0x6c, 0x5c, 0x78, 0x14, 0xf8, + 0x7a, 0xa3, 0x26, 0xd2, 0x3b, 0xfe, 0xa5, 0x5b, 0xdd, 0x00, 0x2b, 0xe4, 0x4c, 0x2a, 0x5f, 0xd9, + 0x99, 0xa6, 0xa4, 0x6e, 0xd8, 0x64, 0xe4, 0x73, 0xe6, 0x23, 0xc4, 0xd8, 0xd4, 0x62, 0x93, 0x6d, + 0xce, 0x7c, 0x79, 0xb6, 0x90, 0x89, 0x6c, 0x34, 0x4d, 0x7c, 0x96, 0x71, 0x84, 0x96, 0xba, 0x34, + 0x1c, 0x91, 0x1d, 0xe1, 0x08, 0xb9, 0x0f, 0x6f, 0x79, 0xe1, 0x54, 0x48, 0x5c, 0x0b, 0xa2, 0x93, + 0x78, 0x14, 0x47, 0xe1, 0x05, 0xea, 0xd7, 0xa6, 0xd7, 0x34, 0x63, 0x27, 0x3a, 0x89, 0x0f, 0xa2, + 0xf0, 0xc2, 0xfd, 0x3f, 0x13, 0x1a, 0x4f, 0x51, 0x0d, 0x0f, 0xa0, 0x39, 0xc1, 0x0b, 0xe5, 0xde, + 0x7b, 0x43, 0x6a, 0x18, 0x79, 0x1b, 0xea, 0xa6, 0xa2, 0x1f, 0x65, 0xe9, 0x05, 0xcd, 0xc5, 0xe4, + 0x8c, 0x8c, 0x1d, 0x87, 0x3c, 0x13, 0xda, 0x22, 0x2a, 0x33, 0x86, 0x8a, 0xa1, 0x67, 0x68, 0xb1, + 0x45, 0xb5, 0xd6, 0x16, 0xd5, 0xba, 0xf2, 0x04, 0xda, 0xd5, 0xbd, 0x64, 0x9c, 0x39, 0xe3, 0x17, + 0xa8, 0xdc, 0x3a, 0x95, 0x9f, 0x64, 0x0d, 0x1a, 0xe8, 0xc5, 0xa8, 0xda, 0xd6, 0x26, 0xc8, 0x2d, + 0xd5, 0x14, 0xaa, 0x18, 0x5f, 0x9b, 0x5f, 0x19, 0x72, 0x9d, 0xea, 0x09, 0xaa, 0xeb, 0x38, 0x57, + 0xaf, 0xa3, 0xa6, 0x54, 0xd6, 0x71, 0xff, 0x6a, 0x42, 0xfb, 0x3b, 0x9e, 0xc6, 0x87, 0x69, 0x9c, + 0xc4, 0x82, 0x85, 0x64, 0x6b, 0xfe, 0x06, 0x4a, 0x53, 0x6b, 0x72, 0x72, 0x55, 0x6c, 0x63, 0x50, + 0x5c, 0x49, 0x69, 0xa0, 0x72, 0x47, 0xe2, 0x82, 0xa5, 0x34, 0x78, 0xc9, 0x15, 0x34, 0x47, 0xca, + 0x28, 0x9d, 0xa1, 0x8e, 0xe6, 0x8f, 0xa7, 0x39, 0x64, 0x15, 0x60, 0xc2, 0x66, 0xbb, 0x9c, 0x09, + 0xbe, 0xe3, 0xe7, 0x26, 0x5a, 0x8e, 0x90, 0x15, 0xb0, 0x27, 0x6c, 0x36, 0x9c, 0x45, 0x43, 0x81, + 0x16, 0x54, 0xa7, 0x05, 0x4d, 0xde, 0x07, 0x67, 0xc2, 0x66, 0xd2, 0x57, 0x76, 0x7c, 0x6d, 0x41, + 0xe5, 0x00, 0xf9, 0x10, 0x6a, 0xd9, 0x2c, 0x42, 0xe0, 0x91, 0xb1, 0x46, 0xe6, 0x07, 0xc3, 0x59, + 0xa4, 0xbd, 0x8a, 0x4a, 0x5e, 0xae, 0x50, 0xbb, 0x54, 0x68, 0x17, 0x6a, 0x5e, 0xe0, 0x63, 0xb0, + 0x71, 0xa8, 0xfc, 0x5c, 0xf9, 0x37, 0xb8, 0xb6, 0xa0, 0x87, 0xea, 0x3b, 0x74, 0xd4, 0xb4, 0xeb, + 0xd5, 0x77, 0xa8, 0x57, 0x75, 0xff, 0xeb, 0x1a, 0x5c, 0xd3, 0xc6, 0x70, 0x1a, 0x24, 0x83, 0x4c, + 0x9a, 0x76, 0x0f, 0x9a, 0x88, 0x28, 0x3c, 0xd5, 0x36, 0x91, 0x93, 0xe4, 0x4b, 0xb0, 0xd0, 0xcb, + 0x72, 0x5b, 0xbc, 0x59, 0x6a, 0xb5, 0x98, 0xae, 0x6c, 0x53, 0x3f, 0x89, 0x16, 0x27, 0x5f, 0x40, + 0xe3, 0x05, 0x4f, 0x63, 0x85, 0x90, 0xad, 0xcd, 0xd5, 0xcb, 0xe6, 0xc9, 0xb7, 0xd5, 0xd3, 0x94, + 0xf0, 0x3f, 0x50, 0xf9, 0xb7, 0x24, 0x26, 0x4e, 0xe2, 0x73, 0xee, 0xf7, 0x9a, 0x78, 0xa2, 0xaa, + 0x7d, 0xe4, 0xac, 0x5c, 0xdb, 0x76, 0xa9, 0xed, 0x6d, 0x68, 0x55, 0xae, 0x77, 0x89, 0xa6, 0x6f, + 0xce, 0x5b, 0xbc, 0x53, 0x38, 0x6b, 0xd5, 0x71, 0xb6, 0x01, 0xca, 0xcb, 0xfe, 0xbd, 0xee, 0xe7, + 0xfe, 0xaf, 0x01, 0xd7, 0x1e, 0xc7, 0x51, 0xc4, 0x31, 0xcd, 0x51, 0x4f, 0x57, 0x9a, 0xbd, 0x71, + 0xa5, 0xd9, 0xdf, 0x83, 0x86, 0x90, 0xc2, 0x7a, 0xf5, 0xb7, 0x2f, 0x79, 0x0b, 0xaa, 0x24, 0x24, + 0x94, 0x4c, 0xd8, 0x6c, 0x94, 0xf0, 0xc8, 0x0f, 0xa2, 0x71, 0x0e, 0x25, 0x13, 0x36, 0x3b, 0x54, + 0x23, 0xee, 0xcf, 0x0c, 0xb0, 0x94, 0xc7, 0xcc, 0x21, 0xb2, 0x31, 0x8f, 0xc8, 0xef, 0x83, 0x93, + 0xa4, 0xdc, 0x0f, 0xbc, 0x7c, 0x57, 0x87, 0x96, 0x03, 0xd2, 0x38, 0x4f, 0xe2, 0xd4, 0xe3, 0xb8, + 0xbc, 0x4d, 0x15, 0x21, 0xb3, 0x46, 0x8c, 0x5a, 0x88, 0xab, 0x0a, 0xb4, 0x6d, 0x39, 0x20, 0x01, + 0x55, 0x4e, 0x11, 0x09, 0xf3, 0x54, 0x1e, 0x57, 0xa3, 0x8a, 0x90, 0x20, 0xaf, 0x5e, 0x0e, 0x5f, + 0xcc, 0xa6, 0x9a, 0x72, 0x7f, 0x61, 0x42, 0x7b, 0x3b, 0x48, 0xb9, 0x97, 0x71, 0xbf, 0xef, 0x8f, + 0x51, 0x90, 0x47, 0x59, 0x90, 0x5d, 0xe8, 0x80, 0xa2, 0xa9, 0x22, 0xde, 0x9b, 0xf3, 0x39, 0xad, + 0x7a, 0x8b, 0x1a, 0xa6, 0xe1, 0x8a, 0x20, 0x9b, 0x00, 0x2a, 0x13, 0xc2, 0x54, 0xbc, 0x7e, 0x75, + 0x2a, 0xee, 0xa0, 0x98, 0xfc, 0x94, 0x0a, 0x52, 0x73, 0x02, 0x15, 0x6c, 0x2c, 0xcc, 0xd3, 0xa7, + 0xd2, 0x90, 0x31, 0x81, 0x38, 0xe6, 0x21, 0x1a, 0x2a, 0x26, 0x10, 0xc7, 0x3c, 0x2c, 0xd2, 0xb6, + 0xa6, 0x3a, 0x8e, 0xfc, 0x26, 0x1f, 0x81, 0x19, 0x27, 0x78, 0x3f, 0xbd, 0x61, 0xf5, 0x62, 0x1b, + 0x07, 0x09, 0x35, 0xe3, 0x44, 0x5a, 0x81, 0xca, 0x3b, 0x7b, 0x8e, 0x36, 0x6e, 0x89, 0x2e, 0x98, + 0x31, 0x51, 0xcd, 0x71, 0x6f, 0x80, 0x79, 0x90, 0x90, 0x26, 0xd4, 0x06, 0xfd, 0x61, 0x77, 0x49, + 0x7e, 0x6c, 0xf7, 0x77, 0xbb, 0x86, 0xfb, 0xd2, 0x00, 0x67, 0x6f, 0x9a, 0x31, 0x69, 0x53, 0xe2, + 0x75, 0x8f, 0xfa, 0x2e, 0xd8, 0x22, 0x63, 0x29, 0x22, 0xb4, 0x82, 0x95, 0x26, 0xd2, 0x43, 0x41, + 0xee, 0x40, 0x83, 0xfb, 0x63, 0x9e, 0x7b, 0x7b, 0x77, 0xf1, 0x9c, 0x54, 0xb1, 0xc9, 0x3a, 0x58, + 0xc2, 0x3b, 0xe5, 0x13, 0xd6, 0xab, 0x97, 0x82, 0x03, 0x1c, 0x51, 0x51, 0x96, 0x6a, 0x3e, 0x96, + 0x09, 0x69, 0x9c, 0x60, 0xde, 0xdc, 0xd0, 0x65, 0x42, 0x1a, 0x27, 0x32, 0x6b, 0xde, 0x84, 0x7f, + 0x0a, 0xc6, 0x51, 0x9c, 0xf2, 0x51, 0x10, 0xf9, 0x7c, 0x36, 0xf2, 0xe2, 0xe8, 0x24, 0x0c, 0xbc, + 0x0c, 0x75, 0x69, 0xd3, 0xb7, 0x15, 0x73, 0x47, 0xf2, 0x1e, 0x6b, 0x96, 0xfb, 0x11, 0x38, 0xcf, + 0xf8, 0x05, 0xe6, 0xac, 0x82, 0xdc, 0x00, 0xf3, 0xec, 0x5c, 0x07, 0x19, 0x4b, 0x9e, 0xe0, 0xd9, + 0x73, 0x6a, 0x9e, 0x9d, 0xbb, 0x33, 0xb0, 0x73, 0x64, 0x25, 0xf7, 0x24, 0x24, 0x22, 0x32, 0x6b, + 0xc7, 0xc2, 0xe2, 0xa0, 0x92, 0x06, 0xd1, 0x9c, 0x2f, 0xdf, 0x12, 0x0f, 0x92, 0x63, 0x2d, 0x12, + 0xd5, 0x24, 0xac, 0x56, 0x4d, 0xc2, 0x30, 0x9f, 0x8c, 0x23, 0xae, 0x4d, 0x1c, 0xbf, 0x65, 0xbe, + 0x60, 0x17, 0xc1, 0xf0, 0x63, 0x70, 0x26, 0xf9, 0x7b, 0x68, 0x97, 0xc5, 0x8c, 0xbb, 0x78, 0x24, + 0x5a, 0xf2, 0xf5, 0x5d, 0xea, 0x8b, 0x77, 0x29, 0x7d, 0xbe, 0xf1, 0x46, 0x9f, 0xbf, 0x0b, 0xd7, + 0xbc, 0x90, 0xb3, 0x68, 0x54, 0xba, 0xac, 0xb2, 0xca, 0x65, 0x1c, 0x3e, 0x2c, 0xfc, 0x56, 0xe3, + 0x56, 0xb3, 0x8c, 0x4e, 0xb7, 0xa1, 0xe1, 0xf3, 0x30, 0x63, 0xd5, 0x02, 0xea, 0x20, 0x65, 0x5e, + 0xc8, 0xb7, 0xe5, 0x30, 0x55, 0x5c, 0xb2, 0x0e, 0x76, 0x1e, 0xa9, 0x75, 0xd9, 0x84, 0xf9, 0x79, + 0xae, 0x6c, 0x5a, 0x70, 0x4b, 0x5d, 0x42, 0x45, 0x97, 0xee, 0x67, 0x50, 0x7b, 0xf6, 0x7c, 0x70, + 0xd5, 0xbb, 0x15, 0x1a, 0x35, 0x2b, 0x1a, 0xfd, 0x1e, 0xcc, 0x67, 0xcf, 0xab, 0x48, 0xdb, 0x2e, + 0xe2, 0xa9, 0x2c, 0xb1, 0xcd, 0xb2, 0xc4, 0x5e, 0x01, 0x7b, 0x2a, 0x78, 0xba, 0xc7, 0x33, 0xa6, + 0x5d, 0xbe, 0xa0, 0x65, 0x60, 0x94, 0xf5, 0x62, 0x10, 0x47, 0x3a, 0x18, 0xe5, 0xa4, 0xfb, 0xe7, + 0x1a, 0x34, 0xb5, 0xeb, 0xcb, 0x35, 0xa7, 0x45, 0xae, 0x2a, 0x3f, 0xe7, 0xc3, 0x6f, 0x81, 0x21, + 0xd5, 0x62, 0xbe, 0xf6, 0xe6, 0x62, 0x9e, 0x7c, 0x0d, 0xed, 0x44, 0xf1, 0xaa, 0xa8, 0xf3, 0x4e, + 0x75, 0x8e, 0xfe, 0xc5, 0x79, 0xad, 0xa4, 0x24, 0xa4, 0xff, 0x60, 0x55, 0x94, 0xb1, 0x31, 0x9a, + 0x40, 0x9b, 0x36, 0x25, 0x3d, 0x64, 0xe3, 0x2b, 0xb0, 0xe7, 0x47, 0x40, 0x88, 0xcc, 0xc9, 0xe3, + 0xa4, 0xd7, 0x46, 0x58, 0x90, 0xb0, 0x53, 0x45, 0x84, 0xce, 0x3c, 0x22, 0xbc, 0x07, 0x8e, 0x17, + 0x4f, 0x26, 0x01, 0xf2, 0x96, 0x55, 0xa8, 0x56, 0x03, 0x43, 0xe1, 0xbe, 0x80, 0xa6, 0xbe, 0x2c, + 0x69, 0x41, 0x73, 0xbb, 0xff, 0x64, 0xeb, 0x68, 0x57, 0x62, 0x12, 0x80, 0xf5, 0x68, 0x67, 0x7f, + 0x8b, 0xfe, 0x67, 0xd7, 0x90, 0xf8, 0xb4, 0xb3, 0x3f, 0xec, 0x9a, 0xc4, 0x81, 0xc6, 0x93, 0xdd, + 0x83, 0xad, 0x61, 0xb7, 0x46, 0x6c, 0xa8, 0x3f, 0x3a, 0x38, 0xd8, 0xed, 0xd6, 0x49, 0x1b, 0xec, + 0xed, 0xad, 0x61, 0x7f, 0xb8, 0xb3, 0xd7, 0xef, 0x36, 0xa4, 0xec, 0xd3, 0xfe, 0x41, 0xd7, 0x92, + 0x1f, 0x47, 0x3b, 0xdb, 0xdd, 0xa6, 0xe4, 0x1f, 0x6e, 0x0d, 0x06, 0xdf, 0x1e, 0xd0, 0xed, 0xae, + 0x2d, 0xd7, 0x1d, 0x0c, 0xe9, 0xce, 0xfe, 0xd3, 0xae, 0xe3, 0x7e, 0x06, 0xad, 0x8a, 0xd2, 0xe4, + 0x0c, 0xda, 0x7f, 0xd2, 0x5d, 0x92, 0xdb, 0x3c, 0xdf, 0xda, 0x3d, 0xea, 0x77, 0x0d, 0xb2, 0x0c, + 0x80, 0x9f, 0xa3, 0xdd, 0xad, 0xfd, 0xa7, 0x5d, 0xd3, 0xfd, 0x57, 0xb0, 0x8f, 0x02, 0xff, 0x51, + 0x18, 0x7b, 0x67, 0xd2, 0xd6, 0x8e, 0x99, 0xe0, 0x3a, 0x78, 0xe3, 0xb7, 0x8c, 0x2e, 0x68, 0xe7, + 0x42, 0x3f, 0xb7, 0xa6, 0xdc, 0x7d, 0x68, 0x1e, 0x05, 0xfe, 0x21, 0xf3, 0xce, 0xc8, 0x07, 0x00, + 0xc7, 0x72, 0xfe, 0x48, 0x04, 0x2f, 0xb8, 0x06, 0x56, 0x07, 0x47, 0x06, 0xc1, 0x0b, 0x4e, 0x6e, + 0x81, 0x85, 0x44, 0x9e, 0x66, 0xa1, 0x7b, 0xe4, 0x7b, 0x52, 0xcd, 0x73, 0xb3, 0xe2, 0xe8, 0x58, + 0xe4, 0xdf, 0x84, 0x7a, 0xc2, 0xbc, 0x33, 0x8d, 0x4f, 0x2d, 0x3d, 0x45, 0x6e, 0x47, 0x91, 0x41, + 0xee, 0x82, 0xad, 0x4d, 0x22, 0x5f, 0xb7, 0x55, 0xb1, 0x1d, 0x5a, 0x30, 0xe7, 0x1f, 0xab, 0xb6, + 0xf0, 0x58, 0x5f, 0x00, 0x94, 0x3d, 0x91, 0x4b, 0x52, 0xfe, 0xeb, 0xd0, 0x60, 0x61, 0xa0, 0x2f, + 0xef, 0x50, 0x45, 0xb8, 0xfb, 0xd0, 0xaa, 0x74, 0x52, 0xa4, 0xa5, 0xb0, 0x30, 0x1c, 0x9d, 0xf1, + 0x0b, 0x81, 0x73, 0x6d, 0xda, 0x64, 0x61, 0xf8, 0x8c, 0x5f, 0x08, 0x72, 0x0b, 0x1a, 0xaa, 0x09, + 0x63, 0x2e, 0xd4, 0xfa, 0x38, 0x95, 0x2a, 0xa6, 0xfb, 0x09, 0x58, 0xaa, 0x01, 0x50, 0x31, 0x54, + 0xe3, 0xca, 0x58, 0xf7, 0x50, 0x9f, 0x19, 0xdb, 0x05, 0xe4, 0x63, 0xdd, 0xec, 0x11, 0xaa, 0xb5, + 0x64, 0x94, 0xf9, 0x9f, 0x12, 0xd2, 0x7d, 0x1e, 0x14, 0x76, 0xb7, 0xc1, 0x7e, 0x6d, 0xfb, 0x4c, + 0x2b, 0xc0, 0x2c, 0x15, 0x70, 0x49, 0x43, 0xcd, 0xfd, 0x6f, 0x80, 0xb2, 0x29, 0xa4, 0xfd, 0x46, + 0xad, 0x22, 0xfd, 0xe6, 0x3e, 0xd8, 0xde, 0x69, 0x10, 0xfa, 0x29, 0x8f, 0xe6, 0x6e, 0x5d, 0xb6, + 0x91, 0x0a, 0x3e, 0x59, 0x83, 0x3a, 0xf6, 0xba, 0x6a, 0x25, 0x6e, 0x16, 0x8d, 0x2e, 0xe4, 0xb8, + 0xc7, 0xd0, 0x51, 0x21, 0x94, 0xf2, 0xff, 0x99, 0x72, 0xf1, 0xda, 0xc4, 0x6c, 0x15, 0xa0, 0x40, + 0xf9, 0xbc, 0x6b, 0x57, 0x19, 0x91, 0xa6, 0x7c, 0x12, 0xf0, 0xd0, 0xcf, 0x6f, 0xa3, 0x29, 0xf7, + 0x4b, 0x68, 0xe7, 0x7b, 0xe8, 0xde, 0x41, 0x1e, 0xc8, 0x95, 0x36, 0x55, 0x39, 0xa3, 0x44, 0xf6, + 0x63, 0xbf, 0x88, 0xe3, 0xee, 0x1f, 0xcc, 0x7c, 0xa6, 0x2e, 0xa3, 0xe7, 0x52, 0x43, 0x63, 0x31, + 0x35, 0x9c, 0x4f, 0xb3, 0xcc, 0x1f, 0x95, 0x66, 0x7d, 0x05, 0x8e, 0x8f, 0xb9, 0x46, 0x70, 0x9e, + 0xe3, 0xea, 0xca, 0x62, 0x5e, 0xa1, 0xb3, 0x91, 0xe0, 0x9c, 0xd3, 0x52, 0x58, 0x9e, 0x25, 0x8b, + 0xcf, 0x78, 0x14, 0xbc, 0xc0, 0x3e, 0x81, 0xbc, 0x70, 0x39, 0x50, 0x36, 0x5d, 0x54, 0xfe, 0xa1, + 0x9b, 0x2e, 0x79, 0xff, 0xc8, 0x2a, 0xfb, 0x47, 0x52, 0x6b, 0xd3, 0x44, 0xf0, 0x34, 0xcb, 0xf3, + 0x50, 0x45, 0x15, 0xf9, 0x9c, 0xa3, 0x65, 0x59, 0x34, 0x76, 0x1f, 0x82, 0x53, 0x9c, 0x45, 0x02, + 0xda, 0xfe, 0xc1, 0x7e, 0x5f, 0xc1, 0xcf, 0xce, 0xfe, 0x76, 0xff, 0x3f, 0xba, 0x86, 0x84, 0x44, + 0xda, 0x7f, 0xde, 0xa7, 0x83, 0x7e, 0xd7, 0x94, 0xd0, 0xb5, 0xdd, 0xdf, 0xed, 0x0f, 0xfb, 0xdd, + 0xda, 0x37, 0x75, 0xbb, 0xd9, 0xb5, 0xa9, 0xcd, 0x67, 0x49, 0x18, 0x78, 0x41, 0xe6, 0x1e, 0x81, + 0xbd, 0xc7, 0x92, 0x57, 0x6a, 0x8a, 0x32, 0xd2, 0x4d, 0x75, 0xaf, 0x44, 0x47, 0xa5, 0xdb, 0xd0, + 0xd4, 0x2e, 0xaf, 0xad, 0x69, 0x0e, 0x0e, 0x72, 0x9e, 0xfb, 0x4b, 0x03, 0xae, 0xef, 0xc5, 0xe7, + 0xbc, 0x08, 0xfc, 0x87, 0xec, 0x22, 0x8c, 0x99, 0xff, 0x86, 0xa7, 0xbb, 0x03, 0xd7, 0x44, 0x3c, + 0x4d, 0x3d, 0x3e, 0x5a, 0xe8, 0xd3, 0x74, 0xd4, 0xf0, 0x53, 0x6d, 0x82, 0x2e, 0x74, 0x7c, 0x2e, + 0xb2, 0x52, 0xaa, 0x86, 0x52, 0x2d, 0x39, 0x98, 0xcb, 0x14, 0xd9, 0x4b, 0xfd, 0x4d, 0xd9, 0x8b, + 0xfb, 0x18, 0x9c, 0xe1, 0x0c, 0x8b, 0xa1, 0xa9, 0x98, 0x0b, 0x48, 0xc6, 0x6b, 0x02, 0x92, 0xb9, + 0x80, 0x71, 0x03, 0x68, 0x55, 0xd2, 0x16, 0xf2, 0x21, 0xd4, 0xb3, 0x59, 0x34, 0xdf, 0x6f, 0xcd, + 0xf7, 0xa0, 0xc8, 0x22, 0x1f, 0x42, 0x5b, 0x16, 0x4a, 0x4c, 0x88, 0x60, 0x1c, 0x71, 0x5f, 0xaf, + 0x28, 0x8b, 0xa7, 0x2d, 0x3d, 0xe4, 0xde, 0x84, 0x8e, 0xac, 0x4c, 0x83, 0x09, 0x17, 0x19, 0x9b, + 0x24, 0x18, 0x3e, 0x35, 0x6a, 0xd5, 0xa9, 0x99, 0x09, 0xf7, 0x0e, 0xb4, 0x0f, 0x39, 0x4f, 0x29, + 0x17, 0x49, 0x1c, 0xa9, 0x38, 0x22, 0x70, 0x0f, 0x0d, 0x91, 0x9a, 0x72, 0xbf, 0x07, 0x47, 0x26, + 0x9e, 0x8f, 0x58, 0xe6, 0x9d, 0xfe, 0x94, 0xc4, 0xf4, 0x0e, 0x34, 0x13, 0xf5, 0x74, 0x3a, 0x8d, + 0x6c, 0xa3, 0x97, 0xea, 0xe7, 0xa4, 0x39, 0xd3, 0xfd, 0x02, 0x6a, 0xfb, 0xd3, 0x49, 0xf5, 0xdf, + 0x87, 0xba, 0x4a, 0x8d, 0xe6, 0x4a, 0x32, 0x73, 0xbe, 0x24, 0x73, 0xbf, 0x83, 0x56, 0x7e, 0xd5, + 0x1d, 0x1f, 0xff, 0x42, 0x40, 0x55, 0xef, 0xf8, 0x73, 0x9a, 0x57, 0xb5, 0x0e, 0x8f, 0xfc, 0x9d, + 0x5c, 0x47, 0x8a, 0x98, 0x5f, 0x5b, 0xd7, 0xf2, 0xc5, 0xda, 0x4f, 0xa0, 0x9d, 0x27, 0x87, 0x98, + 0x87, 0xc9, 0xc7, 0x0b, 0x03, 0x1e, 0x55, 0x1e, 0xd6, 0x56, 0x03, 0x43, 0xf1, 0x9a, 0xce, 0xa0, + 0xbb, 0x01, 0x96, 0xb6, 0x0c, 0x02, 0x75, 0x2f, 0xf6, 0x95, 0xd9, 0x36, 0x28, 0x7e, 0xcb, 0x0b, + 0x4f, 0xc4, 0x38, 0x87, 0xf2, 0x89, 0x18, 0xbb, 0x19, 0x74, 0x1e, 0x31, 0xef, 0x6c, 0x9a, 0xe4, + 0x50, 0x5a, 0xc9, 0xe2, 0x8d, 0xb9, 0x2c, 0xfe, 0x35, 0xed, 0xc8, 0x77, 0xa0, 0x39, 0x8d, 0x82, + 0x59, 0x1e, 0x4b, 0x1d, 0x6a, 0x49, 0x72, 0x88, 0xe0, 0x9a, 0xb1, 0x74, 0xac, 0xfb, 0xb5, 0x0e, + 0xd5, 0x94, 0xfb, 0x5f, 0xd0, 0xe9, 0xcf, 0x12, 0x6c, 0xcc, 0xbe, 0x11, 0xc0, 0x2b, 0x07, 0x32, + 0xe7, 0x0e, 0xb4, 0xb0, 0x6b, 0x2d, 0xdf, 0x75, 0xf3, 0x37, 0x06, 0xd4, 0xa5, 0x79, 0x90, 0x5b, + 0x50, 0xef, 0x7b, 0xa7, 0x31, 0x99, 0xb3, 0x82, 0x95, 0x39, 0xca, 0x5d, 0x22, 0x9f, 0xa8, 0x66, + 0x6f, 0xde, 0xc3, 0xee, 0xe4, 0xd6, 0x85, 0xd6, 0xf7, 0x8a, 0xf4, 0x06, 0xb4, 0xbe, 0x89, 0x83, + 0xe8, 0xb1, 0xea, 0x7f, 0x92, 0x45, 0x5b, 0x7c, 0x45, 0xfe, 0x53, 0xb0, 0x76, 0x84, 0x34, 0xfa, + 0x57, 0x45, 0xb1, 0x16, 0xac, 0xfa, 0x83, 0xbb, 0xb4, 0xf9, 0xab, 0x1a, 0xd4, 0xbf, 0xe3, 0x69, + 0x4c, 0x3e, 0x81, 0xa6, 0xee, 0x7c, 0x90, 0x4a, 0x87, 0x63, 0x05, 0x81, 0x61, 0xa1, 0x25, 0x82, + 0xbb, 0x74, 0x15, 0xec, 0x97, 0x98, 0x41, 0xca, 0xc6, 0xcc, 0x2b, 0x87, 0x7a, 0x08, 0xdd, 0x41, + 0x96, 0x72, 0x36, 0xa9, 0x88, 0xcf, 0x2b, 0xe9, 0x32, 0x00, 0x72, 0x97, 0x1e, 0x18, 0xe4, 0x63, + 0xb0, 0x14, 0x70, 0x2c, 0x4c, 0x58, 0xac, 0x84, 0x50, 0xf8, 0x2e, 0xb4, 0x06, 0xa7, 0xf1, 0x34, + 0xf4, 0x07, 0x3c, 0x3d, 0xe7, 0xa4, 0xd2, 0x7d, 0x5c, 0xa9, 0x7c, 0xbb, 0x4b, 0x64, 0x1d, 0x40, + 0xb9, 0xd6, 0x51, 0xe0, 0x0b, 0xd2, 0x94, 0xbc, 0xfd, 0xe9, 0x44, 0x2d, 0x5a, 0xf1, 0x39, 0x25, + 0x59, 0x01, 0x98, 0xd7, 0x49, 0x7e, 0x0e, 0x9d, 0xc7, 0x08, 0x77, 0x07, 0xe9, 0xd6, 0x71, 0x9c, + 0x66, 0x64, 0xb1, 0x03, 0xb9, 0xb2, 0x38, 0xe0, 0x2e, 0x91, 0x07, 0x60, 0x0f, 0xd3, 0x0b, 0x25, + 0xff, 0x96, 0x86, 0xc1, 0x72, 0xbf, 0x4b, 0x6e, 0xb9, 0xf9, 0xf3, 0x1a, 0x58, 0xdf, 0xc6, 0xe9, + 0x19, 0x4f, 0xc9, 0x7d, 0xb0, 0xb0, 0x64, 0xd5, 0x46, 0x54, 0x94, 0xaf, 0x97, 0x6d, 0x74, 0x0b, + 0x1c, 0x54, 0xca, 0x90, 0x89, 0x33, 0xf5, 0x54, 0xf8, 0xa7, 0xa3, 0xd2, 0x8b, 0xca, 0x39, 0xf0, + 0x5d, 0x97, 0xd5, 0x43, 0x15, 0x65, 0xfa, 0x5c, 0x1d, 0xb9, 0xd2, 0x54, 0x45, 0xe1, 0xc0, 0x5d, + 0x5a, 0x37, 0x1e, 0x18, 0xe4, 0x1e, 0xd4, 0x07, 0xea, 0xa6, 0x52, 0xa8, 0xfc, 0x63, 0x66, 0x65, + 0x39, 0x1f, 0x28, 0x56, 0xfe, 0x67, 0xb0, 0x54, 0xba, 0xa0, 0xae, 0x39, 0x97, 0x4f, 0xad, 0x74, + 0xab, 0x43, 0x7a, 0xc2, 0x3d, 0xb0, 0x14, 0x52, 0xa8, 0x09, 0x73, 0xa8, 0xa1, 0x4e, 0xad, 0x80, + 0x47, 0x89, 0x2a, 0xf7, 0x56, 0xa2, 0x73, 0xae, 0xbe, 0x20, 0xfa, 0x29, 0x74, 0x29, 0xf7, 0x78, + 0x50, 0x09, 0xbe, 0x24, 0xbf, 0xd4, 0xa2, 0xd9, 0xae, 0x1b, 0xe4, 0x21, 0x74, 0xe6, 0x02, 0x35, + 0xe9, 0xa1, 0xa2, 0x2f, 0x89, 0xdd, 0x8b, 0x93, 0x1f, 0x75, 0x7f, 0xf7, 0x72, 0xd5, 0xf8, 0xfd, + 0xcb, 0x55, 0xe3, 0x8f, 0x2f, 0x57, 0x8d, 0x1f, 0xfe, 0xb4, 0xba, 0x74, 0x6c, 0xe1, 0x9f, 0xd5, + 0x9f, 0xff, 0x2d, 0x00, 0x00, 0xff, 0xff, 0x26, 0xdd, 0xb7, 0xb3, 0xc7, 0x1e, 0x00, 0x00, } diff --git a/worker/draft.go b/worker/draft.go index a094b6a1332..7b67c221058 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -115,11 +115,10 @@ func (h *header) Decode(in []byte) { } func (n *node) Ctx(key string) context.Context { - ctx := context.Background() if pctx := n.Proposals.Get(key); pctx != nil { - ctx = pctx.Ctx + return pctx.Ctx } - return ctx + return context.Background() } func (n *node) applyConfChange(e raftpb.Entry) { @@ -159,9 +158,8 @@ func detectPendingTxns(attr string) error { // We don't support schema mutations across nodes in a transaction. // Wait for all transactions to either abort or complete and all write transactions // involving the predicate are aborted until schema mutations are done. -func (n *node) applyMutations(proposal *pb.Proposal) error { - tr := trace.New("Dgraph.Node", "ApplyMutations") - defer tr.Finish() +func (n *node) applyMutations(ctx context.Context, proposal *pb.Proposal) error { + span := otrace.FromContext(ctx) if proposal.Mutations.DropAll { // Ensures nothing get written to disk due to commit proposals. @@ -173,14 +171,10 @@ func (n *node) applyMutations(proposal *pb.Proposal) error { if proposal.Mutations.StartTs == 0 { return errors.New("StartTs must be provided.") } - startTs := proposal.Mutations.StartTs - ctx := n.Ctx(proposal.Key) - ctx, span := otrace.StartSpan(ctx, "node.applyMutations") - defer span.End() if len(proposal.Mutations.Schema) > 0 { - tr.LazyPrintf("Applying Schema") + span.Annotatef(nil, "Applying schema") for _, supdate := range proposal.Mutations.Schema { // This is neceassry to ensure that there is no race between when we start reading // from badger and new mutation getting commited via raft and getting applied. @@ -213,19 +207,17 @@ func (n *node) applyMutations(proposal *pb.Proposal) error { schemaMap := make(map[string]types.TypeID) for _, edge := range proposal.Mutations.Edges { if tablet := groups().Tablet(edge.Attr); tablet != nil && tablet.ReadOnly { - tr.LazyPrintf("Predicate Moving") - tr.SetError() + span.Annotatef(nil, "Tablet moving: %+v. Retry later.", tablet) return errPredicateMoving } if edge.Entity == 0 && bytes.Equal(edge.Value, []byte(x.Star)) { // We should only drop the predicate if there is no pending // transaction. if err := detectPendingTxns(edge.Attr); err != nil { - tr.LazyPrintf("Found pending transactions which obstruct operation.") - tr.SetError() + span.Annotatef(nil, "Found pending transactions. Retry later.") return err } - tr.LazyPrintf("Deleting predicate") + span.Annotatef(nil, "Deleting predicate: %s", edge.Attr) return posting.DeletePredicate(ctx, edge.Attr) } // Dont derive schema when doing deletion. @@ -254,12 +246,10 @@ func (n *node) applyMutations(proposal *pb.Proposal) error { m := proposal.Mutations txn := posting.Oracle().RegisterStartTs(m.StartTs) if txn.ShouldAbort() { - tr.LazyPrintf("Should Abort") - tr.SetError() + span.Annotatef(nil, "Txn %d should abort.", m.StartTs) return dy.ErrConflict } - tr.LazyPrintf("Applying %d edges", len(m.Edges)) span.Annotatef(nil, "To apply: %d edges", len(m.Edges)) var retries int for _, edge := range m.Edges { @@ -269,7 +259,6 @@ func (n *node) applyMutations(proposal *pb.Proposal) error { break } if err != posting.ErrRetry { - tr.SetError() return err } retries++ @@ -278,20 +267,26 @@ func (n *node) applyMutations(proposal *pb.Proposal) error { if retries > 0 { span.Annotatef(nil, "retries=true num=%d", retries) } - tr.LazyPrintf("Done applying %d edges", len(m.Edges)) return nil } func (n *node) applyCommitted(proposal *pb.Proposal) error { - if proposal.Mutations != nil { - // syncmarks for this shouldn't be marked done until it's comitted. - n.elog.Printf("Applying mutations for key: %s", proposal.Key) - return n.applyMutations(proposal) - } - ctx := n.Ctx(proposal.Key) ctx, span := otrace.StartSpan(ctx, "node.applyCommitted") defer span.End() + span.Annotatef(nil, "Node id: %d. Group id: %d. Got proposal key: %s", + n.Id, n.gid, proposal.Key) + + if proposal.Mutations != nil { + // syncmarks for this shouldn't be marked done until it's comitted. + span.Annotate(nil, "Applying mutations") + if err := n.applyMutations(ctx, proposal); err != nil { + span.Annotatef(nil, "While applying mutations: %v", err) + return err + } + span.Annotate(nil, "Done") + return nil + } switch { case len(proposal.Kv) > 0: @@ -856,12 +851,12 @@ func (n *node) rollupLists(readTs uint64) error { return nil } -var errConnection = errors.New("No connection exists") +var errNoConnection = errors.New("No connection exists") func (n *node) blockingAbort(req *pb.TxnTimestamps) error { pl := groups().Leader(0) if pl == nil { - return errConnection + return errNoConnection } zc := pb.NewZeroClient(pl.Get()) ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) diff --git a/worker/groups.go b/worker/groups.go index 6aaf2c3fa41..688d06b72aa 100644 --- a/worker/groups.go +++ b/worker/groups.go @@ -97,7 +97,7 @@ func StartRaftNodes(walStore *badger.DB, bindall bool) { var connState *pb.ConnectionState var err error for { // Keep on retrying. See: https://github.com/dgraph-io/dgraph/issues/2289 - pl := gr.connToZeroLeader(Config.ZeroAddr) + pl := gr.connToZeroLeader() if pl == nil { continue } @@ -129,12 +129,14 @@ func StartRaftNodes(walStore *badger.DB, bindall bool) { x.Checkf(schema.LoadFromDb(), "Error while initializing schema") raftServer.Node = gr.Node.Node gr.Node.InitAndStartNode() - x.UpdateHealthStatus(true) - gr.closer = y.NewCloser(3) // Match CLOSER:1 in this file. - go gr.periodicMembershipUpdate() // Now set it to be run periodically. + + gr.closer = y.NewCloser(4) // Match CLOSER:1 in this file. + go gr.sendMembershipUpdates() + go gr.receiveMembershipUpdates() go gr.cleanupTablets() go gr.processOracleDeltaStream() + gr.proposeInitialSchema() } @@ -303,14 +305,7 @@ func (g *groupi) ServesGroup(gid uint32) bool { } func (g *groupi) BelongsTo(key string) uint32 { - g.RLock() - tablet, ok := g.tablets[key] - g.RUnlock() - - if ok { - return tablet.GroupId - } - tablet = g.Tablet(key) + tablet := g.Tablet(key) if tablet != nil { return tablet.GroupId } @@ -345,10 +340,7 @@ func (g *groupi) Tablet(key string) *pb.Tablet { // We don't know about this tablet. // Check with dgraphzero if we can serve it. - pl := g.Leader(0) - if pl == nil { - return nil - } + pl := g.connToZeroLeader() zc := pb.NewZeroClient(pl.Get()) tablet = &pb.Tablet{GroupId: g.groupId(), Predicate: key} @@ -487,13 +479,30 @@ func (g *groupi) triggerMembershipSync() { const connBaseDelay = 100 * time.Millisecond -func (g *groupi) connToZeroLeader(addr string) *conn.Pool { +func (g *groupi) connToZeroLeader() *conn.Pool { pl := g.Leader(0) if pl != nil { return pl } glog.V(1).Infof("No healthy Zero leader found. Trying to find a Zero leader...") + getLeaderConn := func(zc pb.ZeroClient) *conn.Pool { + ctx, cancel := context.WithTimeout(gr.ctx, 10*time.Second) + defer cancel() + + connState, err := zc.Connect(ctx, &pb.Member{ClusterInfoOnly: true}) + if err != nil || connState == nil { + glog.V(1).Infof("While retrieving Zero leader info. Error: %v. Retrying...", err) + return nil + } + for _, mz := range connState.State.GetZeros() { + if mz.Leader { + return conn.Get().Connect(mz.GetAddr()) + } + } + return nil + } + // No leader found. Let's get the latest membership state from Zero. delay := connBaseDelay maxHalfDelay := time.Second @@ -502,50 +511,141 @@ func (g *groupi) connToZeroLeader(addr string) *conn.Pool { if delay <= maxHalfDelay { delay *= 2 } - var pl *conn.Pool - if len(addr) > 0 { - pl = conn.Get().Connect(addr) - } + pl := g.AnyServer(0) if pl == nil { - pl = g.AnyServer(0) - if pl == nil { - glog.V(1).Infof("No healthy Zero server found. Retrying...") - continue - } + pl = conn.Get().Connect(Config.ZeroAddr) } - zc := pb.NewZeroClient(pl.Get()) - connState, err := zc.Connect(gr.ctx, &pb.Member{ClusterInfoOnly: true}) - if err != nil || connState == nil { - glog.V(1).Infof("While retrieving Zero leader info. Error: %v. Retrying...", err) + if pl == nil { + glog.V(1).Infof("No healthy Zero server found. Retrying...") continue } - for _, mz := range connState.State.GetZeros() { - if mz.Leader { - pl := conn.Get().Connect(mz.GetAddr()) - return pl - } + zc := pb.NewZeroClient(pl.Get()) + if pl := getLeaderConn(zc); pl != nil { + return pl } glog.V(1).Infof("Unable to connect to a healthy Zero leader. Retrying...") } } -// TODO: This function needs to be refactored into smaller functions. It gets hard to reason about. -// TODO: The updates have to be sent to Zero leader. But, the membership update receives can be from -// any Zero server. Let's break that up into two different endpoints. -func (g *groupi) periodicMembershipUpdate() { - defer g.closer.Done() // CLOSER:1 +func (g *groupi) doSendMembership(tablets map[string]*pb.Tablet) error { + leader := g.Node.AmLeader() + member := &pb.Member{ + Id: Config.RaftId, + GroupId: g.groupId(), + Addr: Config.MyAddr, + Leader: leader, + LastUpdate: uint64(time.Now().Unix()), + } + group := &pb.Group{ + Members: make(map[uint64]*pb.Member), + } + group.Members[member.Id] = member + if leader { + // Do not send tablet information, if I'm not the leader. + group.Tablets = tablets + if snap, err := g.Node.Snapshot(); err == nil { + group.SnapshotTs = snap.ReadTs + } + } - // Node might not be the leader when we are calculating size. - // We need to send immediately on start so no leader check inside calculatesize. - tablets := g.calculateTabletSizes() + pl := g.connToZeroLeader() + if pl == nil { + return errNoConnection + } + c := pb.NewZeroClient(pl.Get()) + ctx, cancel := context.WithTimeout(g.ctx, 10*time.Second) + defer cancel() + reply, err := c.UpdateMembership(ctx, group) + if err != nil { + return err + } + if string(reply.GetData()) == "OK" { + return nil + } + return x.Errorf(string(reply.GetData())) +} + +// sendMembershipUpdates sends the membership update to Zero leader. If this Alpha is the leader, it +// would also calculate the tablet sizes and send them to Zero. +func (g *groupi) sendMembershipUpdates() { + defer g.closer.Done() // CLOSER:1 // Calculating tablet sizes is expensive, hence we do it only every 5 mins. slowTicker := time.NewTicker(time.Minute * 5) defer slowTicker.Stop() - fastTicker := time.NewTicker(10 * time.Second) + fastTicker := time.NewTicker(time.Second) defer fastTicker.Stop() + consumeTriggers := func() { + for { + select { + case <-g.triggerCh: + default: + return + } + } + } + + g.triggerMembershipSync() // Ticker doesn't start immediately + var lastSent time.Time + for { + select { + case <-g.closer.HasBeenClosed(): + return + case <-fastTicker.C: + if time.Since(lastSent) > 10*time.Second { + // On start of node if it becomes a leader, we would send tablets size for sure. + g.triggerMembershipSync() + } + case <-g.triggerCh: + // Let's send update even if not leader, zero will know that this node is still active. + // We don't need to send tablet information everytime. So, let's only send it when we + // calculate it. + consumeTriggers() + if err := g.doSendMembership(nil); err != nil { + glog.Errorf("While sending membership update: %v", err) + } else { + lastSent = time.Now() + } + case <-slowTicker.C: + if !g.Node.AmLeader() { + break // breaks select case, not for loop. + } + tablets := g.calculateTabletSizes() + g.RLock() + for attr := range g.tablets { + if tablets[attr] == nil { + // Found an attribute, which is present in the group state by Zero, but not on + // disk. So, we can do some cleanup here by asking Zero to remove this predicate + // from the group's state. + tablets[attr] = &pb.Tablet{ + GroupId: g.gid, + Predicate: attr, + Remove: true, + } + glog.Warningf("Removing tablet: %+v", tablets[attr]) + } + } + g.RUnlock() + if err := g.doSendMembership(tablets); err != nil { + glog.Errorf("While sending membership update with tablet: %v", err) + } else { + lastSent = time.Now() + } + } + } +} + +// receiveMembershipUpdates receives membership updates from ANY Zero server. This is the main +// connection which tells Alpha about the state of the cluster, including the latest Zero leader. +// All the other connections to Zero, are only made only to the leader. +func (g *groupi) receiveMembershipUpdates() { + defer g.closer.Done() // CLOSER:1 + + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() + START: select { case <-g.closer.HasBeenClosed(): @@ -553,7 +653,7 @@ START: default: } - pl := g.connToZeroLeader("") + pl := g.connToZeroLeader() // We should always have some connection to dgraphzero. if pl == nil { glog.Warningln("Membership update: No Zero server known.") @@ -564,7 +664,7 @@ START: c := pb.NewZeroClient(pl.Get()) ctx, cancel := context.WithCancel(context.Background()) - stream, err := c.Update(ctx) + stream, err := c.StreamMembership(ctx, &api.Payload{}) if err != nil { glog.Errorf("Error while calling update %v\n", err) time.Sleep(time.Second) @@ -601,7 +701,6 @@ START: }() lastRecv := time.Now() - g.triggerMembershipSync() // Ticker doesn't start immediately OUTER: for { select { @@ -614,63 +713,16 @@ OUTER: case state := <-stateCh: lastRecv = time.Now() g.applyState(state) - case <-fastTicker.C: + case <-ticker.C: if time.Since(lastRecv) > 10*time.Second { // Zero might have gone under partition. We should recreate our connection. glog.Warningf("No membership update for 10s. Closing connection to Zero.") stream.CloseSend() - cancel() - break OUTER - } - case <-g.triggerCh: - if !g.Node.AmLeader() { - tablets = nil - } - // On start of node if it becomes a leader, we would send tablets size for sure. - if err := g.sendMembership(tablets, stream); err != nil { - stream.CloseSend() - break OUTER - } - case <-slowTicker.C: - // TODO: Zero should have two different RPCs. One for receiving updates, and one for - // sending updates. - - // dgraphzero just adds to the map so check that no data is present for the tablet - // before we remove it to avoid the race condition where a tablet is added recently - // and mutation has not been persisted to disk. - var allTablets map[string]*pb.Tablet - if g.Node.AmLeader() { - prevTablets := tablets - tablets = g.calculateTabletSizes() - if prevTablets != nil { - allTablets = make(map[string]*pb.Tablet) - g.RLock() - for attr := range g.tablets { - if tablets[attr] == nil && prevTablets[attr] == nil { - allTablets[attr] = &pb.Tablet{ - GroupId: g.gid, - Predicate: attr, - Remove: true, - } - } - } - g.RUnlock() - for attr, tab := range tablets { - allTablets[attr] = tab - } - } else { - allTablets = tablets - } - } - // Let's send update even if not leader, zero will know that this node is still - // active. - if err := g.sendMembership(allTablets, stream); err != nil { - glog.Errorf("Error while updating tablets size %v\n", err) - stream.CloseSend() break OUTER } } } + cancel() goto START } @@ -754,27 +806,6 @@ func (g *groupi) cleanupTablets() { } } -func (g *groupi) sendMembership(tablets map[string]*pb.Tablet, - stream pb.Zero_UpdateClient) error { - leader := g.Node.AmLeader() - member := &pb.Member{ - Id: Config.RaftId, - GroupId: g.groupId(), - Addr: Config.MyAddr, - Leader: leader, - LastUpdate: uint64(time.Now().Unix()), - } - group := &pb.Group{ - Members: make(map[uint64]*pb.Member), - } - group.Members[member.Id] = member - if leader { - group.Tablets = tablets - } - - return stream.Send(group) -} - // processOracleDeltaStream is used to process oracle delta stream from Zero. // Zero sends information about aborted/committed transactions and maxPending. func (g *groupi) processOracleDeltaStream() { @@ -888,7 +919,7 @@ func (g *groupi) processOracleDeltaStream() { sort.Slice(delta.Txns, func(i, j int) bool { return delta.Txns[i].CommitTs < delta.Txns[j].CommitTs }) - if glog.V(2) { + if glog.V(3) { glog.Infof("Batched %d updates. Max Assigned: %d. Proposing Deltas:", batch, delta.MaxAssigned) for _, txn := range delta.Txns { diff --git a/worker/mutation.go b/worker/mutation.go index 176ff729a2b..30982d42f7c 100644 --- a/worker/mutation.go +++ b/worker/mutation.go @@ -58,7 +58,7 @@ func runMutation(ctx context.Context, edge *pb.DirectedEdge, txn *posting.Txn) e if !groups().ServesTabletRW(edge.Attr) { // Don't assert, can happen during replay of raft logs if server crashes immediately // after predicate move and before snapshot. - return errUnservedTablet + return x.Errorf("runMutation: Tablet isn't being served by this group.") } su, ok := schema.State().Get(edge.Attr) @@ -116,7 +116,8 @@ func runSchemaMutation(ctx context.Context, update *pb.SchemaUpdate, startTs uin func runSchemaMutationHelper(ctx context.Context, update *pb.SchemaUpdate, startTs uint64) error { n := groups().Node if !groups().ServesTablet(update.Predicate) { - return errUnservedTablet + tablet := groups().Tablet(update.Predicate) + return x.Errorf("Tablet isn't being served by this group. Tablet: %+v", tablet) } if err := checkSchema(update); err != nil { return err @@ -526,6 +527,8 @@ func MutateOverNetwork(ctx context.Context, m *pb.Mutations) (*api.TxnContext, e resCh := make(chan res, len(mutationMap)) for gid, mu := range mutationMap { if gid == 0 { + span.Annotatef(nil, "state: %+v", groups().state) + span.Annotatef(nil, "Group id zero for mutation: %+v", mu) return tctx, errUnservedTablet } mu.StartTs = m.StartTs @@ -578,13 +581,6 @@ func CommitOverNetwork(ctx context.Context, tc *api.TxnContext) (uint64, error) return tctx.CommitTs, nil } -func (w *grpcWorker) PurgeTs(ctx context.Context, - payload *api.Payload) (*pb.Num, error) { - n := &pb.Num{} - n.Val = posting.Oracle().PurgeTs() - return n, nil -} - // Mutate is used to apply mutations over the network on other instances. func (w *grpcWorker) Mutate(ctx context.Context, m *pb.Mutations) (*api.TxnContext, error) { txnCtx := &api.TxnContext{}