diff --git a/conn/node.go b/conn/node.go index a8b0bea2b51..3aca22e7d8a 100644 --- a/conn/node.go +++ b/conn/node.go @@ -50,6 +50,7 @@ type Node struct { messages chan sendmsg RaftContext *intern.RaftContext Store *raftwal.DiskStorage + Rand *rand.Rand // applied is used to keep track of the applied RAFT proposals. // The stages are proposed -> committed (accepted by cluster) -> @@ -57,6 +58,23 @@ type Node struct { Applied x.WaterMark } +type lockedSource struct { + lk sync.Mutex + src rand.Source +} + +func (r *lockedSource) Int63() int64 { + r.lk.Lock() + defer r.lk.Unlock() + return r.src.Int63() +} + +func (r *lockedSource) Seed(seed int64) { + r.lk.Lock() + defer r.lk.Unlock() + r.src.Seed(seed) +} + func NewNode(rc *intern.RaftContext, store *raftwal.DiskStorage) *Node { n := &Node{ Id: rc.Id, @@ -85,6 +103,7 @@ func NewNode(rc *intern.RaftContext, store *raftwal.DiskStorage) *Node { RaftContext: rc, messages: make(chan sendmsg, 100), Applied: x.WaterMark{Name: fmt.Sprintf("Applied watermark")}, + Rand: rand.New(&lockedSource{src: rand.NewSource(time.Now().UnixNano())}), } n.Applied.Init() // TODO: n_ = n is a hack. We should properly init node, and make it part of the server struct. @@ -167,12 +186,21 @@ func (n *Node) Send(m raftpb.Message) { x.AssertTruef(n.Id != m.To, "Sending message to itself") data, err := m.Marshal() x.Check(err) - select { - case n.messages <- sendmsg{to: m.To, data: data}: - // pass - default: - // ignore - } + + // As long as leadership is stable, any attempted Propose() calls should be reflected in the + // next raft.Ready.Messages. Leaders will send MsgApps to the followers; followers will send + // MsgProp to the leader. It is up to the transport layer to get those messages to their + // destination. If a MsgApp gets dropped by the transport layer, it will get retried by raft + // (i.e. it will appear in a future Ready.Messages), but MsgProp will only be sent once. During + // leadership transitions, proposals may get dropped even if the network is reliable. + // + // We can't do a select default here. The messages must be sent to the channel, otherwise we + // should block until the channel can accept these messages. BatchAndSendMessages would take + // care of dropping messages which can't be sent due to network issues to the corresponding + // node. But, we shouldn't take the liberty to do that here. It would take us more time to + // repropose these dropped messages anyway, than to block here a bit waiting for the messages + // channel to clear out. + n.messages <- sendmsg{to: m.To, data: data} } func (n *Node) Snapshot() (raftpb.Snapshot, error) { @@ -270,7 +298,8 @@ func (n *Node) BatchAndSendMessages() { if exists := failedConn[to]; !exists { // So that we print error only the first time we are not able to connect. // Otherwise, the log is polluted with multiple errors. - x.Printf("No healthy connection found to node Id: %d, err: %v\n", to, err) + x.Printf("No healthy connection found to node Id: %d addr: [%s], err: %v\n", + to, addr, err) failedConn[to] = true } continue @@ -286,7 +315,7 @@ func (n *Node) BatchAndSendMessages() { } func (n *Node) doSendMessage(pool *Pool, data []byte) { - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() client := pool.Get() diff --git a/contrib/integration/acctupsert/main.go b/contrib/integration/acctupsert/main.go index d4073dc2fb4..fb89645c403 100644 --- a/contrib/integration/acctupsert/main.go +++ b/contrib/integration/acctupsert/main.go @@ -61,7 +61,9 @@ func main() { flag.Parse() c := newClient() setup(c) + fmt.Println("Doing upserts") doUpserts(c) + fmt.Println("Checking integrity") checkIntegrity(c) } @@ -111,7 +113,7 @@ var ( func upsert(c *dgo.Dgraph, acc account) { for { if time.Since(lastStatus) > 100*time.Millisecond { - fmt.Printf("Success: %d Retries: %d\n", + fmt.Printf("[%s] Success: %d Retries: %d\n", time.Now().Format(time.Stamp), atomic.LoadUint64(&successCount), atomic.LoadUint64(&retryCount)) lastStatus = time.Now() } @@ -119,9 +121,10 @@ func upsert(c *dgo.Dgraph, acc account) { if err == nil { atomic.AddUint64(&successCount, 1) return - } - if err != y.ErrAborted { - x.Check(err) + } else if err == y.ErrAborted { + // pass + } else { + fmt.Errorf("ERROR: %v", err) } atomic.AddUint64(&retryCount, 1) } diff --git a/contrib/scripts/functions.sh b/contrib/scripts/functions.sh index 75a2b1bc5de..f44b062b87e 100755 --- a/contrib/scripts/functions.sh +++ b/contrib/scripts/functions.sh @@ -5,9 +5,12 @@ sleepTime=11 function runCluster { basedir=$GOPATH/src/github.com/dgraph-io/dgraph pushd $basedir/dgraph + sudo rm -Rf /tmp/dg + sudo mkdir /tmp/dg go build . && go install . && md5sum dgraph $GOPATH/bin/dgraph - docker-compose up --force-recreate --remove-orphans --detach + DATA="/tmp/dg" docker-compose up --force-recreate --remove-orphans --detach popd $basedir/contrib/wait-for-it.sh localhost:6080 $basedir/contrib/wait-for-it.sh localhost:9180 + sleep 10 # Sleep 10 seconds to get things ready. } diff --git a/dgraph/cmd/root.go b/dgraph/cmd/root.go index 0b5f06728f8..51c1045c893 100644 --- a/dgraph/cmd/root.go +++ b/dgraph/cmd/root.go @@ -56,6 +56,10 @@ func init() { RootCmd.PersistentFlags().String("config", "", "Configuration file. Takes precedence over default values, but is "+ "overridden to values set with environment variables and flags.") + RootCmd.PersistentFlags().Bool("bindall", true, + "Use 0.0.0.0 instead of localhost to bind to all addresses on local machine.") + RootCmd.PersistentFlags().Bool("expose_trace", false, + "Allow trace endpoint to be accessible from remote") rootConf.BindPFlags(RootCmd.PersistentFlags()) var subcommands = []*x.SubCommand{ diff --git a/dgraph/cmd/server/http_test.go b/dgraph/cmd/server/http_test.go index f7e5dc56a03..701a39e3a4b 100644 --- a/dgraph/cmd/server/http_test.go +++ b/dgraph/cmd/server/http_test.go @@ -12,6 +12,7 @@ import ( "encoding/json" "errors" "fmt" + "io/ioutil" "net/http" "net/http/httptest" "sort" @@ -28,8 +29,10 @@ type res struct { Extensions *query.Extensions `json:"extensions,omitempty"` } +var addr = "http://localhost:8180" + func queryWithTs(q string, ts uint64) (string, uint64, error) { - url := "/query" + url := addr + "/query" if ts != 0 { url += "/" + strconv.FormatUint(ts, 10) } @@ -37,22 +40,13 @@ func queryWithTs(q string, ts uint64) (string, uint64, error) { if err != nil { return "", 0, err } - rr := httptest.NewRecorder() - handler := http.HandlerFunc(queryHandler) - handler.ServeHTTP(rr, req) - - if status := rr.Code; status != http.StatusOK { - return "", 0, fmt.Errorf("Unexpected status code: %v", status) - } - - var qr x.QueryResWithData - json.Unmarshal(rr.Body.Bytes(), &qr) - if len(qr.Errors) > 0 { - return "", 0, errors.New(qr.Errors[0].Message) + _, body, err := runRequest(req) + if err != nil { + return "", 0, err } var r res - x.Check(json.Unmarshal(rr.Body.Bytes(), &r)) + x.Check(json.Unmarshal(body, &r)) startTs := r.Extensions.Txn.StartTs // Remove the extensions. @@ -66,7 +60,7 @@ func queryWithTs(q string, ts uint64) (string, uint64, error) { func mutationWithTs(m string, isJson bool, commitNow bool, ignoreIndexConflict bool, ts uint64) ([]string, uint64, error) { - url := "/mutate" + url := addr + "/mutate" if ts != 0 { url += "/" + strconv.FormatUint(ts, 10) } @@ -82,28 +76,44 @@ func mutationWithTs(m string, isJson bool, commitNow bool, ignoreIndexConflict b if commitNow { req.Header.Set("X-Dgraph-CommitNow", "true") } - rr := httptest.NewRecorder() - handler := http.HandlerFunc(mutationHandler) - handler.ServeHTTP(rr, req) - - if status := rr.Code; status != http.StatusOK { - return keys, 0, fmt.Errorf("Unexpected status code: %v", status) - } - var qr x.QueryResWithData - json.Unmarshal(rr.Body.Bytes(), &qr) - if len(qr.Errors) > 0 { - return keys, 0, errors.New(qr.Errors[0].Message) + _, body, err := runRequest(req) + if err != nil { + return keys, 0, err } var r res - x.Check(json.Unmarshal(rr.Body.Bytes(), &r)) + x.Check(json.Unmarshal(body, &r)) startTs := r.Extensions.Txn.StartTs return r.Extensions.Txn.Keys, startTs, nil } +func runRequest(req *http.Request) (*x.QueryResWithData, []byte, error) { + client := &http.Client{} + resp, err := client.Do(req) + if err != nil { + return nil, nil, err + } + if status := resp.StatusCode; status != http.StatusOK { + return nil, nil, fmt.Errorf("Unexpected status code: %v", status) + } + + defer resp.Body.Close() + body, err := ioutil.ReadAll(resp.Body) + if err != nil { + return nil, nil, err + } + + qr := new(x.QueryResWithData) + json.Unmarshal(body, qr) // Don't check error. + if len(qr.Errors) > 0 { + return nil, nil, errors.New(qr.Errors[0].Message) + } + return qr, body, nil +} + func commitWithTs(keys []string, ts uint64) error { - url := "/commit" + url := addr + "/commit" if ts != 0 { url += "/" + strconv.FormatUint(ts, 10) } @@ -116,22 +126,8 @@ func commitWithTs(keys []string, ts uint64) error { if err != nil { return err } - - rr := httptest.NewRecorder() - handler := http.HandlerFunc(commitHandler) - handler.ServeHTTP(rr, req) - - if status := rr.Code; status != http.StatusOK { - return fmt.Errorf("Unexpected status code: %v", status) - } - - var qr x.QueryResWithData - json.Unmarshal(rr.Body.Bytes(), &qr) - if len(qr.Errors) > 0 { - return errors.New(qr.Errors[0].Message) - } - - return nil + _, _, err = runRequest(req) + return err } func TestTransactionBasic(t *testing.T) { @@ -141,7 +137,6 @@ func TestTransactionBasic(t *testing.T) { q1 := ` { balances(func: anyofterms(name, "Alice Bob")) { - uid name balance } @@ -153,10 +148,10 @@ func TestTransactionBasic(t *testing.T) { m1 := ` { set { - <0x1> "Alice" . - <0x1> "Bob" . - <0x1> "110" . - <0x2> "60" . + _:alice "Alice" . + _:alice "Bob" . + _:alice "110" . + _:bob "60" . } } ` @@ -174,13 +169,13 @@ func TestTransactionBasic(t *testing.T) { // Query with same timestamp. data, _, err = queryWithTs(q1, ts) require.NoError(t, err) - require.Equal(t, `{"data":{"balances":[{"uid":"0x1","name":"Bob","balance":"110"}]}}`, data) + require.Equal(t, `{"data":{"balances":[{"name":"Bob","balance":"110"}]}}`, data) // Commit and query. require.NoError(t, commitWithTs(keys, ts)) data, _, err = queryWithTs(q1, 0) require.NoError(t, err) - require.Equal(t, `{"data":{"balances":[{"uid":"0x1","name":"Bob","balance":"110"}]}}`, data) + require.Equal(t, `{"data":{"balances":[{"name":"Bob","balance":"110"}]}}`, data) } func TestAlterAllFieldsShouldBeSet(t *testing.T) { diff --git a/dgraph/cmd/server/run.go b/dgraph/cmd/server/run.go index f2192240d66..c31f797262c 100644 --- a/dgraph/cmd/server/run.go +++ b/dgraph/cmd/server/run.go @@ -109,11 +109,6 @@ func init() { flag.IntP("port_offset", "o", 0, "Value added to all listening port numbers. [Internal=7080, HTTP=8080, Grpc=9080]") - flag.Bool("bindall", true, - "Use 0.0.0.0 instead of localhost to bind to all addresses on local machine.") - flag.Bool("expose_trace", false, - "Allow trace endpoint to be accessible from remote") - flag.Uint64("query_edge_limit", 1e6, "Limit for the maximum number of edges that can be returned in a query."+ " This is only useful for shortest path queries.") diff --git a/dgraph/cmd/server/run_test.go b/dgraph/cmd/server/run_test.go index bd93191f46a..102e69ce940 100644 --- a/dgraph/cmd/server/run_test.go +++ b/dgraph/cmd/server/run_test.go @@ -10,33 +10,25 @@ package server import ( "bytes" "encoding/json" - "errors" "fmt" - "io/ioutil" "log" "net/http" - "net/http/httptest" "os" - "os/exec" "strings" "sync/atomic" "testing" "time" context "golang.org/x/net/context" + "google.golang.org/grpc" "github.com/stretchr/testify/require" "github.com/dgraph-io/dgo/protos/api" - "github.com/dgraph-io/dgraph/edgraph" "github.com/dgraph-io/dgraph/gql" - "github.com/dgraph-io/dgraph/posting" "github.com/dgraph-io/dgraph/protos/intern" "github.com/dgraph-io/dgraph/query" "github.com/dgraph-io/dgraph/schema" - "github.com/dgraph-io/dgraph/types" - "github.com/dgraph-io/dgraph/worker" - "github.com/dgraph-io/dgraph/x" ) var q0 = ` @@ -71,54 +63,6 @@ func (c *raftServer) JoinCluster(ctx context.Context, in *intern.RaftContext) (* return &api.Payload{}, nil } -func prepare() (dir1, dir2 string, rerr error) { - cmd := exec.Command("go", "install", "github.com/dgraph-io/dgraph/dgraph") - cmd.Env = os.Environ() - if out, err := cmd.CombinedOutput(); err != nil { - log.Fatalf("Could not run %q: %s", cmd.Args, string(out)) - } - zero := exec.Command(os.ExpandEnv("$GOPATH/bin/dgraph"), - "zero", - "-w=wz", - ) - zero.Stdout = os.Stdout - zero.Stderr = os.Stdout - if err := zero.Start(); err != nil { - return "", "", err - } - - var err error - dir1, err = ioutil.TempDir("", "storetest_") - if err != nil { - return "", "", err - } - - dir2, err = ioutil.TempDir("", "wal_") - if err != nil { - return dir1, "", err - } - - edgraph.Config.PostingDir = dir1 - edgraph.Config.BadgerTables = "ram" - edgraph.Config.WALDir = dir2 - edgraph.InitServerState() - - posting.Init(edgraph.State.Pstore) - schema.Init(edgraph.State.Pstore) - worker.Init(edgraph.State.Pstore) - worker.Config.ZeroAddr = fmt.Sprintf("localhost:%d", x.PortZeroGrpc) - x.Config.PortOffset = 1 - worker.Config.RaftId = 1 - go worker.RunServer(false) - worker.StartRaftNodes(edgraph.State.WALstore, false) - return dir1, dir2, nil -} - -func closeAll(dir1, dir2 string) { - os.RemoveAll(dir1) - os.RemoveAll(dir2) -} - func childAttrs(sg *query.SubGraph) []string { var out []string for _, c := range sg.Children { @@ -175,23 +119,12 @@ func runJsonMutation(m string) error { } func alterSchema(s string) error { - req, err := http.NewRequest("PUT", "/alter", bytes.NewBufferString(s)) + req, err := http.NewRequest("PUT", addr+"/alter", bytes.NewBufferString(s)) if err != nil { return err } - rr := httptest.NewRecorder() - handler := http.HandlerFunc(alterHandler) - handler.ServeHTTP(rr, req) - - if status := rr.Code; status != http.StatusOK { - return fmt.Errorf("Unexpected status code: %v", status) - } - var qr x.QueryResWithData - json.Unmarshal(rr.Body.Bytes(), &qr) - if len(qr.Errors) == 0 { - return nil - } - return errors.New(qr.Errors[0].Message) + _, _, err = runRequest(req) + return err } func alterSchemaWithRetry(s string) error { @@ -200,44 +133,22 @@ func alterSchemaWithRetry(s string) error { func dropAll() error { op := `{"drop_all": true}` - req, err := http.NewRequest("PUT", "/alter", bytes.NewBufferString(op)) + req, err := http.NewRequest("PUT", addr+"/alter", bytes.NewBufferString(op)) if err != nil { return err } - rr := httptest.NewRecorder() - handler := http.HandlerFunc(alterHandler) - handler.ServeHTTP(rr, req) - - if status := rr.Code; status != http.StatusOK { - return fmt.Errorf("Unexpected status code: %v", status) - } - var qr x.QueryResWithData - json.Unmarshal(rr.Body.Bytes(), &qr) - if len(qr.Errors) == 0 { - return nil - } - return x.Errorf("Got error while trying to drop all", qr.Errors) + _, _, err = runRequest(req) + return err } func deletePredicate(pred string) error { op := `{"drop_attr": "` + pred + `"}` - req, err := http.NewRequest("PUT", "/alter", bytes.NewBufferString(op)) + req, err := http.NewRequest("PUT", addr+"/alter", bytes.NewBufferString(op)) if err != nil { return err } - rr := httptest.NewRecorder() - handler := http.HandlerFunc(alterHandler) - handler.ServeHTTP(rr, req) - - if status := rr.Code; status != http.StatusOK { - return fmt.Errorf("Unexpected status code: %v", status) - } - var qr x.QueryResWithData - json.Unmarshal(rr.Body.Bytes(), &qr) - if len(qr.Errors) == 0 { - return nil - } - return x.Errorf("Got error while trying to delete predicate", qr.Errors) + _, _, err = runRequest(req) + return err } func TestDeletePredicate(t *testing.T) { @@ -292,7 +203,7 @@ func TestDeletePredicate(t *testing.T) { var q5 = ` { - user(func: uid( 0x3)) { + user(func: uid(0x3)) { age friend { name @@ -310,6 +221,7 @@ func TestDeletePredicate(t *testing.T) { friend: string @index(term) . ` + require.NoError(t, dropAll()) schema.ParseBytes([]byte(""), 1) err := alterSchemaWithRetry(s1) require.NoError(t, err) @@ -368,7 +280,19 @@ func TestDeletePredicate(t *testing.T) { require.NoError(t, err) } +type S struct { + Predicate string `json:"predicate"` + Type string `json:"type"` + Index bool `json:"index"` + Tokenizer []string `json:"tokenizer"` +} + +type Received struct { + Schema []S `json:"schema"` +} + func TestSchemaMutation(t *testing.T) { + require.NoError(t, dropAll()) var m = ` name:string @index(term, exact) . alias:string @index(exact, term) . @@ -381,29 +305,37 @@ func TestSchemaMutation(t *testing.T) { age : int . shadow_deep : int . friend:uid @reverse . - geometry:geo @index(geo) . + geometry:geo @index(geo) . ` -` // reset schema - schema.ParseBytes([]byte(""), 1) - expected := map[string]*intern.SchemaUpdate{ - "name": { - Predicate: "name", - Tokenizer: []string{"term", "exact"}, - ValueType: intern.Posting_ValType(types.StringID), - Directive: intern.SchemaUpdate_INDEX, - }, + expected := S{ + Predicate: "name", + Type: "string", + Index: true, + Tokenizer: []string{"term", "exact"}, } err := alterSchemaWithRetry(m) require.NoError(t, err) - for k, v := range expected { - s, ok := schema.State().Get(k) - require.True(t, ok) - require.Equal(t, *v, s) + + output, err := runQuery("schema {}") + require.NoError(t, err) + got := make(map[string]Received) + require.NoError(t, json.Unmarshal([]byte(output), &got)) + received, ok := got["data"] + require.True(t, ok) + + var found bool + for _, s := range received.Schema { + if s.Predicate == "name" { + found = true + require.Equal(t, expected, s) + } } + require.True(t, found) } func TestSchemaMutation1(t *testing.T) { + require.NoError(t, dropAll()) var m = ` { set { @@ -412,26 +344,28 @@ func TestSchemaMutation1(t *testing.T) { } } -` // reset schema - schema.ParseBytes([]byte(""), 1) - expected := map[string]*intern.SchemaUpdate{ - "pred1": { - ValueType: intern.Posting_ValType(types.StringID), - Predicate: "pred1", - }, - "pred2": { - ValueType: intern.Posting_ValType(types.DefaultID), - Predicate: "pred2", - }, - } - +` err := runMutation(m) require.NoError(t, err) - for k, v := range expected { - s, ok := schema.State().Get(k) - require.True(t, ok) - require.Equal(t, *v, s) + + output, err := runQuery("schema {}") + require.NoError(t, err) + got := make(map[string]Received) + require.NoError(t, json.Unmarshal([]byte(output), &got)) + received, ok := got["data"] + require.True(t, ok) + + var count int + for _, s := range received.Schema { + if s.Predicate == "pred1" { + require.Equal(t, "string", s.Type) + count++ + } else if s.Predicate == "pred2" { + require.Equal(t, "default", s.Type) + count++ + } } + require.Equal(t, 2, count) } // reverse on scalar type @@ -1024,14 +958,8 @@ var q1 = ` } ` +// TODO: This might not work. Fix it later, if needed. func BenchmarkQuery(b *testing.B) { - dir1, dir2, err := prepare() - if err != nil { - b.Error(err) - return - } - defer closeAll(dir1, dir2) - b.ResetTimer() for i := 0; i < b.N; i++ { processToFastJSON(q1) @@ -1263,6 +1191,7 @@ func TestMultipleValues(t *testing.T) { } func TestListTypeSchemaChange(t *testing.T) { + require.NoError(t, dropAll()) schema.ParseBytes([]byte(""), 1) m := ` occupations: [string] @index(term) . @@ -1490,26 +1419,16 @@ func TestIllegalCountInQueryFn(t *testing.T) { } func TestMain(m *testing.M) { - dc := edgraph.DefaultConfig - dc.AllottedMemory = 2048.0 - edgraph.SetConfiguration(dc) - x.Init(true) - - dir1, dir2, err := prepare() + // Increment lease, so that mutations work. + conn, err := grpc.Dial("localhost:5080", grpc.WithInsecure()) if err != nil { log.Fatal(err) } - time.Sleep(10 * time.Millisecond) - - // Increment lease, so that mutations work. - _, err = worker.AssignUidsOverNetwork(context.Background(), &intern.Num{Val: 10e6}) - if err != nil { + zc := intern.NewZeroClient(conn) + if _, err := zc.AssignUids(context.Background(), &intern.Num{Val: 1e6}); err != nil { log.Fatal(err) } - // Parse GQL into intern.query representation. + r := m.Run() - closeAll(dir1, dir2) - exec.Command("killall", "-9", "dgraph").Run() - os.RemoveAll("wz") os.Exit(r) } diff --git a/dgraph/cmd/zero/raft.go b/dgraph/cmd/zero/raft.go index 4937dfaa2fd..971c8b3262c 100644 --- a/dgraph/cmd/zero/raft.go +++ b/dgraph/cmd/zero/raft.go @@ -10,6 +10,7 @@ package zero import ( "encoding/binary" "errors" + "fmt" "log" "math/rand" "sync" @@ -34,33 +35,48 @@ type proposalCtx struct { type proposals struct { sync.RWMutex - ids map[uint32]*proposalCtx + all map[string]*proposalCtx } -func (p *proposals) Store(pid uint32, pctx *proposalCtx) bool { - if pid == 0 { +func (p *proposals) Store(key string, pctx *proposalCtx) bool { + if len(key) == 0 { return false } p.Lock() defer p.Unlock() - if p.ids == nil { - p.ids = make(map[uint32]*proposalCtx) + if p.all == nil { + p.all = make(map[string]*proposalCtx) } - if _, has := p.ids[pid]; has { + if _, has := p.all[key]; has { return false } - p.ids[pid] = pctx + p.all[key] = pctx return true } -func (p *proposals) Done(pid uint32, err error) { +func (p *proposals) Delete(key string) { + if len(key) == 0 { + return + } p.Lock() defer p.Unlock() - pd, has := p.ids[pid] + delete(p.all, key) +} + +func (p *proposals) Done(key string, err error) { + if len(key) == 0 { + return + } + p.Lock() + defer p.Unlock() + pd, has := p.all[key] if !has { + // If we assert here, there would be a race condition between a context + // timing out, and a proposal getting applied immediately after. That + // would cause assert to fail. So, don't assert. return } - delete(p.ids, pid) + delete(p.all, key) pd.ch <- err } @@ -175,46 +191,70 @@ func (n *node) AmLeader() bool { return r.Status().Lead == r.Status().ID } +func (n *node) uniqueKey() string { + return fmt.Sprintf("z%d-%d", n.Id, n.Rand.Uint64()) +} + +var errInternalRetry = errors.New("Retry Raft proposal internally") + func (n *node) proposeAndWait(ctx context.Context, proposal *intern.ZeroProposal) error { if n.Raft() == nil { return x.Errorf("Raft isn't initialized yet.") } - if ctx.Err() != nil { return ctx.Err() } - che := make(chan error, 1) - pctx := &proposalCtx{ - ch: che, - ctx: ctx, - } - for { - id := rand.Uint32() + 1 - if n.props.Store(id, pctx) { - proposal.Id = id - break + propose := func() error { + cctx, cancel := context.WithTimeout(ctx, 15*time.Second) + defer cancel() + + che := make(chan error, 1) + pctx := &proposalCtx{ + ch: che, + // Don't use the original context, because that's not what we're passing to Raft. + ctx: cctx, } - } - data, err := proposal.Marshal() - if err != nil { - return err - } + key := n.uniqueKey() + x.AssertTruef(n.props.Store(key, pctx), "Found existing proposal with key: [%v]", key) + defer n.props.Delete(key) + proposal.Key = key - cctx, cancel := context.WithTimeout(ctx, time.Minute) - defer cancel() - // Propose the change. - if err := n.Raft().Propose(cctx, data); err != nil { - return x.Wrapf(err, "While proposing") + if tr, ok := trace.FromContext(ctx); ok { + tr.LazyPrintf("Proposing with key: %X", key) + } + + data, err := proposal.Marshal() + if err != nil { + return err + } + + // Propose the change. + if err := n.Raft().Propose(cctx, data); err != nil { + return x.Wrapf(err, "While proposing") + } + + // Wait for proposal to be applied or timeout. + select { + case err := <-che: + // We arrived here by a call to n.props.Done(). + return err + case <-ctx.Done(): + return ctx.Err() + case <-cctx.Done(): + return errInternalRetry + } } - // Wait for proposal to be applied or timeout. - select { - case err := <-che: - return err - case <-cctx.Done(): - return cctx.Err() + // Some proposals can be stuck if leader change happens. For e.g. MsgProp message from follower + // to leader can be dropped/end up appearing with empty Data in CommittedEntries. + // Having a timeout here prevents the mutation being stuck forever in case they don't have a + // timeout. We should always try with a timeout and optionally retry. + err := errInternalRetry + for err == errInternalRetry { + err = propose() } + return err } var ( @@ -229,17 +269,20 @@ func newGroup() *intern.Group { } } -func (n *node) applyProposal(e raftpb.Entry) (uint32, error) { +func (n *node) applyProposal(e raftpb.Entry) (string, error) { var p intern.ZeroProposal // Raft commits empty entry on becoming a leader. if len(e.Data) == 0 { - return p.Id, nil + return p.Key, nil } if err := p.Unmarshal(e.Data); err != nil { - return p.Id, err + return p.Key, err + } + if p.DeprecatedId != 0 { + p.Key = fmt.Sprint(p.DeprecatedId) } - if p.Id == 0 { - return 0, errInvalidProposal + if len(p.Key) == 0 { + return p.Key, errInvalidProposal } n.server.Lock() @@ -249,7 +292,7 @@ func (n *node) applyProposal(e raftpb.Entry) (uint32, error) { state.Counter = e.Index if p.MaxRaftId > 0 { if p.MaxRaftId <= state.MaxRaftId { - return p.Id, errInvalidProposal + return p.Key, errInvalidProposal } state.MaxRaftId = p.MaxRaftId } @@ -257,7 +300,7 @@ func (n *node) applyProposal(e raftpb.Entry) (uint32, error) { 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.Id, errInvalidAddress + return p.Key, errInvalidAddress } if p.Member.GroupId == 0 { state.Zeros[p.Member.Id] = p.Member @@ -270,7 +313,7 @@ func (n *node) applyProposal(e raftpb.Entry) (uint32, error) { } } } - return p.Id, nil + return p.Key, nil } group := state.Groups[p.Member.GroupId] if group == nil { @@ -285,11 +328,11 @@ func (n *node) applyProposal(e raftpb.Entry) (uint32, error) { conn.Get().Remove(m.Addr) } // else already removed. - return p.Id, nil + 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.Id, errInvalidProposal + return p.Key, errInvalidProposal } // Create a connection to this server. @@ -317,7 +360,7 @@ func (n *node) applyProposal(e raftpb.Entry) (uint32, error) { } if p.Tablet != nil { if p.Tablet.GroupId == 0 { - return p.Id, errInvalidProposal + return p.Key, errInvalidProposal } group := state.Groups[p.Tablet.GroupId] if p.Tablet.Remove { @@ -325,7 +368,7 @@ func (n *node) applyProposal(e raftpb.Entry) (uint32, error) { if group != nil { delete(group.Tablets, p.Tablet.Predicate) } - return p.Id, nil + return p.Key, nil } if group == nil { group = newGroup() @@ -343,7 +386,7 @@ func (n *node) applyProposal(e raftpb.Entry) (uint32, error) { if tablet.GroupId != p.Tablet.GroupId { x.Printf("Tablet for attr: [%s], gid: [%d] is already being served by group: [%d]\n", tablet.Predicate, p.Tablet.GroupId, tablet.GroupId) - return p.Id, errTabletAlreadyServed + return p.Key, errTabletAlreadyServed } // This update can come from tablet size. p.Tablet.ReadOnly = tablet.ReadOnly @@ -366,7 +409,7 @@ func (n *node) applyProposal(e raftpb.Entry) (uint32, error) { n.server.orc.updateCommitStatus(e.Index, p.Txn) } - return p.Id, nil + return p.Key, nil } func (n *node) applyConfChange(e raftpb.Entry) { @@ -574,11 +617,11 @@ func (n *node) Run() { n.applyConfChange(entry) } else if entry.Type == raftpb.EntryNormal { - pid, err := n.applyProposal(entry) + key, err := n.applyProposal(entry) if err != nil && err != errTabletAlreadyServed { x.Printf("While applying proposal: %v\n", err) } - n.props.Done(pid, err) + n.props.Done(key, err) } else { x.Printf("Unhandled entry: %+v\n", entry) diff --git a/dgraph/cmd/zero/run.go b/dgraph/cmd/zero/run.go index 4211aff9ff6..4cb3680e045 100644 --- a/dgraph/cmd/zero/run.go +++ b/dgraph/cmd/zero/run.go @@ -19,6 +19,7 @@ import ( "time" "golang.org/x/net/context" + "golang.org/x/net/trace" "google.golang.org/grpc" "github.com/dgraph-io/badger" @@ -62,8 +63,6 @@ instances to achieve high-availability. Zero.EnvPrefix = "DGRAPH_ZERO" flag := Zero.Cmd.Flags() - flag.Bool("bindall", true, - "Use 0.0.0.0 instead of localhost to bind to all addresses on local machine.") flag.String("my", "", "addr:port of this server, so other Dgraph servers can talk to this.") flag.IntP("port_offset", "o", 0, @@ -142,6 +141,11 @@ func run() { rebalanceInterval: Zero.Conf.GetDuration("rebalance_interval"), } + if Zero.Conf.GetBool("expose_trace") { + trace.AuthRequest = func(req *http.Request) (any, sensitive bool) { + return true, true + } + } grpc.EnableTracing = false addr := "localhost" diff --git a/dgraph/docker-compose.yml b/dgraph/docker-compose.yml index 5cd3f455a24..021584d5bbe 100644 --- a/dgraph/docker-compose.yml +++ b/dgraph/docker-compose.yml @@ -7,22 +7,26 @@ services: zero1: image: debian:latest container_name: bank-dg0.1 + working_dir: /data/dg0.1 ports: - 5080:5080 - 6080:6080 - command: /gobin/dgraph zero --my=zero1:5080 --replicas 3 --idx 1 --bindall + command: /gobin/dgraph zero --my=zero1:5080 --replicas 3 --idx 1 --bindall --expose_trace --profile_mode block --block_rate 10 volumes: - type: bind - source: $HOME/go/bin + source: $GOPATH/bin target: /gobin read_only: true + - type: bind + source: "${DATA}" + target: /data # zero2: # image: debian:latest # container_name: bank-dg0.2 # command: /gobin/dgraph zero --my=zero2:5080 --replicas 3 --peer=zero1:5080 --idx 2 # volumes: # - type: bind - # source: $HOME/go/bin + # source: $GOPATH/bin # target: /gobin # read_only: true # zero3: @@ -31,43 +35,57 @@ services: # command: /gobin/dgraph zero --my=zero3:5080 --replicas 3 --peer=zero1:5080 --idx 3 # volumes: # - type: bind - # source: $HOME/go/bin + # source: $GOPATH/bin # target: /gobin # read_only: true dg1: image: debian:latest container_name: bank-dg1 + working_dir: /data/dg1 volumes: - type: bind - source: $HOME/go/bin + source: $GOPATH/bin target: /gobin read_only: true + - type: bind + source: "${DATA}" + target: /data ports: - 8180:8180 - 9180:9180 - command: /gobin/dgraph server --my=dg1:7180 --lru_mb=1024 --zero=zero1:5080 -o 100 --expose_trace # --trace 1.0 + command: /gobin/dgraph server --my=dg1:7180 --lru_mb=1024 --zero=zero1:5080 -o 100 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 dg2: image: debian:latest container_name: bank-dg2 + working_dir: /data/dg2 volumes: - type: bind - source: $HOME/go/bin + source: $GOPATH/bin target: /gobin read_only: true + - type: bind + source: "${DATA}" + target: /data ports: - 8181:8181 - 9181:9181 - command: /gobin/dgraph server --my=dg2:7181 --lru_mb=1024 --zero=zero1:5080 -o 101 --expose_trace # --trace 1.0 + command: /gobin/dgraph server --my=dg2:7181 --lru_mb=1024 --zero=zero1:5080 -o 101 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 + dg3: image: debian:latest container_name: bank-dg3 + working_dir: /data/dg3 volumes: - type: bind - source: $HOME/go/bin + source: $GOPATH/bin target: /gobin read_only: true + - type: bind + source: "${DATA}" + target: /data ports: - 8182:8182 - 9182:9182 - command: /gobin/dgraph server --my=dg3:7182 --lru_mb=1024 --zero=zero1:5080 -o 102 --expose_trace # --trace 1.0 + command: /gobin/dgraph server --my=dg3:7182 --lru_mb=1024 --zero=zero1:5080 -o 102 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 + diff --git a/edgraph/server.go b/edgraph/server.go index 11677c301ff..6403a1bbb82 100644 --- a/edgraph/server.go +++ b/edgraph/server.go @@ -308,7 +308,7 @@ func (s *Server) Mutate(ctx context.Context, mu *api.Mutation) (resp *api.Assign } if rand.Float64() < worker.Config.Tracing { var tr trace.Trace - tr, ctx = x.NewTrace("GrpcMutate", ctx) + tr, ctx = x.NewTrace("Server.Mutate", ctx) defer tr.Finish() } diff --git a/posting/list.go b/posting/list.go index 314c6a29985..1cbaa7c9432 100644 --- a/posting/list.go +++ b/posting/list.go @@ -455,6 +455,7 @@ func (l *List) Conflicts(readTs uint64) []uint64 { } func (l *List) pickPostings(readTs uint64) (*intern.PostingList, []*intern.Posting) { + // This function would return zero ts for entries above readTs. effective := func(start, commit uint64) uint64 { if commit > 0 && commit <= readTs { // Has been committed and below the readTs. @@ -684,10 +685,9 @@ func (l *List) rollup() error { // Pick all committed entries x.AssertTrue(l.minTs <= l.commitTs) err := l.iterate(l.commitTs, 0, func(p *intern.Posting) bool { - commitTs := atomic.LoadUint64(&p.CommitTs) - if commitTs == 0 || commitTs > l.commitTs { - return true - } + // iterate already takes care of not returning entries whose commitTs is above l.commitTs. + // So, we don't need to do any filtering here. In fact, doing filtering here could result + // in a bug. buf = append(buf, p.Uid) if len(buf) == bp128.BlockSize { bp.PackAppend(buf) diff --git a/posting/list_test.go b/posting/list_test.go index 2d00909186e..a16f5860883 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -424,6 +424,31 @@ func TestAddMutation_mrjn1(t *testing.T) { require.Equal(t, 0, ol.Length(txn.StartTs, 0)) } +func TestMillion(t *testing.T) { + ctx := context.Background() + key := x.DataKey("bal", 1331) + ol, err := getNew(key, ps) + require.NoError(t, err) + var commits int + N := int(1e6) + for i := 2; i <= N; i += 2 { + edge := &intern.DirectedEdge{ + ValueId: uint64(i), + } + txn := Txn{StartTs: uint64(i)} + addMutationHelper(t, ol, edge, Set, &txn) + require.NoError(t, ol.CommitMutation(ctx, uint64(i), uint64(i)+1)) + commits++ + } + opt := ListOptions{ReadTs: uint64(N) + 1} + l, err := ol.Uids(opt) + require.NoError(t, err) + require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) + for i, uid := range l.Uids { + require.Equal(t, uint64(i+1)*2, uid) + } +} + // Test the various mutate, commit and abort sequences. func TestAddMutation_mrjn2(t *testing.T) { ctx := context.Background() diff --git a/posting/mvcc.go b/posting/mvcc.go index 2f755775c4e..4486c0948a3 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -199,6 +199,9 @@ func (t *Txn) SetAbort() { } func (t *Txn) ShouldAbort() bool { + if t == nil { + return false + } return atomic.LoadUint32(&t.shouldAbort) > 0 } diff --git a/protos/intern/internal.pb.go b/protos/intern/internal.pb.go index ff282f57302..2af0f5b9e71 100644 --- a/protos/intern/internal.pb.go +++ b/protos/intern/internal.pb.go @@ -1,6 +1,5 @@ -// Code generated by protoc-gen-gogo. +// Code generated by protoc-gen-gogo. DO NOT EDIT. // source: internal.proto -// DO NOT EDIT! /* Package intern is a generated protocol buffer package. @@ -59,10 +58,10 @@ import fmt "fmt" import math "math" import api "github.com/dgraph-io/dgo/protos/api" -import ( - context "golang.org/x/net/context" - grpc "google.golang.org/grpc" -) +import context "golang.org/x/net/context" +import grpc "google.golang.org/grpc" + +import binary "encoding/binary" import io "io" @@ -753,13 +752,14 @@ func (m *Group) GetTablets() map[string]*Tablet { } type ZeroProposal struct { - Id uint32 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` - 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"` + DeprecatedId uint32 `protobuf:"varint,1,opt,name=deprecated_id,json=deprecatedId,proto3" json:"deprecated_id,omitempty"` + 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"` } func (m *ZeroProposal) Reset() { *m = ZeroProposal{} } @@ -767,9 +767,9 @@ func (m *ZeroProposal) String() string { return proto.CompactTextStri func (*ZeroProposal) ProtoMessage() {} func (*ZeroProposal) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{13} } -func (m *ZeroProposal) GetId() uint32 { +func (m *ZeroProposal) GetDeprecatedId() uint32 { if m != nil { - return m.Id + return m.DeprecatedId } return 0 } @@ -816,6 +816,13 @@ func (m *ZeroProposal) GetTxn() *api.TxnContext { return nil } +func (m *ZeroProposal) GetKey() string { + if m != nil { + return m.Key + } + return "" +} + // MembershipState is used to pack together the current membership state of all the nodes // in the caller server; and the membership updates recorded by the callee server since // the provided lastUpdate. @@ -2849,22 +2856,8 @@ func (m *List) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintInternal(dAtA, i, uint64(len(m.Uids)*8)) for _, num := range m.Uids { - dAtA[i] = uint8(num) - i++ - dAtA[i] = uint8(num >> 8) - i++ - dAtA[i] = uint8(num >> 16) - i++ - dAtA[i] = uint8(num >> 24) - i++ - dAtA[i] = uint8(num >> 32) - i++ - dAtA[i] = uint8(num >> 40) - i++ - dAtA[i] = uint8(num >> 48) - i++ - dAtA[i] = uint8(num >> 56) - i++ + binary.LittleEndian.PutUint64(dAtA[i:], uint64(num)) + i += 8 } } return i, nil @@ -2987,7 +2980,8 @@ func (m *Query) MarshalTo(dAtA []byte) (int, error) { if m.AfterUid != 0 { dAtA[i] = 0x19 i++ - i = encodeFixed64Internal(dAtA, i, uint64(m.AfterUid)) + binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.AfterUid)) + i += 8 } if m.DoCount { dAtA[i] = 0x20 @@ -3427,7 +3421,8 @@ func (m *RaftContext) MarshalTo(dAtA []byte) (int, error) { if m.Id != 0 { dAtA[i] = 0x9 i++ - i = encodeFixed64Internal(dAtA, i, uint64(m.Id)) + binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.Id)) + i += 8 } if m.Group != 0 { dAtA[i] = 0x10 @@ -3466,7 +3461,8 @@ func (m *Member) MarshalTo(dAtA []byte) (int, error) { if m.Id != 0 { dAtA[i] = 0x9 i++ - i = encodeFixed64Internal(dAtA, i, uint64(m.Id)) + binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.Id)) + i += 8 } if m.GroupId != 0 { dAtA[i] = 0x10 @@ -3605,10 +3601,10 @@ func (m *ZeroProposal) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Id != 0 { + if m.DeprecatedId != 0 { dAtA[i] = 0x8 i++ - i = encodeVarintInternal(dAtA, i, uint64(m.Id)) + i = encodeVarintInternal(dAtA, i, uint64(m.DeprecatedId)) } if m.Member != nil { dAtA[i] = 0x12 @@ -3655,6 +3651,12 @@ func (m *ZeroProposal) MarshalTo(dAtA []byte) (int, error) { } i += n15 } + if len(m.Key) > 0 { + dAtA[i] = 0x42 + i++ + i = encodeVarintInternal(dAtA, i, uint64(len(m.Key))) + i += copy(dAtA[i:], m.Key) + } return i, nil } @@ -3887,7 +3889,8 @@ func (m *DirectedEdge) MarshalTo(dAtA []byte) (int, error) { if m.Entity != 0 { dAtA[i] = 0x9 i++ - i = encodeFixed64Internal(dAtA, i, uint64(m.Entity)) + binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.Entity)) + i += 8 } if len(m.Attr) > 0 { dAtA[i] = 0x12 @@ -3909,7 +3912,8 @@ func (m *DirectedEdge) MarshalTo(dAtA []byte) (int, error) { if m.ValueId != 0 { dAtA[i] = 0x29 i++ - i = encodeFixed64Internal(dAtA, i, uint64(m.ValueId)) + binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.ValueId)) + i += 8 } if len(m.Label) > 0 { dAtA[i] = 0x32 @@ -4211,7 +4215,8 @@ func (m *Posting) MarshalTo(dAtA []byte) (int, error) { if m.Uid != 0 { dAtA[i] = 0x9 i++ - i = encodeFixed64Internal(dAtA, i, uint64(m.Uid)) + binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.Uid)) + i += 8 } if len(m.Value) > 0 { dAtA[i] = 0x12 @@ -4735,7 +4740,8 @@ func (m *MapEntry) MarshalTo(dAtA []byte) (int, error) { if m.Uid != 0 { dAtA[i] = 0x11 i++ - i = encodeFixed64Internal(dAtA, i, uint64(m.Uid)) + binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.Uid)) + i += 8 } if m.Posting != nil { dAtA[i] = 0x1a @@ -5001,24 +5007,6 @@ func (m *SnapshotMeta) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func encodeFixed64Internal(dAtA []byte, offset int, v uint64) int { - dAtA[offset] = uint8(v) - dAtA[offset+1] = uint8(v >> 8) - dAtA[offset+2] = uint8(v >> 16) - dAtA[offset+3] = uint8(v >> 24) - dAtA[offset+4] = uint8(v >> 32) - dAtA[offset+5] = uint8(v >> 40) - dAtA[offset+6] = uint8(v >> 48) - dAtA[offset+7] = uint8(v >> 56) - return offset + 8 -} -func encodeFixed32Internal(dAtA []byte, offset int, v uint32) int { - dAtA[offset] = uint8(v) - dAtA[offset+1] = uint8(v >> 8) - dAtA[offset+2] = uint8(v >> 16) - dAtA[offset+3] = uint8(v >> 24) - return offset + 4 -} func encodeVarintInternal(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -5339,8 +5327,8 @@ func (m *Group) Size() (n int) { func (m *ZeroProposal) Size() (n int) { var l int _ = l - if m.Id != 0 { - n += 1 + sovInternal(uint64(m.Id)) + if m.DeprecatedId != 0 { + n += 1 + sovInternal(uint64(m.DeprecatedId)) } if m.Member != nil { l = m.Member.Size() @@ -5363,6 +5351,10 @@ func (m *ZeroProposal) Size() (n int) { l = m.Txn.Size() n += 1 + l + sovInternal(uint64(l)) } + l = len(m.Key) + if l > 0 { + n += 1 + l + sovInternal(uint64(l)) + } return n } @@ -5998,7 +5990,15 @@ func (m *List) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType == 2 { + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Uids = append(m.Uids, v) + } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { @@ -6026,32 +6026,10 @@ func (m *List) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } + v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 - v = uint64(dAtA[iNdEx-8]) - v |= uint64(dAtA[iNdEx-7]) << 8 - v |= uint64(dAtA[iNdEx-6]) << 16 - v |= uint64(dAtA[iNdEx-5]) << 24 - v |= uint64(dAtA[iNdEx-4]) << 32 - v |= uint64(dAtA[iNdEx-3]) << 40 - v |= uint64(dAtA[iNdEx-2]) << 48 - v |= uint64(dAtA[iNdEx-1]) << 56 m.Uids = append(m.Uids, v) } - } else if wireType == 1 { - var v uint64 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - iNdEx += 8 - v = uint64(dAtA[iNdEx-8]) - v |= uint64(dAtA[iNdEx-7]) << 8 - v |= uint64(dAtA[iNdEx-6]) << 16 - v |= uint64(dAtA[iNdEx-5]) << 24 - v |= uint64(dAtA[iNdEx-4]) << 32 - v |= uint64(dAtA[iNdEx-3]) << 40 - v |= uint64(dAtA[iNdEx-2]) << 48 - v |= uint64(dAtA[iNdEx-1]) << 56 - m.Uids = append(m.Uids, v) } else { return fmt.Errorf("proto: wrong wireType = %d for field Uids", wireType) } @@ -6399,15 +6377,8 @@ func (m *Query) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } + m.AfterUid = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 - m.AfterUid = uint64(dAtA[iNdEx-8]) - m.AfterUid |= uint64(dAtA[iNdEx-7]) << 8 - m.AfterUid |= uint64(dAtA[iNdEx-6]) << 16 - m.AfterUid |= uint64(dAtA[iNdEx-5]) << 24 - m.AfterUid |= uint64(dAtA[iNdEx-4]) << 32 - m.AfterUid |= uint64(dAtA[iNdEx-3]) << 40 - m.AfterUid |= uint64(dAtA[iNdEx-2]) << 48 - m.AfterUid |= uint64(dAtA[iNdEx-1]) << 56 case 4: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field DoCount", wireType) @@ -6925,7 +6896,24 @@ func (m *Result) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 3: - if wireType == 2 { + if wireType == 0 { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Counts = append(m.Counts, v) + } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { @@ -6966,23 +6954,6 @@ func (m *Result) Unmarshal(dAtA []byte) error { } m.Counts = append(m.Counts, v) } - } else if wireType == 0 { - var v uint32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (uint32(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.Counts = append(m.Counts, v) } else { return fmt.Errorf("proto: wrong wireType = %d for field Counts", wireType) } @@ -7623,15 +7594,8 @@ func (m *RaftContext) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } + m.Id = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 - m.Id = uint64(dAtA[iNdEx-8]) - m.Id |= uint64(dAtA[iNdEx-7]) << 8 - m.Id |= uint64(dAtA[iNdEx-6]) << 16 - m.Id |= uint64(dAtA[iNdEx-5]) << 24 - m.Id |= uint64(dAtA[iNdEx-4]) << 32 - m.Id |= uint64(dAtA[iNdEx-3]) << 40 - m.Id |= uint64(dAtA[iNdEx-2]) << 48 - m.Id |= uint64(dAtA[iNdEx-1]) << 56 case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Group", wireType) @@ -7757,15 +7721,8 @@ func (m *Member) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } + m.Id = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 - m.Id = uint64(dAtA[iNdEx-8]) - m.Id |= uint64(dAtA[iNdEx-7]) << 8 - m.Id |= uint64(dAtA[iNdEx-6]) << 16 - m.Id |= uint64(dAtA[iNdEx-5]) << 24 - m.Id |= uint64(dAtA[iNdEx-4]) << 32 - m.Id |= uint64(dAtA[iNdEx-3]) << 40 - m.Id |= uint64(dAtA[iNdEx-2]) << 48 - m.Id |= uint64(dAtA[iNdEx-1]) << 56 case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field GroupId", wireType) @@ -7969,41 +7926,14 @@ func (m *Group) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - var keykey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - keykey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - var mapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapkey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } if m.Members == nil { m.Members = make(map[uint64]*Member) } - if iNdEx < postIndex { - var valuekey uint64 + var mapkey uint64 + var mapvalue *Member + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowInternal @@ -8013,46 +7943,74 @@ func (m *Group) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - valuekey |= (uint64(b) & 0x7F) << shift + wire |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLengthInternal + } + postmsgIndex := iNdEx + mapmsglen + if mapmsglen < 0 { + return ErrInvalidLengthInternal + } + if postmsgIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break + mapvalue = &Member{} + if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { + return err } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipInternal(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthInternal + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy } - if mapmsglen < 0 { - return ErrInvalidLengthInternal - } - postmsgIndex := iNdEx + mapmsglen - if mapmsglen < 0 { - return ErrInvalidLengthInternal - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue := &Member{} - if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - m.Members[mapkey] = mapvalue - } else { - var mapvalue *Member - m.Members[mapkey] = mapvalue } + m.Members[mapkey] = mapvalue iNdEx = postIndex case 2: if wireType != 2 { @@ -8080,51 +8038,14 @@ func (m *Group) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - var keykey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - keykey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLenmapkey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLengthInternal - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey := string(dAtA[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey if m.Tablets == nil { m.Tablets = make(map[string]*Tablet) } - if iNdEx < postIndex { - var valuekey uint64 + var mapkey string + var mapvalue *Tablet + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowInternal @@ -8134,46 +8055,85 @@ func (m *Group) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - valuekey |= (uint64(b) & 0x7F) << shift + wire |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLengthInternal + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } } + if mapmsglen < 0 { + return ErrInvalidLengthInternal + } + postmsgIndex := iNdEx + mapmsglen + if mapmsglen < 0 { + return ErrInvalidLengthInternal + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &Tablet{} + if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipInternal(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthInternal + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy } - if mapmsglen < 0 { - return ErrInvalidLengthInternal - } - postmsgIndex := iNdEx + mapmsglen - if mapmsglen < 0 { - return ErrInvalidLengthInternal - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue := &Tablet{} - if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - m.Tablets[mapkey] = mapvalue - } else { - var mapvalue *Tablet - m.Tablets[mapkey] = mapvalue } + m.Tablets[mapkey] = mapvalue iNdEx = postIndex default: iNdEx = preIndex @@ -8227,9 +8187,9 @@ func (m *ZeroProposal) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedId", wireType) } - m.Id = 0 + m.DeprecatedId = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowInternal @@ -8239,7 +8199,7 @@ func (m *ZeroProposal) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Id |= (uint32(b) & 0x7F) << shift + m.DeprecatedId |= (uint32(b) & 0x7F) << shift if b < 0x80 { break } @@ -8400,19 +8360,48 @@ func (m *ZeroProposal) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipInternal(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthInternal - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) } - iNdEx += skippy + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthInternal + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipInternal(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthInternal + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy } } @@ -8495,41 +8484,14 @@ func (m *MembershipState) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - var keykey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - keykey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - var mapkey uint32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapkey |= (uint32(b) & 0x7F) << shift - if b < 0x80 { - break - } - } if m.Groups == nil { m.Groups = make(map[uint32]*Group) } - if iNdEx < postIndex { - var valuekey uint64 + var mapkey uint32 + var mapvalue *Group + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowInternal @@ -8539,46 +8501,74 @@ func (m *MembershipState) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - valuekey |= (uint64(b) & 0x7F) << shift + wire |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLengthInternal + } + postmsgIndex := iNdEx + mapmsglen + if mapmsglen < 0 { + return ErrInvalidLengthInternal + } + if postmsgIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break + mapvalue = &Group{} + if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { + return err } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipInternal(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthInternal + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy } - if mapmsglen < 0 { - return ErrInvalidLengthInternal - } - postmsgIndex := iNdEx + mapmsglen - if mapmsglen < 0 { - return ErrInvalidLengthInternal - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue := &Group{} - if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - m.Groups[mapkey] = mapvalue - } else { - var mapvalue *Group - m.Groups[mapkey] = mapvalue } + m.Groups[mapkey] = mapvalue iNdEx = postIndex case 3: if wireType != 2 { @@ -8606,41 +8596,14 @@ func (m *MembershipState) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - var keykey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - keykey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - var mapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapkey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } if m.Zeros == nil { m.Zeros = make(map[uint64]*Member) } - if iNdEx < postIndex { - var valuekey uint64 + var mapkey uint64 + var mapvalue *Member + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowInternal @@ -8650,46 +8613,74 @@ func (m *MembershipState) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - valuekey |= (uint64(b) & 0x7F) << shift + wire |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLengthInternal + } + postmsgIndex := iNdEx + mapmsglen + if mapmsglen < 0 { + return ErrInvalidLengthInternal + } + if postmsgIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break + mapvalue = &Member{} + if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { + return err } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipInternal(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthInternal + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy } - if mapmsglen < 0 { - return ErrInvalidLengthInternal - } - postmsgIndex := iNdEx + mapmsglen - if mapmsglen < 0 { - return ErrInvalidLengthInternal - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue := &Member{} - if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - m.Zeros[mapkey] = mapvalue - } else { - var mapvalue *Member - m.Zeros[mapkey] = mapvalue } + m.Zeros[mapkey] = mapvalue iNdEx = postIndex case 4: if wireType != 0 { @@ -9149,15 +9140,8 @@ func (m *DirectedEdge) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } + m.Entity = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 - m.Entity = uint64(dAtA[iNdEx-8]) - m.Entity |= uint64(dAtA[iNdEx-7]) << 8 - m.Entity |= uint64(dAtA[iNdEx-6]) << 16 - m.Entity |= uint64(dAtA[iNdEx-5]) << 24 - m.Entity |= uint64(dAtA[iNdEx-4]) << 32 - m.Entity |= uint64(dAtA[iNdEx-3]) << 40 - m.Entity |= uint64(dAtA[iNdEx-2]) << 48 - m.Entity |= uint64(dAtA[iNdEx-1]) << 56 case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Attr", wireType) @@ -9245,15 +9229,8 @@ func (m *DirectedEdge) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } + m.ValueId = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 - m.ValueId = uint64(dAtA[iNdEx-8]) - m.ValueId |= uint64(dAtA[iNdEx-7]) << 8 - m.ValueId |= uint64(dAtA[iNdEx-6]) << 16 - m.ValueId |= uint64(dAtA[iNdEx-5]) << 24 - m.ValueId |= uint64(dAtA[iNdEx-4]) << 32 - m.ValueId |= uint64(dAtA[iNdEx-3]) << 40 - m.ValueId |= uint64(dAtA[iNdEx-2]) << 48 - m.ValueId |= uint64(dAtA[iNdEx-1]) << 56 case 6: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Label", wireType) @@ -10191,15 +10168,8 @@ func (m *Posting) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } + m.Uid = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 - m.Uid = uint64(dAtA[iNdEx-8]) - m.Uid |= uint64(dAtA[iNdEx-7]) << 8 - m.Uid |= uint64(dAtA[iNdEx-6]) << 16 - m.Uid |= uint64(dAtA[iNdEx-5]) << 24 - m.Uid |= uint64(dAtA[iNdEx-4]) << 32 - m.Uid |= uint64(dAtA[iNdEx-3]) << 40 - m.Uid |= uint64(dAtA[iNdEx-2]) << 48 - m.Uid |= uint64(dAtA[iNdEx-1]) << 56 case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) @@ -11753,15 +11723,8 @@ func (m *MapEntry) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } + m.Uid = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 - m.Uid = uint64(dAtA[iNdEx-8]) - m.Uid |= uint64(dAtA[iNdEx-7]) << 8 - m.Uid |= uint64(dAtA[iNdEx-6]) << 16 - m.Uid |= uint64(dAtA[iNdEx-5]) << 24 - m.Uid |= uint64(dAtA[iNdEx-4]) << 32 - m.Uid |= uint64(dAtA[iNdEx-3]) << 40 - m.Uid |= uint64(dAtA[iNdEx-2]) << 48 - m.Uid |= uint64(dAtA[iNdEx-1]) << 56 case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Posting", wireType) @@ -12147,41 +12110,14 @@ func (m *OracleDelta) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - var keykey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - keykey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - var mapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapkey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } if m.Commits == nil { m.Commits = make(map[uint64]uint64) } - if iNdEx < postIndex { - var valuekey uint64 + var mapkey uint64 + var mapvalue uint64 + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowInternal @@ -12191,12 +12127,62 @@ func (m *OracleDelta) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - valuekey |= (uint64(b) & 0x7F) << shift + wire |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - var mapvalue uint64 + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + } else if fieldNum == 2 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapvalue |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + } else { + iNdEx = entryPreIndex + skippy, err := skipInternal(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthInternal + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.Commits[mapkey] = mapvalue + iNdEx = postIndex + case 2: + if wireType == 0 { + var v uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowInternal @@ -12206,19 +12192,13 @@ func (m *OracleDelta) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - mapvalue |= (uint64(b) & 0x7F) << shift + v |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - m.Commits[mapkey] = mapvalue - } else { - var mapvalue uint64 - m.Commits[mapkey] = mapvalue - } - iNdEx = postIndex - case 2: - if wireType == 2 { + m.Aborts = append(m.Aborts, v) + } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { @@ -12259,23 +12239,6 @@ func (m *OracleDelta) Unmarshal(dAtA []byte) error { } m.Aborts = append(m.Aborts, v) } - } else if wireType == 0 { - var v uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.Aborts = append(m.Aborts, v) } else { return fmt.Errorf("proto: wrong wireType = %d for field Aborts", wireType) } @@ -12349,7 +12312,24 @@ func (m *TxnTimestamps) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType == 2 { + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Ts = append(m.Ts, v) + } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { @@ -12390,23 +12370,6 @@ func (m *TxnTimestamps) Unmarshal(dAtA []byte) error { } m.Ts = append(m.Ts, v) } - } else if wireType == 0 { - var v uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInternal - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.Ts = append(m.Ts, v) } else { return fmt.Errorf("proto: wrong wireType = %d for field Ts", wireType) } @@ -12766,194 +12729,194 @@ var ( func init() { proto.RegisterFile("internal.proto", fileDescriptorInternal) } var fileDescriptorInternal = []byte{ - // 3013 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x39, 0xcb, 0x6e, 0x23, 0xc7, - 0xb5, 0xec, 0x26, 0xd9, 0x6c, 0x1e, 0x92, 0x1a, 0xba, 0x3c, 0x9e, 0xa1, 0x69, 0x5f, 0x59, 0xae, - 0xf1, 0xf5, 0xc8, 0x2f, 0xd9, 0x23, 0x8f, 0x1f, 0x77, 0xee, 0xf5, 0x0d, 0x64, 0x91, 0x1a, 0xd3, - 0xa3, 0x97, 0x8b, 0x94, 0x1c, 0x67, 0x11, 0xa2, 0xc4, 0x2e, 0x51, 0x0d, 0x35, 0xbb, 0xdb, 0x5d, - 0x4d, 0x81, 0xf2, 0x32, 0xbb, 0x20, 0x3f, 0xe0, 0x75, 0x80, 0xec, 0x02, 0x04, 0xc8, 0x32, 0x8b, - 0xac, 0x12, 0x20, 0x8b, 0x00, 0x49, 0xfe, 0x20, 0x71, 0x36, 0x41, 0xb2, 0xca, 0x1f, 0x04, 0xf5, - 0xe8, 0x07, 0x29, 0x8e, 0x2c, 0xe4, 0xb1, 0x62, 0x9f, 0x53, 0xe7, 0xd4, 0xe3, 0xbc, 0xcf, 0x21, - 0xac, 0xb8, 0x7e, 0xcc, 0x22, 0x9f, 0x7a, 0x1b, 0x61, 0x14, 0xc4, 0x01, 0xb2, 0x14, 0xdc, 0xae, - 0xd2, 0xd0, 0x55, 0x28, 0xdc, 0x86, 0xd2, 0xae, 0xcb, 0x63, 0x84, 0xa0, 0x34, 0x75, 0x1d, 0xde, - 0x32, 0xd6, 0x8a, 0xeb, 0x16, 0x91, 0xdf, 0xf8, 0x33, 0xa8, 0x0e, 0x28, 0x3f, 0x3f, 0xa6, 0xde, - 0x94, 0xa1, 0x26, 0x14, 0x2f, 0xa8, 0xd7, 0x32, 0xd6, 0x8c, 0xf5, 0x3a, 0x11, 0x9f, 0x68, 0x13, - 0xec, 0x0b, 0xea, 0x0d, 0xe3, 0xcb, 0x90, 0xb5, 0xcc, 0x35, 0x63, 0x7d, 0x65, 0xf3, 0xee, 0x86, - 0x3a, 0x60, 0xe3, 0x30, 0xe0, 0xb1, 0xeb, 0x8f, 0x37, 0x8e, 0xa9, 0x37, 0xb8, 0x0c, 0x19, 0xa9, - 0x5c, 0xa8, 0x0f, 0x7c, 0x00, 0xb5, 0x7e, 0x34, 0xda, 0x99, 0xfa, 0xa3, 0xd8, 0x0d, 0x7c, 0x71, - 0xaa, 0x4f, 0x27, 0x4c, 0xee, 0x5a, 0x25, 0xf2, 0x5b, 0xe0, 0x68, 0x34, 0xe6, 0xad, 0xe2, 0x5a, - 0x51, 0xe0, 0xc4, 0x37, 0x6a, 0x41, 0xc5, 0xe5, 0xdb, 0xc1, 0xd4, 0x8f, 0x5b, 0xa5, 0x35, 0x63, - 0xdd, 0x26, 0x09, 0x88, 0x7f, 0x5a, 0x84, 0xf2, 0x67, 0x53, 0x16, 0x5d, 0x4a, 0xbe, 0x38, 0x8e, - 0x92, 0xbd, 0xc4, 0x37, 0xba, 0x0d, 0x65, 0x8f, 0xfa, 0x63, 0xde, 0x32, 0xe5, 0x66, 0x0a, 0x40, - 0x2f, 0x40, 0x95, 0x9e, 0xc6, 0x2c, 0x1a, 0x4e, 0x5d, 0xa7, 0x55, 0x5c, 0x33, 0xd6, 0x2d, 0x62, - 0x4b, 0xc4, 0x91, 0xeb, 0xa0, 0xe7, 0xc1, 0x76, 0x82, 0xe1, 0x28, 0x7f, 0x96, 0x13, 0xc8, 0xb3, - 0xd0, 0x7d, 0xb0, 0xa7, 0xae, 0x33, 0xf4, 0x5c, 0x1e, 0xb7, 0xca, 0x6b, 0xc6, 0x7a, 0x6d, 0xb3, - 0x9e, 0x3c, 0x58, 0xc8, 0x90, 0x54, 0xa6, 0xae, 0x23, 0x85, 0xb9, 0x01, 0x36, 0x8f, 0x46, 0xc3, - 0xd3, 0xa9, 0x3f, 0x6a, 0x59, 0x92, 0xf0, 0xd9, 0x84, 0x30, 0xf7, 0x7a, 0x52, 0xe1, 0x0a, 0x10, - 0xcf, 0x8b, 0xd8, 0x05, 0x8b, 0x38, 0x6b, 0x55, 0xd4, 0x91, 0x1a, 0x44, 0x0f, 0xa1, 0x76, 0x4a, - 0x47, 0x2c, 0x1e, 0x86, 0x34, 0xa2, 0x93, 0x96, 0x3d, 0xbf, 0xd9, 0x8e, 0x58, 0x3a, 0x14, 0x2b, - 0x9c, 0xc0, 0x69, 0x0a, 0xa0, 0x0f, 0xa0, 0x21, 0x21, 0x3e, 0x3c, 0x75, 0xbd, 0x98, 0x45, 0xad, - 0xaa, 0xe4, 0x43, 0x29, 0x9f, 0xc4, 0x0e, 0x22, 0xc6, 0x48, 0x5d, 0x11, 0x2a, 0x0c, 0xfa, 0x2f, - 0x00, 0x36, 0x0b, 0xa9, 0xef, 0x0c, 0xa9, 0xe7, 0xb5, 0x40, 0xde, 0xa5, 0xaa, 0x30, 0x5b, 0x9e, - 0x87, 0xee, 0x8a, 0x7b, 0x52, 0x67, 0x18, 0xf3, 0x56, 0x63, 0xcd, 0x58, 0x2f, 0x11, 0x4b, 0x80, - 0x03, 0x2e, 0x24, 0xe3, 0xb9, 0xfe, 0x50, 0x40, 0xad, 0x15, 0x2d, 0x19, 0x61, 0x63, 0xbb, 0xae, - 0x4f, 0x18, 0x75, 0x48, 0xc5, 0x53, 0x1f, 0xf8, 0x7d, 0xa8, 0x4a, 0x73, 0x92, 0x62, 0x7a, 0x0d, - 0xac, 0x0b, 0x01, 0x28, 0xab, 0xab, 0x6d, 0x3e, 0x93, 0xdc, 0x2f, 0xb5, 0x3a, 0xa2, 0x09, 0xf0, - 0x2a, 0xd8, 0xbb, 0xd4, 0x1f, 0x27, 0xa6, 0x2a, 0xf4, 0x28, 0x99, 0xaa, 0x44, 0x7e, 0xe3, 0x3f, - 0x98, 0x60, 0x11, 0xc6, 0xa7, 0x5e, 0x8c, 0xde, 0x00, 0x10, 0x5a, 0x9a, 0xd0, 0x38, 0x72, 0x67, - 0x7a, 0xe7, 0x79, 0x3d, 0x55, 0xa7, 0xae, 0xb3, 0x27, 0x97, 0xd1, 0x43, 0xa8, 0xcb, 0x13, 0x12, - 0x72, 0x73, 0xfe, 0x22, 0xe9, 0x5d, 0x49, 0x4d, 0x92, 0x69, 0xae, 0x3b, 0x60, 0x49, 0x03, 0x51, - 0x46, 0xda, 0x20, 0x1a, 0x42, 0xff, 0xad, 0x3d, 0x8e, 0xb3, 0x51, 0x3c, 0x74, 0x18, 0x4f, 0x2c, - 0xa8, 0x91, 0x62, 0x3b, 0x8c, 0xc7, 0xe8, 0x3d, 0x50, 0x52, 0x4f, 0x0e, 0x2d, 0xcb, 0x43, 0xd1, - 0x9c, 0x56, 0xb9, 0x3a, 0x55, 0xd2, 0xe9, 0x53, 0x1f, 0x40, 0x4d, 0xbc, 0x35, 0xe1, 0xb2, 0x24, - 0x57, 0x33, 0x7d, 0x99, 0x16, 0x0f, 0x01, 0x41, 0xa4, 0x59, 0x84, 0xa8, 0x84, 0xb5, 0x2a, 0xab, - 0x92, 0xdf, 0x37, 0xd7, 0x55, 0x17, 0xca, 0x07, 0x91, 0xc3, 0xa2, 0xa5, 0x9e, 0x85, 0xa0, 0xe4, - 0x30, 0x3e, 0x92, 0x8e, 0x6f, 0x13, 0xf9, 0x9d, 0x79, 0x5b, 0x31, 0xe7, 0x6d, 0xf8, 0x77, 0x06, - 0xd4, 0xfa, 0x41, 0x14, 0xef, 0x31, 0xce, 0xe9, 0x98, 0xa1, 0x7b, 0x50, 0x0e, 0xc4, 0xb6, 0x5a, - 0x35, 0x8d, 0xe4, 0x01, 0xf2, 0x2c, 0xa2, 0xd6, 0x16, 0x94, 0x68, 0x5e, 0xaf, 0xc4, 0xdb, 0x50, - 0x56, 0xfe, 0x2a, 0x7c, 0xb9, 0x4c, 0x14, 0x20, 0x94, 0x14, 0x9c, 0x9e, 0x72, 0xa6, 0x94, 0x50, - 0x26, 0x1a, 0xfa, 0x37, 0x18, 0xf1, 0x09, 0x80, 0x78, 0xd0, 0x3f, 0x63, 0x6f, 0x37, 0x3e, 0xe3, - 0x0c, 0x6a, 0x84, 0x9e, 0xc6, 0xdb, 0x81, 0x1f, 0xb3, 0x59, 0x8c, 0x56, 0xc0, 0x74, 0x1d, 0xa9, - 0x00, 0x8b, 0x98, 0xae, 0x23, 0x9e, 0x3c, 0x8e, 0x82, 0x69, 0x28, 0xe5, 0xdf, 0x20, 0x0a, 0x90, - 0x8a, 0x72, 0x9c, 0x48, 0xca, 0x41, 0x28, 0xca, 0x71, 0x22, 0xf4, 0x12, 0xd4, 0xb8, 0x4f, 0x43, - 0x7e, 0x16, 0xc4, 0xe2, 0xc9, 0x25, 0xf9, 0x64, 0x48, 0x50, 0x03, 0x8e, 0x7f, 0x6d, 0x80, 0xb5, - 0xc7, 0x26, 0x27, 0x2c, 0xba, 0x72, 0xca, 0xf3, 0x60, 0xcb, 0x8d, 0x87, 0xae, 0xa3, 0x0f, 0xaa, - 0x48, 0xb8, 0xe7, 0x2c, 0x3d, 0xea, 0x0e, 0x58, 0x1e, 0xa3, 0x42, 0xb5, 0xca, 0xec, 0x35, 0x24, - 0x24, 0x4e, 0x27, 0x43, 0x47, 0xbc, 0xb9, 0xac, 0x16, 0xe8, 0xa4, 0xc3, 0xa8, 0x23, 0xee, 0xe6, - 0x51, 0x1e, 0x0f, 0xa7, 0xa1, 0x43, 0x63, 0x26, 0x43, 0x65, 0x49, 0xd8, 0x2f, 0x8f, 0x8f, 0x24, - 0x06, 0xbd, 0x0e, 0xcf, 0x8c, 0xbc, 0x29, 0x17, 0xb1, 0xda, 0xf5, 0x4f, 0x83, 0x61, 0xe0, 0x7b, - 0x97, 0x52, 0x6b, 0x36, 0xb9, 0xa5, 0x17, 0x7a, 0xfe, 0x69, 0x70, 0xe0, 0x7b, 0x97, 0xf8, 0x47, - 0x26, 0x94, 0x1f, 0x4b, 0x31, 0x3c, 0x84, 0xca, 0x44, 0x3e, 0x28, 0x09, 0x2c, 0xed, 0x44, 0x1d, - 0x72, 0x7d, 0x43, 0xbd, 0x96, 0x77, 0xfd, 0x38, 0xba, 0x24, 0x09, 0xa9, 0xe0, 0x8a, 0xe9, 0x89, - 0xc7, 0x62, 0xae, 0xed, 0x6d, 0x81, 0x6b, 0xa0, 0x16, 0x35, 0x97, 0x26, 0x6d, 0x7f, 0x0a, 0xf5, - 0xfc, 0x76, 0x22, 0x4d, 0x9e, 0xb3, 0x4b, 0x29, 0xc3, 0x12, 0x11, 0x9f, 0xe8, 0x15, 0x28, 0xcb, - 0xd8, 0x21, 0x25, 0x58, 0xdb, 0x5c, 0x49, 0x76, 0x55, 0x6c, 0x44, 0x2d, 0x3e, 0x32, 0x3f, 0x34, - 0xc4, 0x5e, 0xf9, 0x43, 0xf2, 0x7b, 0x55, 0xaf, 0xdf, 0x4b, 0xb1, 0xe5, 0xf6, 0xc2, 0x7f, 0x33, - 0xa0, 0xfe, 0x3d, 0x16, 0x05, 0x87, 0x51, 0x10, 0x06, 0x9c, 0x7a, 0x39, 0xdd, 0x36, 0xa4, 0x6e, - 0x5f, 0x05, 0x4b, 0xbd, 0xfc, 0x29, 0xf7, 0xd2, 0xab, 0x82, 0x4e, 0xbd, 0x55, 0xaa, 0xfa, 0xea, - 0x99, 0x7a, 0x15, 0xad, 0x02, 0x4c, 0xe8, 0x6c, 0x97, 0x51, 0xce, 0x7a, 0x4e, 0x62, 0x66, 0x19, - 0x06, 0xb5, 0xc1, 0x9e, 0xd0, 0xd9, 0x60, 0xe6, 0x0f, 0xb8, 0xb4, 0x82, 0x12, 0x49, 0x61, 0xf4, - 0x22, 0x54, 0x27, 0x74, 0x26, 0xec, 0xbd, 0xe7, 0x68, 0x2b, 0xc8, 0x10, 0xe8, 0x65, 0x28, 0xc6, - 0x33, 0x5f, 0xc6, 0xb0, 0xda, 0xe6, 0x2d, 0xe9, 0x2e, 0x83, 0x99, 0xaf, 0x3d, 0x83, 0x88, 0x35, - 0xfc, 0x8b, 0x22, 0xdc, 0xd2, 0x6a, 0x38, 0x73, 0xc3, 0x7e, 0x2c, 0x6c, 0xa7, 0x05, 0x15, 0x19, - 0x08, 0x58, 0xa4, 0xb5, 0x91, 0x80, 0xe8, 0x7f, 0xc1, 0x92, 0x66, 0x9c, 0x28, 0xfa, 0xde, 0xfc, - 0xd3, 0xd3, 0x2d, 0x94, 0xe2, 0xb5, 0xc6, 0x35, 0x0b, 0xfa, 0x10, 0xca, 0x5f, 0xb1, 0x28, 0x50, - 0x41, 0xae, 0xb6, 0x89, 0x9f, 0xc6, 0x2b, 0x84, 0xaf, 0x59, 0x15, 0xc3, 0x7f, 0x50, 0x42, 0xeb, - 0x22, 0xa4, 0x4d, 0x82, 0x0b, 0xe6, 0xb4, 0x2a, 0xf2, 0x56, 0x8b, 0xca, 0x4c, 0x96, 0xdb, 0x9f, - 0x40, 0x2d, 0xf7, 0xa8, 0xbc, 0x85, 0x35, 0x94, 0x85, 0xdd, 0x9b, 0xb7, 0xb0, 0xc6, 0x9c, 0x0f, - 0xe4, 0x8d, 0xf5, 0x13, 0x80, 0xec, 0x89, 0xff, 0x8a, 0xd9, 0xe3, 0x1f, 0x1a, 0x70, 0x6b, 0x3b, - 0xf0, 0x7d, 0x26, 0xab, 0x22, 0xa5, 0xbc, 0xcc, 0x3a, 0x8d, 0x6b, 0xad, 0xf3, 0x2d, 0x28, 0x73, - 0xc1, 0xa0, 0x4f, 0xb9, 0xfb, 0x14, 0x6d, 0x10, 0x45, 0x25, 0x02, 0xce, 0x84, 0xce, 0x86, 0x21, - 0xf3, 0x1d, 0xd7, 0x1f, 0x4b, 0x8b, 0x56, 0x3a, 0x38, 0x54, 0x18, 0xfc, 0x63, 0x03, 0x2c, 0x65, - 0xd8, 0x73, 0xc1, 0xcf, 0x98, 0x0f, 0x7e, 0x2f, 0x42, 0x35, 0x8c, 0x98, 0xe3, 0x8e, 0x92, 0x93, - 0xab, 0x24, 0x43, 0x88, 0xd8, 0x7c, 0x1a, 0x44, 0x23, 0x26, 0xb7, 0xb7, 0x89, 0x02, 0x44, 0xd1, - 0x29, 0xd3, 0x8e, 0x0c, 0x61, 0x2a, 0x3e, 0xda, 0x02, 0x21, 0x62, 0x97, 0x60, 0xe1, 0x21, 0x1d, - 0xa9, 0xf2, 0xaf, 0x48, 0x14, 0x20, 0xe2, 0xa9, 0xd2, 0x9b, 0xac, 0xfb, 0x6c, 0xa2, 0x21, 0xfc, - 0x73, 0x13, 0xea, 0x1d, 0x37, 0x62, 0xa3, 0x98, 0x39, 0x5d, 0x67, 0x2c, 0x09, 0x99, 0x1f, 0xbb, - 0xf1, 0xa5, 0x8e, 0xdd, 0x1a, 0x4a, 0x13, 0xb7, 0x39, 0x5f, 0x12, 0x2b, 0xbd, 0x14, 0x65, 0x25, - 0xaf, 0x00, 0xf4, 0x3e, 0x80, 0xaa, 0x83, 0x64, 0x35, 0x5f, 0xba, 0xbe, 0x9a, 0xaf, 0x4a, 0x52, - 0xf1, 0x29, 0x84, 0xa4, 0xf8, 0x5c, 0x15, 0xdb, 0x2d, 0x59, 0xea, 0x4f, 0x85, 0x39, 0xcb, 0x6a, - 0xe0, 0x84, 0x79, 0xd2, 0x5c, 0x65, 0x35, 0x70, 0xc2, 0xbc, 0xb4, 0x78, 0xab, 0xa8, 0x2b, 0x89, - 0x6f, 0x74, 0x1f, 0xcc, 0x20, 0x94, 0x6f, 0xcc, 0x1d, 0x9a, 0x7f, 0xe0, 0xc6, 0x41, 0x48, 0xcc, - 0x20, 0x44, 0x18, 0x2c, 0x55, 0xae, 0xb6, 0xaa, 0xd2, 0xcc, 0x41, 0x06, 0x03, 0x59, 0x2f, 0x11, - 0xbd, 0x82, 0xef, 0x80, 0x79, 0x10, 0xa2, 0x0a, 0x14, 0xfb, 0xdd, 0x41, 0xb3, 0x20, 0x3e, 0x3a, - 0xdd, 0xdd, 0xa6, 0x81, 0xff, 0x6a, 0x40, 0x75, 0x6f, 0x1a, 0x53, 0x61, 0x63, 0xfc, 0x3a, 0xe5, - 0x3e, 0x0f, 0x36, 0x8f, 0x69, 0x24, 0xb3, 0xa5, 0xa9, 0x02, 0x87, 0x84, 0x07, 0x1c, 0xbd, 0x0e, - 0x65, 0xe6, 0x8c, 0x59, 0xe2, 0xfb, 0xb7, 0x97, 0xdd, 0x95, 0x28, 0x12, 0xf4, 0x26, 0x58, 0x7c, - 0x74, 0xc6, 0x26, 0xb4, 0x55, 0x9a, 0x27, 0xee, 0x4b, 0xac, 0x4a, 0x70, 0x44, 0xd3, 0xc8, 0xae, - 0x23, 0x0a, 0x42, 0x59, 0x76, 0x97, 0x75, 0xd7, 0x11, 0x05, 0xa1, 0x28, 0xba, 0x37, 0xe1, 0x39, - 0x77, 0xec, 0x07, 0x11, 0x1b, 0xba, 0xbe, 0xc3, 0x66, 0xc3, 0x51, 0xe0, 0x9f, 0x7a, 0xee, 0x28, - 0x96, 0x72, 0xb5, 0xc9, 0xb3, 0x6a, 0xb1, 0x27, 0xd6, 0xb6, 0xf5, 0x12, 0xbe, 0x0f, 0xd5, 0x27, - 0xec, 0x52, 0x56, 0xaf, 0x1c, 0xb5, 0xc1, 0x3c, 0xbf, 0xd0, 0x99, 0x10, 0x92, 0x5b, 0x3c, 0x39, - 0x26, 0xe6, 0xf9, 0x05, 0xfe, 0xda, 0x04, 0x3b, 0x4d, 0x11, 0xf7, 0xa0, 0xe1, 0xb0, 0x30, 0x62, - 0xc2, 0x8a, 0x9d, 0x4c, 0x32, 0xf5, 0x0c, 0xd9, 0x73, 0xd0, 0xdb, 0x50, 0x9d, 0x24, 0x62, 0xd4, - 0x5e, 0x97, 0x96, 0xcb, 0xa9, 0x7c, 0x49, 0x46, 0x83, 0xde, 0x81, 0x5a, 0x3c, 0xf3, 0xc5, 0xb5, - 0x45, 0xbc, 0xd6, 0x59, 0xe4, 0x4a, 0x18, 0x87, 0x38, 0xfd, 0xd6, 0x17, 0x2e, 0x2d, 0xbb, 0x70, - 0xe6, 0xf0, 0xe5, 0x1b, 0x39, 0xfc, 0x7d, 0xb8, 0x35, 0xf2, 0x18, 0xf5, 0x87, 0x99, 0xbf, 0x2a, - 0x73, 0x5c, 0x91, 0xe8, 0xc3, 0xd4, 0x69, 0x75, 0x00, 0xab, 0xa4, 0xb9, 0x16, 0xbf, 0x0c, 0xc5, - 0x27, 0xc7, 0xfd, 0x6b, 0xa5, 0xf7, 0x7d, 0x30, 0x9f, 0x1c, 0xe7, 0x63, 0x5f, 0x5d, 0xc5, 0x3e, - 0xdd, 0x2b, 0x9b, 0x59, 0xaf, 0xdc, 0x06, 0x7b, 0xca, 0x59, 0xb4, 0xc7, 0x62, 0xaa, 0x1d, 0x2f, - 0x85, 0x45, 0xa2, 0x12, 0xcd, 0x9e, 0x1b, 0xf8, 0x3a, 0x29, 0x24, 0x20, 0xfe, 0x7b, 0x11, 0x2a, - 0xda, 0xf9, 0xc4, 0x9e, 0xd3, 0xb4, 0x38, 0x13, 0x9f, 0x99, 0x27, 0x9b, 0x79, 0x4f, 0xce, 0x77, - 0xe5, 0xc5, 0x9b, 0x75, 0xe5, 0xe8, 0xff, 0xa1, 0x1e, 0xaa, 0xb5, 0xbc, 0xff, 0xbf, 0xb0, 0xc8, - 0xa7, 0x7f, 0x25, 0x6f, 0x2d, 0xcc, 0x00, 0x61, 0xbd, 0xb2, 0x33, 0x89, 0xe9, 0x58, 0xea, 0xa5, - 0x4e, 0x2a, 0x02, 0x1e, 0xd0, 0xf1, 0x53, 0xa2, 0xc0, 0x0d, 0x1c, 0x59, 0x14, 0x2c, 0x41, 0xd8, - 0xaa, 0xab, 0x82, 0x25, 0x08, 0xe7, 0xfc, 0xb2, 0x31, 0xef, 0x97, 0x2f, 0x40, 0x75, 0x14, 0x4c, - 0x26, 0xae, 0x5c, 0x5b, 0x51, 0xa9, 0x53, 0x21, 0x06, 0x1c, 0x7f, 0x05, 0x15, 0xfd, 0x60, 0x54, - 0x83, 0x4a, 0xa7, 0xbb, 0xb3, 0x75, 0xb4, 0x2b, 0x22, 0x03, 0x80, 0xf5, 0x71, 0x6f, 0x7f, 0x8b, - 0x7c, 0xd1, 0x34, 0x44, 0x94, 0xe8, 0xed, 0x0f, 0x9a, 0x26, 0xaa, 0x42, 0x79, 0x67, 0xf7, 0x60, - 0x6b, 0xd0, 0x2c, 0x22, 0x1b, 0x4a, 0x1f, 0x1f, 0x1c, 0xec, 0x36, 0x4b, 0xa8, 0x0e, 0x76, 0x67, - 0x6b, 0xd0, 0x1d, 0xf4, 0xf6, 0xba, 0xcd, 0xb2, 0xa0, 0x7d, 0xdc, 0x3d, 0x68, 0x5a, 0xe2, 0xe3, - 0xa8, 0xd7, 0x69, 0x56, 0xc4, 0xfa, 0xe1, 0x56, 0xbf, 0xff, 0xf9, 0x01, 0xe9, 0x34, 0x6d, 0xb1, - 0x6f, 0x7f, 0x40, 0x7a, 0xfb, 0x8f, 0x9b, 0x55, 0xfc, 0x00, 0x6a, 0x39, 0xa1, 0x09, 0x0e, 0xd2, - 0xdd, 0x69, 0x16, 0xc4, 0x31, 0xc7, 0x5b, 0xbb, 0x47, 0xdd, 0xa6, 0x81, 0x56, 0x00, 0xe4, 0xe7, - 0x70, 0x77, 0x6b, 0xff, 0x71, 0xd3, 0xc4, 0x3f, 0x30, 0x52, 0x1e, 0xd9, 0xed, 0xbe, 0x01, 0xb6, - 0x16, 0x75, 0x52, 0xcd, 0xde, 0x5a, 0xd0, 0x0b, 0x49, 0x09, 0x84, 0x99, 0x8d, 0xce, 0xd8, 0xe8, - 0x9c, 0x4f, 0x27, 0xda, 0x2a, 0x52, 0x58, 0x35, 0xad, 0x42, 0x26, 0x3a, 0xed, 0x69, 0x28, 0x9d, - 0xfc, 0x94, 0x24, 0xbd, 0x9a, 0xfc, 0x3c, 0x04, 0xc8, 0x66, 0x0b, 0x4b, 0xea, 0xd0, 0xdb, 0x50, - 0xa6, 0x9e, 0x4b, 0xb9, 0xce, 0x2c, 0x0a, 0xc0, 0x04, 0x6a, 0xb9, 0x89, 0x84, 0x50, 0x18, 0xf5, - 0xbc, 0xe1, 0x39, 0xbb, 0xe4, 0x92, 0xd7, 0x26, 0x15, 0xea, 0x79, 0x4f, 0xd8, 0x25, 0x47, 0xeb, - 0x50, 0x56, 0x03, 0x0d, 0x73, 0x49, 0xeb, 0x2b, 0xd9, 0x89, 0x22, 0xc0, 0x6f, 0x82, 0xa5, 0xfa, - 0xe1, 0x9c, 0xcd, 0x18, 0x4f, 0x0d, 0xfe, 0x1f, 0xe9, 0x7b, 0xcb, 0xee, 0x19, 0xbd, 0xad, 0x87, - 0x27, 0x5c, 0x8d, 0x6c, 0x8c, 0xf9, 0xd2, 0x48, 0x11, 0xea, 0xb9, 0x89, 0x64, 0xc0, 0x1d, 0xb0, - 0xaf, 0x1d, 0x4d, 0x69, 0x41, 0x98, 0x99, 0x20, 0x96, 0x0c, 0xab, 0x70, 0x04, 0x90, 0x0d, 0x58, - 0xb4, 0x19, 0xab, 0x5d, 0x84, 0x19, 0x6f, 0x08, 0x15, 0xb9, 0x9e, 0x13, 0x31, 0xff, 0xca, 0xeb, - 0xb3, 0xb1, 0x4c, 0x4a, 0x83, 0x5e, 0x81, 0x92, 0x9c, 0x23, 0xa9, 0xb8, 0x99, 0xb6, 0xfb, 0xe9, - 0x10, 0x49, 0xae, 0xe2, 0x13, 0x68, 0xa8, 0xbc, 0x42, 0xd8, 0x97, 0x53, 0xc6, 0xaf, 0xad, 0x5e, - 0x56, 0x01, 0xd2, 0x68, 0x98, 0x4c, 0xc6, 0x72, 0x18, 0x61, 0x28, 0xa7, 0x2e, 0xf3, 0x9c, 0xe4, - 0x55, 0x1a, 0xc2, 0x1f, 0x40, 0x3d, 0x39, 0x43, 0x36, 0xbe, 0xf7, 0xd3, 0x0c, 0x97, 0xd8, 0xa5, - 0x50, 0x88, 0x22, 0xd9, 0x0f, 0x9c, 0x34, 0xb9, 0xe1, 0x3f, 0x99, 0x09, 0xa7, 0x6e, 0xeb, 0xe6, - 0xea, 0x27, 0x63, 0xb1, 0x7e, 0x9a, 0xaf, 0x45, 0xcc, 0x1b, 0xd7, 0x22, 0xff, 0x07, 0x55, 0x47, - 0x26, 0x62, 0xf7, 0x22, 0x09, 0x7d, 0xab, 0xcb, 0x92, 0xae, 0x4e, 0xd7, 0xee, 0x05, 0x23, 0x19, - 0x83, 0xb8, 0x53, 0x1c, 0x9c, 0x33, 0xdf, 0xfd, 0x4a, 0xf6, 0xaf, 0xe2, 0xe1, 0x19, 0x22, 0x1b, - 0x31, 0xa8, 0xe4, 0xac, 0x47, 0x0c, 0xc9, 0x78, 0xc5, 0xca, 0x8d, 0x57, 0xee, 0x80, 0x35, 0x0d, - 0x39, 0x8b, 0xe2, 0xa4, 0x68, 0x53, 0x50, 0x5a, 0xf8, 0x54, 0x35, 0x2d, 0xf5, 0xc7, 0xf8, 0x7f, - 0xa0, 0x9a, 0xde, 0x45, 0xc4, 0x9b, 0xfd, 0x83, 0xfd, 0xae, 0x8a, 0x0e, 0xbd, 0xfd, 0x4e, 0xf7, - 0xbb, 0x4d, 0x43, 0x44, 0x2c, 0xd2, 0x3d, 0xee, 0x92, 0x7e, 0xb7, 0x69, 0x8a, 0xc8, 0xd2, 0xe9, - 0xee, 0x76, 0x07, 0xdd, 0x66, 0xf1, 0xd3, 0x92, 0x5d, 0x69, 0xda, 0xc4, 0x66, 0xb3, 0xd0, 0x73, - 0x47, 0x6e, 0x8c, 0xbf, 0x00, 0x7b, 0x8f, 0x86, 0x57, 0x8a, 0xf1, 0x2c, 0x21, 0x4d, 0x75, 0x0f, - 0xaf, 0x93, 0xc7, 0x6b, 0x50, 0xd1, 0x51, 0x23, 0xcd, 0xc8, 0x0b, 0x51, 0x25, 0x59, 0xc7, 0x3f, - 0x33, 0xe0, 0xf6, 0x5e, 0x70, 0xc1, 0xd2, 0x64, 0x79, 0x48, 0x2f, 0xbd, 0x80, 0x3a, 0xdf, 0xa2, - 0xc6, 0x57, 0xe1, 0x16, 0x0f, 0xa6, 0xd1, 0x88, 0x0d, 0x17, 0x66, 0x08, 0x0d, 0x85, 0x7e, 0xac, - 0xcd, 0x11, 0x8b, 0xaa, 0x83, 0xc7, 0x19, 0x55, 0x51, 0x52, 0xd5, 0x04, 0x32, 0xa1, 0x49, 0xb3, - 0x7e, 0xe9, 0x26, 0x59, 0x1f, 0xff, 0xd6, 0x80, 0x46, 0x77, 0x16, 0x06, 0x51, 0x9c, 0x5c, 0xf5, - 0x39, 0x51, 0x4a, 0x7f, 0x99, 0x38, 0x43, 0x89, 0x94, 0x23, 0xf6, 0x65, 0xef, 0xda, 0x01, 0xc7, - 0x43, 0xb0, 0xc4, 0x66, 0x53, 0xae, 0x4d, 0xe9, 0xc5, 0xe4, 0xcc, 0xb9, 0x8d, 0x37, 0xfa, 0x92, - 0x86, 0x68, 0xda, 0xfc, 0x70, 0xa9, 0x94, 0x1f, 0x2e, 0xe1, 0x47, 0x60, 0x29, 0xd2, 0x9c, 0x9e, - 0x6b, 0x50, 0xe9, 0x1f, 0x6d, 0x6f, 0x77, 0xfb, 0xfd, 0xa6, 0x81, 0x1a, 0x50, 0xed, 0x1c, 0x1d, - 0xee, 0xf6, 0xb6, 0xb7, 0x06, 0x5a, 0xd7, 0x3b, 0x5b, 0xbd, 0xdd, 0x6e, 0xa7, 0x59, 0xc4, 0xbf, - 0x34, 0xa0, 0x76, 0x10, 0xd1, 0x91, 0xc7, 0x3a, 0xcc, 0x8b, 0x29, 0x7a, 0x24, 0x3a, 0x5b, 0x11, - 0xbb, 0x93, 0x50, 0xb8, 0x96, 0xcd, 0xd0, 0x52, 0xaa, 0x8d, 0x6d, 0x45, 0xa2, 0xe7, 0x15, 0x9a, - 0x41, 0x98, 0x27, 0x3d, 0x09, 0x22, 0x3d, 0xe4, 0x28, 0x11, 0x0d, 0x7d, 0x6b, 0x67, 0xd4, 0x7e, - 0x04, 0xf5, 0xfc, 0x8e, 0x4b, 0x3a, 0xbe, 0xb9, 0x7a, 0xa4, 0x94, 0xef, 0xf0, 0x5e, 0x82, 0x86, - 0x68, 0x63, 0xdd, 0x09, 0xe3, 0x31, 0x9d, 0x84, 0x32, 0xb7, 0xeb, 0xcb, 0x97, 0x88, 0x19, 0x73, - 0xfc, 0x2a, 0xd4, 0x0f, 0x19, 0x8b, 0x08, 0xe3, 0x61, 0xe0, 0x73, 0xd9, 0xd0, 0x68, 0xe1, 0xab, - 0xc4, 0xa1, 0x21, 0x7c, 0x17, 0x8a, 0xfb, 0xd3, 0x49, 0xfe, 0xbf, 0x88, 0x92, 0xac, 0xaf, 0xf0, - 0x0e, 0xd4, 0xfb, 0x7a, 0xa4, 0x25, 0x6b, 0x2a, 0x51, 0x11, 0x78, 0x2e, 0xf3, 0x65, 0x45, 0x60, - 0xe8, 0x8a, 0x40, 0x22, 0x06, 0xfc, 0x1a, 0xad, 0x6f, 0xfe, 0xca, 0x80, 0x92, 0x68, 0xaa, 0x45, - 0xd8, 0xed, 0x8e, 0xce, 0x02, 0xa4, 0xc6, 0x73, 0x5a, 0xdb, 0xed, 0x39, 0x08, 0x17, 0xd0, 0x1b, - 0x6a, 0x4a, 0x97, 0x8c, 0x36, 0xaf, 0x27, 0xde, 0x84, 0xda, 0xa7, 0x81, 0xeb, 0x6f, 0xab, 0xb9, - 0x15, 0x4a, 0xa7, 0xf8, 0xb9, 0x39, 0xdf, 0x15, 0x9e, 0xf7, 0xc0, 0xea, 0x71, 0x21, 0x9a, 0xe5, - 0xe4, 0x69, 0x53, 0x91, 0x97, 0x1e, 0x2e, 0x6c, 0xfe, 0xa4, 0x08, 0x25, 0xd1, 0x9d, 0xa3, 0x87, - 0x50, 0xd1, 0xad, 0x35, 0x5a, 0x68, 0xa1, 0xdb, 0xa9, 0x13, 0x2d, 0xf4, 0xde, 0xb8, 0x80, 0xde, - 0x07, 0x4b, 0x47, 0xea, 0xf9, 0xfe, 0xbf, 0xfd, 0x34, 0xc7, 0xc3, 0x85, 0x75, 0xe3, 0x1d, 0x03, - 0xbd, 0x0d, 0x96, 0xb2, 0xc0, 0x05, 0x49, 0x3c, 0xbb, 0xc4, 0x3e, 0x71, 0x41, 0x32, 0xd4, 0xfa, - 0x67, 0xc1, 0xd4, 0x73, 0xfa, 0x2c, 0xba, 0x60, 0x68, 0x61, 0xb6, 0xd4, 0x5e, 0x80, 0x71, 0x01, - 0xbd, 0x05, 0xb0, 0xc5, 0xb9, 0x3b, 0xf6, 0x8f, 0x5c, 0x87, 0xa3, 0x5a, 0xb2, 0xbe, 0x3f, 0x9d, - 0xb4, 0x9b, 0xf2, 0x48, 0xb5, 0x2a, 0x3a, 0x15, 0xae, 0xc8, 0x73, 0x56, 0xf7, 0xad, 0xe4, 0xef, - 0x42, 0x43, 0xd9, 0xf8, 0x41, 0xb4, 0x25, 0xdc, 0x02, 0x2d, 0xb6, 0x29, 0xed, 0x45, 0x04, 0x2e, - 0xa0, 0x47, 0x60, 0x0f, 0xa2, 0x4b, 0x45, 0xff, 0x5c, 0x7a, 0xe1, 0xbc, 0xb9, 0xb7, 0x97, 0xa3, - 0x71, 0x61, 0xf3, 0x2f, 0x45, 0xb0, 0x3e, 0x0f, 0xa2, 0x73, 0x16, 0xa1, 0x0d, 0xb0, 0x64, 0xfb, - 0xc4, 0xd0, 0xd5, 0x76, 0x6a, 0xd9, 0xb1, 0x6f, 0x42, 0x55, 0x0a, 0x6d, 0x40, 0xf9, 0x79, 0xa6, - 0x26, 0xf9, 0x57, 0x58, 0x26, 0x37, 0x95, 0xa9, 0x71, 0x01, 0x7d, 0x07, 0xee, 0xa4, 0xe1, 0x7b, - 0xcb, 0x77, 0x54, 0x3a, 0xec, 0xd0, 0x98, 0xa2, 0xac, 0x2f, 0xcd, 0xf9, 0x4f, 0xbb, 0x96, 0x75, - 0x3a, 0x7d, 0xa9, 0xa9, 0x07, 0x50, 0xea, 0x8b, 0x17, 0x66, 0x7f, 0x64, 0x65, 0x23, 0xfd, 0x36, - 0xca, 0x23, 0xd3, 0x33, 0x3f, 0x00, 0x4b, 0x9d, 0x93, 0x89, 0x65, 0xae, 0x46, 0x69, 0xdf, 0x5e, - 0x44, 0x6b, 0xc6, 0xfb, 0x60, 0xef, 0xb9, 0xbe, 0x1a, 0x7c, 0xcd, 0x1b, 0x52, 0x5e, 0x83, 0xb8, - 0x80, 0x3e, 0x04, 0x4b, 0x45, 0xe3, 0xec, 0x84, 0xb9, 0xe8, 0xdc, 0x5e, 0x8e, 0xc6, 0x05, 0xf4, - 0x00, 0x9a, 0x84, 0x8d, 0x98, 0x9b, 0xcb, 0x6a, 0x28, 0xff, 0xe6, 0x45, 0x47, 0x5c, 0x37, 0xd0, - 0x47, 0xd0, 0x98, 0xcb, 0x82, 0x28, 0xcd, 0x08, 0xcb, 0x92, 0xe3, 0xe2, 0x06, 0x1f, 0x37, 0x7f, - 0xf3, 0xcd, 0xaa, 0xf1, 0xfb, 0x6f, 0x56, 0x8d, 0x3f, 0x7e, 0xb3, 0x6a, 0x7c, 0xfd, 0xe7, 0xd5, - 0xc2, 0x89, 0x25, 0xff, 0x81, 0x7d, 0xf7, 0x1f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x6c, 0x30, 0xc1, - 0x11, 0xa6, 0x1d, 0x00, 0x00, + // 3014 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x19, 0xcb, 0x72, 0x23, 0x57, + 0x55, 0xdd, 0x6a, 0xb5, 0x5a, 0x47, 0x92, 0x47, 0xb9, 0x99, 0xcc, 0x28, 0x4a, 0x70, 0x9c, 0x9e, + 0x90, 0x71, 0x5e, 0x4e, 0xc6, 0x99, 0x3c, 0x18, 0x08, 0x94, 0x63, 0x69, 0x26, 0xca, 0xf8, 0x95, + 0x2b, 0xd9, 0x21, 0x2c, 0x50, 0x5d, 0xab, 0xaf, 0xe5, 0x2e, 0xb7, 0xba, 0x3b, 0x7d, 0xbb, 0x5d, + 0x72, 0x96, 0xec, 0x28, 0x76, 0xac, 0xb2, 0xa6, 0x8a, 0x1d, 0x55, 0x54, 0xb1, 0x64, 0xc1, 0x0a, + 0xaa, 0x58, 0x50, 0x05, 0xfc, 0x01, 0x84, 0x0d, 0xc5, 0x8e, 0x3f, 0xa0, 0xee, 0xa3, 0x1f, 0x92, + 0x35, 0x8e, 0x79, 0xad, 0xd4, 0xe7, 0xdc, 0x73, 0xee, 0xe3, 0xbc, 0xcf, 0x11, 0xac, 0xb8, 0x7e, + 0x4c, 0x23, 0x9f, 0x78, 0x1b, 0x61, 0x14, 0xc4, 0x01, 0x32, 0x25, 0xdc, 0xa9, 0x91, 0xd0, 0x95, + 0x28, 0xbb, 0x03, 0xc6, 0x8e, 0xcb, 0x62, 0x84, 0xc0, 0x48, 0x5c, 0x87, 0xb5, 0xb5, 0xb5, 0xf2, + 0xba, 0x89, 0xc5, 0xb7, 0xfd, 0x09, 0xd4, 0x86, 0x84, 0x9d, 0x1d, 0x11, 0x2f, 0xa1, 0xa8, 0x05, + 0xe5, 0x73, 0xe2, 0xb5, 0xb5, 0x35, 0x6d, 0xbd, 0x81, 0xf9, 0x27, 0xda, 0x04, 0xeb, 0x9c, 0x78, + 0xa3, 0xf8, 0x22, 0xa4, 0x6d, 0x7d, 0x4d, 0x5b, 0x5f, 0xd9, 0xbc, 0xbd, 0x21, 0x0f, 0xd8, 0x38, + 0x08, 0x58, 0xec, 0xfa, 0x93, 0x8d, 0x23, 0xe2, 0x0d, 0x2f, 0x42, 0x8a, 0xab, 0xe7, 0xf2, 0xc3, + 0xde, 0x87, 0xfa, 0x20, 0x1a, 0x3f, 0x4c, 0xfc, 0x71, 0xec, 0x06, 0x3e, 0x3f, 0xd5, 0x27, 0x53, + 0x2a, 0x76, 0xad, 0x61, 0xf1, 0xcd, 0x71, 0x24, 0x9a, 0xb0, 0x76, 0x79, 0xad, 0xcc, 0x71, 0xfc, + 0x1b, 0xb5, 0xa1, 0xea, 0xb2, 0xed, 0x20, 0xf1, 0xe3, 0xb6, 0xb1, 0xa6, 0xad, 0x5b, 0x38, 0x05, + 0xed, 0x5f, 0x94, 0xa1, 0xf2, 0x49, 0x42, 0xa3, 0x0b, 0xc1, 0x17, 0xc7, 0x51, 0xba, 0x17, 0xff, + 0x46, 0x37, 0xa1, 0xe2, 0x11, 0x7f, 0xc2, 0xda, 0xba, 0xd8, 0x4c, 0x02, 0xe8, 0x39, 0xa8, 0x91, + 0x93, 0x98, 0x46, 0xa3, 0xc4, 0x75, 0xda, 0xe5, 0x35, 0x6d, 0xdd, 0xc4, 0x96, 0x40, 0x1c, 0xba, + 0x0e, 0x7a, 0x16, 0x2c, 0x27, 0x18, 0x8d, 0x8b, 0x67, 0x39, 0x81, 0x38, 0x0b, 0xdd, 0x05, 0x2b, + 0x71, 0x9d, 0x91, 0xe7, 0xb2, 0xb8, 0x5d, 0x59, 0xd3, 0xd6, 0xeb, 0x9b, 0x8d, 0xf4, 0xc1, 0x5c, + 0x86, 0xb8, 0x9a, 0xb8, 0x8e, 0x10, 0xe6, 0x06, 0x58, 0x2c, 0x1a, 0x8f, 0x4e, 0x12, 0x7f, 0xdc, + 0x36, 0x05, 0xe1, 0xd3, 0x29, 0x61, 0xe1, 0xf5, 0xb8, 0xca, 0x24, 0xc0, 0x9f, 0x17, 0xd1, 0x73, + 0x1a, 0x31, 0xda, 0xae, 0xca, 0x23, 0x15, 0x88, 0xee, 0x43, 0xfd, 0x84, 0x8c, 0x69, 0x3c, 0x0a, + 0x49, 0x44, 0xa6, 0x6d, 0x6b, 0x7e, 0xb3, 0x87, 0x7c, 0xe9, 0x80, 0xaf, 0x30, 0x0c, 0x27, 0x19, + 0x80, 0xde, 0x83, 0xa6, 0x80, 0xd8, 0xe8, 0xc4, 0xf5, 0x62, 0x1a, 0xb5, 0x6b, 0x82, 0x0f, 0x65, + 0x7c, 0x02, 0x3b, 0x8c, 0x28, 0xc5, 0x0d, 0x49, 0x28, 0x31, 0xe8, 0x1b, 0x00, 0x74, 0x16, 0x12, + 0xdf, 0x19, 0x11, 0xcf, 0x6b, 0x83, 0xb8, 0x4b, 0x4d, 0x62, 0xb6, 0x3c, 0x0f, 0xdd, 0xe6, 0xf7, + 0x24, 0xce, 0x28, 0x66, 0xed, 0xe6, 0x9a, 0xb6, 0x6e, 0x60, 0x93, 0x83, 0x43, 0xc6, 0x25, 0xe3, + 0xb9, 0xfe, 0x88, 0x43, 0xed, 0x15, 0x25, 0x19, 0x6e, 0x63, 0x3b, 0xae, 0x8f, 0x29, 0x71, 0x70, + 0xd5, 0x93, 0x1f, 0xf6, 0xbb, 0x50, 0x13, 0xe6, 0x24, 0xc4, 0xf4, 0x0a, 0x98, 0xe7, 0x1c, 0x90, + 0x56, 0x57, 0xdf, 0x7c, 0x2a, 0xbd, 0x5f, 0x66, 0x75, 0x58, 0x11, 0xd8, 0xab, 0x60, 0xed, 0x10, + 0x7f, 0x92, 0x9a, 0x2a, 0xd7, 0xa3, 0x60, 0xaa, 0x61, 0xf1, 0x6d, 0xff, 0x59, 0x07, 0x13, 0x53, + 0x96, 0x78, 0x31, 0x7a, 0x0d, 0x80, 0x6b, 0x69, 0x4a, 0xe2, 0xc8, 0x9d, 0xa9, 0x9d, 0xe7, 0xf5, + 0x54, 0x4b, 0x5c, 0x67, 0x57, 0x2c, 0xa3, 0xfb, 0xd0, 0x10, 0x27, 0xa4, 0xe4, 0xfa, 0xfc, 0x45, + 0xb2, 0xbb, 0xe2, 0xba, 0x20, 0x53, 0x5c, 0xb7, 0xc0, 0x14, 0x06, 0x22, 0x8d, 0xb4, 0x89, 0x15, + 0x84, 0xbe, 0xa9, 0x3c, 0x8e, 0xd1, 0x71, 0x3c, 0x72, 0x28, 0x4b, 0x2d, 0xa8, 0x99, 0x61, 0xbb, + 0x94, 0xc5, 0xe8, 0x1d, 0x90, 0x52, 0x4f, 0x0f, 0xad, 0x88, 0x43, 0xd1, 0x9c, 0x56, 0x99, 0x3c, + 0x55, 0xd0, 0xa9, 0x53, 0xef, 0x41, 0x9d, 0xbf, 0x35, 0xe5, 0x32, 0x05, 0x57, 0x2b, 0x7b, 0x99, + 0x12, 0x0f, 0x06, 0x4e, 0xa4, 0x58, 0xb8, 0xa8, 0xb8, 0xb5, 0x4a, 0xab, 0x12, 0xdf, 0xd7, 0xd7, + 0x55, 0x0f, 0x2a, 0xfb, 0x91, 0x43, 0xa3, 0xa5, 0x9e, 0x85, 0xc0, 0x70, 0x28, 0x1b, 0x0b, 0xc7, + 0xb7, 0xb0, 0xf8, 0xce, 0xbd, 0xad, 0x5c, 0xf0, 0x36, 0xfb, 0x8f, 0x1a, 0xd4, 0x07, 0x41, 0x14, + 0xef, 0x52, 0xc6, 0xc8, 0x84, 0xa2, 0x3b, 0x50, 0x09, 0xf8, 0xb6, 0x4a, 0x35, 0xcd, 0xf4, 0x01, + 0xe2, 0x2c, 0x2c, 0xd7, 0x16, 0x94, 0xa8, 0x5f, 0xad, 0xc4, 0x9b, 0x50, 0x91, 0xfe, 0xca, 0x7d, + 0xb9, 0x82, 0x25, 0xc0, 0x95, 0x14, 0x9c, 0x9c, 0x30, 0x2a, 0x95, 0x50, 0xc1, 0x0a, 0xfa, 0x1f, + 0x18, 0xf1, 0x31, 0x00, 0x7f, 0xd0, 0x7f, 0x62, 0x6f, 0xd7, 0x3e, 0xe3, 0x14, 0xea, 0x98, 0x9c, + 0xc4, 0xdb, 0x81, 0x1f, 0xd3, 0x59, 0x8c, 0x56, 0x40, 0x77, 0x1d, 0xa1, 0x00, 0x13, 0xeb, 0xae, + 0xc3, 0x9f, 0x3c, 0x89, 0x82, 0x24, 0x14, 0xf2, 0x6f, 0x62, 0x09, 0x08, 0x45, 0x39, 0x4e, 0x24, + 0xe4, 0xc0, 0x15, 0xe5, 0x38, 0x11, 0x7a, 0x01, 0xea, 0xcc, 0x27, 0x21, 0x3b, 0x0d, 0x62, 0xfe, + 0x64, 0x43, 0x3c, 0x19, 0x52, 0xd4, 0x90, 0xd9, 0xbf, 0xd3, 0xc0, 0xdc, 0xa5, 0xd3, 0x63, 0x1a, + 0x5d, 0x3a, 0xe5, 0x59, 0xb0, 0xc4, 0xc6, 0x23, 0xd7, 0x51, 0x07, 0x55, 0x05, 0xdc, 0x77, 0x96, + 0x1e, 0x75, 0x0b, 0x4c, 0x8f, 0x12, 0xae, 0x5a, 0x69, 0xf6, 0x0a, 0xe2, 0x12, 0x27, 0xd3, 0x91, + 0xc3, 0xdf, 0x5c, 0x91, 0x0b, 0x64, 0xda, 0xa5, 0xc4, 0xe1, 0x77, 0xf3, 0x08, 0x8b, 0x47, 0x49, + 0xe8, 0x90, 0x98, 0x8a, 0x50, 0x69, 0x70, 0xfb, 0x65, 0xf1, 0xa1, 0xc0, 0xa0, 0x57, 0xe1, 0xa9, + 0xb1, 0x97, 0x30, 0x1e, 0xab, 0x5d, 0xff, 0x24, 0x18, 0x05, 0xbe, 0x77, 0x21, 0xb4, 0x66, 0xe1, + 0x1b, 0x6a, 0xa1, 0xef, 0x9f, 0x04, 0xfb, 0xbe, 0x77, 0x61, 0xff, 0x44, 0x87, 0xca, 0x23, 0x21, + 0x86, 0xfb, 0x50, 0x9d, 0x8a, 0x07, 0xa5, 0x81, 0xa5, 0x93, 0xaa, 0x43, 0xac, 0x6f, 0xc8, 0xd7, + 0xb2, 0x9e, 0x1f, 0x47, 0x17, 0x38, 0x25, 0xe5, 0x5c, 0x31, 0x39, 0xf6, 0x68, 0xcc, 0x94, 0xbd, + 0x2d, 0x70, 0x0d, 0xe5, 0xa2, 0xe2, 0x52, 0xa4, 0x9d, 0x8f, 0xa1, 0x51, 0xdc, 0x8e, 0xa7, 0xc9, + 0x33, 0x7a, 0x21, 0x64, 0x68, 0x60, 0xfe, 0x89, 0x5e, 0x82, 0x8a, 0x88, 0x1d, 0x42, 0x82, 0xf5, + 0xcd, 0x95, 0x74, 0x57, 0xc9, 0x86, 0xe5, 0xe2, 0x03, 0xfd, 0x7d, 0x8d, 0xef, 0x55, 0x3c, 0xa4, + 0xb8, 0x57, 0xed, 0xea, 0xbd, 0x24, 0x5b, 0x61, 0x2f, 0xfb, 0xa7, 0x3a, 0x34, 0x7e, 0x40, 0xa3, + 0xe0, 0x20, 0x0a, 0xc2, 0x80, 0x11, 0x0f, 0xdd, 0x81, 0xa6, 0x43, 0xc3, 0x88, 0x8e, 0x49, 0x4c, + 0x9d, 0x91, 0x52, 0x73, 0x13, 0x37, 0x72, 0x64, 0xdf, 0x41, 0x2f, 0x83, 0x29, 0xc5, 0xf1, 0x84, + 0xcb, 0xaa, 0x55, 0x4e, 0x27, 0x05, 0x20, 0xf4, 0x7f, 0xf9, 0x22, 0x6a, 0x15, 0xad, 0x02, 0x4c, + 0xc9, 0x6c, 0x87, 0x12, 0x46, 0xfb, 0x4e, 0x6a, 0x7b, 0x39, 0x06, 0x75, 0xc0, 0x9a, 0x92, 0xd9, + 0x70, 0xe6, 0x0f, 0x99, 0x30, 0x0d, 0x03, 0x67, 0x30, 0x7a, 0x1e, 0x6a, 0x53, 0x32, 0xe3, 0x4e, + 0xd0, 0x77, 0x94, 0x69, 0xe4, 0x08, 0xf4, 0x22, 0x94, 0xe3, 0x99, 0x2f, 0x02, 0x5b, 0x7d, 0xf3, + 0x86, 0xf0, 0xa1, 0xe1, 0xcc, 0x57, 0xee, 0x82, 0xf9, 0x5a, 0x2a, 0x3e, 0x2b, 0x13, 0x9f, 0xfd, + 0xeb, 0x32, 0xdc, 0x50, 0xda, 0x3a, 0x75, 0xc3, 0x41, 0xcc, 0x4d, 0xac, 0x0d, 0x55, 0x11, 0x2f, + 0x68, 0xa4, 0x94, 0x96, 0x82, 0xe8, 0xdb, 0x60, 0x0a, 0x6b, 0x4f, 0xed, 0xe1, 0xce, 0xbc, 0x30, + 0xb2, 0x2d, 0xa4, 0x7d, 0x28, 0xc3, 0x50, 0x2c, 0xe8, 0x7d, 0xa8, 0x7c, 0x41, 0xa3, 0x40, 0xc6, + 0xc2, 0xfa, 0xa6, 0xfd, 0x24, 0x5e, 0xae, 0x23, 0xc5, 0x2a, 0x19, 0xfe, 0x8f, 0x32, 0x5b, 0xe7, + 0x91, 0x6f, 0x1a, 0x9c, 0x53, 0xa7, 0x5d, 0x15, 0xb7, 0x5a, 0x54, 0x6f, 0xba, 0xdc, 0xf9, 0x08, + 0xea, 0x85, 0x47, 0x15, 0x0d, 0xb1, 0x29, 0x0d, 0xf1, 0xce, 0xbc, 0x21, 0x36, 0xe7, 0x5c, 0xa5, + 0x68, 0xd3, 0x1f, 0x01, 0xe4, 0x4f, 0xfc, 0x6f, 0xbc, 0xc3, 0xfe, 0xb1, 0x06, 0x37, 0xb6, 0x03, + 0xdf, 0xa7, 0xa2, 0x78, 0x92, 0xca, 0xcb, 0xed, 0x55, 0xbb, 0xd2, 0x5e, 0xdf, 0x80, 0x0a, 0xe3, + 0x0c, 0xea, 0x94, 0xdb, 0x4f, 0xd0, 0x06, 0x96, 0x54, 0x3c, 0x2e, 0x4d, 0xc9, 0x6c, 0x14, 0x52, + 0xdf, 0x71, 0xfd, 0x89, 0xb0, 0x71, 0xa9, 0x83, 0x03, 0x89, 0xb1, 0x7f, 0xa6, 0x81, 0x29, 0x4d, + 0x7d, 0x2e, 0x46, 0x6a, 0xf3, 0x31, 0xf2, 0x79, 0xa8, 0x85, 0x11, 0x75, 0xdc, 0x71, 0x7a, 0x72, + 0x0d, 0xe7, 0x08, 0x1e, 0xc2, 0x4f, 0x82, 0x68, 0x4c, 0xc5, 0xf6, 0x16, 0x96, 0x00, 0xaf, 0x4d, + 0x45, 0x76, 0x12, 0x91, 0x4e, 0x86, 0x51, 0x8b, 0x23, 0x78, 0x88, 0xe3, 0x2c, 0x2c, 0x24, 0x63, + 0x59, 0x25, 0x96, 0xb1, 0x04, 0x78, 0xd8, 0x95, 0x7a, 0x13, 0xa6, 0x6e, 0x61, 0x05, 0xd9, 0xbf, + 0xd2, 0xa1, 0xd1, 0x75, 0x23, 0x3a, 0x8e, 0xa9, 0xd3, 0x73, 0x26, 0x82, 0x90, 0xfa, 0xb1, 0x1b, + 0x5f, 0xa8, 0x10, 0xaf, 0xa0, 0x2c, 0xbf, 0xeb, 0xf3, 0x95, 0xb3, 0xd4, 0x4b, 0x59, 0x14, 0xfc, + 0x12, 0x40, 0xef, 0x02, 0xc8, 0x72, 0x49, 0x14, 0xfd, 0xc6, 0xd5, 0x45, 0x7f, 0x4d, 0x90, 0xf2, + 0x4f, 0x2e, 0x24, 0xc9, 0xe7, 0xca, 0x14, 0x60, 0x8a, 0x8e, 0x20, 0xe1, 0xe6, 0x2c, 0x8a, 0x86, + 0x63, 0xea, 0x09, 0x73, 0x15, 0x45, 0xc3, 0x31, 0xf5, 0xb2, 0x1a, 0xaf, 0x2a, 0xaf, 0xc4, 0xbf, + 0xd1, 0x5d, 0xd0, 0x83, 0x50, 0xbc, 0xb1, 0x70, 0x68, 0xf1, 0x81, 0x1b, 0xfb, 0x21, 0xd6, 0x83, + 0x10, 0xd9, 0x60, 0xca, 0xaa, 0xb6, 0x5d, 0x13, 0x66, 0x0e, 0x22, 0x3c, 0x88, 0xb2, 0x0a, 0xab, + 0x15, 0xfb, 0x16, 0xe8, 0xfb, 0x21, 0xaa, 0x42, 0x79, 0xd0, 0x1b, 0xb6, 0x4a, 0xfc, 0xa3, 0xdb, + 0xdb, 0x69, 0x69, 0xf6, 0x3f, 0x34, 0xa8, 0xed, 0x26, 0x31, 0xe1, 0x36, 0xc6, 0xae, 0x52, 0xee, + 0xb3, 0x60, 0xb1, 0x98, 0x44, 0x22, 0xa9, 0xea, 0x32, 0x70, 0x08, 0x78, 0xc8, 0xd0, 0xab, 0x50, + 0xa1, 0xce, 0x84, 0xa6, 0xbe, 0x7f, 0x73, 0xd9, 0x5d, 0xb1, 0x24, 0x41, 0xaf, 0x83, 0xc9, 0xc6, + 0xa7, 0x74, 0x4a, 0xda, 0xc6, 0x3c, 0xf1, 0x40, 0x60, 0x65, 0x1e, 0xc4, 0x8a, 0x46, 0x34, 0x27, + 0x51, 0x10, 0x8a, 0xea, 0xbc, 0xa2, 0x9a, 0x93, 0x28, 0x08, 0x79, 0x6d, 0xbe, 0x09, 0xcf, 0xb8, + 0x13, 0x3f, 0x88, 0xe8, 0xc8, 0xf5, 0x1d, 0x3a, 0x1b, 0x8d, 0x03, 0xff, 0xc4, 0x73, 0xc7, 0xb1, + 0x90, 0xab, 0x85, 0x9f, 0x96, 0x8b, 0x7d, 0xbe, 0xb6, 0xad, 0x96, 0xec, 0xbb, 0x50, 0x7b, 0x4c, + 0x2f, 0x44, 0x91, 0xcb, 0x50, 0x07, 0xf4, 0xb3, 0x73, 0x95, 0x30, 0x21, 0xbd, 0xc5, 0xe3, 0x23, + 0xac, 0x9f, 0x9d, 0xdb, 0x5f, 0xea, 0x60, 0xfd, 0x7b, 0x99, 0xe4, 0x4d, 0xa8, 0x4d, 0x53, 0x31, + 0x2a, 0xaf, 0xcb, 0xaa, 0xea, 0x4c, 0xbe, 0x38, 0xa7, 0x41, 0x6f, 0x41, 0x3d, 0x9e, 0xf9, 0xfc, + 0xda, 0x3c, 0x82, 0xab, 0xbc, 0x72, 0x29, 0xb0, 0x43, 0x9c, 0x7d, 0xab, 0x0b, 0x1b, 0xcb, 0x2e, + 0x9c, 0x3b, 0x7c, 0xe5, 0x5a, 0x0e, 0x7f, 0x17, 0x6e, 0x8c, 0x3d, 0x4a, 0xfc, 0x51, 0xee, 0xaf, + 0xd2, 0x1c, 0x57, 0x04, 0xfa, 0x20, 0x73, 0x5a, 0x15, 0xc0, 0xaa, 0x79, 0x4e, 0x79, 0x11, 0xca, + 0x8f, 0x8f, 0x06, 0x57, 0x4a, 0xef, 0x87, 0xa0, 0x3f, 0x3e, 0x2a, 0xc6, 0xbe, 0x86, 0x8c, 0x7d, + 0xaa, 0xa5, 0xd6, 0xf3, 0x96, 0xba, 0x03, 0x56, 0xc2, 0x68, 0xb4, 0x4b, 0x63, 0xa2, 0x1c, 0x2f, + 0x83, 0x79, 0xa2, 0xe2, 0x3d, 0xa1, 0x1b, 0xf8, 0x2a, 0x29, 0xa4, 0xa0, 0xfd, 0xcf, 0x32, 0x54, + 0x95, 0xf3, 0xf1, 0x3d, 0x93, 0xac, 0x86, 0xe3, 0x9f, 0xb9, 0x27, 0xeb, 0x45, 0x4f, 0x2e, 0x36, + 0xef, 0xe5, 0xeb, 0x35, 0xef, 0xe8, 0xbb, 0xd0, 0x08, 0xe5, 0x5a, 0xd1, 0xff, 0x9f, 0x5b, 0xe4, + 0x53, 0xbf, 0x82, 0xb7, 0x1e, 0xe6, 0x00, 0xb7, 0x5e, 0xd1, 0xc0, 0xc4, 0x64, 0x22, 0xf4, 0xd2, + 0xc0, 0x55, 0x0e, 0x0f, 0xc9, 0xe4, 0x09, 0x51, 0xe0, 0x1a, 0x8e, 0xcc, 0x6b, 0xd6, 0x20, 0x6c, + 0x37, 0x84, 0x09, 0x72, 0xe7, 0x2f, 0xfa, 0x65, 0x73, 0xde, 0x2f, 0x9f, 0x83, 0xda, 0x38, 0x98, + 0x4e, 0x5d, 0xb1, 0xb6, 0x22, 0x53, 0xa7, 0x44, 0x0c, 0x99, 0xfd, 0x05, 0x54, 0xd5, 0x83, 0x51, + 0x1d, 0xaa, 0xdd, 0xde, 0xc3, 0xad, 0xc3, 0x1d, 0x1e, 0x19, 0x00, 0xcc, 0x0f, 0xfb, 0x7b, 0x5b, + 0xf8, 0xb3, 0x96, 0xc6, 0xa3, 0x44, 0x7f, 0x6f, 0xd8, 0xd2, 0x51, 0x0d, 0x2a, 0x0f, 0x77, 0xf6, + 0xb7, 0x86, 0xad, 0x32, 0xb2, 0xc0, 0xf8, 0x70, 0x7f, 0x7f, 0xa7, 0x65, 0xa0, 0x06, 0x58, 0xdd, + 0xad, 0x61, 0x6f, 0xd8, 0xdf, 0xed, 0xb5, 0x2a, 0x9c, 0xf6, 0x51, 0x6f, 0xbf, 0x65, 0xf2, 0x8f, + 0xc3, 0x7e, 0xb7, 0x55, 0xe5, 0xeb, 0x07, 0x5b, 0x83, 0xc1, 0xa7, 0xfb, 0xb8, 0xdb, 0xb2, 0xf8, + 0xbe, 0x83, 0x21, 0xee, 0xef, 0x3d, 0x6a, 0xd5, 0xec, 0x7b, 0x50, 0x2f, 0x08, 0x8d, 0x73, 0xe0, + 0xde, 0xc3, 0x56, 0x89, 0x1f, 0x73, 0xb4, 0xb5, 0x73, 0xd8, 0x6b, 0x69, 0x68, 0x05, 0x40, 0x7c, + 0x8e, 0x76, 0xb6, 0xf6, 0x1e, 0xb5, 0x74, 0xfb, 0x47, 0x5a, 0xc6, 0x23, 0x9a, 0xe2, 0xd7, 0xc0, + 0x52, 0xa2, 0x4e, 0x8b, 0xde, 0x1b, 0x0b, 0x7a, 0xc1, 0x19, 0x01, 0x37, 0xb3, 0xf1, 0x29, 0x1d, + 0x9f, 0xb1, 0x64, 0xaa, 0xac, 0x22, 0x83, 0x65, 0x6f, 0xcb, 0x65, 0xa2, 0xd2, 0x9e, 0x82, 0xb2, + 0x01, 0x91, 0x21, 0xe8, 0xe5, 0x80, 0xe8, 0x3e, 0x40, 0x3e, 0x82, 0x58, 0x52, 0xae, 0xde, 0x84, + 0x0a, 0xf1, 0x5c, 0xc2, 0x54, 0x66, 0x91, 0x80, 0x8d, 0xa1, 0x5e, 0x18, 0x5c, 0x70, 0x85, 0x11, + 0xcf, 0x1b, 0x9d, 0xd1, 0x0b, 0x26, 0x78, 0x2d, 0x5c, 0x25, 0x9e, 0xf7, 0x98, 0x5e, 0x30, 0xb4, + 0x0e, 0x15, 0x39, 0xf7, 0xd0, 0x97, 0x74, 0xc8, 0x82, 0x1d, 0x4b, 0x02, 0xfb, 0x75, 0x30, 0x65, + 0xdb, 0x5c, 0xb0, 0x19, 0xed, 0x89, 0xc1, 0xff, 0x03, 0x75, 0x6f, 0xd1, 0x64, 0xa3, 0x37, 0xd5, + 0x8c, 0x85, 0xc9, 0xc9, 0x8e, 0x36, 0x5f, 0x1a, 0x49, 0x42, 0x35, 0x5e, 0x11, 0x0c, 0x76, 0x17, + 0xac, 0x2b, 0x27, 0x58, 0x4a, 0x10, 0x7a, 0x2e, 0x88, 0x25, 0x33, 0x2d, 0x3b, 0x02, 0xc8, 0xe7, + 0x30, 0xca, 0x8c, 0xe5, 0x2e, 0xdc, 0x8c, 0x37, 0xb8, 0x8a, 0x5c, 0xcf, 0x89, 0xa8, 0x7f, 0xe9, + 0xf5, 0xf9, 0xf4, 0x26, 0xa3, 0x41, 0x2f, 0x81, 0x21, 0xc6, 0x4d, 0x32, 0x6e, 0x66, 0x53, 0x81, + 0x6c, 0xd6, 0x24, 0x56, 0xed, 0x63, 0x68, 0xca, 0xbc, 0x82, 0xe9, 0xe7, 0x09, 0x65, 0x57, 0x56, + 0x2f, 0xab, 0x00, 0x59, 0x34, 0x4c, 0x07, 0x68, 0x05, 0x0c, 0x37, 0x94, 0x13, 0x97, 0x7a, 0x4e, + 0xfa, 0x2a, 0x05, 0xd9, 0xef, 0x41, 0x23, 0x3d, 0x43, 0xf4, 0xc7, 0x77, 0xb3, 0x0c, 0x97, 0xda, + 0x25, 0x57, 0x88, 0x24, 0xd9, 0x0b, 0x9c, 0x2c, 0xb9, 0xd9, 0x7f, 0xd5, 0x53, 0x4e, 0xd5, 0xfd, + 0xcd, 0xd5, 0x4f, 0xda, 0x62, 0xfd, 0x34, 0x5f, 0x8b, 0xe8, 0xd7, 0xae, 0x45, 0xbe, 0x03, 0x35, + 0x47, 0x24, 0x62, 0xf7, 0x3c, 0x0d, 0x7d, 0xab, 0xcb, 0x92, 0xae, 0x4a, 0xd7, 0xee, 0x39, 0xc5, + 0x39, 0x03, 0xbf, 0x53, 0x1c, 0x9c, 0x51, 0xdf, 0xfd, 0x42, 0xb4, 0xb9, 0xfc, 0xe1, 0x39, 0x22, + 0x9f, 0x44, 0xc8, 0xe4, 0xac, 0x26, 0x11, 0xe9, 0x14, 0xc6, 0x2c, 0x4c, 0x61, 0x6e, 0x81, 0x99, + 0x84, 0x8c, 0x46, 0x71, 0x5a, 0xb4, 0x49, 0x28, 0x2b, 0x7c, 0x6a, 0x8a, 0x96, 0xf8, 0x13, 0xfb, + 0x5b, 0x50, 0xcb, 0xee, 0xc2, 0xe3, 0xcd, 0xde, 0xfe, 0x5e, 0x4f, 0x46, 0x87, 0xfe, 0x5e, 0xb7, + 0xf7, 0xfd, 0x96, 0xc6, 0x23, 0x16, 0xee, 0x1d, 0xf5, 0xf0, 0xa0, 0xd7, 0xd2, 0x79, 0x64, 0xe9, + 0xf6, 0x76, 0x7a, 0xc3, 0x5e, 0xab, 0xfc, 0xb1, 0x61, 0x55, 0x5b, 0x16, 0xb6, 0xe8, 0x2c, 0xf4, + 0xdc, 0xb1, 0x1b, 0xdb, 0x9f, 0x81, 0xb5, 0x4b, 0xc2, 0x4b, 0xc5, 0x78, 0x9e, 0x90, 0x12, 0xd5, + 0xea, 0xab, 0xe4, 0xf1, 0x0a, 0x54, 0x55, 0xd4, 0xc8, 0x32, 0xf2, 0x42, 0x54, 0x49, 0xd7, 0xed, + 0x5f, 0x6a, 0x70, 0x73, 0x37, 0x38, 0xa7, 0x59, 0xb2, 0x3c, 0x20, 0x17, 0x5e, 0x40, 0x9c, 0xaf, + 0x51, 0xe3, 0xcb, 0x70, 0x83, 0x05, 0x49, 0x34, 0xa6, 0xa3, 0x85, 0x51, 0x43, 0x53, 0xa2, 0x1f, + 0x29, 0x73, 0xb4, 0x79, 0xd5, 0xc1, 0xe2, 0x9c, 0xaa, 0x2c, 0xa8, 0xea, 0x1c, 0x99, 0xd2, 0x64, + 0x59, 0xdf, 0xb8, 0x4e, 0xd6, 0xb7, 0xff, 0xa0, 0x41, 0xb3, 0x37, 0x0b, 0x83, 0x28, 0x4e, 0xaf, + 0xfa, 0x0c, 0x2f, 0xa5, 0x3f, 0x4f, 0x9d, 0xc1, 0xc0, 0x95, 0x88, 0x7e, 0xde, 0xbf, 0x72, 0x0e, + 0x72, 0x1f, 0x4c, 0xbe, 0x59, 0xc2, 0x94, 0x29, 0x3d, 0x9f, 0x9e, 0x39, 0xb7, 0xf1, 0xc6, 0x40, + 0xd0, 0x60, 0x45, 0x5b, 0x9c, 0x41, 0x19, 0xc5, 0x19, 0x94, 0xfd, 0x00, 0x4c, 0x49, 0x5a, 0xd0, + 0x73, 0x1d, 0xaa, 0x83, 0xc3, 0xed, 0xed, 0xde, 0x60, 0xd0, 0xd2, 0x50, 0x13, 0x6a, 0xdd, 0xc3, + 0x83, 0x9d, 0xfe, 0xf6, 0xd6, 0x50, 0xe9, 0xfa, 0xe1, 0x56, 0x7f, 0xa7, 0xd7, 0x6d, 0x95, 0xed, + 0xdf, 0x68, 0x50, 0xdf, 0x8f, 0xc8, 0xd8, 0xa3, 0x5d, 0xea, 0xc5, 0x04, 0x3d, 0xe0, 0x9d, 0x2d, + 0x8f, 0xdd, 0x69, 0x28, 0x5c, 0xcb, 0x47, 0x6d, 0x19, 0xd5, 0xc6, 0xb6, 0x24, 0x51, 0x63, 0x0d, + 0xc5, 0xc0, 0xcd, 0x93, 0x1c, 0x07, 0x91, 0x9a, 0x85, 0x18, 0x58, 0x41, 0x5f, 0xdb, 0x19, 0x75, + 0x1e, 0x40, 0xa3, 0xb8, 0xe3, 0x92, 0x8e, 0x6f, 0xae, 0x1e, 0x31, 0x8a, 0x1d, 0xde, 0x0b, 0xd0, + 0xe4, 0x6d, 0xac, 0x3b, 0xa5, 0x2c, 0x26, 0xd3, 0x50, 0xe4, 0x76, 0x75, 0x79, 0x03, 0xeb, 0x31, + 0xb3, 0x5f, 0x86, 0xc6, 0x01, 0xa5, 0x11, 0xa6, 0x2c, 0x0c, 0x7c, 0x26, 0x1a, 0x1a, 0x25, 0x7c, + 0x99, 0x38, 0x14, 0x64, 0xdf, 0x86, 0xf2, 0x5e, 0x32, 0x2d, 0xfe, 0x65, 0x61, 0x88, 0xfa, 0xca, + 0x7e, 0x08, 0x8d, 0x81, 0x9a, 0x7c, 0x89, 0x9a, 0x8a, 0x57, 0x04, 0x9e, 0x4b, 0x7d, 0x51, 0x11, + 0x68, 0xaa, 0x22, 0x10, 0x88, 0x21, 0xbb, 0x42, 0xeb, 0x9b, 0xbf, 0xd5, 0xc0, 0xe0, 0x4d, 0x35, + 0x0f, 0xbb, 0xbd, 0xf1, 0x69, 0x80, 0xe4, 0x14, 0x4f, 0x69, 0xbb, 0x33, 0x07, 0xd9, 0x25, 0xf4, + 0x9a, 0x1c, 0xe6, 0xa5, 0x13, 0xd0, 0xab, 0x89, 0x37, 0xa1, 0xfe, 0x71, 0xe0, 0xfa, 0xdb, 0x72, + 0xbc, 0x85, 0xb2, 0x61, 0x7f, 0x61, 0x1c, 0x78, 0x89, 0xe7, 0x1d, 0x30, 0xfb, 0x8c, 0x8b, 0x66, + 0x39, 0x79, 0xd6, 0x54, 0x14, 0xa5, 0x67, 0x97, 0x36, 0x7f, 0x5e, 0x06, 0x83, 0x77, 0xe7, 0xe8, + 0x3e, 0x54, 0x55, 0x6b, 0x8d, 0x16, 0x5a, 0xe8, 0x4e, 0xe6, 0x44, 0x0b, 0xbd, 0xb7, 0x5d, 0x42, + 0xef, 0x82, 0xa9, 0x22, 0xf5, 0x7c, 0xff, 0xdf, 0x79, 0x92, 0xe3, 0xd9, 0xa5, 0x75, 0xed, 0x2d, + 0x0d, 0xbd, 0x09, 0xa6, 0xb4, 0xc0, 0x05, 0x49, 0x3c, 0xbd, 0xc4, 0x3e, 0xed, 0x92, 0x60, 0xa8, + 0x0f, 0x4e, 0x83, 0xc4, 0x73, 0x06, 0x34, 0x3a, 0xa7, 0x68, 0x61, 0xda, 0xd4, 0x59, 0x80, 0xed, + 0x12, 0x7a, 0x03, 0x60, 0x8b, 0x31, 0x77, 0xe2, 0x1f, 0xba, 0x0e, 0x43, 0xf5, 0x74, 0x7d, 0x2f, + 0x99, 0x76, 0x5a, 0xe2, 0x48, 0xb9, 0xca, 0x3b, 0x15, 0x26, 0xc9, 0x0b, 0x56, 0xf7, 0xb5, 0xe4, + 0x6f, 0x43, 0x53, 0xda, 0xf8, 0x7e, 0xb4, 0xc5, 0xdd, 0x02, 0x2d, 0xb6, 0x29, 0x9d, 0x45, 0x84, + 0x5d, 0x42, 0x0f, 0xc0, 0x1a, 0x46, 0x17, 0x92, 0xfe, 0x99, 0xec, 0xc2, 0x45, 0x73, 0xef, 0x2c, + 0x47, 0xdb, 0xa5, 0xcd, 0xbf, 0x97, 0xc1, 0xfc, 0x34, 0x88, 0xce, 0x68, 0x84, 0x36, 0xc0, 0x14, + 0xed, 0x13, 0x45, 0x97, 0xdb, 0xa9, 0x65, 0xc7, 0xbe, 0x0e, 0x35, 0x21, 0xb4, 0x21, 0x61, 0x67, + 0xb9, 0x9a, 0xc4, 0x3f, 0x66, 0xb9, 0xdc, 0x64, 0xa6, 0xb6, 0x4b, 0xe8, 0x7b, 0x70, 0x2b, 0x0b, + 0xdf, 0x5b, 0xbe, 0x23, 0xd3, 0x61, 0x97, 0xc4, 0x04, 0xe5, 0x7d, 0x69, 0xc1, 0x7f, 0x3a, 0xf5, + 0xbc, 0xd3, 0x19, 0x08, 0x4d, 0xdd, 0x03, 0x63, 0xc0, 0x5f, 0x98, 0xff, 0xdf, 0x95, 0x4f, 0xfe, + 0x3b, 0xa8, 0x88, 0xcc, 0xce, 0x7c, 0x0f, 0x4c, 0x79, 0x4e, 0x2e, 0x96, 0xb9, 0x1a, 0xa5, 0x73, + 0x73, 0x11, 0xad, 0x18, 0xef, 0x82, 0xb5, 0xeb, 0xfa, 0x72, 0xf0, 0x35, 0x6f, 0x48, 0x45, 0x0d, + 0xda, 0x25, 0xf4, 0x3e, 0x98, 0x32, 0x1a, 0xe7, 0x27, 0xcc, 0x45, 0xe7, 0xce, 0x72, 0xb4, 0x5d, + 0x42, 0xf7, 0xa0, 0x85, 0xe9, 0x98, 0xba, 0x85, 0xac, 0x86, 0x8a, 0x6f, 0x5e, 0x74, 0xc4, 0x75, + 0x0d, 0x7d, 0x00, 0xcd, 0xb9, 0x2c, 0x88, 0xb2, 0x8c, 0xb0, 0x2c, 0x39, 0x2e, 0x6e, 0xf0, 0x61, + 0xeb, 0xf7, 0x5f, 0xad, 0x6a, 0x7f, 0xfa, 0x6a, 0x55, 0xfb, 0xcb, 0x57, 0xab, 0xda, 0x97, 0x7f, + 0x5b, 0x2d, 0x1d, 0x9b, 0xe2, 0x8f, 0xda, 0xb7, 0xff, 0x15, 0x00, 0x00, 0xff, 0xff, 0x1c, 0xcf, + 0xad, 0x21, 0xcd, 0x1d, 0x00, 0x00, } diff --git a/protos/internal.proto b/protos/internal.proto index f0a8545af15..5a9b43c4ccc 100644 --- a/protos/internal.proto +++ b/protos/internal.proto @@ -127,13 +127,14 @@ message Group { } message ZeroProposal { - uint32 id = 1; + uint32 deprecated_id = 1; // delete this field in later versions. Member member = 2; Tablet tablet = 3; uint64 maxLeaseId = 4; uint64 maxTxnTs = 5; uint64 maxRaftId = 6; api.TxnContext txn = 7; + string key = 8; // Used as unique identifier for proposal id. } // MembershipState is used to pack together the current membership state of all the nodes diff --git a/query/query_test.go b/query/query_test.go index 6e328daa1ed..b6456728cb9 100644 --- a/query/query_test.go +++ b/query/query_test.go @@ -6125,6 +6125,7 @@ func TestMain(m *testing.M) { zero := exec.Command(os.ExpandEnv("$GOPATH/bin/dgraph"), "zero", "--wal", zw, + "-o", "10", ) zero.Stdout = os.Stdout zero.Stderr = os.Stdout @@ -6145,7 +6146,7 @@ func TestMain(m *testing.M) { worker.Config.RaftId = 1 posting.Config.AllottedMemory = 1024.0 posting.Config.CommitFraction = 0.10 - worker.Config.ZeroAddr = fmt.Sprintf("localhost:%d", x.PortZeroGrpc) + worker.Config.ZeroAddr = fmt.Sprintf("localhost:%d", x.PortZeroGrpc+10) worker.Config.RaftId = 1 worker.Config.MyAddr = "localhost:12345" worker.Config.ExpandEdge = true diff --git a/test b/test.sh similarity index 58% rename from test rename to test.sh index d1cf865da99..eff231a4b11 100755 --- a/test +++ b/test.sh @@ -1,5 +1,6 @@ #!/bin/bash +source contrib/scripts/functions.sh function run { go test -short=true $@ |\ GREP_COLORS='mt=01;32' egrep --line-buffered --color=always '^ok\ .*|$' |\ @@ -7,12 +8,25 @@ function run { GREP_COLORS='mt=01;31' egrep --line-buffered --color=always '.*FAIL.*|$' } +function runAll { + runCluster + # pushd dgraph/cmd/server + # go test -v . + # popd + + for PKG in $(go list ./...|grep -v -E 'vendor|contrib|wiki|customtok'); do + echo "Running test for $PKG" + run $PKG + done +} + # For piped commands return non-zero status if any command # in the pipe returns a non-zero status set -o pipefail echo echo "Running tests. Ignoring vendor folder." -for PKG in $(go list ./...|grep -v -E 'vendor|contrib|wiki|customtok'); do - echo "Running test for $PKG" - run $PKG -done +runAll + +echo +echo "Running load-test.sh" +./contrib/scripts/load-test.sh diff --git a/vendor/github.com/coreos/etcd/raft/README.md b/vendor/github.com/coreos/etcd/raft/README.md index f485b839771..fde22b16519 100644 --- a/vendor/github.com/coreos/etcd/raft/README.md +++ b/vendor/github.com/coreos/etcd/raft/README.md @@ -25,12 +25,12 @@ This raft implementation is a full feature implementation of Raft protocol. Feat - Membership changes - Leadership transfer extension - Efficient linearizable read-only queries served by both the leader and followers - - leader checks with quorum and bypasses Raft log before processing read-only queries - - followers asks leader to get a safe read index before processing read-only queries + - leader checks with quorum and bypasses Raft log before processing read-only queries + - followers asks leader to get a safe read index before processing read-only queries - More efficient lease-based linearizable read-only queries served by both the leader and followers - - leader bypasses Raft log and processing read-only queries locally - - followers asks leader to get a safe read index before processing read-only queries - - this approach relies on the clock of the all the machines in raft group + - leader bypasses Raft log and processing read-only queries locally + - followers asks leader to get a safe read index before processing read-only queries + - this approach relies on the clock of the all the machines in raft group This raft implementation also includes a few optional enhancements: @@ -112,7 +112,7 @@ After creating a Node, the user has a few responsibilities: First, read from the Node.Ready() channel and process the updates it contains. These steps may be performed in parallel, except as noted in step 2. -1. Write HardState, Entries, and Snapshot to persistent storage if they are not empty. Note that when writing an Entry with Index i, any previously-persisted entries with Index >= i must be discarded. +1. Write Entries, HardState and Snapshot to persistent storage in order, i.e. Entries first, then HardState and Snapshot if they are not empty. If persistent storage supports atomic writes then all of them can be written together. Note that when writing an Entry with Index i, any previously-persisted entries with Index >= i must be discarded. 2. Send all Messages to the nodes named in the To field. It is important that no messages be sent until the latest HardState has been persisted to disk, and all Entries written by any previous Ready batch (Messages may be sent while entries from the same batch are being persisted). To reduce the I/O latency, an optimization can be applied to make leader write to disk in parallel with its followers (as explained at section 10.2.1 in Raft thesis). If any Message has type MsgSnap, call Node.ReportSnapshot() after it has been sent (these messages may be large). Note: Marshalling messages is not thread-safe; it is important to make sure that no new entries are persisted while marshalling. The easiest way to achieve this is to serialise the messages directly inside the main raft loop. diff --git a/vendor/github.com/coreos/etcd/raft/node.go b/vendor/github.com/coreos/etcd/raft/node.go index 5da1c1193b2..33a9db84001 100644 --- a/vendor/github.com/coreos/etcd/raft/node.go +++ b/vendor/github.com/coreos/etcd/raft/node.go @@ -15,10 +15,10 @@ package raft import ( + "context" "errors" pb "github.com/coreos/etcd/raft/raftpb" - "golang.org/x/net/context" ) type SnapshotStatus int @@ -319,7 +319,7 @@ func (n *node) run(r *raft) { r.Step(m) case m := <-n.recvc: // filter out response message from unknown From. - if _, ok := r.prs[m.From]; ok || !IsResponseMsg(m.Type) { + if pr := r.getProgress(m.From); pr != nil || !IsResponseMsg(m.Type) { r.Step(m) // raft never returns an error } case cc := <-n.confc: @@ -334,6 +334,8 @@ func (n *node) run(r *raft) { switch cc.Type { case pb.ConfChangeAddNode: r.addNode(cc.NodeID) + case pb.ConfChangeAddLearnerNode: + r.addLearner(cc.NodeID) case pb.ConfChangeRemoveNode: // block incoming proposal when local node is // removed diff --git a/vendor/github.com/coreos/etcd/raft/progress.go b/vendor/github.com/coreos/etcd/raft/progress.go index 77c7b52efe3..ef3787db65d 100644 --- a/vendor/github.com/coreos/etcd/raft/progress.go +++ b/vendor/github.com/coreos/etcd/raft/progress.go @@ -48,6 +48,7 @@ type Progress struct { // When in ProgressStateSnapshot, leader should have sent out snapshot // before and stops sending any replication message. State ProgressStateType + // Paused is used in ProgressStateProbe. // When Paused is true, raft should pause sending replication message to this peer. Paused bool @@ -76,6 +77,9 @@ type Progress struct { // be freed by calling inflights.freeTo with the index of the last // received entry. ins *inflights + + // IsLearner is true if this progress is tracked for a learner. + IsLearner bool } func (pr *Progress) resetState(state ProgressStateType) { @@ -243,7 +247,8 @@ func (in *inflights) freeTo(to uint64) { return } - i, idx := 0, in.start + idx := in.start + var i int for i = 0; i < in.count; i++ { if to < in.buffer[idx] { // found the first large inflight break diff --git a/vendor/github.com/coreos/etcd/raft/raft.go b/vendor/github.com/coreos/etcd/raft/raft.go index 29f20398203..b4c0f0248ca 100644 --- a/vendor/github.com/coreos/etcd/raft/raft.go +++ b/vendor/github.com/coreos/etcd/raft/raft.go @@ -116,6 +116,10 @@ type Config struct { // used for testing right now. peers []uint64 + // learners contains the IDs of all leaner nodes (including self if the local node is a leaner) in the raft cluster. + // learners only receives entries from the leader node. It does not vote or promote itself. + learners []uint64 + // ElectionTick is the number of Node.Tick invocations that must pass between // elections. That is, if a follower does not receive any message from the // leader of current term before ElectionTick has elapsed, it will become @@ -171,11 +175,22 @@ type Config struct { // If the clock drift is unbounded, leader might keep the lease longer than it // should (clock can move backward/pause without any bound). ReadIndex is not safe // in that case. + // CheckQuorum MUST be enabled if ReadOnlyOption is ReadOnlyLeaseBased. ReadOnlyOption ReadOnlyOption // Logger is the logger used for raft log. For multinode which can host // multiple raft group, each raft group can have its own logger Logger Logger + + // DisableProposalForwarding set to true means that followers will drop + // proposals, rather than forwarding them to the leader. One use case for + // this feature would be in a situation where the Raft leader is used to + // compute the data of a proposal, for example, adding a timestamp from a + // hybrid logical clock to data in a monotonically increasing way. Forwarding + // should be disabled to prevent a follower with an innaccurate hybrid + // logical clock from assigning the timestamp and then forwarding the data + // to the leader. + DisableProposalForwarding bool } func (c *Config) validate() error { @@ -203,6 +218,10 @@ func (c *Config) validate() error { c.Logger = raftLogger } + if c.ReadOnlyOption == ReadOnlyLeaseBased && !c.CheckQuorum { + return errors.New("CheckQuorum must be enabled when ReadOnlyOption is ReadOnlyLeaseBased") + } + return nil } @@ -220,9 +239,13 @@ type raft struct { maxInflight int maxMsgSize uint64 prs map[uint64]*Progress + learnerPrs map[uint64]*Progress state StateType + // isLearner is true if the local raft node is a learner. + isLearner bool + votes map[uint64]bool msgs []pb.Message @@ -256,6 +279,7 @@ type raft struct { // [electiontimeout, 2 * electiontimeout - 1]. It gets reset // when raft changes its state to follower or candidate. randomizedElectionTimeout int + disableProposalForwarding bool tick func() step stepFunc @@ -273,32 +297,47 @@ func newRaft(c *Config) *raft { panic(err) // TODO(bdarnell) } peers := c.peers - if len(cs.Nodes) > 0 { - if len(peers) > 0 { + learners := c.learners + if len(cs.Nodes) > 0 || len(cs.Learners) > 0 { + if len(peers) > 0 || len(learners) > 0 { // TODO(bdarnell): the peers argument is always nil except in // tests; the argument should be removed and these tests should be // updated to specify their nodes through a snapshot. - panic("cannot specify both newRaft(peers) and ConfState.Nodes)") + panic("cannot specify both newRaft(peers, learners) and ConfState.(Nodes, Learners)") } peers = cs.Nodes + learners = cs.Learners } r := &raft{ - id: c.ID, - lead: None, - raftLog: raftlog, - maxMsgSize: c.MaxSizePerMsg, - maxInflight: c.MaxInflightMsgs, - prs: make(map[uint64]*Progress), - electionTimeout: c.ElectionTick, - heartbeatTimeout: c.HeartbeatTick, - logger: c.Logger, - checkQuorum: c.CheckQuorum, - preVote: c.PreVote, - readOnly: newReadOnly(c.ReadOnlyOption), + id: c.ID, + lead: None, + isLearner: false, + raftLog: raftlog, + maxMsgSize: c.MaxSizePerMsg, + maxInflight: c.MaxInflightMsgs, + prs: make(map[uint64]*Progress), + learnerPrs: make(map[uint64]*Progress), + electionTimeout: c.ElectionTick, + heartbeatTimeout: c.HeartbeatTick, + logger: c.Logger, + checkQuorum: c.CheckQuorum, + preVote: c.PreVote, + readOnly: newReadOnly(c.ReadOnlyOption), + disableProposalForwarding: c.DisableProposalForwarding, } for _, p := range peers { r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)} } + for _, p := range learners { + if _, ok := r.prs[p]; ok { + panic(fmt.Sprintf("node %x is in both learner and peer list", p)) + } + r.learnerPrs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight), IsLearner: true} + if r.id == p { + r.isLearner = true + } + } + if !isHardStateEqual(hs, emptyState) { r.loadState(hs) } @@ -332,10 +371,13 @@ func (r *raft) hardState() pb.HardState { func (r *raft) quorum() int { return len(r.prs)/2 + 1 } func (r *raft) nodes() []uint64 { - nodes := make([]uint64, 0, len(r.prs)) + nodes := make([]uint64, 0, len(r.prs)+len(r.learnerPrs)) for id := range r.prs { nodes = append(nodes, id) } + for id := range r.learnerPrs { + nodes = append(nodes, id) + } sort.Sort(uint64Slice(nodes)) return nodes } @@ -343,10 +385,20 @@ func (r *raft) nodes() []uint64 { // send persists state to stable storage and then sends to its mailbox. func (r *raft) send(m pb.Message) { m.From = r.id - if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote { + if m.Type == pb.MsgVote || m.Type == pb.MsgVoteResp || m.Type == pb.MsgPreVote || m.Type == pb.MsgPreVoteResp { if m.Term == 0 { - // PreVote RPCs are sent at a term other than our actual term, so the code - // that sends these messages is responsible for setting the term. + // All {pre-,}campaign messages need to have the term set when + // sending. + // - MsgVote: m.Term is the term the node is campaigning for, + // non-zero as we increment the term when campaigning. + // - MsgVoteResp: m.Term is the new r.Term if the MsgVote was + // granted, non-zero for the same reason MsgVote is + // - MsgPreVote: m.Term is the term the node will campaign, + // non-zero as we use m.Term to indicate the next term we'll be + // campaigning for + // - MsgPreVoteResp: m.Term is the term received in the original + // MsgPreVote if the pre-vote was granted, non-zero for the + // same reasons MsgPreVote is panic(fmt.Sprintf("term should be set when sending %s", m.Type)) } } else { @@ -364,9 +416,17 @@ func (r *raft) send(m pb.Message) { r.msgs = append(r.msgs, m) } +func (r *raft) getProgress(id uint64) *Progress { + if pr, ok := r.prs[id]; ok { + return pr + } + + return r.learnerPrs[id] +} + // sendAppend sends RPC, with entries to the given peer. func (r *raft) sendAppend(to uint64) { - pr := r.prs[to] + pr := r.getProgress(to) if pr.IsPaused() { return } @@ -431,7 +491,7 @@ func (r *raft) sendHeartbeat(to uint64, ctx []byte) { // or it might not have all the committed entries. // The leader MUST NOT forward the follower's commit to // an unmatched index. - commit := min(r.prs[to].Match, r.raftLog.committed) + commit := min(r.getProgress(to).Match, r.raftLog.committed) m := pb.Message{ To: to, Type: pb.MsgHeartbeat, @@ -442,15 +502,26 @@ func (r *raft) sendHeartbeat(to uint64, ctx []byte) { r.send(m) } +func (r *raft) forEachProgress(f func(id uint64, pr *Progress)) { + for id, pr := range r.prs { + f(id, pr) + } + + for id, pr := range r.learnerPrs { + f(id, pr) + } +} + // bcastAppend sends RPC, with entries to all peers that are not up-to-date // according to the progress recorded in r.prs. func (r *raft) bcastAppend() { - for id := range r.prs { + r.forEachProgress(func(id uint64, _ *Progress) { if id == r.id { - continue + return } + r.sendAppend(id) - } + }) } // bcastHeartbeat sends RPC, without entries to all the peers. @@ -464,12 +535,12 @@ func (r *raft) bcastHeartbeat() { } func (r *raft) bcastHeartbeatWithCtx(ctx []byte) { - for id := range r.prs { + r.forEachProgress(func(id uint64, _ *Progress) { if id == r.id { - continue + return } r.sendHeartbeat(id, ctx) - } + }) } // maybeCommit attempts to advance the commit index. Returns true if @@ -478,8 +549,8 @@ func (r *raft) bcastHeartbeatWithCtx(ctx []byte) { func (r *raft) maybeCommit() bool { // TODO(bmizerany): optimize.. Currently naive mis := make(uint64Slice, 0, len(r.prs)) - for id := range r.prs { - mis = append(mis, r.prs[id].Match) + for _, p := range r.prs { + mis = append(mis, p.Match) } sort.Sort(sort.Reverse(mis)) mci := mis[r.quorum()-1] @@ -500,12 +571,13 @@ func (r *raft) reset(term uint64) { r.abortLeaderTransfer() r.votes = make(map[uint64]bool) - for id := range r.prs { - r.prs[id] = &Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight)} + r.forEachProgress(func(id uint64, pr *Progress) { + *pr = Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight), IsLearner: pr.IsLearner} if id == r.id { - r.prs[id].Match = r.raftLog.lastIndex() + pr.Match = r.raftLog.lastIndex() } - } + }) + r.pendingConf = false r.readOnly = newReadOnly(r.readOnly.option) } @@ -517,7 +589,7 @@ func (r *raft) appendEntry(es ...pb.Entry) { es[i].Index = li + 1 + uint64(i) } r.raftLog.append(es...) - r.prs[r.id].maybeUpdate(r.raftLog.lastIndex()) + r.getProgress(r.id).maybeUpdate(r.raftLog.lastIndex()) // Regardless of maybeCommit's return, our caller will call bcastAppend. r.maybeCommit() } @@ -589,6 +661,7 @@ func (r *raft) becomePreCandidate() { // but doesn't change anything else. In particular it does not increase // r.Term or change r.Vote. r.step = stepCandidate + r.votes = make(map[uint64]bool) r.tick = r.tickElection r.state = StatePreCandidate r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term) @@ -682,7 +755,6 @@ func (r *raft) Step(m pb.Message) error { case m.Term == 0: // local message case m.Term > r.Term: - lead := m.From if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote { force := bytes.Equal(m.Context, []byte(campaignTransfer)) inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout @@ -693,7 +765,6 @@ func (r *raft) Step(m pb.Message) error { r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed) return nil } - lead = None } switch { case m.Type == pb.MsgPreVote: @@ -707,7 +778,11 @@ func (r *raft) Step(m pb.Message) error { default: r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]", r.id, r.Term, m.Type, m.From, m.Term) - r.becomeFollower(m.Term, lead) + if m.Type == pb.MsgApp || m.Type == pb.MsgHeartbeat || m.Type == pb.MsgSnap { + r.becomeFollower(m.Term, m.From) + } else { + r.becomeFollower(m.Term, None) + } } case m.Term < r.Term: @@ -757,12 +832,27 @@ func (r *raft) Step(m pb.Message) error { } case pb.MsgVote, pb.MsgPreVote: + if r.isLearner { + // TODO: learner may need to vote, in case of node down when confchange. + r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored %s from %x [logterm: %d, index: %d] at term %d: learner can not vote", + r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term) + return nil + } // The m.Term > r.Term clause is for MsgPreVote. For MsgVote m.Term should // always equal r.Term. if (r.Vote == None || m.Term > r.Term || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) { r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] cast %s for %x [logterm: %d, index: %d] at term %d", r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term) - r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type)}) + // When responding to Msg{Pre,}Vote messages we include the term + // from the message, not the local term. To see why consider the + // case where a single node was previously partitioned away and + // it's local term is now of date. If we include the local term + // (recall that for pre-votes we don't update the local term), the + // (pre-)campaigning node on the other end will proceed to ignore + // the message (it ignores all out of date messages). + // The term in the original message and current local term are the + // same in the case of regular votes, but different for pre-votes. + r.send(pb.Message{To: m.From, Term: m.Term, Type: voteRespMsgType(m.Type)}) if m.Type == pb.MsgVote { // Only record real votes. r.electionElapsed = 0 @@ -771,7 +861,7 @@ func (r *raft) Step(m pb.Message) error { } else { r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d", r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term) - r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type), Reject: true}) + r.send(pb.Message{To: m.From, Term: r.Term, Type: voteRespMsgType(m.Type), Reject: true}) } default: @@ -836,10 +926,7 @@ func stepLeader(r *raft, m pb.Message) { r.readOnly.addRequest(r.raftLog.committed, m) r.bcastHeartbeatWithCtx(m.Entries[0].Data) case ReadOnlyLeaseBased: - var ri uint64 - if r.checkQuorum { - ri = r.raftLog.committed - } + ri := r.raftLog.committed if m.From == None || m.From == r.id { // from local member r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data}) } else { @@ -854,8 +941,8 @@ func stepLeader(r *raft, m pb.Message) { } // All other message types require a progress for m.From (pr). - pr, prOk := r.prs[m.From] - if !prOk { + pr := r.getProgress(m.From) + if pr == nil { r.logger.Debugf("%x no progress available for %x", r.id, m.From) return } @@ -954,6 +1041,10 @@ func stepLeader(r *raft, m pb.Message) { } r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr) case pb.MsgTransferLeader: + if pr.IsLearner { + r.logger.Debugf("%x is learner. Ignored transferring leadership", r.id) + return + } leadTransferee := m.From lastLeadTransferee := r.leadTransferee if lastLeadTransferee != None { @@ -1033,6 +1124,9 @@ func stepFollower(r *raft, m pb.Message) { if r.lead == None { r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term) return + } else if r.disableProposalForwarding { + r.logger.Infof("%x not forwarding to leader %x at term %d; dropping proposal", r.id, r.lead, r.Term) + return } m.To = r.lead r.send(m) @@ -1127,20 +1221,37 @@ func (r *raft) restore(s pb.Snapshot) bool { return false } + // The normal peer can't become learner. + if !r.isLearner { + for _, id := range s.Metadata.ConfState.Learners { + if id == r.id { + r.logger.Errorf("%x can't become learner when restores snapshot [index: %d, term: %d]", r.id, s.Metadata.Index, s.Metadata.Term) + return false + } + } + } + r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]", r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term) r.raftLog.restore(s) r.prs = make(map[uint64]*Progress) - for _, n := range s.Metadata.ConfState.Nodes { + r.learnerPrs = make(map[uint64]*Progress) + r.restoreNode(s.Metadata.ConfState.Nodes, false) + r.restoreNode(s.Metadata.ConfState.Learners, true) + return true +} + +func (r *raft) restoreNode(nodes []uint64, isLearner bool) { + for _, n := range nodes { match, next := uint64(0), r.raftLog.lastIndex()+1 if n == r.id { match = next - 1 + r.isLearner = isLearner } - r.setProgress(n, match, next) - r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n]) + r.setProgress(n, match, next, isLearner) + r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.getProgress(n)) } - return true } // promotable indicates whether state machine can be promoted to leader, @@ -1151,18 +1262,46 @@ func (r *raft) promotable() bool { } func (r *raft) addNode(id uint64) { + r.addNodeOrLearnerNode(id, false) +} + +func (r *raft) addLearner(id uint64) { + r.addNodeOrLearnerNode(id, true) +} + +func (r *raft) addNodeOrLearnerNode(id uint64, isLearner bool) { r.pendingConf = false - if _, ok := r.prs[id]; ok { - // Ignore any redundant addNode calls (which can happen because the - // initial bootstrapping entries are applied twice). - return + pr := r.getProgress(id) + if pr == nil { + r.setProgress(id, 0, r.raftLog.lastIndex()+1, isLearner) + } else { + if isLearner && !pr.IsLearner { + // can only change Learner to Voter + r.logger.Infof("%x ignored addLeaner: do not support changing %x from raft peer to learner.", r.id, id) + return + } + + if isLearner == pr.IsLearner { + // Ignore any redundant addNode calls (which can happen because the + // initial bootstrapping entries are applied twice). + return + } + + // change Learner to Voter, use origin Learner progress + delete(r.learnerPrs, id) + pr.IsLearner = false + r.prs[id] = pr + } + + if r.id == id { + r.isLearner = isLearner } - r.setProgress(id, 0, r.raftLog.lastIndex()+1) // When a node is first added, we should mark it as recently active. // Otherwise, CheckQuorum may cause us to step down if it is invoked // before the added node has a chance to communicate with us. - r.prs[id].RecentActive = true + pr = r.getProgress(id) + pr.RecentActive = true } func (r *raft) removeNode(id uint64) { @@ -1170,7 +1309,7 @@ func (r *raft) removeNode(id uint64) { r.pendingConf = false // do not try to commit or abort transferring if there is no nodes in the cluster. - if len(r.prs) == 0 { + if len(r.prs) == 0 && len(r.learnerPrs) == 0 { return } @@ -1187,12 +1326,22 @@ func (r *raft) removeNode(id uint64) { func (r *raft) resetPendingConf() { r.pendingConf = false } -func (r *raft) setProgress(id, match, next uint64) { - r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)} +func (r *raft) setProgress(id, match, next uint64, isLearner bool) { + if !isLearner { + delete(r.learnerPrs, id) + r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)} + return + } + + if _, ok := r.prs[id]; ok { + panic(fmt.Sprintf("%x unexpected changing from voter to learner for %x", r.id, id)) + } + r.learnerPrs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight), IsLearner: true} } func (r *raft) delProgress(id uint64) { delete(r.prs, id) + delete(r.learnerPrs, id) } func (r *raft) loadState(state pb.HardState) { @@ -1222,18 +1371,18 @@ func (r *raft) resetRandomizedElectionTimeout() { func (r *raft) checkQuorumActive() bool { var act int - for id := range r.prs { + r.forEachProgress(func(id uint64, pr *Progress) { if id == r.id { // self is always active act++ - continue + return } - if r.prs[id].RecentActive { + if pr.RecentActive && !pr.IsLearner { act++ } - r.prs[id].RecentActive = false - } + pr.RecentActive = false + }) return act >= r.quorum() } diff --git a/vendor/github.com/coreos/etcd/raft/raftpb/raft.pb.go b/vendor/github.com/coreos/etcd/raft/raftpb/raft.pb.go index 3c45eef003c..fd9ee3729ec 100644 --- a/vendor/github.com/coreos/etcd/raft/raftpb/raft.pb.go +++ b/vendor/github.com/coreos/etcd/raft/raftpb/raft.pb.go @@ -1,6 +1,5 @@ -// Code generated by protoc-gen-gogo. +// Code generated by protoc-gen-gogo. DO NOT EDIT. // source: raft.proto -// DO NOT EDIT! /* Package raftpb is a generated protocol buffer package. @@ -26,6 +25,8 @@ import ( math "math" + _ "github.com/gogo/protobuf/gogoproto" + io "io" ) @@ -162,20 +163,23 @@ func (MessageType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, type ConfChangeType int32 const ( - ConfChangeAddNode ConfChangeType = 0 - ConfChangeRemoveNode ConfChangeType = 1 - ConfChangeUpdateNode ConfChangeType = 2 + ConfChangeAddNode ConfChangeType = 0 + ConfChangeRemoveNode ConfChangeType = 1 + ConfChangeUpdateNode ConfChangeType = 2 + ConfChangeAddLearnerNode ConfChangeType = 3 ) var ConfChangeType_name = map[int32]string{ 0: "ConfChangeAddNode", 1: "ConfChangeRemoveNode", 2: "ConfChangeUpdateNode", + 3: "ConfChangeAddLearnerNode", } var ConfChangeType_value = map[string]int32{ - "ConfChangeAddNode": 0, - "ConfChangeRemoveNode": 1, - "ConfChangeUpdateNode": 2, + "ConfChangeAddNode": 0, + "ConfChangeRemoveNode": 1, + "ConfChangeUpdateNode": 2, + "ConfChangeAddLearnerNode": 3, } func (x ConfChangeType) Enum() *ConfChangeType { @@ -267,6 +271,7 @@ func (*HardState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []in type ConfState struct { Nodes []uint64 `protobuf:"varint,1,rep,name=nodes" json:"nodes,omitempty"` + Learners []uint64 `protobuf:"varint,2,rep,name=learners" json:"learners,omitempty"` XXX_unrecognized []byte `json:"-"` } @@ -537,6 +542,13 @@ func (m *ConfState) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintRaft(dAtA, i, uint64(num)) } } + if len(m.Learners) > 0 { + for _, num := range m.Learners { + dAtA[i] = 0x10 + i++ + i = encodeVarintRaft(dAtA, i, uint64(num)) + } + } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) } @@ -579,24 +591,6 @@ func (m *ConfChange) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func encodeFixed64Raft(dAtA []byte, offset int, v uint64) int { - dAtA[offset] = uint8(v) - dAtA[offset+1] = uint8(v >> 8) - dAtA[offset+2] = uint8(v >> 16) - dAtA[offset+3] = uint8(v >> 24) - dAtA[offset+4] = uint8(v >> 32) - dAtA[offset+5] = uint8(v >> 40) - dAtA[offset+6] = uint8(v >> 48) - dAtA[offset+7] = uint8(v >> 56) - return offset + 8 -} -func encodeFixed32Raft(dAtA []byte, offset int, v uint32) int { - dAtA[offset] = uint8(v) - dAtA[offset+1] = uint8(v >> 8) - dAtA[offset+2] = uint8(v >> 16) - dAtA[offset+3] = uint8(v >> 24) - return offset + 4 -} func encodeVarintRaft(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -700,6 +694,11 @@ func (m *ConfState) Size() (n int) { n += 1 + sovRaft(uint64(e)) } } + if len(m.Learners) > 0 { + for _, e := range m.Learners { + n += 1 + sovRaft(uint64(e)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -1558,25 +1557,129 @@ func (m *ConfState) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType != 0 { + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Nodes = append(m.Nodes, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Nodes = append(m.Nodes, v) + } + } else { return fmt.Errorf("proto: wrong wireType = %d for field Nodes", wireType) } - var v uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRaft + case 2: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { + m.Learners = append(m.Learners, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + packedLen + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - v |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Learners = append(m.Learners, v) } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Learners", wireType) } - m.Nodes = append(m.Nodes, v) default: iNdEx = preIndex skippy, err := skipRaft(dAtA[iNdEx:]) @@ -1846,55 +1949,56 @@ var ( func init() { proto.RegisterFile("raft.proto", fileDescriptorRaft) } var fileDescriptorRaft = []byte{ - // 790 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x54, 0xcd, 0x6e, 0xdb, 0x46, - 0x10, 0x16, 0x29, 0xea, 0x6f, 0x28, 0xcb, 0xab, 0xb5, 0x5a, 0x2c, 0x0c, 0x43, 0x55, 0x85, 0x1e, - 0x04, 0x17, 0x76, 0x5b, 0x1d, 0x7a, 0xe8, 0xcd, 0x96, 0x0a, 0x58, 0x40, 0x65, 0xb8, 0xb2, 0xdc, - 0x43, 0x83, 0x20, 0x58, 0x8b, 0x2b, 0x4a, 0x89, 0xc9, 0x25, 0x96, 0x2b, 0xc7, 0xbe, 0x04, 0x79, - 0x80, 0x3c, 0x40, 0x2e, 0x79, 0x1f, 0x1f, 0x0d, 0xe4, 0x1e, 0xc4, 0xce, 0x8b, 0x04, 0xbb, 0x5c, - 0x4a, 0x94, 0x74, 0xdb, 0xf9, 0xbe, 0xe1, 0xcc, 0x37, 0xdf, 0xce, 0x12, 0x40, 0xd0, 0xa9, 0x3c, - 0x8e, 0x04, 0x97, 0x1c, 0x17, 0xd5, 0x39, 0xba, 0xde, 0x6f, 0xf8, 0xdc, 0xe7, 0x1a, 0xfa, 0x4d, - 0x9d, 0x12, 0xb6, 0xfd, 0x0e, 0x0a, 0x7f, 0x87, 0x52, 0xdc, 0xe3, 0x5f, 0xc1, 0x19, 0xdf, 0x47, - 0x8c, 0x58, 0x2d, 0xab, 0x53, 0xeb, 0xd6, 0x8f, 0x93, 0xaf, 0x8e, 0x35, 0xa9, 0x88, 0x53, 0xe7, - 0xe1, 0xcb, 0x4f, 0xb9, 0x91, 0x4e, 0xc2, 0x04, 0x9c, 0x31, 0x13, 0x01, 0xb1, 0x5b, 0x56, 0xc7, - 0x59, 0x32, 0x4c, 0x04, 0x78, 0x1f, 0x0a, 0x83, 0xd0, 0x63, 0x77, 0x24, 0x9f, 0xa1, 0x12, 0x08, - 0x63, 0x70, 0xfa, 0x54, 0x52, 0xe2, 0xb4, 0xac, 0x4e, 0x75, 0xa4, 0xcf, 0xed, 0xf7, 0x16, 0xa0, - 0xcb, 0x90, 0x46, 0xf1, 0x8c, 0xcb, 0x21, 0x93, 0xd4, 0xa3, 0x92, 0xe2, 0x3f, 0x01, 0x26, 0x3c, - 0x9c, 0xbe, 0x8a, 0x25, 0x95, 0x89, 0x22, 0x77, 0xa5, 0xa8, 0xc7, 0xc3, 0xe9, 0xa5, 0x22, 0x4c, - 0xf1, 0xca, 0x24, 0x05, 0x54, 0xf3, 0xb9, 0x6e, 0x9e, 0xd5, 0x95, 0x40, 0x4a, 0xb2, 0x54, 0x92, - 0xb3, 0xba, 0x34, 0xd2, 0xfe, 0x1f, 0xca, 0xa9, 0x02, 0x25, 0x51, 0x29, 0xd0, 0x3d, 0xab, 0x23, - 0x7d, 0xc6, 0x7f, 0x41, 0x39, 0x30, 0xca, 0x74, 0x61, 0xb7, 0x4b, 0x52, 0x2d, 0x9b, 0xca, 0x4d, - 0xdd, 0x65, 0x7e, 0xfb, 0x53, 0x1e, 0x4a, 0x43, 0x16, 0xc7, 0xd4, 0x67, 0xf8, 0x08, 0x1c, 0xb9, - 0x72, 0x78, 0x2f, 0xad, 0x61, 0xe8, 0xac, 0xc7, 0x2a, 0x0d, 0x37, 0xc0, 0x96, 0x7c, 0x6d, 0x12, - 0x5b, 0x72, 0x35, 0xc6, 0x54, 0xf0, 0x8d, 0x31, 0x14, 0xb2, 0x1c, 0xd0, 0xd9, 0x1c, 0x10, 0x37, - 0xa1, 0x74, 0xc3, 0x7d, 0x7d, 0x61, 0x85, 0x0c, 0x99, 0x82, 0x2b, 0xdb, 0x8a, 0xdb, 0xb6, 0x1d, - 0x41, 0x89, 0x85, 0x52, 0xcc, 0x59, 0x4c, 0x4a, 0xad, 0x7c, 0xc7, 0xed, 0xee, 0xac, 0x6d, 0x46, - 0x5a, 0xca, 0xe4, 0xe0, 0x03, 0x28, 0x4e, 0x78, 0x10, 0xcc, 0x25, 0x29, 0x67, 0x6a, 0x19, 0x0c, - 0x77, 0xa1, 0x1c, 0x1b, 0xc7, 0x48, 0x45, 0x3b, 0x89, 0x36, 0x9d, 0x4c, 0x1d, 0x4c, 0xf3, 0x54, - 0x45, 0xc1, 0x5e, 0xb3, 0x89, 0x24, 0xd0, 0xb2, 0x3a, 0xe5, 0xb4, 0x62, 0x82, 0xe1, 0x5f, 0x00, - 0x92, 0xd3, 0xd9, 0x3c, 0x94, 0xc4, 0xcd, 0xf4, 0xcc, 0xe0, 0x98, 0x40, 0x69, 0xc2, 0x43, 0xc9, - 0xee, 0x24, 0xa9, 0xea, 0x8b, 0x4d, 0xc3, 0xf6, 0x4b, 0xa8, 0x9c, 0x51, 0xe1, 0x25, 0xeb, 0x93, - 0x3a, 0x68, 0x6d, 0x39, 0x48, 0xc0, 0xb9, 0xe5, 0x92, 0xad, 0xef, 0xbb, 0x42, 0x32, 0x03, 0xe7, - 0xb7, 0x07, 0x6e, 0xff, 0x0c, 0x95, 0xe5, 0xba, 0xe2, 0x06, 0x14, 0x42, 0xee, 0xb1, 0x98, 0x58, - 0xad, 0x7c, 0xc7, 0x19, 0x25, 0x41, 0xfb, 0x83, 0x05, 0xa0, 0x72, 0x7a, 0x33, 0x1a, 0xfa, 0xfa, - 0xd6, 0x07, 0xfd, 0x35, 0x05, 0xf6, 0xa0, 0x8f, 0x7f, 0x37, 0x8f, 0xd3, 0xd6, 0xab, 0xf3, 0x63, - 0xf6, 0x29, 0x24, 0xdf, 0x6d, 0xbd, 0xd0, 0x03, 0x28, 0x9e, 0x73, 0x8f, 0x0d, 0xfa, 0xeb, 0xba, - 0x12, 0x4c, 0x19, 0xd2, 0x33, 0x86, 0x24, 0x8f, 0x31, 0x0d, 0x0f, 0xff, 0x80, 0xca, 0xf2, 0xc9, - 0xe3, 0x5d, 0x70, 0x75, 0x70, 0xce, 0x45, 0x40, 0x6f, 0x50, 0x0e, 0xef, 0xc1, 0xae, 0x06, 0x56, - 0x8d, 0x91, 0x75, 0xf8, 0xd9, 0x06, 0x37, 0xb3, 0xc4, 0x18, 0xa0, 0x38, 0x8c, 0xfd, 0xb3, 0x45, - 0x84, 0x72, 0xd8, 0x85, 0xd2, 0x30, 0xf6, 0x4f, 0x19, 0x95, 0xc8, 0x32, 0xc1, 0x85, 0xe0, 0x11, - 0xb2, 0x4d, 0xd6, 0x49, 0x14, 0xa1, 0x3c, 0xae, 0x01, 0x24, 0xe7, 0x11, 0x8b, 0x23, 0xe4, 0x98, - 0xc4, 0xff, 0xb8, 0x64, 0xa8, 0xa0, 0x44, 0x98, 0x40, 0xb3, 0x45, 0xc3, 0xaa, 0x85, 0x41, 0x25, - 0x8c, 0xa0, 0xaa, 0x9a, 0x31, 0x2a, 0xe4, 0xb5, 0xea, 0x52, 0xc6, 0x0d, 0x40, 0x59, 0x44, 0x7f, - 0x54, 0xc1, 0x18, 0x6a, 0xc3, 0xd8, 0xbf, 0x0a, 0x05, 0xa3, 0x93, 0x19, 0xbd, 0xbe, 0x61, 0x08, - 0x70, 0x1d, 0x76, 0x4c, 0x21, 0x75, 0x41, 0x8b, 0x18, 0xb9, 0x26, 0xad, 0x37, 0x63, 0x93, 0x37, - 0xff, 0x2e, 0xb8, 0x58, 0x04, 0xa8, 0x8a, 0x7f, 0x80, 0xfa, 0x30, 0xf6, 0xc7, 0x82, 0x86, 0xf1, - 0x94, 0x89, 0x7f, 0x18, 0xf5, 0x98, 0x40, 0x3b, 0xe6, 0xeb, 0xf1, 0x3c, 0x60, 0x7c, 0x21, 0xcf, - 0xf9, 0x5b, 0x54, 0x33, 0x62, 0x46, 0x8c, 0x7a, 0xfa, 0x87, 0x87, 0x76, 0x8d, 0x98, 0x25, 0xa2, - 0xc5, 0x20, 0x33, 0xef, 0x85, 0x60, 0x7a, 0xc4, 0xba, 0xe9, 0x6a, 0x62, 0x9d, 0x83, 0x0f, 0x5f, - 0x40, 0x6d, 0xfd, 0x7a, 0x95, 0x8e, 0x15, 0x72, 0xe2, 0x79, 0xea, 0x2e, 0x51, 0x0e, 0x13, 0x68, - 0xac, 0xe0, 0x11, 0x0b, 0xf8, 0x2d, 0xd3, 0x8c, 0xb5, 0xce, 0x5c, 0x45, 0x1e, 0x95, 0x09, 0x63, - 0x9f, 0x92, 0x87, 0xa7, 0x66, 0xee, 0xf1, 0xa9, 0x99, 0x7b, 0x78, 0x6e, 0x5a, 0x8f, 0xcf, 0x4d, - 0xeb, 0xeb, 0x73, 0xd3, 0xfa, 0xf8, 0xad, 0x99, 0xfb, 0x1e, 0x00, 0x00, 0xff, 0xff, 0xcf, 0x30, - 0x01, 0x41, 0x3a, 0x06, 0x00, 0x00, + // 815 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x54, 0xcd, 0x6e, 0x23, 0x45, + 0x10, 0xf6, 0x8c, 0xc7, 0x7f, 0x35, 0x8e, 0xd3, 0xa9, 0x35, 0xa8, 0x15, 0x45, 0xc6, 0xb2, 0x38, + 0x58, 0x41, 0x1b, 0x20, 0x07, 0x0e, 0x48, 0x1c, 0x36, 0x09, 0x52, 0x22, 0xad, 0xa3, 0xc5, 0x9b, + 0xe5, 0x80, 0x84, 0x50, 0xc7, 0x53, 0x9e, 0x18, 0x32, 0xd3, 0xa3, 0x9e, 0xf6, 0xb2, 0xb9, 0x20, + 0x1e, 0x80, 0x07, 0xe0, 0xc2, 0xfb, 0xe4, 0xb8, 0x12, 0x77, 0xc4, 0x86, 0x17, 0x41, 0xdd, 0xd3, + 0x63, 0xcf, 0x24, 0xb7, 0xae, 0xef, 0xab, 0xae, 0xfa, 0xea, 0xeb, 0x9a, 0x01, 0x50, 0x62, 0xa9, + 0x8f, 0x32, 0x25, 0xb5, 0xc4, 0xb6, 0x39, 0x67, 0xd7, 0xfb, 0xc3, 0x58, 0xc6, 0xd2, 0x42, 0x9f, + 0x9b, 0x53, 0xc1, 0x4e, 0x7e, 0x83, 0xd6, 0xb7, 0xa9, 0x56, 0x77, 0xf8, 0x19, 0x04, 0x57, 0x77, + 0x19, 0x71, 0x6f, 0xec, 0x4d, 0x07, 0xc7, 0x7b, 0x47, 0xc5, 0xad, 0x23, 0x4b, 0x1a, 0xe2, 0x24, + 0xb8, 0xff, 0xe7, 0x93, 0xc6, 0xdc, 0x26, 0x21, 0x87, 0xe0, 0x8a, 0x54, 0xc2, 0xfd, 0xb1, 0x37, + 0x0d, 0x36, 0x0c, 0xa9, 0x04, 0xf7, 0xa1, 0x75, 0x91, 0x46, 0xf4, 0x8e, 0x37, 0x2b, 0x54, 0x01, + 0x21, 0x42, 0x70, 0x26, 0xb4, 0xe0, 0xc1, 0xd8, 0x9b, 0xf6, 0xe7, 0xf6, 0x3c, 0xf9, 0xdd, 0x03, + 0xf6, 0x3a, 0x15, 0x59, 0x7e, 0x23, 0xf5, 0x8c, 0xb4, 0x88, 0x84, 0x16, 0xf8, 0x15, 0xc0, 0x42, + 0xa6, 0xcb, 0x9f, 0x72, 0x2d, 0x74, 0xa1, 0x28, 0xdc, 0x2a, 0x3a, 0x95, 0xe9, 0xf2, 0xb5, 0x21, + 0x5c, 0xf1, 0xde, 0xa2, 0x04, 0x4c, 0xf3, 0x95, 0x6d, 0x5e, 0xd5, 0x55, 0x40, 0x46, 0xb2, 0x36, + 0x92, 0xab, 0xba, 0x2c, 0x32, 0xf9, 0x01, 0xba, 0xa5, 0x02, 0x23, 0xd1, 0x28, 0xb0, 0x3d, 0xfb, + 0x73, 0x7b, 0xc6, 0xaf, 0xa1, 0x9b, 0x38, 0x65, 0xb6, 0x70, 0x78, 0xcc, 0x4b, 0x2d, 0x8f, 0x95, + 0xbb, 0xba, 0x9b, 0xfc, 0xc9, 0x5f, 0x4d, 0xe8, 0xcc, 0x28, 0xcf, 0x45, 0x4c, 0xf8, 0x1c, 0x02, + 0xbd, 0x75, 0xf8, 0x59, 0x59, 0xc3, 0xd1, 0x55, 0x8f, 0x4d, 0x1a, 0x0e, 0xc1, 0xd7, 0xb2, 0x36, + 0x89, 0xaf, 0xa5, 0x19, 0x63, 0xa9, 0xe4, 0xa3, 0x31, 0x0c, 0xb2, 0x19, 0x30, 0x78, 0x3c, 0x20, + 0x8e, 0xa0, 0x73, 0x2b, 0x63, 0xfb, 0x60, 0xad, 0x0a, 0x59, 0x82, 0x5b, 0xdb, 0xda, 0x4f, 0x6d, + 0x7b, 0x0e, 0x1d, 0x4a, 0xb5, 0x5a, 0x51, 0xce, 0x3b, 0xe3, 0xe6, 0x34, 0x3c, 0xde, 0xa9, 0x6d, + 0x46, 0x59, 0xca, 0xe5, 0xe0, 0x01, 0xb4, 0x17, 0x32, 0x49, 0x56, 0x9a, 0x77, 0x2b, 0xb5, 0x1c, + 0x86, 0xc7, 0xd0, 0xcd, 0x9d, 0x63, 0xbc, 0x67, 0x9d, 0x64, 0x8f, 0x9d, 0x2c, 0x1d, 0x2c, 0xf3, + 0x4c, 0x45, 0x45, 0x3f, 0xd3, 0x42, 0x73, 0x18, 0x7b, 0xd3, 0x6e, 0x59, 0xb1, 0xc0, 0xf0, 0x53, + 0x80, 0xe2, 0x74, 0xbe, 0x4a, 0x35, 0x0f, 0x2b, 0x3d, 0x2b, 0x38, 0x72, 0xe8, 0x2c, 0x64, 0xaa, + 0xe9, 0x9d, 0xe6, 0x7d, 0xfb, 0xb0, 0x65, 0x38, 0xf9, 0x11, 0x7a, 0xe7, 0x42, 0x45, 0xc5, 0xfa, + 0x94, 0x0e, 0x7a, 0x4f, 0x1c, 0xe4, 0x10, 0xbc, 0x95, 0x9a, 0xea, 0xfb, 0x6e, 0x90, 0xca, 0xc0, + 0xcd, 0xa7, 0x03, 0x4f, 0xbe, 0x81, 0xde, 0x66, 0x5d, 0x71, 0x08, 0xad, 0x54, 0x46, 0x94, 0x73, + 0x6f, 0xdc, 0x9c, 0x06, 0xf3, 0x22, 0xc0, 0x7d, 0xe8, 0xde, 0x92, 0x50, 0x29, 0xa9, 0x9c, 0xfb, + 0x96, 0xd8, 0xc4, 0x93, 0x3f, 0x3c, 0x00, 0x73, 0xff, 0xf4, 0x46, 0xa4, 0xb1, 0xdd, 0x88, 0x8b, + 0xb3, 0x9a, 0x3a, 0xff, 0xe2, 0x0c, 0xbf, 0x70, 0x1f, 0xae, 0x6f, 0xd7, 0xea, 0xe3, 0xea, 0x67, + 0x52, 0xdc, 0x7b, 0xf2, 0xf5, 0x1e, 0x40, 0xfb, 0x52, 0x46, 0x74, 0x71, 0x56, 0xd7, 0x5c, 0x60, + 0xc6, 0xac, 0x53, 0x67, 0x56, 0xf1, 0xa1, 0x96, 0xe1, 0xe1, 0x97, 0xd0, 0xdb, 0xfc, 0x0e, 0x70, + 0x17, 0x42, 0x1b, 0x5c, 0x4a, 0x95, 0x88, 0x5b, 0xd6, 0xc0, 0x67, 0xb0, 0x6b, 0x81, 0x6d, 0x63, + 0xe6, 0x1d, 0xfe, 0xed, 0x43, 0x58, 0x59, 0x70, 0x04, 0x68, 0xcf, 0xf2, 0xf8, 0x7c, 0x9d, 0xb1, + 0x06, 0x86, 0xd0, 0x99, 0xe5, 0xf1, 0x09, 0x09, 0xcd, 0x3c, 0x17, 0xbc, 0x52, 0x32, 0x63, 0xbe, + 0xcb, 0x7a, 0x91, 0x65, 0xac, 0x89, 0x03, 0x80, 0xe2, 0x3c, 0xa7, 0x3c, 0x63, 0x81, 0x4b, 0xfc, + 0x5e, 0x6a, 0x62, 0x2d, 0x23, 0xc2, 0x05, 0x96, 0x6d, 0x3b, 0xd6, 0x2c, 0x13, 0xeb, 0x20, 0x83, + 0xbe, 0x69, 0x46, 0x42, 0xe9, 0x6b, 0xd3, 0xa5, 0x8b, 0x43, 0x60, 0x55, 0xc4, 0x5e, 0xea, 0x21, + 0xc2, 0x60, 0x96, 0xc7, 0x6f, 0x52, 0x45, 0x62, 0x71, 0x23, 0xae, 0x6f, 0x89, 0x01, 0xee, 0xc1, + 0x8e, 0x2b, 0x64, 0x1e, 0x6f, 0x9d, 0xb3, 0xd0, 0xa5, 0x9d, 0xde, 0xd0, 0xe2, 0x97, 0xef, 0xd6, + 0x52, 0xad, 0x13, 0xd6, 0xc7, 0x8f, 0x60, 0x6f, 0x96, 0xc7, 0x57, 0x4a, 0xa4, 0xf9, 0x92, 0xd4, + 0x4b, 0x12, 0x11, 0x29, 0xb6, 0xe3, 0x6e, 0x5f, 0xad, 0x12, 0x92, 0x6b, 0x7d, 0x29, 0x7f, 0x65, + 0x03, 0x27, 0x66, 0x4e, 0x22, 0xb2, 0x3f, 0x43, 0xb6, 0xeb, 0xc4, 0x6c, 0x10, 0x2b, 0x86, 0xb9, + 0x79, 0x5f, 0x29, 0xb2, 0x23, 0xee, 0xb9, 0xae, 0x2e, 0xb6, 0x39, 0x78, 0x78, 0x07, 0x83, 0xfa, + 0xf3, 0x1a, 0x1d, 0x5b, 0xe4, 0x45, 0x14, 0x99, 0xb7, 0x64, 0x0d, 0xe4, 0x30, 0xdc, 0xc2, 0x73, + 0x4a, 0xe4, 0x5b, 0xb2, 0x8c, 0x57, 0x67, 0xde, 0x64, 0x91, 0xd0, 0x05, 0xe3, 0xe3, 0x01, 0xf0, + 0x5a, 0xa9, 0x97, 0xc5, 0x36, 0x5a, 0xb6, 0x79, 0xc2, 0xef, 0x3f, 0x8c, 0x1a, 0xef, 0x3f, 0x8c, + 0x1a, 0xf7, 0x0f, 0x23, 0xef, 0xfd, 0xc3, 0xc8, 0xfb, 0xf7, 0x61, 0xe4, 0xfd, 0xf9, 0xdf, 0xa8, + 0xf1, 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x86, 0x52, 0x5b, 0xe0, 0x74, 0x06, 0x00, 0x00, } diff --git a/vendor/github.com/coreos/etcd/raft/raftpb/raft.proto b/vendor/github.com/coreos/etcd/raft/raftpb/raft.proto index 806a43634fd..644ce7b8f2f 100644 --- a/vendor/github.com/coreos/etcd/raft/raftpb/raft.proto +++ b/vendor/github.com/coreos/etcd/raft/raftpb/raft.proto @@ -76,13 +76,15 @@ message HardState { } message ConfState { - repeated uint64 nodes = 1; + repeated uint64 nodes = 1; + repeated uint64 learners = 2; } enum ConfChangeType { - ConfChangeAddNode = 0; - ConfChangeRemoveNode = 1; - ConfChangeUpdateNode = 2; + ConfChangeAddNode = 0; + ConfChangeRemoveNode = 1; + ConfChangeUpdateNode = 2; + ConfChangeAddLearnerNode = 3; } message ConfChange { diff --git a/vendor/github.com/coreos/etcd/raft/rawnode.go b/vendor/github.com/coreos/etcd/raft/rawnode.go index b950d5169a5..925cb851c4a 100644 --- a/vendor/github.com/coreos/etcd/raft/rawnode.go +++ b/vendor/github.com/coreos/etcd/raft/rawnode.go @@ -175,6 +175,8 @@ func (rn *RawNode) ApplyConfChange(cc pb.ConfChange) *pb.ConfState { switch cc.Type { case pb.ConfChangeAddNode: rn.raft.addNode(cc.NodeID) + case pb.ConfChangeAddLearnerNode: + rn.raft.addLearner(cc.NodeID) case pb.ConfChangeRemoveNode: rn.raft.removeNode(cc.NodeID) case pb.ConfChangeUpdateNode: @@ -191,7 +193,7 @@ func (rn *RawNode) Step(m pb.Message) error { if IsLocalMsg(m.Type) { return ErrStepLocalMsg } - if _, ok := rn.raft.prs[m.From]; ok || !IsResponseMsg(m.Type) { + if pr := rn.raft.getProgress(m.From); pr != nil || !IsResponseMsg(m.Type) { return rn.raft.Step(m) } return ErrStepPeerNotFound diff --git a/vendor/github.com/coreos/etcd/raft/read_only.go b/vendor/github.com/coreos/etcd/raft/read_only.go index d0085237e36..ae746fa73eb 100644 --- a/vendor/github.com/coreos/etcd/raft/read_only.go +++ b/vendor/github.com/coreos/etcd/raft/read_only.go @@ -18,7 +18,7 @@ import pb "github.com/coreos/etcd/raft/raftpb" // ReadState provides state for read only query. // It's caller's responsibility to call ReadIndex first before getting -// this state from ready, It's also caller's duty to differentiate if this +// this state from ready, it's also caller's duty to differentiate if this // state is what it requests through RequestCtx, eg. given a unique id as // RequestCtx type ReadState struct { diff --git a/vendor/github.com/coreos/etcd/raft/status.go b/vendor/github.com/coreos/etcd/raft/status.go index b690fa56b95..f4d3d86a4e3 100644 --- a/vendor/github.com/coreos/etcd/raft/status.go +++ b/vendor/github.com/coreos/etcd/raft/status.go @@ -28,11 +28,17 @@ type Status struct { Applied uint64 Progress map[uint64]Progress + + LeadTransferee uint64 } // getStatus gets a copy of the current raft status. func getStatus(r *raft) Status { - s := Status{ID: r.id} + s := Status{ + ID: r.id, + LeadTransferee: r.leadTransferee, + } + s.HardState = r.hardState() s.SoftState = *r.softState() @@ -43,6 +49,10 @@ func getStatus(r *raft) Status { for id, p := range r.prs { s.Progress[id] = *p } + + for id, p := range r.learnerPrs { + s.Progress[id] = *p + } } return s @@ -51,19 +61,21 @@ func getStatus(r *raft) Status { // MarshalJSON translates the raft status into JSON. // TODO: try to simplify this by introducing ID type into raft func (s Status) MarshalJSON() ([]byte, error) { - j := fmt.Sprintf(`{"id":"%x","term":%d,"vote":"%x","commit":%d,"lead":"%x","raftState":%q,"progress":{`, - s.ID, s.Term, s.Vote, s.Commit, s.Lead, s.RaftState) + j := fmt.Sprintf(`{"id":"%x","term":%d,"vote":"%x","commit":%d,"lead":"%x","raftState":%q,"applied":%d,"progress":{`, + s.ID, s.Term, s.Vote, s.Commit, s.Lead, s.RaftState, s.Applied) if len(s.Progress) == 0 { - j += "}}" + j += "}," } else { for k, v := range s.Progress { subj := fmt.Sprintf(`"%x":{"match":%d,"next":%d,"state":%q},`, k, v.Match, v.Next, v.State) j += subj } // remove the trailing "," - j = j[:len(j)-1] + "}}" + j = j[:len(j)-1] + "}," } + + j += fmt.Sprintf(`"leadtransferee":"%x"}`, s.LeadTransferee) return []byte(j), nil } diff --git a/vendor/github.com/gogo/protobuf/LICENSE b/vendor/github.com/gogo/protobuf/LICENSE index 335e38e19b9..7be0cc7b62c 100644 --- a/vendor/github.com/gogo/protobuf/LICENSE +++ b/vendor/github.com/gogo/protobuf/LICENSE @@ -1,7 +1,7 @@ -Extensions for Protocol Buffers to create more go like structures. +Protocol Buffers for Go with Gadgets -Copyright (c) 2013, Vastech SA (PTY) LTD. All rights reserved. -http://github.com/gogo/protobuf/gogoproto +Copyright (c) 2013, The GoGo Authors. All rights reserved. +http://github.com/gogo/protobuf Go support for Protocol Buffers - Google's data interchange format diff --git a/vendor/vendor.json b/vendor/vendor.json index 09587fd99d7..ed532f4ad0c 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -141,20 +141,20 @@ "revisionTime": "2016-10-10T02:54:55Z" }, { - "checksumSHA1": "h1nLibY0IliypSG0cwbXpSpcsMA=", + "checksumSHA1": "4VUg2Be1lkd0wm8iVTkoMTa58Ow=", "path": "github.com/coreos/etcd/raft", - "revision": "9d43462d174c664f5edf313dec0de31e1ef4ed47", - "revisionTime": "2017-08-21T17:40:55Z", - "version": "v3.2.6", - "versionExact": "v3.2.6" + "revision": "33245c6b5b49130ca99280408fadfab01aac0e48", + "revisionTime": "2018-06-15T16:41:56Z", + "version": "v3.3.8", + "versionExact": "v3.3.8" }, { - "checksumSHA1": "L0Ds4Qp/I/bPN9+y+0zN2SF0KJg=", + "checksumSHA1": "cwEnAGl7uzwDepjDZcIocMVEVEE=", "path": "github.com/coreos/etcd/raft/raftpb", - "revision": "9d43462d174c664f5edf313dec0de31e1ef4ed47", - "revisionTime": "2017-08-21T17:40:55Z", - "version": "v3.2.6", - "versionExact": "v3.2.6" + "revision": "33245c6b5b49130ca99280408fadfab01aac0e48", + "revisionTime": "2018-06-15T16:41:56Z", + "version": "v3.3.8", + "versionExact": "v3.3.8" }, { "checksumSHA1": "Lf3uUXTkKK5DJ37BxQvxO1Fq+K8=", diff --git a/worker/draft.go b/worker/draft.go index e06b4092303..ba02107ab66 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -10,10 +10,8 @@ package worker import ( "bytes" "encoding/binary" - "encoding/hex" "errors" "fmt" - "math/rand" "sync" "time" @@ -47,54 +45,71 @@ type proposalCtx struct { type proposals struct { sync.RWMutex - // The key is hex encoded version of - // This should make sure its not same across replicas. - keys map[string]*proposalCtx + all map[string]*proposalCtx } +// uniqueKey is meant to be unique across all the replicas. func uniqueKey() string { - b := make([]byte, 16) - copy(b[:8], groups().Node.raftIdBuffer) - groups().Node.rand.Read(b[8:]) - return hex.EncodeToString(b) + return fmt.Sprintf("%02d-%d", groups().Node.Id, groups().Node.Rand.Uint64()) } func (p *proposals) Store(key string, pctx *proposalCtx) bool { p.Lock() defer p.Unlock() - if _, has := p.keys[key]; has { + if _, has := p.all[key]; has { return false } - p.keys[key] = pctx + p.all[key] = pctx return true } +func (p *proposals) Delete(key string) { + if len(key) == 0 { + return + } + p.Lock() + defer p.Unlock() + delete(p.all, key) +} + func (p *proposals) pctx(key string) *proposalCtx { p.RLock() defer p.RUnlock() - return p.keys[key] + if pctx := p.all[key]; pctx != nil { + return pctx + } + return new(proposalCtx) } func (p *proposals) CtxAndTxn(key string) (context.Context, *posting.Txn) { p.RLock() defer p.RUnlock() - pd, has := p.keys[key] - x.AssertTrue(has) + pd, has := p.all[key] + if !has { + // See the race condition note in Done. + return context.Background(), new(posting.Txn) + } return pd.ctx, pd.txn } func (p *proposals) Done(key string, err error) { + if len(key) == 0 { + return + } p.Lock() defer p.Unlock() - pd, has := p.keys[key] + pd, has := p.all[key] if !has { + // If we assert here, there would be a race condition between a context + // timing out, and a proposal getting applied immediately after. That + // would cause assert to fail. So, don't assert. return } x.AssertTrue(pd.index != 0) if err != nil { pd.err = err } - delete(p.keys, key) + delete(p.all, key) pd.ch <- pd.err } @@ -112,9 +127,8 @@ type node struct { gid uint32 props proposals - canCampaign bool - rand *rand.Rand - raftIdBuffer []byte + canCampaign bool + elog trace.EventLog } func (n *node) WaitForMinProposal(ctx context.Context, read *api.LinRead) error { @@ -132,23 +146,6 @@ func (n *node) WaitForMinProposal(ctx context.Context, read *api.LinRead) error return n.Applied.WaitForMark(ctx, min) } -type lockedSource struct { - lk sync.Mutex - src rand.Source -} - -func (r *lockedSource) Int63() int64 { - r.lk.Lock() - defer r.lk.Unlock() - return r.src.Int63() -} - -func (r *lockedSource) Seed(seed int64) { - r.lk.Lock() - defer r.lk.Unlock() - r.src.Seed(seed) -} - func newNode(store *raftwal.DiskStorage, gid uint32, id uint64, myAddr string) *node { x.Printf("Node ID: %v with GroupID: %v\n", id, gid) @@ -159,12 +156,9 @@ func newNode(store *raftwal.DiskStorage, gid uint32, id uint64, myAddr string) * } m := conn.NewNode(rc, store) props := proposals{ - keys: make(map[string]*proposalCtx), + all: make(map[string]*proposalCtx), } - b := make([]byte, 8) - binary.LittleEndian.PutUint64(b, id) - n := &node{ Node: m, requestCh: make(chan linReadReq), @@ -172,12 +166,11 @@ func newNode(store *raftwal.DiskStorage, gid uint32, id uint64, myAddr string) * gid: gid, // processConfChange etc are not throttled so some extra delta, so that we don't // block tick when applyCh is full - applyCh: make(chan raftpb.Entry, Config.NumPendingProposals+1000), - props: props, - stop: make(chan struct{}), - done: make(chan struct{}), - rand: rand.New(&lockedSource{src: rand.NewSource(time.Now().UnixNano())}), - raftIdBuffer: b, + applyCh: make(chan raftpb.Entry, Config.NumPendingProposals+1000), + props: props, + stop: make(chan struct{}), + done: make(chan struct{}), + elog: trace.NewEventLog("Dgraph", "ApplyCh"), } return n } @@ -203,6 +196,8 @@ func (h *header) Decode(in []byte) { h.msgId = binary.LittleEndian.Uint16(in[4:6]) } +var errInternalRetry = errors.New("Retry Raft proposal internally") + // proposeAndWait sends a proposal through RAFT. It waits on a channel for the proposal // to be applied(written to WAL) to all the nodes in the group. func (n *node) proposeAndWait(ctx context.Context, proposal *intern.Proposal) error { @@ -245,49 +240,64 @@ func (n *node) proposeAndWait(ctx context.Context, proposal *intern.Proposal) er } } - che := make(chan error, 1) - pctx := &proposalCtx{ - ch: che, - ctx: ctx, - } - - key := uniqueKey() - x.AssertTruef(n.props.Store(key, pctx), "Found existing proposal with key: [%v]", key) - proposal.Key = key + propose := func() error { + cctx, cancel := context.WithTimeout(ctx, 15*time.Second) + defer cancel() - sz := proposal.Size() - slice := make([]byte, sz) + che := make(chan error, 1) + pctx := &proposalCtx{ + ch: che, + ctx: cctx, + } + key := uniqueKey() + x.AssertTruef(n.props.Store(key, pctx), "Found existing proposal with key: [%v]", key) + defer n.props.Delete(key) // Ensure that it gets deleted on return. + proposal.Key = key - upto, err := proposal.MarshalTo(slice) - if err != nil { - return err - } + if tr, ok := trace.FromContext(ctx); ok { + tr.LazyPrintf("Proposing data with key: %s", key) + } - // Some proposals can be stuck if leader change happens. For e.g. MsgProp message from follower - // to leader can be dropped/end up appearing with empty Data in CommittedEntries. - // Having a timeout here prevents the mutation being stuck forever in case they don't have a - // timeout. - cctx, cancel := context.WithTimeout(ctx, 10*time.Minute) - defer cancel() - if err = n.Raft().Propose(cctx, slice[:upto]); err != nil { - return x.Wrapf(err, "While proposing") - } + data, err := proposal.Marshal() + if err != nil { + return err + } - if tr, ok := trace.FromContext(ctx); ok { - tr.LazyPrintf("Waiting for the proposal.") - } + if err = n.Raft().Propose(cctx, data); err != nil { + return x.Wrapf(err, "While proposing") + } + if tr, ok := trace.FromContext(ctx); ok { + tr.LazyPrintf("Waiting for the proposal.") + } - select { - case err = <-che: - if err != nil { + select { + case err = <-che: + // We arrived here by a call to n.props.Done(). + if tr, ok := trace.FromContext(ctx); ok { + tr.LazyPrintf("Done with error: %v", err) + } + return err + case <-ctx.Done(): if tr, ok := trace.FromContext(ctx); ok { - tr.LazyPrintf("Raft Propose error: %v", err) + tr.LazyPrintf("External context timed out with error: %v.", ctx.Err()) } + return ctx.Err() + case <-cctx.Done(): + if tr, ok := trace.FromContext(ctx); ok { + tr.LazyPrintf("Internal context timed out with error: %v. Retrying...", cctx.Err()) + } + return errInternalRetry } - case <-cctx.Done(): - return fmt.Errorf("While proposing to Raft group, err: %+v\n", cctx.Err()) } + // Some proposals can be stuck if leader change happens. For e.g. MsgProp message from follower + // to leader can be dropped/end up appearing with empty Data in CommittedEntries. + // Having a timeout here prevents the mutation being stuck forever in case they don't have a + // timeout. We should always try with a timeout and optionally retry. + err := errInternalRetry + for err == errInternalRetry { + err = propose() + } return err } @@ -304,6 +314,8 @@ func (n *node) processEdge(ridx uint64, pkey string, edge *intern.DirectedEdge) // while applying the second mutation we check the old value // of name and delete it from "janardhan"'s index. If we don't // wait for commit information then mutation won't see the value + + // TODO: We should propose Oracle via Raft, and not wait here. posting.Oracle().WaitForTs(context.Background(), txn.StartTs) if err := runMutation(ctx, edge, txn); err != nil { if tr, ok := trace.FromContext(ctx); ok { @@ -493,6 +505,7 @@ func (n *node) applyCommitted(proposal *intern.Proposal, index uint64) error { posting.TxnMarks().Begin(index) 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, index) } @@ -501,6 +514,7 @@ func (n *node) applyCommitted(proposal *intern.Proposal, index uint64) error { return n.processKeyValues(proposal.Key, proposal.Kv) } else if proposal.State != nil { + n.elog.Printf("Applying state for key: %s", proposal.Key) // This state needn't be snapshotted in this group, on restart we would fetch // a state which is latest or equal to this. groups().applyState(proposal.State) @@ -510,6 +524,7 @@ func (n *node) applyCommitted(proposal *intern.Proposal, index uint64) error { return n.deletePredicate(proposal.Key, proposal.CleanPredicate) } else if proposal.TxnContext != nil { + n.elog.Printf("Applying txncontext for key: %s", proposal.Key) return n.commitOrAbort(proposal.Key, proposal.TxnContext) } else { x.Fatalf("Unknown proposal") @@ -524,6 +539,7 @@ func (n *node) processApplyCh() { x.Fatalf("Unable to unmarshal proposal: %v %q\n", err, e.Data) } err := n.applyCommitted(proposal, e.Index) + n.elog.Printf("Applied proposal with key: %s, index: %d. Err: %v", proposal.Key, e.Index, err) n.props.Done(proposal.Key, err) n.Applied.Done(e.Index) } @@ -637,7 +653,7 @@ func (n *node) runReadIndexLoop(closer *y.Closer, readStateCh <-chan raft.ReadSt } } activeRctx := make([]byte, 8) - x.Check2(n.rand.Read(activeRctx[:])) + x.Check2(n.Rand.Read(activeRctx[:])) // To see if the ReadIndex request succeeds, we need to use a timeout and wait for a // successful response. If we don't see one, the raft leader wasn't configured, or the // raft leader didn't respond. @@ -698,8 +714,14 @@ func (n *node) Run() { // That way we know sending to readStateCh will not deadlock. go n.runReadIndexLoop(closer, readStateCh) + logTicker := time.NewTicker(time.Minute) + defer logTicker.Stop() + for { select { + case <-logTicker.C: + n.elog.Printf("Size of applyCh: %d", len(n.applyCh)) + case <-ticker.C: n.Raft().Tick() @@ -779,6 +801,8 @@ func (n *node) Run() { } else if len(entry.Data) == 0 { // TODO: Say something. Do something. + tr.LazyPrintf("Found empty data at index: %d", entry.Index) + tr.SetError() n.Applied.Done(entry.Index) } else { diff --git a/worker/mutation.go b/worker/mutation.go index ab746c8078d..8fa6056bbaf 100644 --- a/worker/mutation.go +++ b/worker/mutation.go @@ -589,7 +589,7 @@ func (w *grpcWorker) Mutate(ctx context.Context, m *intern.Mutations) (*api.TxnC node := groups().Node if rand.Float64() < Config.Tracing { var tr trace.Trace - tr, ctx = x.NewTrace("GrpcMutate", ctx) + tr, ctx = x.NewTrace("grpcWorker.Mutate", ctx) defer tr.Finish() }