diff --git a/Makefile b/Makefile index 03cd62c5..a00cd76b 100644 --- a/Makefile +++ b/Makefile @@ -23,8 +23,9 @@ GO_VERSION := $(shell go version|cut -f3 -d' '|dd bs=1 count=5 2>/dev/null) all: test .PHONY: install +install: INSTALL_FLAGS += install: - go install $(BINARIES) + go install $(INSTALL_FLAGS) $(BINARIES) .PHONY: test test: @@ -62,6 +63,10 @@ protobufs: protobufs-requirements clean-protobufs (cd ${API_PKG}; protoc --proto_path="${PROTO_PATH}" --gogo_out=. ./scheduler/*.proto) (cd ${API_PKG}; protoc --proto_path="${PROTO_PATH}" --gogo_out=. ./executor/*.proto) (cd ${API_PKG}; protoc --proto_path="${PROTO_PATH}" --gogo_out=. ./agent/*.proto) + (cd ${API_PKG}; protoc --proto_path="${PROTO_PATH}" --gogo_out=. ./quota/*.proto) + (cd ${API_PKG}; protoc --proto_path="${PROTO_PATH}" --gogo_out=. ./allocator/*.proto) + (cd ${API_PKG}; protoc --proto_path="${PROTO_PATH}" --gogo_out=. ./maintenance/*.proto) + (cd ${API_PKG}; protoc --proto_path="${PROTO_PATH}" --gogo_out=. ./master/*.proto) .PHONY: clean-protobufs clean-protobufs: @@ -73,6 +78,10 @@ ffjson: clean-ffjson (cd ${API_PKG}; ffjson scheduler/*.pb.go) (cd ${API_PKG}; ffjson executor/*.pb.go) (cd ${API_PKG}; ffjson agent/*.pb.go) + (cd ${API_PKG}; ffjson quota/*.pb.go) + (cd ${API_PKG}; ffjson allocator/*.pb.go) + (cd ${API_PKG}; ffjson maintenance/*.pb.go) + (cd ${API_PKG}; ffjson master/*.pb.go) .PHONY: clean-ffjson clean-ffjson: @@ -85,7 +94,7 @@ sync: (cd ${CMD_VENDOR}; govendor sync) .PHONY: generate -generate: GENERATE_PACKAGES = ./api/v1/lib/extras/executor/eventrules ./api/v1/lib/extras/executor/callrules ./api/v1/lib/extras/scheduler/eventrules ./api/v1/lib/extras/scheduler/callrules ./api/v1/lib/executor/events ./api/v1/lib/executor/calls ./api/v1/lib/scheduler/events ./api/v1/lib/scheduler/calls +generate: GENERATE_PACKAGES = ./api/v1/lib/extras/executor/eventrules ./api/v1/lib/extras/executor/callrules ./api/v1/lib/extras/scheduler/eventrules ./api/v1/lib/extras/scheduler/callrules ./api/v1/lib/executor/events ./api/v1/lib/executor/calls ./api/v1/lib/scheduler/events ./api/v1/lib/scheduler/calls ./api/v1/lib/agent/calls ./api/v1/lib/master/calls ./api/v1/lib/httpcli/httpagent ./api/v1/lib/httpcli/httpmaster generate: go generate -x ${GENERATE_PACKAGES} @@ -116,7 +125,7 @@ docker: make -C api/${MESOS_API_VERSION}/docker .PHONY: coveralls -coveralls: IGNORE_FILES = $(shell { find api/v1/cmd -type d ; ls api/v1/lib{,/scheduler,/executor,/agent}/*.pb{,_ffjson}.go ; find api/v0 -type d; } | tr '\n' ,) +coveralls: IGNORE_FILES = $(shell { find api/v1/cmd -type d ; ls api/v1/lib{,/scheduler,/executor,/agent,/quota,/allocator,/maintenance,/master}/*.pb{,_ffjson}.go ; find api/v0 -type d; } | tr '\n' ,) coveralls: SHELL := /bin/bash coveralls: test "$(TRAVIS)" = "" || rm -rf $$HOME/gopath/pkg diff --git a/api/v1/cmd/example-scheduler/app/flags.go b/api/v1/cmd/example-scheduler/app/flags.go index bf950fb5..77995397 100644 --- a/api/v1/cmd/example-scheduler/app/flags.go +++ b/api/v1/cmd/example-scheduler/app/flags.go @@ -30,7 +30,7 @@ type codec struct{ encoding.Codec } func (c *codec) Set(value string) error { v := strings.ToLower(value) for _, codec := range encoding.DefaultCodecs { - if v == codec.Name() { + if v == codec.Name { c.Codec = codec return nil } diff --git a/api/v1/cmd/example-scheduler/app/state.go b/api/v1/cmd/example-scheduler/app/state.go index efd74981..55ee5b8d 100644 --- a/api/v1/cmd/example-scheduler/app/state.go +++ b/api/v1/cmd/example-scheduler/app/state.go @@ -32,7 +32,7 @@ func prepareExecutorInfo( Type: mesos.ExecutorInfo_CUSTOM, ExecutorID: mesos.ExecutorID{Value: "default"}, Name: proto.String("Test Executor"), - Command: mesos.CommandInfo{ + Command: &mesos.CommandInfo{ Shell: func() *bool { x := false; return &x }(), }, Container: &mesos.ContainerInfo{ @@ -77,7 +77,7 @@ func prepareExecutorInfo( Type: mesos.ExecutorInfo_CUSTOM, ExecutorID: mesos.ExecutorID{Value: "default"}, Name: proto.String("Test Executor"), - Command: mesos.CommandInfo{ + Command: &mesos.CommandInfo{ Value: proto.String(executorCommand), URIs: executorUris, }, diff --git a/api/v1/cmd/msh/msh.go b/api/v1/cmd/msh/msh.go index 1e329ee6..0e397153 100644 --- a/api/v1/cmd/msh/msh.go +++ b/api/v1/cmd/msh/msh.go @@ -15,17 +15,21 @@ import ( "fmt" "io" "log" + "net" "os" "time" "github.com/gogo/protobuf/proto" "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/agent" + agentcalls "github.com/mesos/mesos-go/api/v1/lib/agent/calls" "github.com/mesos/mesos-go/api/v1/lib/extras/scheduler/callrules" "github.com/mesos/mesos-go/api/v1/lib/extras/scheduler/controller" "github.com/mesos/mesos-go/api/v1/lib/extras/scheduler/eventrules" "github.com/mesos/mesos-go/api/v1/lib/extras/scheduler/offers" "github.com/mesos/mesos-go/api/v1/lib/extras/store" "github.com/mesos/mesos-go/api/v1/lib/httpcli" + "github.com/mesos/mesos-go/api/v1/lib/httpcli/httpagent" "github.com/mesos/mesos-go/api/v1/lib/httpcli/httpsched" "github.com/mesos/mesos-go/api/v1/lib/resources" "github.com/mesos/mesos-go/api/v1/lib/scheduler" @@ -46,11 +50,22 @@ var ( CPUs = float64(0.010) Memory = float64(64) - fidStore store.Singleton - declineAndSuppress bool - refuseSeconds = calls.RefuseSeconds(5 * time.Second) - wantsResources mesos.Resources - taskPrototype mesos.TaskInfo + fidStore store.Singleton + declineAndSuppress bool + refuseSeconds = calls.RefuseSeconds(5 * time.Second) + wantsResources mesos.Resources + taskPrototype mesos.TaskInfo + interactive bool + tty bool + pod bool + executorPrototype mesos.ExecutorInfo + wantsExecutorResources = mesos.Resources{ + resources.NewCPUs(0.01).Resource, + resources.NewMemory(32).Resource, + resources.NewDisk(5).Resource, + } + agentDirectory = make(map[mesos.AgentID]string) + uponExit = new(cleanups) ) func init() { @@ -60,6 +75,9 @@ func init() { flag.StringVar(&User, "user", User, "OS user that owns the launched task") flag.Float64Var(&CPUs, "cpus", CPUs, "CPU resources to allocate for the remote command") flag.Float64Var(&Memory, "memory", Memory, "Memory resources to allocate for the remote command") + flag.BoolVar(&tty, "tty", tty, "Route all container stdio, stdout, stderr communication through a TTY device") + flag.BoolVar(&pod, "pod", pod, "Launch the remote command in a mesos task-group") + flag.BoolVar(&interactive, "interactive", interactive, "Attach to the task's stdin, stdout, and stderr") fidStore = store.DecorateSingleton( store.NewInMemorySingleton(), @@ -89,17 +107,33 @@ func main() { }, } taskPrototype.Command.Arguments = args + if interactive { + taskPrototype.Container = &mesos.ContainerInfo{ + Type: mesos.ContainerInfo_MESOS.Enum(), + TTYInfo: &mesos.TTYInfo{}, + } + } + if term := os.Getenv("TERM"); term != "" && tty { + taskPrototype.Command.Environment = &mesos.Environment{ + Variables: []mesos.Environment_Variable{ + mesos.Environment_Variable{Name: "TERM", Value: &term}, + }, + } + } if err := run(); err != nil { if exitErr, ok := err.(ExitError); ok { if code := int(exitErr); code != 0 { log.Println(exitErr) + uponExit.unwind() os.Exit(code) } // else, code=0 indicates success, exit normally } else { + uponExit.unwind() log.Fatalf("%#v", err) } } + uponExit.unwind() } func run() error { @@ -132,12 +166,54 @@ func buildClient() calls.Caller { func buildEventHandler(caller calls.Caller) events.Handler { logger := controller.LogEvents(nil) return controller.LiftErrors().Handle(events.Handlers{ - scheduler.Event_SUBSCRIBED: eventrules.Rules{logger, controller.TrackSubscription(fidStore, 0)}, - scheduler.Event_OFFERS: maybeDeclineOffers(caller).AndThen().Handle(resourceOffers(caller)), - scheduler.Event_UPDATE: controller.AckStatusUpdates(caller).AndThen().HandleF(statusUpdate), + scheduler.Event_SUBSCRIBED: eventrules.Rules{ + logger, + controller.TrackSubscription(fidStore, 0), + updateExecutor, + }, + + scheduler.Event_OFFERS: eventrules.Rules{ + trackAgents, + maybeDeclineOffers(caller), + eventrules.DropOnError(), + eventrules.Handle(resourceOffers(caller)), + }, + + scheduler.Event_UPDATE: controller.AckStatusUpdates(caller).AndThen().HandleF(statusUpdate), }.Otherwise(logger.HandleEvent)) } +func updateExecutor(ctx context.Context, e *scheduler.Event, err error, chain eventrules.Chain) (context.Context, *scheduler.Event, error) { + if err != nil { + return chain(ctx, e, err) + } + if e.GetType() != scheduler.Event_SUBSCRIBED { + return chain(ctx, e, err) + } + if pod { + executorPrototype = mesos.ExecutorInfo{ + Type: mesos.ExecutorInfo_DEFAULT, + FrameworkID: e.GetSubscribed().FrameworkID, + } + } + return chain(ctx, e, err) +} + +func trackAgents(ctx context.Context, e *scheduler.Event, err error, chain eventrules.Chain) (context.Context, *scheduler.Event, error) { + if err != nil { + return chain(ctx, e, err) + } + if e.GetType() != scheduler.Event_OFFERS { + return chain(ctx, e, err) + } + off := e.GetOffers().GetOffers() + for i := range off { + // TODO(jdef) eventually implement an algorithm to purge agents that are gone + agentDirectory[off[i].GetAgentID()] = off[i].GetHostname() + } + return chain(ctx, e, err) +} + func maybeDeclineOffers(caller calls.Caller) eventrules.Rule { return func(ctx context.Context, e *scheduler.Event, err error, chain eventrules.Chain) (context.Context, *scheduler.Event, error) { if err != nil { @@ -159,22 +235,42 @@ func maybeDeclineOffers(caller calls.Caller) eventrules.Rule { func resourceOffers(caller calls.Caller) events.HandlerFunc { return func(ctx context.Context, e *scheduler.Event) (err error) { var ( - off = e.GetOffers().GetOffers() - index = offers.NewIndex(off, nil) - match = index.Find(offers.ContainsResources(wantsResources)) + off = e.GetOffers().GetOffers() + index = offers.NewIndex(off, nil) + matchResources = func() mesos.Resources { + if pod { + return wantsResources.Plus(wantsExecutorResources...) + } else { + return wantsResources + } + }() + match = index.Find(offers.ContainsResources(matchResources)) ) if match != nil { + ts := time.Now().Format(RFC3339a) task := taskPrototype - task.TaskID = mesos.TaskID{Value: time.Now().Format(RFC3339a)} + task.TaskID = mesos.TaskID{Value: ts} task.AgentID = match.AgentID task.Resources = resources.Find( resources.Flatten(wantsResources, Role.Assign()), match.Resources..., ) - err = calls.CallNoData(ctx, caller, calls.Accept( - calls.OfferOperations{calls.OpLaunch(task)}.WithOffers(match.ID), - )) + if pod { + executor := executorPrototype + executor.ExecutorID = mesos.ExecutorID{Value: "msh_" + ts} + executor.Resources = resources.Find( + resources.Flatten(wantsExecutorResources, Role.Assign()), + match.Resources..., + ) + err = calls.CallNoData(ctx, caller, calls.Accept( + calls.OfferOperations{calls.OpLaunchGroup(executor, task)}.WithOffers(match.ID), + )) + } else { + err = calls.CallNoData(ctx, caller, calls.Accept( + calls.OfferOperations{calls.OpLaunch(task)}.WithOffers(match.ID), + )) + } if err != nil { return } @@ -201,6 +297,13 @@ func statusUpdate(_ context.Context, e *scheduler.Event) error { switch st := s.GetState(); st { case mesos.TASK_FINISHED, mesos.TASK_RUNNING, mesos.TASK_STAGING, mesos.TASK_STARTING: log.Printf("status update from agent %q: %v", s.GetAgentID().GetValue(), st) + if st == mesos.TASK_RUNNING && interactive && s.AgentID != nil { + cid := s.GetContainerStatus().GetContainerID() + if cid != nil { + log.Printf("attaching for interactive session to agent %q container %q", s.AgentID.Value, cid.Value) + return tryInteractive(agentDirectory[*s.AgentID], *cid) + } + } if st != mesos.TASK_FINISHED { return nil } @@ -221,3 +324,155 @@ func statusUpdate(_ context.Context, e *scheduler.Event) error { type ExitError int func (e ExitError) Error() string { return fmt.Sprintf("exit code %d", int(e)) } + +func tryInteractive(agentHost string, cid mesos.ContainerID) (err error) { + // TODO(jdef) only re-attach if we're disconnected (guard against redundant TASK_RUNNING) + var ( + ctx, cancel = context.WithCancel(context.TODO()) + winCh <-chan mesos.TTYInfo_WindowSize + ) + if tty { + ttyd, err := initTTY() + if err != nil { + cancel() // stop go-vet from complaining + return err + } + + uponExit.push(ttyd.Close) // fail-safe + + go func() { + <-ctx.Done() + ttyd.Close() + }() + + winCh = ttyd.winch + } + + var ( + cli = httpagent.NewSender( + httpcli.New( + httpcli.Endpoint(fmt.Sprintf("http://%s/api/v1", net.JoinHostPort(agentHost, "5051"))), + ), + ) + aciCh = make(chan *agent.Call, 1) // must be buffered to avoid blocking below + ) + aciCh <- agentcalls.AttachContainerInput(cid) // very first input message MUST be this + go func() { + defer cancel() + acif := agentcalls.FromChan(aciCh) + + // blocking call, hence the goroutine; Send only returns when the input stream is severed + err2 := agentcalls.SendNoData(ctx, cli, acif) + if err2 != nil && err2 != io.EOF { + log.Printf("attached input stream error %v", err2) + } + }() + + // attach to container stdout, stderr; Send returns immediately with a Response from which output + // may be decoded. + output, err := cli.Send(ctx, agentcalls.NonStreaming(agentcalls.AttachContainerOutput(cid))) + if err != nil { + log.Printf("attach output stream error: %v", err) + if output != nil { + output.Close() + } + cancel() + return + } + + go func() { + defer cancel() + attachContainerOutput(output, os.Stdout, os.Stderr) + }() + + go attachContainerInput(ctx, os.Stdin, winCh, aciCh) + + return nil +} + +func attachContainerInput(ctx context.Context, stdin io.Reader, winCh <-chan mesos.TTYInfo_WindowSize, aciCh chan<- *agent.Call) { + defer close(aciCh) + + input := make(chan []byte) + go func() { + defer close(input) + for { + buf := make([]byte, 512) // not efficient to always do this + n, err := stdin.Read(buf) + if n > 0 { + buf = buf[:n] + select { + case input <- buf: + case <-ctx.Done(): + return + } + } + // TODO(jdef) check for temporary error? + if err != nil { + return + } + } + }() + for { + select { + case <-ctx.Done(): + return + // TODO(jdef) send a heartbeat message every so often + // attach_container_input process_io heartbeats may act as keepalive's, `interval` field is ignored: + // https://github.com/apache/mesos/blob/4e200e55d8ed282b892f650983ebdf516680d90d/src/slave/containerizer/mesos/io/switchboard.cpp#L1608 + case data, ok := <-input: + if !ok { + return + } + c := agentcalls.AttachContainerInputData(data) + select { + case aciCh <- c: + case <-ctx.Done(): + return + } + case ws := <-winCh: + c := agentcalls.AttachContainerInputTTY(&mesos.TTYInfo{WindowSize: &ws}) + select { + case aciCh <- c: + case <-ctx.Done(): + return + } + } + } +} + +func attachContainerOutput(resp mesos.Response, stdout, stderr io.Writer) error { + defer resp.Close() + forward := func(b []byte, out io.Writer) error { + n, err := out.Write(b) + if err == nil && len(b) != n { + err = io.ErrShortWrite + } + return err + } + for { + var pio agent.ProcessIO + err := resp.Decode(&pio) + if err != nil { + return err + } + switch pio.GetType() { + case agent.ProcessIO_DATA: + data := pio.GetData() + switch data.GetType() { + case agent.ProcessIO_Data_STDOUT: + if err := forward(data.GetData(), stdout); err != nil { + return err + } + case agent.ProcessIO_Data_STDERR: + if err := forward(data.GetData(), stderr); err != nil { + return err + } + default: + // ignore + } + default: + // ignore + } + } +} diff --git a/api/v1/cmd/msh/tty.go b/api/v1/cmd/msh/tty.go new file mode 100644 index 00000000..ca818838 --- /dev/null +++ b/api/v1/cmd/msh/tty.go @@ -0,0 +1,207 @@ +package main + +import ( + "fmt" + "log" + "os" + "os/signal" + "sync" + "syscall" + + "github.com/mesos/mesos-go/api/v1/lib" +) + +// #include +// #include +// #include +// #include +// #include +// #include +// +// /* because golang doesn't like the ... param of ioctl */ +// int ioctl_winsize(int d, unsigned long request, void *buf) { +// return ioctl(d, request, buf); +// } +// +import "C" +import "unsafe" + +type cleanups struct { + ops []func() + once sync.Once +} + +func (c *cleanups) unwind() { + c.once.Do(func() { + for _, f := range c.ops { + defer f() + } + }) +} + +func (c *cleanups) push(f func()) { + if f != nil { + c.ops = append(c.ops, f) + } +} + +type ttyDevice struct { + fd int + cancel chan struct{} + winch chan mesos.TTYInfo_WindowSize + cleanups *cleanups + original_winsize C.struct_winsize +} + +func (t *ttyDevice) Done() <-chan struct{} { return t.cancel } +func (t *ttyDevice) Close() { t.cleanups.unwind() } + +func initTTY() (_ *ttyDevice, err error) { + return newTTY( + ttyConsoleAttach(&os.Stdin, &os.Stdout, &os.Stderr), + ttyWinch, + ttyTermReset, + ) +} + +func newTTY(opts ...ttyOption) (_ *ttyDevice, err error) { + tty := ttyDevice{ + cancel: make(chan struct{}), + cleanups: new(cleanups), + } + tty.cleanups.push(func() { close(tty.cancel) }) + defer func() { + if err != nil { + tty.Close() + } + }() + + ttyname := C.ctermid((*C.char)(unsafe.Pointer(nil))) + if p := (*C.char)(unsafe.Pointer(ttyname)); p == nil { + err = fmt.Errorf("failed to get tty name") + return + } + + tty.fd, _ = syscall.Open(C.GoString(ttyname), syscall.O_RDWR, 0) + if tty.fd < 0 { + err = fmt.Errorf("failed to open tty device: %d", tty.fd) + return + } + tty.cleanups.push(func() { syscall.Close(tty.fd) }) + + var original_termios C.struct_termios + result := C.tcgetattr(C.int(tty.fd), &original_termios) + if result < 0 { + err = fmt.Errorf("failed getting termios: %d", result) + return + } + + new_termios := original_termios + C.cfmakeraw(&new_termios) + result = C.tcsetattr(C.int(tty.fd), C.TCSANOW, &new_termios) + if result < 0 { + err = fmt.Errorf("failed setting termios: %d", result) + return + } + tty.cleanups.push(func() { + r := C.tcsetattr(C.int(tty.fd), C.TCSANOW, &original_termios) + if r < 0 { + log.Printf("failed to set original termios: %d", r) + } + }) + + // use this local var instead of tty.original_winsize to avoid cgo complaints about double-pointers + var original_winsize C.struct_winsize + result = C.ioctl_winsize(0, C.TIOCGWINSZ, unsafe.Pointer(&original_winsize)) + if result < 0 { + err = fmt.Errorf("failed to get winsize: %d", result) + return + } + tty.original_winsize = original_winsize + tty.cleanups.push(func() { + r := C.ioctl_winsize(0, C.TIOCSWINSZ, unsafe.Pointer(&original_winsize)) + if r < 0 { + log.Printf("failed to set winsize: %d", r) + } + }) + + log.Printf("original window size is %d x %d\n", tty.original_winsize.ws_col, tty.original_winsize.ws_row) + + for _, f := range opts { + if f != nil { + f(&tty) + } + } + + return &tty, nil +} + +type ttyOption func(*ttyDevice) + +func ttyConsoleAttach(stdin, stdout, stderr **os.File) ttyOption { + swapfd := func(newfd uintptr, name string, target **os.File) func() { + f := os.NewFile(newfd, name) + if f == nil { + panic(fmt.Sprintf("failed to swap fd for %q", name)) + } + old := *target + *target = f + return func() { + *target = old + } + } + return func(tty *ttyDevice) { + tty.cleanups.push(swapfd(uintptr(tty.fd), "tty", stdout)) + tty.cleanups.push(swapfd(uintptr(tty.fd), "tty", stderr)) + tty.cleanups.push(swapfd(uintptr(tty.fd), "tty", stdin)) + } +} + +func ttyWinch(tty *ttyDevice) { + // translate window-size signals into chan events + c := make(chan os.Signal, 1) + tty.winch = make(chan mesos.TTYInfo_WindowSize, 1) + tty.winch <- mesos.TTYInfo_WindowSize{ + Rows: uint32(tty.original_winsize.ws_row), + Columns: uint32(tty.original_winsize.ws_col), + } + go func() { + defer signal.Ignore(os.Signal(syscall.SIGWINCH)) + for { + select { + case <-c: + signal.Ignore(os.Signal(syscall.SIGWINCH)) + var temp_winsize C.struct_winsize + r := C.ioctl_winsize(0, C.TIOCGWINSZ, unsafe.Pointer(&temp_winsize)) + if r < 0 { + panic(fmt.Sprintf("failed to get winsize: %d", r)) + } + ws := mesos.TTYInfo_WindowSize{ + Rows: uint32(temp_winsize.ws_row), + Columns: uint32(temp_winsize.ws_col), + } + select { + case <-tty.Done(): + return + case tty.winch <- ws: + signal.Notify(c, os.Signal(syscall.SIGWINCH)) + } + case <-tty.Done(): + return + } + } + }() + signal.Notify(c, os.Signal(syscall.SIGWINCH)) +} + +func ttyTermReset(tty *ttyDevice) { + // cleanup properly upon SIGTERM + term := make(chan os.Signal, 1) + go func() { + <-term + tty.cleanups.unwind() + os.Exit(0) + }() + tty.cleanups.push(func() { signal.Ignore(os.Signal(syscall.SIGTERM)) }) + signal.Notify(term, os.Signal(syscall.SIGTERM)) +} diff --git a/api/v1/lib/agent/calls/calls.go b/api/v1/lib/agent/calls/calls.go new file mode 100644 index 00000000..7d62aae2 --- /dev/null +++ b/api/v1/lib/agent/calls/calls.go @@ -0,0 +1,167 @@ +package calls + +import ( + "time" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/agent" +) + +func GetHealth() *agent.Call { return &agent.Call{Type: agent.Call_GET_HEALTH} } + +func GetFlags() *agent.Call { return &agent.Call{Type: agent.Call_GET_FLAGS} } + +func GetVersion() *agent.Call { return &agent.Call{Type: agent.Call_GET_VERSION} } + +func GetMetrics(d *time.Duration) (call *agent.Call) { + call = &agent.Call{ + Type: agent.Call_GET_METRICS, + GetMetrics: &agent.Call_GetMetrics{}, + } + if d != nil { + call.GetMetrics.Timeout = &mesos.DurationInfo{ + Nanoseconds: d.Nanoseconds(), + } + } + return +} + +func GetLoggingLevel() *agent.Call { return &agent.Call{Type: agent.Call_GET_LOGGING_LEVEL} } + +func SetLoggingLevel(level uint32, d time.Duration) *agent.Call { + return &agent.Call{ + Type: agent.Call_SET_LOGGING_LEVEL, + SetLoggingLevel: &agent.Call_SetLoggingLevel{ + Duration: mesos.DurationInfo{Nanoseconds: d.Nanoseconds()}, + Level: level, + }, + } +} + +func ListFiles(path string) *agent.Call { + return &agent.Call{ + Type: agent.Call_LIST_FILES, + ListFiles: &agent.Call_ListFiles{ + Path: path, + }, + } +} + +func ReadFile(path string, offset uint64) *agent.Call { + return &agent.Call{ + Type: agent.Call_READ_FILE, + ReadFile: &agent.Call_ReadFile{ + Path: path, + Offset: offset, + }, + } +} + +func ReadFileWithLength(path string, offset, length uint64) (call *agent.Call) { + call = ReadFile(path, offset) + call.ReadFile.Length = &length + return +} + +func GetState() *agent.Call { return &agent.Call{Type: agent.Call_GET_STATE} } + +func GetContainers() *agent.Call { return &agent.Call{Type: agent.Call_GET_CONTAINERS} } + +func GetFrameworks() *agent.Call { return &agent.Call{Type: agent.Call_GET_FRAMEWORKS} } + +func GetExecutors() *agent.Call { return &agent.Call{Type: agent.Call_GET_EXECUTORS} } + +func GetTasks() *agent.Call { return &agent.Call{Type: agent.Call_GET_TASKS} } + +func LaunchNestedContainer(cid mesos.ContainerID, cmd *mesos.CommandInfo, ci *mesos.ContainerInfo) *agent.Call { + return &agent.Call{ + Type: agent.Call_LAUNCH_NESTED_CONTAINER, + LaunchNestedContainer: &agent.Call_LaunchNestedContainer{ + ContainerID: cid, + Command: cmd, + Container: ci, + }, + } +} + +func WaitNestedContainer(cid mesos.ContainerID) *agent.Call { + return &agent.Call{ + Type: agent.Call_WAIT_NESTED_CONTAINER, + WaitNestedContainer: &agent.Call_WaitNestedContainer{ + ContainerID: cid, + }, + } +} + +func KillNestedContainer(cid mesos.ContainerID) *agent.Call { + return &agent.Call{ + Type: agent.Call_KILL_NESTED_CONTAINER, + KillNestedContainer: &agent.Call_KillNestedContainer{ + ContainerID: cid, + }, + } +} + +func LaunchNestedContainerSession(cid mesos.ContainerID, cmd *mesos.CommandInfo, ci *mesos.ContainerInfo) *agent.Call { + return &agent.Call{ + Type: agent.Call_LAUNCH_NESTED_CONTAINER_SESSION, + LaunchNestedContainerSession: &agent.Call_LaunchNestedContainerSession{ + ContainerID: cid, + Command: cmd, + Container: ci, + }, + } +} + +func AttachContainerOutput(cid mesos.ContainerID) *agent.Call { + return &agent.Call{ + Type: agent.Call_ATTACH_CONTAINER_OUTPUT, + AttachContainerOutput: &agent.Call_AttachContainerOutput{ + ContainerID: cid, + }, + } +} + +// AttachContainerInput returns a Call that is used to initiate attachment to a container's stdin. +// Callers should first send this Call followed by one or more AttachContainerInputXxx calls. +func AttachContainerInput(cid mesos.ContainerID) *agent.Call { + return &agent.Call{ + Type: agent.Call_ATTACH_CONTAINER_INPUT, + AttachContainerInput: &agent.Call_AttachContainerInput{ + Type: agent.Call_AttachContainerInput_CONTAINER_ID, + ContainerID: &cid, + }, + } +} + +func AttachContainerInputData(data []byte) *agent.Call { + return &agent.Call{ + Type: agent.Call_ATTACH_CONTAINER_INPUT, + AttachContainerInput: &agent.Call_AttachContainerInput{ + Type: agent.Call_AttachContainerInput_PROCESS_IO, + ProcessIO: &agent.ProcessIO{ + Type: agent.ProcessIO_DATA, + Data: &agent.ProcessIO_Data{ + Type: agent.ProcessIO_Data_STDIN, + Data: data, + }, + }, + }, + } +} + +func AttachContainerInputTTY(t *mesos.TTYInfo) *agent.Call { + return &agent.Call{ + Type: agent.Call_ATTACH_CONTAINER_INPUT, + AttachContainerInput: &agent.Call_AttachContainerInput{ + Type: agent.Call_AttachContainerInput_PROCESS_IO, + ProcessIO: &agent.ProcessIO{ + Type: agent.ProcessIO_CONTROL, + Control: &agent.ProcessIO_Control{ + Type: agent.ProcessIO_Control_TTY_INFO, + TTYInfo: t, + }, + }, + }, + } +} diff --git a/api/v1/lib/agent/calls/calls_generated.go b/api/v1/lib/agent/calls/calls_generated.go new file mode 100644 index 00000000..98816f55 --- /dev/null +++ b/api/v1/lib/agent/calls/calls_generated.go @@ -0,0 +1,129 @@ +package calls + +// go generate -import github.com/mesos/mesos-go/api/v1/lib/agent -type C:agent.Call +// GENERATED CODE FOLLOWS; DO NOT EDIT. + +import ( + "context" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/encoding" + + "github.com/mesos/mesos-go/api/v1/lib/agent" +) + +type ( + // Request generates a Call that's sent to a Mesos agent. Subsequent invocations are expected to + // yield equivalent calls. Intended for use w/ non-streaming requests to an agent. + Request interface { + Call() *agent.Call + } + + // RequestFunc is the functional adaptation of Request. + RequestFunc func() *agent.Call + + // RequestStreaming generates a Call that's send to a Mesos agent. Subsequent invocations MAY generate + // different Call objects. No more Call objects are expected once a nil is returned to signal the end of + // of the request stream. + RequestStreaming interface { + Request + IsStreaming() + } + + // RequestStreamingFunc is the functional adaptation of RequestStreaming. + RequestStreamingFunc func() *agent.Call + + // Send issues a Request to a Mesos agent and properly manages Call-specific mechanics. + Sender interface { + Send(context.Context, Request) (mesos.Response, error) + } + + // SenderFunc is the functional adaptation of the Sender interface + SenderFunc func(context.Context, Request) (mesos.Response, error) +) + +func (f RequestFunc) Call() *agent.Call { return f() } + +func (f RequestFunc) Marshaler() encoding.Marshaler { + // avoid returning (*agent.Call)(nil) for interface type + if call := f(); call != nil { + return call + } + return nil +} + +func (f RequestStreamingFunc) Push(c ...*agent.Call) RequestStreamingFunc { return Push(f, c...) } + +func (f RequestStreamingFunc) Marshaler() encoding.Marshaler { + // avoid returning (*agent.Call)(nil) for interface type + if call := f(); call != nil { + return call + } + return nil +} + +func (f RequestStreamingFunc) IsStreaming() {} + +func (f RequestStreamingFunc) Call() *agent.Call { return f() } + +// Push prepends one or more calls onto a request stream. If no calls are given then the original stream is returned. +func Push(r RequestStreaming, c ...*agent.Call) RequestStreamingFunc { + return func() *agent.Call { + if len(c) == 0 { + return r.Call() + } + head := c[0] + c = c[1:] + return head + } +} + +// Empty generates a stream that always returns nil. +func Empty() RequestStreamingFunc { return func() *agent.Call { return nil } } + +var ( + _ = Request(RequestFunc(nil)) + _ = RequestStreaming(RequestStreamingFunc(nil)) + _ = Sender(SenderFunc(nil)) +) + +// NonStreaming returns a RequestFunc that always generates the same Call. +func NonStreaming(c *agent.Call) RequestFunc { return func() *agent.Call { return c } } + +// FromChan returns a streaming request that fetches calls from the given channel until it closes. +// If a nil chan is specified then the returned func will always generate nil. +func FromChan(ch <-chan *agent.Call) RequestStreamingFunc { + if ch == nil { + // avoid blocking forever if we're handed a nil chan + return func() *agent.Call { return nil } + } + return func() *agent.Call { + if m, ok := <-ch; ok { + return m + } + return nil + } +} + +// Send implements the Sender interface for SenderFunc +func (f SenderFunc) Send(ctx context.Context, r Request) (mesos.Response, error) { + return f(ctx, r) +} + +// IgnoreResponse generates a sender that closes any non-nil response received by Mesos. +func IgnoreResponse(s Sender) SenderFunc { + return func(ctx context.Context, r Request) (mesos.Response, error) { + resp, err := s.Send(ctx, r) + if resp != nil { + resp.Close() + } + return nil, err + } +} + +// SendNoData is a convenience func that executes the given Call using the provided Sender +// and always drops the response data. +func SendNoData(ctx context.Context, sender Sender, r Request) (err error) { + _, err = IgnoreResponse(sender).Send(ctx, r) + return +} diff --git a/api/v1/lib/agent/calls/calls_generated_test.go b/api/v1/lib/agent/calls/calls_generated_test.go new file mode 100644 index 00000000..cdba8b97 --- /dev/null +++ b/api/v1/lib/agent/calls/calls_generated_test.go @@ -0,0 +1,99 @@ +package calls + +// go generate -import github.com/mesos/mesos-go/api/v1/lib/agent -type C:agent.Call +// GENERATED CODE FOLLOWS; DO NOT EDIT. + +import ( + "context" + "testing" + + "github.com/mesos/mesos-go/api/v1/lib" + + "github.com/mesos/mesos-go/api/v1/lib/agent" +) + +func TestNonStreaming(t *testing.T) { + c := new(agent.Call) + f := NonStreaming(c) + if x := f.Call(); x != c { + t.Fatalf("expected %#v instead of %#v", c, x) + } + if x := f.Marshaler(); x == nil { + t.Fatal("expected non-nil Marshaler") + } + f = NonStreaming(nil) + if x := f.Marshaler(); x != nil { + t.Fatalf("expected nil Marshaler instead of %#v", x) + } +} + +func TestStreaming(t *testing.T) { + f := Empty() + + f.IsStreaming() + + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + if x := f.Marshaler(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + + c := new(agent.Call) + + f = f.Push(c) + if x := f.Marshaler(); x == nil { + t.Fatal("expected non-nil Marshaler") + } + if x := f.Marshaler(); x != nil { + t.Fatalf("expected nil Marshaler instead of %#v", x) + } + + c2 := new(agent.Call) + + f = Empty().Push(c, c2) + if x := f.Call(); x != c { + t.Fatalf("expected %#v instead of %#v", c, x) + } + if x := f.Call(); x != c2 { + t.Fatalf("expected %#v instead of %#v", c2, x) + } + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + + ch := make(chan *agent.Call, 2) + ch <- c + ch <- c2 + close(ch) + f = FromChan(ch) + if x := f.Call(); x != c { + t.Fatalf("expected %#v instead of %#v", c, x) + } + if x := f.Call(); x != c2 { + t.Fatalf("expected %#v instead of %#v", c2, x) + } + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + + f = FromChan(nil) + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } +} + +func TestIgnoreResponse(t *testing.T) { + var closed bool + + IgnoreResponse(SenderFunc(func(_ context.Context, _ Request) (mesos.Response, error) { + return &mesos.ResponseWrapper{Closer: mesos.CloseFunc(func() error { + closed = true + return nil + })}, nil + })).Send(nil, nil) + + if !closed { + t.Fatal("expected response to be closed") + } +} diff --git a/api/v1/lib/agent/calls/calls_test.go b/api/v1/lib/agent/calls/calls_test.go new file mode 100644 index 00000000..c9efe88e --- /dev/null +++ b/api/v1/lib/agent/calls/calls_test.go @@ -0,0 +1,69 @@ +package calls_test + +import ( + "context" + "time" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/agent" + . "github.com/mesos/mesos-go/api/v1/lib/agent/calls" +) + +func Example() { + var ( + check = func(err error) { + if err != nil { + panic(err) + } + } + swallow = func(_ mesos.Response, err error) { check(err) } + + ctx = context.Background() + sender = SenderFunc(func(_ context.Context, _ Request) (_ mesos.Response, _ error) { return }) + blackhole = func(calls ...*agent.Call) { + for i := range calls { + swallow(sender.Send(ctx, NonStreaming(calls[i]))) + } + } + + d = time.Duration(0) + ) + blackhole( + GetHealth(), + GetFlags(), + GetVersion(), + GetMetrics(nil), + GetMetrics(&d), + GetLoggingLevel(), + ListFiles(""), + ReadFile("", 0), + ReadFileWithLength("", 0, 0), + GetState(), + GetContainers(), + GetFrameworks(), + GetExecutors(), + GetTasks(), + WaitNestedContainer(mesos.ContainerID{}), + LaunchNestedContainerSession(mesos.ContainerID{}, nil, nil), + ) + + blackhole = func(calls ...*agent.Call) { + for i := range calls { + check(SendNoData(ctx, sender, NonStreaming(calls[i]))) + } + } + blackhole( + SetLoggingLevel(0, d), + LaunchNestedContainer(mesos.ContainerID{}, nil, nil), + KillNestedContainer(mesos.ContainerID{}), + AttachContainerOutput(mesos.ContainerID{}), + ) + + swallow(sender.Send(ctx, Empty().Push( + AttachContainerInput(mesos.ContainerID{}), + AttachContainerInputTTY(nil), + AttachContainerInputData(nil), + ))) + + // Output: +} diff --git a/api/v1/lib/agent/calls/gen.go b/api/v1/lib/agent/calls/gen.go new file mode 100644 index 00000000..53693c06 --- /dev/null +++ b/api/v1/lib/agent/calls/gen.go @@ -0,0 +1,3 @@ +package calls + +//go:generate go run ../../extras/gen/sender.go ../../extras/gen/gen.go -import github.com/mesos/mesos-go/api/v1/lib/agent -type C:agent.Call diff --git a/api/v1/lib/allocator/allocator.pb.go b/api/v1/lib/allocator/allocator.pb.go new file mode 100644 index 00000000..7fa1c9dd --- /dev/null +++ b/api/v1/lib/allocator/allocator.pb.go @@ -0,0 +1,717 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: allocator/allocator.proto + +/* + Package allocator is a generated protocol buffer package. + + It is generated from these files: + allocator/allocator.proto + + It has these top-level messages: + InverseOfferStatus +*/ +package allocator + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" +import mesos "github.com/mesos/mesos-go/api/v1/lib" +import _ "github.com/gogo/protobuf/gogoproto" + +import strconv "strconv" + +import strings "strings" +import reflect "reflect" + +import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" + +import io "io" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +type InverseOfferStatus_Status int32 + +const ( + // We have not received a response yet. This is the default state before + // receiving a response. + InverseOfferStatus_UNKNOWN InverseOfferStatus_Status = 1 + // The framework is ok with the inverse offer. This means it will not + // violate any SLAs and will attempt to evacuate any tasks running on the + // agent. If the tasks are not evacuated by the framework, the operator can + // manually shut down the slave knowing that the framework will not have + // violated its SLAs. + InverseOfferStatus_ACCEPT InverseOfferStatus_Status = 2 + // The framework wants to block the maintenance operation from happening. An + // example would be that it cannot meet its SLA by losing resources. + InverseOfferStatus_DECLINE InverseOfferStatus_Status = 3 +) + +var InverseOfferStatus_Status_name = map[int32]string{ + 1: "UNKNOWN", + 2: "ACCEPT", + 3: "DECLINE", +} +var InverseOfferStatus_Status_value = map[string]int32{ + "UNKNOWN": 1, + "ACCEPT": 2, + "DECLINE": 3, +} + +func (x InverseOfferStatus_Status) Enum() *InverseOfferStatus_Status { + p := new(InverseOfferStatus_Status) + *p = x + return p +} +func (x InverseOfferStatus_Status) MarshalJSON() ([]byte, error) { + return proto.MarshalJSONEnum(InverseOfferStatus_Status_name, int32(x)) +} +func (x *InverseOfferStatus_Status) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(InverseOfferStatus_Status_value, data, "InverseOfferStatus_Status") + if err != nil { + return err + } + *x = InverseOfferStatus_Status(value) + return nil +} +func (InverseOfferStatus_Status) EnumDescriptor() ([]byte, []int) { + return fileDescriptorAllocator, []int{0, 0} +} + +// * +// Describes the status of an inverse offer. +// +// This is a protobuf so as to be able to share the status to inverse offers +// through endpoints such as the maintenance status endpoint. +type InverseOfferStatus struct { + Status *InverseOfferStatus_Status `protobuf:"varint,1,req,name=status,enum=mesos.allocator.InverseOfferStatus_Status" json:"status,omitempty"` + FrameworkID mesos.FrameworkID `protobuf:"bytes,2,req,name=framework_id,json=frameworkId" json:"framework_id"` + // Time, since the epoch, when this status was last updated. + Timestamp mesos.TimeInfo `protobuf:"bytes,3,req,name=timestamp" json:"timestamp"` +} + +func (m *InverseOfferStatus) Reset() { *m = InverseOfferStatus{} } +func (*InverseOfferStatus) ProtoMessage() {} +func (*InverseOfferStatus) Descriptor() ([]byte, []int) { return fileDescriptorAllocator, []int{0} } + +func (m *InverseOfferStatus) GetStatus() InverseOfferStatus_Status { + if m != nil && m.Status != nil { + return *m.Status + } + return InverseOfferStatus_UNKNOWN +} + +func (m *InverseOfferStatus) GetFrameworkID() mesos.FrameworkID { + if m != nil { + return m.FrameworkID + } + return mesos.FrameworkID{} +} + +func (m *InverseOfferStatus) GetTimestamp() mesos.TimeInfo { + if m != nil { + return m.Timestamp + } + return mesos.TimeInfo{} +} + +func init() { + proto.RegisterType((*InverseOfferStatus)(nil), "mesos.allocator.InverseOfferStatus") + proto.RegisterEnum("mesos.allocator.InverseOfferStatus_Status", InverseOfferStatus_Status_name, InverseOfferStatus_Status_value) +} +func (x InverseOfferStatus_Status) String() string { + s, ok := InverseOfferStatus_Status_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (this *InverseOfferStatus) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*InverseOfferStatus) + if !ok { + that2, ok := that.(InverseOfferStatus) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *InverseOfferStatus") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *InverseOfferStatus but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *InverseOfferStatus but is not nil && this == nil") + } + if this.Status != nil && that1.Status != nil { + if *this.Status != *that1.Status { + return fmt.Errorf("Status this(%v) Not Equal that(%v)", *this.Status, *that1.Status) + } + } else if this.Status != nil { + return fmt.Errorf("this.Status == nil && that.Status != nil") + } else if that1.Status != nil { + return fmt.Errorf("Status this(%v) Not Equal that(%v)", this.Status, that1.Status) + } + if !this.FrameworkID.Equal(&that1.FrameworkID) { + return fmt.Errorf("FrameworkID this(%v) Not Equal that(%v)", this.FrameworkID, that1.FrameworkID) + } + if !this.Timestamp.Equal(&that1.Timestamp) { + return fmt.Errorf("Timestamp this(%v) Not Equal that(%v)", this.Timestamp, that1.Timestamp) + } + return nil +} +func (this *InverseOfferStatus) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*InverseOfferStatus) + if !ok { + that2, ok := that.(InverseOfferStatus) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Status != nil && that1.Status != nil { + if *this.Status != *that1.Status { + return false + } + } else if this.Status != nil { + return false + } else if that1.Status != nil { + return false + } + if !this.FrameworkID.Equal(&that1.FrameworkID) { + return false + } + if !this.Timestamp.Equal(&that1.Timestamp) { + return false + } + return true +} +func (this *InverseOfferStatus) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&allocator.InverseOfferStatus{") + if this.Status != nil { + s = append(s, "Status: "+valueToGoStringAllocator(this.Status, "InverseOfferStatus_Status")+",\n") + } + s = append(s, "FrameworkID: "+strings.Replace(this.FrameworkID.GoString(), `&`, ``, 1)+",\n") + s = append(s, "Timestamp: "+strings.Replace(this.Timestamp.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringAllocator(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} +func (m *InverseOfferStatus) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *InverseOfferStatus) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Status == nil { + return 0, github_com_gogo_protobuf_proto.NewRequiredNotSetError("status") + } else { + dAtA[i] = 0x8 + i++ + i = encodeVarintAllocator(dAtA, i, uint64(*m.Status)) + } + dAtA[i] = 0x12 + i++ + i = encodeVarintAllocator(dAtA, i, uint64(m.FrameworkID.ProtoSize())) + n1, err := m.FrameworkID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n1 + dAtA[i] = 0x1a + i++ + i = encodeVarintAllocator(dAtA, i, uint64(m.Timestamp.ProtoSize())) + n2, err := m.Timestamp.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + return i, nil +} + +func encodeFixed64Allocator(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 encodeFixed32Allocator(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 encodeVarintAllocator(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func NewPopulatedInverseOfferStatus(r randyAllocator, easy bool) *InverseOfferStatus { + this := &InverseOfferStatus{} + v1 := InverseOfferStatus_Status([]int32{1, 2, 3}[r.Intn(3)]) + this.Status = &v1 + v2 := mesos.NewPopulatedFrameworkID(r, easy) + this.FrameworkID = *v2 + v3 := mesos.NewPopulatedTimeInfo(r, easy) + this.Timestamp = *v3 + if !easy && r.Intn(10) != 0 { + } + return this +} + +type randyAllocator interface { + Float32() float32 + Float64() float64 + Int63() int64 + Int31() int32 + Uint32() uint32 + Intn(n int) int +} + +func randUTF8RuneAllocator(r randyAllocator) rune { + ru := r.Intn(62) + if ru < 10 { + return rune(ru + 48) + } else if ru < 36 { + return rune(ru + 55) + } + return rune(ru + 61) +} +func randStringAllocator(r randyAllocator) string { + v4 := r.Intn(100) + tmps := make([]rune, v4) + for i := 0; i < v4; i++ { + tmps[i] = randUTF8RuneAllocator(r) + } + return string(tmps) +} +func randUnrecognizedAllocator(r randyAllocator, maxFieldNumber int) (dAtA []byte) { + l := r.Intn(5) + for i := 0; i < l; i++ { + wire := r.Intn(4) + if wire == 3 { + wire = 5 + } + fieldNumber := maxFieldNumber + r.Intn(100) + dAtA = randFieldAllocator(dAtA, r, fieldNumber, wire) + } + return dAtA +} +func randFieldAllocator(dAtA []byte, r randyAllocator, fieldNumber int, wire int) []byte { + key := uint32(fieldNumber)<<3 | uint32(wire) + switch wire { + case 0: + dAtA = encodeVarintPopulateAllocator(dAtA, uint64(key)) + v5 := r.Int63() + if r.Intn(2) == 0 { + v5 *= -1 + } + dAtA = encodeVarintPopulateAllocator(dAtA, uint64(v5)) + case 1: + dAtA = encodeVarintPopulateAllocator(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + case 2: + dAtA = encodeVarintPopulateAllocator(dAtA, uint64(key)) + ll := r.Intn(100) + dAtA = encodeVarintPopulateAllocator(dAtA, uint64(ll)) + for j := 0; j < ll; j++ { + dAtA = append(dAtA, byte(r.Intn(256))) + } + default: + dAtA = encodeVarintPopulateAllocator(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + } + return dAtA +} +func encodeVarintPopulateAllocator(dAtA []byte, v uint64) []byte { + for v >= 1<<7 { + dAtA = append(dAtA, uint8(uint64(v)&0x7f|0x80)) + v >>= 7 + } + dAtA = append(dAtA, uint8(v)) + return dAtA +} +func (m *InverseOfferStatus) ProtoSize() (n int) { + var l int + _ = l + if m.Status != nil { + n += 1 + sovAllocator(uint64(*m.Status)) + } + l = m.FrameworkID.ProtoSize() + n += 1 + l + sovAllocator(uint64(l)) + l = m.Timestamp.ProtoSize() + n += 1 + l + sovAllocator(uint64(l)) + return n +} + +func sovAllocator(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozAllocator(x uint64) (n int) { + return sovAllocator(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *InverseOfferStatus) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&InverseOfferStatus{`, + `Status:` + valueToStringAllocator(this.Status) + `,`, + `FrameworkID:` + strings.Replace(strings.Replace(this.FrameworkID.String(), "FrameworkID", "mesos.FrameworkID", 1), `&`, ``, 1) + `,`, + `Timestamp:` + strings.Replace(strings.Replace(this.Timestamp.String(), "TimeInfo", "mesos.TimeInfo", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func valueToStringAllocator(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *InverseOfferStatus) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAllocator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: InverseOfferStatus: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: InverseOfferStatus: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var v InverseOfferStatus_Status + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAllocator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (InverseOfferStatus_Status(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Status = &v + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FrameworkID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAllocator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthAllocator + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.FrameworkID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000002) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAllocator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthAllocator + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Timestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000004) + default: + iNdEx = preIndex + skippy, err := skipAllocator(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthAllocator + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("status") + } + if hasFields[0]&uint64(0x00000002) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("framework_id") + } + if hasFields[0]&uint64(0x00000004) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("timestamp") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipAllocator(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowAllocator + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowAllocator + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowAllocator + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthAllocator + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowAllocator + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipAllocator(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthAllocator = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowAllocator = fmt.Errorf("proto: integer overflow") +) + +func init() { proto.RegisterFile("allocator/allocator.proto", fileDescriptorAllocator) } + +var fileDescriptorAllocator = []byte{ + // 361 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4c, 0xcc, 0xc9, 0xc9, + 0x4f, 0x4e, 0x2c, 0xc9, 0x2f, 0xd2, 0x87, 0xb3, 0xf4, 0x0a, 0x8a, 0xf2, 0x4b, 0xf2, 0x85, 0xf8, + 0x73, 0x53, 0x8b, 0xf3, 0x8b, 0xf5, 0xe0, 0xc2, 0x52, 0x06, 0xe9, 0x99, 0x25, 0x19, 0xa5, 0x49, + 0x7a, 0xc9, 0xf9, 0xb9, 0xfa, 0x60, 0x39, 0x08, 0xa9, 0x9b, 0x9e, 0xaf, 0x9f, 0x58, 0x90, 0xa9, + 0x5f, 0x66, 0xa8, 0x9f, 0x93, 0x99, 0x04, 0x11, 0x83, 0x18, 0x21, 0xa5, 0x8b, 0xa4, 0x23, 0x3d, + 0x3f, 0x3d, 0x5f, 0x1f, 0x2c, 0x9c, 0x54, 0x9a, 0x06, 0xe6, 0x81, 0x39, 0x60, 0x16, 0x44, 0xb9, + 0x52, 0x3f, 0x13, 0x97, 0x90, 0x67, 0x5e, 0x59, 0x6a, 0x51, 0x71, 0xaa, 0x7f, 0x5a, 0x5a, 0x6a, + 0x51, 0x70, 0x49, 0x62, 0x49, 0x69, 0xb1, 0x90, 0x13, 0x17, 0x5b, 0x31, 0x98, 0x25, 0xc1, 0xa8, + 0xc0, 0xa4, 0xc1, 0x67, 0xa4, 0xa5, 0x87, 0xe6, 0x32, 0x3d, 0x4c, 0x4d, 0x7a, 0x10, 0x2a, 0x08, + 0xaa, 0x53, 0xc8, 0x8b, 0x8b, 0x27, 0xad, 0x28, 0x31, 0x37, 0xb5, 0x3c, 0xbf, 0x28, 0x3b, 0x3e, + 0x33, 0x45, 0x82, 0x49, 0x81, 0x49, 0x83, 0xdb, 0x48, 0x08, 0x6a, 0x92, 0x1b, 0x4c, 0xca, 0xd3, + 0xc5, 0x49, 0xf8, 0xc4, 0x3d, 0x79, 0x86, 0x47, 0xf7, 0xe4, 0xb9, 0x91, 0x04, 0x83, 0xb8, 0xe1, + 0x9a, 0x3d, 0x53, 0x84, 0x8c, 0xb9, 0x38, 0x4b, 0x32, 0x73, 0x53, 0x8b, 0x4b, 0x12, 0x73, 0x0b, + 0x24, 0x98, 0xc1, 0x06, 0xf1, 0x43, 0x0d, 0x0a, 0xc9, 0xcc, 0x4d, 0xf5, 0xcc, 0x4b, 0xcb, 0x77, + 0x62, 0x01, 0x99, 0x12, 0x84, 0x50, 0xa7, 0x64, 0xc2, 0xc5, 0x06, 0xf5, 0x0e, 0x37, 0x17, 0x7b, + 0xa8, 0x9f, 0xb7, 0x9f, 0x7f, 0xb8, 0x9f, 0x00, 0xa3, 0x10, 0x17, 0x17, 0x9b, 0xa3, 0xb3, 0xb3, + 0x6b, 0x40, 0x88, 0x00, 0x13, 0x48, 0xc2, 0xc5, 0xd5, 0xd9, 0xc7, 0xd3, 0xcf, 0x55, 0x80, 0x59, + 0x8a, 0xa5, 0x63, 0xb1, 0x1c, 0xa3, 0x93, 0xfb, 0x85, 0x87, 0x72, 0x0c, 0x37, 0x1e, 0xca, 0x31, + 0x3c, 0x78, 0x28, 0xc7, 0xf8, 0xe1, 0xa1, 0x1c, 0xe3, 0x8f, 0x87, 0x72, 0x8c, 0x0d, 0x8f, 0xe4, + 0x18, 0x57, 0x3c, 0x92, 0x63, 0xdc, 0xf1, 0x48, 0x8e, 0xf1, 0xc0, 0x23, 0x39, 0xc6, 0x13, 0x8f, + 0xe4, 0x18, 0x2f, 0x3c, 0x92, 0x63, 0x7c, 0xf1, 0x48, 0x8e, 0xe1, 0xc3, 0x23, 0x39, 0xc6, 0x09, + 0x8f, 0xe5, 0x18, 0x16, 0x3c, 0x96, 0x63, 0x8c, 0xe2, 0x84, 0x87, 0x10, 0x20, 0x00, 0x00, 0xff, + 0xff, 0x9b, 0x93, 0x27, 0x49, 0xe8, 0x01, 0x00, 0x00, +} diff --git a/api/v1/lib/allocator/allocator.pb_ffjson.go b/api/v1/lib/allocator/allocator.pb_ffjson.go new file mode 100644 index 00000000..6ca0579e --- /dev/null +++ b/api/v1/lib/allocator/allocator.pb_ffjson.go @@ -0,0 +1,326 @@ +// DO NOT EDIT! +// Code generated by ffjson +// source: allocator/allocator.pb.go +// DO NOT EDIT! + +package allocator + +import ( + "bytes" + "fmt" + fflib "github.com/pquerna/ffjson/fflib/v1" +) + +func (mj *InverseOfferStatus) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *InverseOfferStatus) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteByte('{') + if mj.Status != nil { + if true { + buf.WriteString(`"status":`) + + { + + obj, err = mj.Status.MarshalJSON() + if err != nil { + return err + } + buf.Write(obj) + + } + buf.WriteByte(',') + } + } + buf.WriteString(`"framework_id":`) + + { + + err = mj.FrameworkID.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteString(`,"timestamp":`) + + { + + err = mj.Timestamp.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_InverseOfferStatusbase = iota + ffj_t_InverseOfferStatusno_such_key + + ffj_t_InverseOfferStatus_Status + + ffj_t_InverseOfferStatus_FrameworkID + + ffj_t_InverseOfferStatus_Timestamp +) + +var ffj_key_InverseOfferStatus_Status = []byte("status") + +var ffj_key_InverseOfferStatus_FrameworkID = []byte("framework_id") + +var ffj_key_InverseOfferStatus_Timestamp = []byte("timestamp") + +func (uj *InverseOfferStatus) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *InverseOfferStatus) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_InverseOfferStatusbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_InverseOfferStatusno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'f': + + if bytes.Equal(ffj_key_InverseOfferStatus_FrameworkID, kn) { + currentKey = ffj_t_InverseOfferStatus_FrameworkID + state = fflib.FFParse_want_colon + goto mainparse + } + + case 's': + + if bytes.Equal(ffj_key_InverseOfferStatus_Status, kn) { + currentKey = ffj_t_InverseOfferStatus_Status + state = fflib.FFParse_want_colon + goto mainparse + } + + case 't': + + if bytes.Equal(ffj_key_InverseOfferStatus_Timestamp, kn) { + currentKey = ffj_t_InverseOfferStatus_Timestamp + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_InverseOfferStatus_Timestamp, kn) { + currentKey = ffj_t_InverseOfferStatus_Timestamp + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_InverseOfferStatus_FrameworkID, kn) { + currentKey = ffj_t_InverseOfferStatus_FrameworkID + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_InverseOfferStatus_Status, kn) { + currentKey = ffj_t_InverseOfferStatus_Status + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_InverseOfferStatusno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_InverseOfferStatus_Status: + goto handle_Status + + case ffj_t_InverseOfferStatus_FrameworkID: + goto handle_FrameworkID + + case ffj_t_InverseOfferStatus_Timestamp: + goto handle_Timestamp + + case ffj_t_InverseOfferStatusno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Status: + + /* handler: uj.Status type=allocator.InverseOfferStatus_Status kind=int32 quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.Status = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + tbuf, err := fs.CaptureField(tok) + if err != nil { + return fs.WrapErr(err) + } + + if uj.Status == nil { + uj.Status = new(InverseOfferStatus_Status) + } + + err = uj.Status.UnmarshalJSON(tbuf) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_FrameworkID: + + /* handler: uj.FrameworkID type=mesos.FrameworkID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.FrameworkID.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Timestamp: + + /* handler: uj.Timestamp type=mesos.TimeInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Timestamp.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} diff --git a/api/v1/lib/allocator/allocator.proto b/api/v1/lib/allocator/allocator.proto new file mode 100644 index 00000000..64564983 --- /dev/null +++ b/api/v1/lib/allocator/allocator.proto @@ -0,0 +1,74 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto2"; + +package mesos.allocator; + +import "github.com/mesos/mesos-go/api/v1/lib/mesos.proto"; +import "github.com/gogo/protobuf/gogoproto/gogo.proto"; + +option go_package = "allocator"; +option (gogoproto.benchgen_all) = true; +option (gogoproto.enum_stringer_all) = true; +option (gogoproto.equal_all) = true; +option (gogoproto.goproto_enum_prefix_all) = false; +option (gogoproto.goproto_enum_stringer_all) = false; +option (gogoproto.goproto_stringer_all) = false; +option (gogoproto.goproto_unrecognized_all) = false; +option (gogoproto.gostring_all) = true; +option (gogoproto.marshaler_all) = true; +option (gogoproto.populate_all) = true; +option (gogoproto.protosizer_all) = true; +option (gogoproto.stringer_all) = true; +option (gogoproto.testgen_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.verbose_equal_all) = true; + + +/** + * Describes the status of an inverse offer. + * + * This is a protobuf so as to be able to share the status to inverse offers + * through endpoints such as the maintenance status endpoint. + */ +message InverseOfferStatus { + enum Status { + // We have not received a response yet. This is the default state before + // receiving a response. + UNKNOWN = 1; + // The framework is ok with the inverse offer. This means it will not + // violate any SLAs and will attempt to evacuate any tasks running on the + // agent. If the tasks are not evacuated by the framework, the operator can + // manually shut down the slave knowing that the framework will not have + // violated its SLAs. + ACCEPT = 2; + // The framework wants to block the maintenance operation from happening. An + // example would be that it cannot meet its SLA by losing resources. + DECLINE = 3; + + option (gogoproto.goproto_enum_prefix) = true; + } + + required Status status = 1; + required FrameworkID framework_id = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "FrameworkID"]; + + // Time, since the epoch, when this status was last updated. + required TimeInfo timestamp = 3 [(gogoproto.nullable) = false]; + + // TODO(jmlvanre): Capture decline message. +} diff --git a/api/v1/lib/allocator/allocatorpb_test.go b/api/v1/lib/allocator/allocatorpb_test.go new file mode 100644 index 00000000..98ea05af --- /dev/null +++ b/api/v1/lib/allocator/allocatorpb_test.go @@ -0,0 +1,263 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: allocator/allocator.proto + +/* +Package allocator is a generated protocol buffer package. + +It is generated from these files: + allocator/allocator.proto + +It has these top-level messages: + InverseOfferStatus +*/ +package allocator + +import testing "testing" +import math_rand "math/rand" +import time "time" +import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" +import github_com_gogo_protobuf_jsonpb "github.com/gogo/protobuf/jsonpb" +import fmt "fmt" +import go_parser "go/parser" +import proto "github.com/gogo/protobuf/proto" +import math "math" +import _ "github.com/mesos/mesos-go/api/v1/lib" +import _ "github.com/gogo/protobuf/gogoproto" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +func TestInverseOfferStatusProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedInverseOfferStatus(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &InverseOfferStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestInverseOfferStatusMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedInverseOfferStatus(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &InverseOfferStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkInverseOfferStatusProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*InverseOfferStatus, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedInverseOfferStatus(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkInverseOfferStatusProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedInverseOfferStatus(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &InverseOfferStatus{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestInverseOfferStatusJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedInverseOfferStatus(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &InverseOfferStatus{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestInverseOfferStatusProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedInverseOfferStatus(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &InverseOfferStatus{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestInverseOfferStatusProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedInverseOfferStatus(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &InverseOfferStatus{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestInverseOfferStatusVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedInverseOfferStatus(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &InverseOfferStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestInverseOfferStatusGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedInverseOfferStatus(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestInverseOfferStatusProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedInverseOfferStatus(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkInverseOfferStatusProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*InverseOfferStatus, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedInverseOfferStatus(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestInverseOfferStatusStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedInverseOfferStatus(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} + +//These tests are generated by github.com/gogo/protobuf/plugin/testgen diff --git a/api/v1/lib/client.go b/api/v1/lib/client.go index 5cc641f6..233fe362 100644 --- a/api/v1/lib/client.go +++ b/api/v1/lib/client.go @@ -1,5 +1,7 @@ package mesos +// DEPRECATED in favor of github.com/mesos/mesos-go/api/v1/lib/client + import ( "io" diff --git a/api/v1/lib/client/client.go b/api/v1/lib/client/client.go new file mode 100644 index 00000000..fcc15607 --- /dev/null +++ b/api/v1/lib/client/client.go @@ -0,0 +1,53 @@ +package client + +import ( + "github.com/mesos/mesos-go/api/v1/lib/encoding" +) + +type ( + // ResponseClass indicates the kind of response that a caller is expecting from Mesos. + ResponseClass int + + // Request is a non-streaming request from the client to the server. + // Marshaler always returns the same object; the object is sent once to the server and then + // a response is expected. + Request interface { + Marshaler() encoding.Marshaler + } + + // RequestStreaming is a streaming request from the client to the server. + // Marshaler returns a new object for upon each invocation, nil when there are no more objects to send. + // Client implementations are expected to differentiate between Request and RequestStreaming either by + // type-switching or by attempting interface conversion. + RequestStreaming interface { + Request + IsStreaming() + } + + RequestFunc func() encoding.Marshaler + RequestStreamingFunc func() encoding.Marshaler +) + +var ( + _ = Request(RequestFunc(nil)) + _ = RequestStreaming(RequestStreamingFunc(nil)) +) + +func (f RequestFunc) Marshaler() encoding.Marshaler { return f() } +func (f RequestStreamingFunc) Marshaler() encoding.Marshaler { return f() } +func (f RequestStreamingFunc) IsStreaming() {} + +// RequestSingleton generates a non-streaming Request that always returns the same marshaler +func RequestSingleton(m encoding.Marshaler) Request { + return RequestFunc(func() encoding.Marshaler { return m }) +} + +const ( + ResponseClassSingleton ResponseClass = iota + ResponseClassStreaming + + // ResponseClassAuto should be used with versions of Mesos prior to 1.2.x. + // Otherwise, this type is deprecated and callers should use ResponseClassSingleton + // or ResponseClassStreaming instead. + ResponseClassAuto +) diff --git a/api/v1/lib/encoding/codec.go b/api/v1/lib/encoding/codec.go index 120af1eb..caabcc10 100644 --- a/api/v1/lib/encoding/codec.go +++ b/api/v1/lib/encoding/codec.go @@ -2,13 +2,11 @@ package encoding import ( "encoding/json" - "fmt" "io" - - "github.com/mesos/mesos-go/api/v1/lib/encoding/framing" - "github.com/mesos/mesos-go/api/v1/lib/encoding/proto" + "io/ioutil" pb "github.com/gogo/protobuf/proto" + "github.com/mesos/mesos-go/api/v1/lib/encoding/framing" ) type MediaType string @@ -22,18 +20,18 @@ const ( // DefaultCodecs are pre-configured default Codecs, ready to use OOTB var DefaultCodecs = map[MediaType]Codec{ - MediaTypeProtobuf: Codec(&codec{ - name: "protobuf", - mediaTypes: [2]MediaType{MediaTypeProtobuf, MediaTypeProtobuf}, - newEncoder: NewProtobufEncoder, - newDecoder: NewProtobufDecoder, - }), - MediaTypeJSON: Codec(&codec{ - name: "json", - mediaTypes: [2]MediaType{MediaTypeJSON, MediaTypeJSON}, - newEncoder: NewJSONEncoder, - newDecoder: NewJSONDecoder, - }), + MediaTypeProtobuf: Codec{ + Name: "protobuf", + Type: MediaTypeProtobuf, + NewEncoder: NewProtobufEncoder, + NewDecoder: NewProtobufDecoder, + }, + MediaTypeJSON: Codec{ + Name: "json", + Type: MediaTypeJSON, + NewEncoder: NewJSONEncoder, + NewDecoder: NewJSONDecoder, + }, } // Codec returns the configured Codec for the media type, or nil if no such Codec has been configured. @@ -43,45 +41,76 @@ func (m MediaType) Codec() Codec { return DefaultCodecs[m] } func (m MediaType) ContentType() string { return string(m) } type ( + Source func() framing.Reader + Sink func() framing.Writer + // A Codec composes encoding and decoding of a serialization format. - Codec interface { - fmt.Stringer - Name() string - RequestType() MediaType - ResponseType() MediaType - // NewEncoder returns a new encoder for the defined media type. - NewEncoder(io.Writer) Encoder - // NewDecoder returns a new decoder for the defined media type. - NewDecoder(framing.Reader) Decoder + Codec struct { + Name string + Type MediaType + NewEncoder func(Sink) Encoder + NewDecoder func(Source) Decoder + } + + SourceFactory interface { + NewSource(r io.Reader) Source + } + SourceFactoryFunc func(r io.Reader) Source + + SinkFactory interface { + NewSink(w io.Writer) Sink } + SinkFactoryFunc func(w io.Writer) Sink - codec struct { - // Name holds the codec name. - name string - // MediaTypes holds the media types of the codec encoding and decoding - // formats, respectively. - mediaTypes [2]MediaType - // NewEncoder returns a new encoder for the defined media type. - newEncoder func(io.Writer) EncoderFunc - // NewDecoder returns a new decoder for the defined media type. - newDecoder func(framing.Reader) DecoderFunc + Stream interface { + SourceFactory + SinkFactory } ) +func (f SourceFactoryFunc) NewSource(r io.Reader) Source { return f(r) } +func (f SinkFactoryFunc) NewSink(w io.Writer) Sink { return f(w) } + +// SourceReader returns a Source that buffers all input from the given io.Reader +// and returns the contents in a single frame. +func SourceReader(r io.Reader) Source { + return func() framing.Reader { + b, err := ioutil.ReadAll(r) + return framing.ReaderFunc(func() (f []byte, e error) { + // only return a non-nil frame ONCE + f = b + b = nil + e = err + + if e == nil { + e = io.EOF + } + return + }) + } +} + +// SinkWriter returns a Sink that sends a frame to an io.Writer with no decoration. +func SinkWriter(w io.Writer) Sink { + return func() framing.Writer { + return framing.WriterFunc(func(b []byte) error { + n, err := w.Write(b) + if err == nil && n != len(b) { + return io.ErrShortWrite + } + return err + }) + } +} + // String implements the fmt.Stringer interface. -func (c *codec) String() string { +func (c *Codec) String() string { if c == nil { return "" } - return c.name + return c.Name } -func (c *codec) Name() string { return c.name } -func (c *codec) RequestType() MediaType { return c.mediaTypes[0] } -func (c *codec) ResponseType() MediaType { return c.mediaTypes[1] } -func (c *codec) NewEncoder(w io.Writer) Encoder { return c.newEncoder(w) } -func (c *codec) NewDecoder(r framing.Reader) Decoder { return c.newDecoder(r) } - type ( // Marshaler composes the supported marshaling formats. Marshaler interface { @@ -118,31 +147,44 @@ func (f EncoderFunc) Encode(m Marshaler) error { return f(m) } // NewProtobufEncoder returns a new Encoder of Calls to Protobuf messages written to // the given io.Writer. -func NewProtobufEncoder(w io.Writer) EncoderFunc { - enc := proto.NewEncoder(w) - return func(m Marshaler) error { return enc.Encode(m) } +func NewProtobufEncoder(s Sink) Encoder { + w := s() + return EncoderFunc(func(m Marshaler) error { + b, err := pb.Marshal(m.(pb.Message)) + if err != nil { + return err + } + return w.WriteFrame(b) + }) } // NewJSONEncoder returns a new Encoder of Calls to JSON messages written to // the given io.Writer. -func NewJSONEncoder(w io.Writer) EncoderFunc { - enc := json.NewEncoder(w) - return func(m Marshaler) error { return enc.Encode(m) } +func NewJSONEncoder(s Sink) Encoder { + w := s() + return EncoderFunc(func(m Marshaler) error { + b, err := json.Marshal(m) + if err != nil { + return err + } + return w.WriteFrame(b) + }) } -// NewProtobufDecoder returns a new Decoder of Protobuf messages read from the -// given framing.Reader to Events. -func NewProtobufDecoder(r framing.Reader) DecoderFunc { +// NewProtobufDecoder returns a new Decoder of Protobuf messages read from the given Source. +func NewProtobufDecoder(s Source) Decoder { + r := s() var ( uf = func(b []byte, m interface{}) error { return pb.Unmarshal(b, m.(pb.Message)) } dec = framing.NewDecoder(r, uf) ) - return func(u Unmarshaler) error { return dec.Decode(u) } + return DecoderFunc(func(u Unmarshaler) error { return dec.Decode(u) }) + } -// NewJSONDecoder returns a new Decoder of JSON messages read from the -// given framing.Reader to Events. -func NewJSONDecoder(r framing.Reader) DecoderFunc { +// NewJSONDecoder returns a new Decoder of JSON messages read from the given source. +func NewJSONDecoder(s Source) Decoder { + r := s() dec := framing.NewDecoder(r, json.Unmarshal) - return func(u Unmarshaler) error { return dec.Decode(u) } + return DecoderFunc(func(u Unmarshaler) error { return dec.Decode(u) }) } diff --git a/api/v1/lib/encoding/framing/decoder.go b/api/v1/lib/encoding/framing/decoder.go index cabb8832..2c873e17 100644 --- a/api/v1/lib/encoding/framing/decoder.go +++ b/api/v1/lib/encoding/framing/decoder.go @@ -23,7 +23,7 @@ func (f DecoderFunc) Decode(m interface{}) error { return f(m) } var _ = Decoder(DecoderFunc(nil)) -// NewDecoder returns a new Decoder that reads from the given io.Reader. +// NewDecoder returns a new Decoder that reads from the given frame Reader. func NewDecoder(r Reader, uf UnmarshalFunc) DecoderFunc { return func(m interface{}) error { // Note: the buf returned by ReadFrame will change over time, it can't be sub-sliced diff --git a/api/v1/lib/encoding/framing/framing.go b/api/v1/lib/encoding/framing/framing.go index 054cb24b..c034221b 100644 --- a/api/v1/lib/encoding/framing/framing.go +++ b/api/v1/lib/encoding/framing/framing.go @@ -16,10 +16,20 @@ type ( ReadFrame() (frame []byte, err error) } - // ReaderFunc is the functional adaptation of Reader + // ReaderFunc is the functional adaptation of Reader. ReaderFunc func() ([]byte, error) + + // Writer sends whole frames to some endpoint; returns io.ErrShortWrite if the frame is only partially written. + Writer interface { + WriteFrame(frame []byte) error + } + + // WriterFunc is the functional adaptation of Writer. + WriterFunc func([]byte) error ) func (f ReaderFunc) ReadFrame() ([]byte, error) { return f() } +func (f WriterFunc) WriteFrame(b []byte) error { return f(b) } var _ = Reader(ReaderFunc(nil)) +var _ = Writer(WriterFunc(nil)) diff --git a/api/v1/lib/extras/gen/httpsender.go b/api/v1/lib/extras/gen/httpsender.go new file mode 100644 index 00000000..4ec1fa8c --- /dev/null +++ b/api/v1/lib/extras/gen/httpsender.go @@ -0,0 +1,60 @@ +// +build ignore + +package main + +import ( + "os" + "text/template" +) + +func main() { + Run(handlersTemplate, nil, os.Args...) +} + +var handlersTemplate = template.Must(template.New("").Parse(`package {{.Package}} + +// go generate {{.Args}} +// GENERATED CODE FOLLOWS; DO NOT EDIT. + +import ( + "context" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/client" + "github.com/mesos/mesos-go/api/v1/lib/httpcli" +{{range .Imports}}{{/* a "calls" package is assumed to be imported */}} + {{ printf "%q" . -}} +{{end}} +) + +{{.RequireType "C" -}}{{/* C is assumed to be a pointer type */ -}} +type ResponseClassifier func({{.Type "C"}}) (client.ResponseClass, error) + +{{/* classifyResponse must be declared in the package wherein this generated module lives */ -}} +var DefaultResponseClassifier = ResponseClassifier(classifyResponse) + +// NewSender generates a sender that uses the Mesos v1 HTTP API for encoding/decoding requests/responses. +// The ResponseClass is inferred from the first object generated by the given Request. +func NewSender(c *httpcli.Client) calls.Sender { + return calls.SenderFunc(func(ctx context.Context, r calls.Request) (mesos.Response, error) { + var ( + obj = r.Call() + rc, err = DefaultResponseClassifier(obj) + ) + if err != nil { + return nil, err + } + + var req client.Request + + switch r := r.(type) { + case calls.RequestStreaming: + req = calls.Push(r, obj) + default: + req = calls.NonStreaming(obj) + } + + return c.Send(req, rc, httpcli.Context(ctx)) + }) +} +`)) diff --git a/api/v1/lib/extras/gen/sender.go b/api/v1/lib/extras/gen/sender.go new file mode 100644 index 00000000..1183d205 --- /dev/null +++ b/api/v1/lib/extras/gen/sender.go @@ -0,0 +1,247 @@ +// +build ignore + +package main + +import ( + "os" + "text/template" +) + +func main() { + Run(srcTemplate, testTemplate, os.Args...) +} + +var srcTemplate = template.Must(template.New("").Parse(`package {{.Package}} + +// go generate {{.Args}} +// GENERATED CODE FOLLOWS; DO NOT EDIT. + +import ( + "context" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/encoding" +{{range .Imports}} + {{ printf "%q" . -}} +{{end}} +) + +{{.RequireType "C" -}}{{/* C is assumed to be a struct or primitive type */ -}} +type ( + // Request generates a Call that's sent to a Mesos agent. Subsequent invocations are expected to + // yield equivalent calls. Intended for use w/ non-streaming requests to an agent. + Request interface { + Call() *{{.Type "C"}} + } + + // RequestFunc is the functional adaptation of Request. + RequestFunc func() *{{.Type "C"}} + + // RequestStreaming generates a Call that's send to a Mesos agent. Subsequent invocations MAY generate + // different Call objects. No more Call objects are expected once a nil is returned to signal the end of + // of the request stream. + RequestStreaming interface { + Request + IsStreaming() + } + + // RequestStreamingFunc is the functional adaptation of RequestStreaming. + RequestStreamingFunc func() *{{.Type "C"}} + + // Send issues a Request to a Mesos agent and properly manages Call-specific mechanics. + Sender interface { + Send(context.Context, Request) (mesos.Response, error) + } + + // SenderFunc is the functional adaptation of the Sender interface + SenderFunc func(context.Context, Request) (mesos.Response, error) +) + +func (f RequestFunc) Call() *{{.Type "C"}} { return f() } + +func (f RequestFunc) Marshaler() encoding.Marshaler { + // avoid returning (*{{.Type "C"}})(nil) for interface type + if call := f(); call != nil { + return call + } + return nil +} + +func (f RequestStreamingFunc) Push(c ...*{{.Type "C"}}) RequestStreamingFunc { return Push(f, c...) } + +func (f RequestStreamingFunc) Marshaler() encoding.Marshaler { + // avoid returning (*{{.Type "C"}})(nil) for interface type + if call := f(); call != nil { + return call + } + return nil +} + +func (f RequestStreamingFunc) IsStreaming() {} + +func (f RequestStreamingFunc) Call() *{{.Type "C"}} { return f() } + +// Push prepends one or more calls onto a request stream. If no calls are given then the original stream is returned. +func Push(r RequestStreaming, c ...*{{.Type "C"}}) RequestStreamingFunc { + return func() *{{.Type "C"}} { + if len(c) == 0 { + return r.Call() + } + head := c[0] + c = c[1:] + return head + } +} + +// Empty generates a stream that always returns nil. +func Empty() RequestStreamingFunc { return func() *{{.Type "C"}} { return nil } } + +var ( + _ = Request(RequestFunc(nil)) + _ = RequestStreaming(RequestStreamingFunc(nil)) + _ = Sender(SenderFunc(nil)) +) + +// NonStreaming returns a RequestFunc that always generates the same Call. +func NonStreaming(c *{{.Type "C"}}) RequestFunc { return func() *{{.Type "C"}} { return c } } + +// FromChan returns a streaming request that fetches calls from the given channel until it closes. +// If a nil chan is specified then the returned func will always generate nil. +func FromChan(ch <-chan *{{.Type "C"}}) RequestStreamingFunc { + if ch == nil { + // avoid blocking forever if we're handed a nil chan + return func() *{{.Type "C"}} { return nil } + } + return func() *{{.Type "C"}} { + if m, ok := <-ch; ok { + return m + } + return nil + } +} + +// Send implements the Sender interface for SenderFunc +func (f SenderFunc) Send(ctx context.Context, r Request) (mesos.Response, error) { + return f(ctx, r) +} + +// IgnoreResponse generates a sender that closes any non-nil response received by Mesos. +func IgnoreResponse(s Sender) SenderFunc { + return func(ctx context.Context, r Request) (mesos.Response, error) { + resp, err := s.Send(ctx, r) + if resp != nil { + resp.Close() + } + return nil, err + } +} + +// SendNoData is a convenience func that executes the given Call using the provided Sender +// and always drops the response data. +func SendNoData(ctx context.Context, sender Sender, r Request) (err error) { + _, err = IgnoreResponse(sender).Send(ctx, r) + return +} +`)) + +var testTemplate = template.Must(template.New("").Parse(`package {{.Package}} + +// go generate {{.Args}} +// GENERATED CODE FOLLOWS; DO NOT EDIT. + +import ( + "context" + "testing" + + "github.com/mesos/mesos-go/api/v1/lib" +{{range .Imports}} + {{ printf "%q" . -}} +{{end}} +) + +func TestNonStreaming(t *testing.T) { + c := new({{.Type "C"}}) + f := NonStreaming(c) + if x := f.Call(); x != c { + t.Fatalf("expected %#v instead of %#v", c, x) + } + if x := f.Marshaler(); x == nil { + t.Fatal("expected non-nil Marshaler") + } + f = NonStreaming(nil) + if x := f.Marshaler(); x != nil { + t.Fatalf("expected nil Marshaler instead of %#v", x) + } +} + +func TestStreaming(t *testing.T) { + f := Empty() + + f.IsStreaming() + + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + if x := f.Marshaler(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + + c := new({{.Type "C"}}) + + f = f.Push(c) + if x := f.Marshaler(); x == nil { + t.Fatal("expected non-nil Marshaler") + } + if x := f.Marshaler(); x != nil { + t.Fatalf("expected nil Marshaler instead of %#v", x) + } + + c2 := new({{.Type "C"}}) + + f = Empty().Push(c, c2) + if x := f.Call(); x != c { + t.Fatalf("expected %#v instead of %#v", c, x) + } + if x := f.Call(); x != c2 { + t.Fatalf("expected %#v instead of %#v", c2, x) + } + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + + ch := make(chan *{{.Type "C"}}, 2) + ch <- c + ch <- c2 + close(ch) + f = FromChan(ch) + if x := f.Call(); x != c { + t.Fatalf("expected %#v instead of %#v", c, x) + } + if x := f.Call(); x != c2 { + t.Fatalf("expected %#v instead of %#v", c2, x) + } + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + + f = FromChan(nil) + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } +} + +func TestIgnoreResponse(t *testing.T) { + var closed bool + + IgnoreResponse(SenderFunc(func(_ context.Context, _ Request) (mesos.Response, error) { + return &mesos.ResponseWrapper{Closer: mesos.CloseFunc(func() error { + closed = true + return nil + })}, nil + })).Send(nil, nil) + + if !closed { + t.Fatal("expected response to be closed") + } +} +`)) diff --git a/api/v1/lib/httpcli/http.go b/api/v1/lib/httpcli/http.go index d4d8cbcb..41b9d8b3 100644 --- a/api/v1/lib/httpcli/http.go +++ b/api/v1/lib/httpcli/http.go @@ -6,13 +6,16 @@ import ( "crypto/tls" "fmt" "io" - "log" "net" "net/http" + "sync" "time" "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/client" + logger "github.com/mesos/mesos-go/api/v1/lib/debug" "github.com/mesos/mesos-go/api/v1/lib/encoding" + "github.com/mesos/mesos-go/api/v1/lib/encoding/framing" "github.com/mesos/mesos-go/api/v1/lib/httpcli/apierrors" "github.com/mesos/mesos-go/api/v1/lib/recordio" ) @@ -27,7 +30,8 @@ type ProtocolError string func (pe ProtocolError) Error() string { return string(pe) } const ( - debug = false // TODO(jdef) kill me at some point + debug = logger.Logger(false) + mediaTypeRecordIO = encoding.MediaType("application/recordio") ) // DoFunc sends an HTTP request and returns an HTTP response. @@ -58,19 +62,20 @@ type Response struct { // ErrorMapperFunc generates an error for the given response. type ErrorMapperFunc func(*http.Response) error -// ResponseHandler is invoked to process an HTTP response -type ResponseHandler func(*http.Response, error) (mesos.Response, error) +// ResponseHandler is invoked to process an HTTP response. Callers SHALL invoke Close for +// a non-nil Response, even when errors are returned. +type ResponseHandler func(*http.Response, client.ResponseClass, error) (mesos.Response, error) // A Client is a Mesos HTTP APIs client. type Client struct { - url string - do DoFunc - header http.Header - codec encoding.Codec - errorMapper ErrorMapperFunc - requestOpts []RequestOpt - buildRequest func(encoding.Marshaler, ...RequestOpt) (*http.Request, error) - handleResponse ResponseHandler + url string + do DoFunc + header http.Header + codec encoding.Codec + errorMapper ErrorMapperFunc + requestOpts []RequestOpt + buildRequestFunc func(client.Request, client.ResponseClass, ...RequestOpt) (*http.Request, error) + handleResponse ResponseHandler } var ( @@ -99,7 +104,7 @@ func New(opts ...Opt) *Client { header: DefaultHeaders, errorMapper: DefaultErrorMapper, } - c.buildRequest = c.BuildRequest + c.buildRequestFunc = c.buildRequest c.handleResponse = c.HandleResponse c.With(opts...) return c @@ -142,18 +147,45 @@ func (c *Client) WithTemporary(opt Opt, f func() error) error { return f() } -// Mesos returns a mesos.Client variant backed by this implementation +// Mesos returns a mesos.Client variant backed by this implementation. +// Deprecated. func (c *Client) Mesos(opts ...RequestOpt) mesos.Client { return mesos.ClientFunc(func(m encoding.Marshaler) (mesos.Response, error) { return c.Do(m, opts...) }) } -// BuildRequest is a factory func that generates and returns an http.Request for the +func prepareForResponse(rc client.ResponseClass, codec encoding.Codec) (RequestOpts, error) { + // We need to tell Mesos both the content-type and message-content-type that we're expecting, otherwise + // the server may give us validation problems, or else send back a vague content-type (w/o a + // message-content-type). In order to communicate these things we need to understand the desired response + // type from the perspective of the caller --> client.ResponseClass. + var accept RequestOpts + switch rc { + case client.ResponseClassSingleton, client.ResponseClassAuto: + accept = append(accept, Header("Accept", codec.Type.ContentType())) + case client.ResponseClassStreaming: + accept = append(accept, Header("Accept", mediaTypeRecordIO.ContentType())) + accept = append(accept, Header("Message-Accept", codec.Type.ContentType())) + default: + return nil, ProtocolError(fmt.Sprintf("illegal response class requested: %v", rc)) + } + return accept, nil +} + +// buildRequest is a factory func that generates and returns an http.Request for the // given marshaler and request options. -func (c *Client) BuildRequest(m encoding.Marshaler, opt ...RequestOpt) (*http.Request, error) { +func (c *Client) buildRequest(cr client.Request, rc client.ResponseClass, opt ...RequestOpt) (*http.Request, error) { + if crs, ok := cr.(client.RequestStreaming); ok { + return c.buildRequestStream(crs.Marshaler, rc, opt...) + } + accept, err := prepareForResponse(rc, c.codec) + if err != nil { + return nil, err + } + var body bytes.Buffer //TODO(jdef): use a pool to allocate these (and reduce garbage)? - if err := c.codec.NewEncoder(&body).Encode(m); err != nil { + if err := c.codec.NewEncoder(encoding.SinkWriter(&body)).Encode(cr.Marshaler()); err != nil { return nil, err } @@ -166,14 +198,105 @@ func (c *Client) BuildRequest(m encoding.Marshaler, opt ...RequestOpt) (*http.Re return helper. withOptions(c.requestOpts, opt). withHeaders(c.header). - withHeader("Content-Type", c.codec.RequestType().ContentType()). - withHeader("Accept", c.codec.ResponseType().ContentType()). + withHeader("Content-Type", c.codec.Type.ContentType()). + withHeader("Accept", c.codec.Type.ContentType()). + withOptions(accept). Request, nil } +func (c *Client) buildRequestStream(f func() encoding.Marshaler, rc client.ResponseClass, opt ...RequestOpt) (*http.Request, error) { + accept, err := prepareForResponse(rc, c.codec) + if err != nil { + return nil, err + } + + var ( + pr, pw = io.Pipe() + enc = c.codec.NewEncoder(func() framing.Writer { return recordio.NewWriter(pw) }) + ) + req, err := http.NewRequest("POST", c.url, pr) + if err != nil { + pw.Close() // ignore error + return nil, err + } + + go func() { + var closeOnce sync.Once + defer closeOnce.Do(func() { + pw.Close() + }) + for { + m := f() + if m == nil { + // no more messages to send; end of the stream + break + } + err := enc.Encode(m) + if err != nil { + closeOnce.Do(func() { + pw.CloseWithError(err) + }) + break + } + } + }() + + helper := HTTPRequestHelper{req} + return helper. + withOptions(c.requestOpts, opt). + withHeaders(c.header). + withHeader("Content-Type", mediaTypeRecordIO.ContentType()). + withHeader("Message-Content-Type", c.codec.Type.ContentType()). + withOptions(accept). + Request, nil +} + +func validateSuccessfulResponse(codec encoding.Codec, res *http.Response, rc client.ResponseClass) error { + switch res.StatusCode { + case http.StatusOK: + ct := res.Header.Get("Content-Type") + switch rc { + case client.ResponseClassSingleton, client.ResponseClassAuto: + if ct != codec.Type.ContentType() { + return ProtocolError(fmt.Sprintf("unexpected content type: %q", ct)) + } + case client.ResponseClassStreaming: + if ct != mediaTypeRecordIO.ContentType() { + return ProtocolError(fmt.Sprintf("unexpected content type: %q", ct)) + } + ct = res.Header.Get("Message-Content-Type") + if ct != codec.Type.ContentType() { + return ProtocolError(fmt.Sprintf("unexpected message content type: %q", ct)) + } + default: + return ProtocolError(fmt.Sprintf("unsupported response-class: %q", rc)) + } + + case http.StatusAccepted: + // nothing to validate, we're not expecting any response entity in this case. + // TODO(jdef) perhaps check Content-Length == 0 here? + } + return nil +} + +func responseToSource(res *http.Response, rc client.ResponseClass) encoding.SourceFactoryFunc { + switch rc { + case client.ResponseClassSingleton: + return encoding.SourceReader + case client.ResponseClassStreaming, client.ResponseClassAuto: + return func(r io.Reader) encoding.Source { + return func() framing.Reader { + return recordio.NewReader(r) + } + } + default: + panic(fmt.Sprintf("unsupported response-class: %q", rc)) + } +} + // HandleResponse parses an HTTP response from a Mesos service endpoint, transforming the // raw HTTP response into a mesos.Response. -func (c *Client) HandleResponse(res *http.Response, err error) (mesos.Response, error) { +func (c *Client) HandleResponse(res *http.Response, rc client.ResponseClass, err error) (mesos.Response, error) { if err != nil { if res != nil && res.Body != nil { res.Body.Close() @@ -189,41 +312,53 @@ func (c *Client) HandleResponse(res *http.Response, err error) (mesos.Response, return result, err } + err = validateSuccessfulResponse(c.codec, res, rc) + if err != nil { + res.Body.Close() + return nil, err + } + switch res.StatusCode { case http.StatusOK: - if debug { - log.Println("request OK, decoding response") - } - ct := res.Header.Get("Content-Type") - if ct != c.codec.ResponseType().ContentType() { - res.Body.Close() - return nil, ProtocolError(fmt.Sprintf("unexpected content type: %q", ct)) - } - result.Decoder = c.codec.NewDecoder(recordio.NewReader(res.Body)) + debug.Log("request OK, decoding response") + sf := responseToSource(res, rc) + result.Decoder = c.codec.NewDecoder(sf.NewSource(res.Body)) case http.StatusAccepted: - if debug { - log.Println("request Accepted") - } + debug.Log("request Accepted") // noop; no decoder for these types of calls default: + // don't close the response here because the caller may want to evaluate the entity. + // it's the caller's job to Close the returned response. return result, ProtocolError(fmt.Sprintf("unexpected mesos HTTP response code: %d", res.StatusCode)) } return result, nil } -// Do sends a Call and returns (a) a Response (should be closed when finished) that -// contains a streaming Decoder from which callers can read Events from, and; (b) an -// error in case of failure. Callers are expected to *always* close a non-nil Response -// if one is returned. For operations which are successful but also for which there is -// no expected object stream as a result the embedded Decoder will be nil. +// Do is deprecated in favor of Send. func (c *Client) Do(m encoding.Marshaler, opt ...RequestOpt) (res mesos.Response, err error) { - var req *http.Request - req, err = c.buildRequest(m, opt...) + return c.Send(client.RequestSingleton(m), client.ResponseClassAuto, opt...) +} + +// Send sends a Call and returns (a) a Response (should be closed when finished) that +// contains a either a streaming or non-streaming Decoder from which callers can read +// objects from, and; (b) an error in case of failure. Callers are expected to *always* +// close a non-nil Response if one is returned. For operations which are successful but +// also for which there are no expected result objects the embedded Decoder will be nil. +// The provided ResponseClass determines whether the client implementation will attempt +// to decode a result as a single obeject or as an object stream. When working with +// versions of Mesos prior to v1.2.x callers MUST use ResponseClassAuto. +func (c *Client) Send(cr client.Request, rc client.ResponseClass, opt ...RequestOpt) (res mesos.Response, err error) { + var ( + hreq *http.Request + hres *http.Response + ) + hreq, err = c.buildRequestFunc(cr, rc, opt...) if err == nil { - res, err = c.handleResponse(c.do(req)) + hres, err = c.do(hreq) + res, err = c.handleResponse(hres, rc, err) } return } @@ -425,9 +560,7 @@ func (r *HTTPRequestHelper) withOptions(optsets ...RequestOpts) *HTTPRequestHelp func (r *HTTPRequestHelper) withHeaders(hh http.Header) *HTTPRequestHelper { for k, v := range hh { r.Header[k] = v - if debug { - log.Println("request header " + k + ": " + v[0]) - } + debug.Log("request header " + k + ": " + v[0]) } return r } diff --git a/api/v1/lib/httpcli/httpagent/gen.go b/api/v1/lib/httpcli/httpagent/gen.go new file mode 100644 index 00000000..18f6d5ff --- /dev/null +++ b/api/v1/lib/httpcli/httpagent/gen.go @@ -0,0 +1,3 @@ +package httpagent + +//go:generate go run ../../extras/gen/httpsender.go ../../extras/gen/gen.go -import github.com/mesos/mesos-go/api/v1/lib/agent -import github.com/mesos/mesos-go/api/v1/lib/agent/calls -type C:*agent.Call diff --git a/api/v1/lib/httpcli/httpagent/httpagent_generated.go b/api/v1/lib/httpcli/httpagent/httpagent_generated.go new file mode 100644 index 00000000..e6d2c546 --- /dev/null +++ b/api/v1/lib/httpcli/httpagent/httpagent_generated.go @@ -0,0 +1,44 @@ +package httpagent + +// go generate -import github.com/mesos/mesos-go/api/v1/lib/agent -import github.com/mesos/mesos-go/api/v1/lib/agent/calls -type C:*agent.Call +// GENERATED CODE FOLLOWS; DO NOT EDIT. + +import ( + "context" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/client" + "github.com/mesos/mesos-go/api/v1/lib/httpcli" + + "github.com/mesos/mesos-go/api/v1/lib/agent" + "github.com/mesos/mesos-go/api/v1/lib/agent/calls" +) + +type ResponseClassifier func(*agent.Call) (client.ResponseClass, error) + +var DefaultResponseClassifier = ResponseClassifier(classifyResponse) + +// NewSender generates a sender that uses the Mesos v1 HTTP API for encoding/decoding requests/responses. +// The ResponseClass is inferred from the first object generated by the given Request. +func NewSender(c *httpcli.Client) calls.Sender { + return calls.SenderFunc(func(ctx context.Context, r calls.Request) (mesos.Response, error) { + var ( + obj = r.Call() + rc, err = DefaultResponseClassifier(obj) + ) + if err != nil { + return nil, err + } + + var req client.Request + + switch r := r.(type) { + case calls.RequestStreaming: + req = calls.Push(r, obj) + default: + req = calls.NonStreaming(obj) + } + + return c.Send(req, rc, httpcli.Context(ctx)) + }) +} diff --git a/api/v1/lib/httpcli/httpagent/util.go b/api/v1/lib/httpcli/httpagent/util.go new file mode 100644 index 00000000..4f8bcad0 --- /dev/null +++ b/api/v1/lib/httpcli/httpagent/util.go @@ -0,0 +1,50 @@ +package httpagent + +import ( + "fmt" + + "github.com/mesos/mesos-go/api/v1/lib/agent" + "github.com/mesos/mesos-go/api/v1/lib/client" + "github.com/mesos/mesos-go/api/v1/lib/httpcli" +) + +func classifyResponse(c *agent.Call) (rc client.ResponseClass, err error) { + if c == nil { + err = httpcli.ProtocolError("nil agent.Call not allowed") + return + } + + switch t := c.GetType(); t { + // singleton + case agent.Call_GET_HEALTH, + agent.Call_GET_FLAGS, + agent.Call_GET_VERSION, + agent.Call_GET_METRICS, + agent.Call_GET_LOGGING_LEVEL, + agent.Call_LIST_FILES, + agent.Call_READ_FILE, + agent.Call_GET_STATE, + agent.Call_GET_CONTAINERS, + agent.Call_GET_FRAMEWORKS, + agent.Call_GET_EXECUTORS, + agent.Call_GET_TASKS, + agent.Call_WAIT_NESTED_CONTAINER: + rc = client.ResponseClassSingleton + + // streaming + case agent.Call_LAUNCH_NESTED_CONTAINER_SESSION, + agent.Call_ATTACH_CONTAINER_OUTPUT: + rc = client.ResponseClassStreaming + + // no-data + case agent.Call_SET_LOGGING_LEVEL, + agent.Call_LAUNCH_NESTED_CONTAINER, + agent.Call_KILL_NESTED_CONTAINER, + agent.Call_ATTACH_CONTAINER_INPUT: + rc = client.ResponseClassAuto + + default: + err = httpcli.ProtocolError(fmt.Sprintf("unsupported agent.Call type: %v", t)) + } + return +} diff --git a/api/v1/lib/httpcli/httpmaster/gen.go b/api/v1/lib/httpcli/httpmaster/gen.go new file mode 100644 index 00000000..c5f34fae --- /dev/null +++ b/api/v1/lib/httpcli/httpmaster/gen.go @@ -0,0 +1,3 @@ +package httpmaster + +//go:generate go run ../../extras/gen/httpsender.go ../../extras/gen/gen.go -import github.com/mesos/mesos-go/api/v1/lib/master -import github.com/mesos/mesos-go/api/v1/lib/master/calls -type C:*master.Call diff --git a/api/v1/lib/httpcli/httpmaster/httpmaster_generated.go b/api/v1/lib/httpcli/httpmaster/httpmaster_generated.go new file mode 100644 index 00000000..56e7d411 --- /dev/null +++ b/api/v1/lib/httpcli/httpmaster/httpmaster_generated.go @@ -0,0 +1,44 @@ +package httpmaster + +// go generate -import github.com/mesos/mesos-go/api/v1/lib/master -import github.com/mesos/mesos-go/api/v1/lib/master/calls -type C:*master.Call +// GENERATED CODE FOLLOWS; DO NOT EDIT. + +import ( + "context" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/client" + "github.com/mesos/mesos-go/api/v1/lib/httpcli" + + "github.com/mesos/mesos-go/api/v1/lib/master" + "github.com/mesos/mesos-go/api/v1/lib/master/calls" +) + +type ResponseClassifier func(*master.Call) (client.ResponseClass, error) + +var DefaultResponseClassifier = ResponseClassifier(classifyResponse) + +// NewSender generates a sender that uses the Mesos v1 HTTP API for encoding/decoding requests/responses. +// The ResponseClass is inferred from the first object generated by the given Request. +func NewSender(c *httpcli.Client) calls.Sender { + return calls.SenderFunc(func(ctx context.Context, r calls.Request) (mesos.Response, error) { + var ( + obj = r.Call() + rc, err = DefaultResponseClassifier(obj) + ) + if err != nil { + return nil, err + } + + var req client.Request + + switch r := r.(type) { + case calls.RequestStreaming: + req = calls.Push(r, obj) + default: + req = calls.NonStreaming(obj) + } + + return c.Send(req, rc, httpcli.Context(ctx)) + }) +} diff --git a/api/v1/lib/httpcli/httpmaster/util.go b/api/v1/lib/httpcli/httpmaster/util.go new file mode 100644 index 00000000..04101b90 --- /dev/null +++ b/api/v1/lib/httpcli/httpmaster/util.go @@ -0,0 +1,61 @@ +package httpmaster + +import ( + "fmt" + + "github.com/mesos/mesos-go/api/v1/lib/client" + "github.com/mesos/mesos-go/api/v1/lib/httpcli" + "github.com/mesos/mesos-go/api/v1/lib/master" +) + +func classifyResponse(c *master.Call) (rc client.ResponseClass, err error) { + if c == nil { + err = httpcli.ProtocolError("nil master.Call not allowed") + return + } + + switch t := c.GetType(); t { + // singleton + case master.Call_GET_HEALTH, + master.Call_GET_FLAGS, + master.Call_GET_VERSION, + master.Call_GET_METRICS, + master.Call_GET_LOGGING_LEVEL, + master.Call_LIST_FILES, + master.Call_READ_FILE, + master.Call_GET_STATE, + master.Call_GET_AGENTS, + master.Call_GET_FRAMEWORKS, + master.Call_GET_EXECUTORS, + master.Call_GET_TASKS, + master.Call_GET_ROLES, + master.Call_GET_WEIGHTS, + master.Call_GET_MASTER, + master.Call_GET_MAINTENANCE_STATUS, + master.Call_GET_MAINTENANCE_SCHEDULE, + master.Call_GET_QUOTA: + rc = client.ResponseClassSingleton + + // streaming + case master.Call_SUBSCRIBE: + rc = client.ResponseClassStreaming + + // no-data + case master.Call_SET_LOGGING_LEVEL, + master.Call_UPDATE_WEIGHTS, + master.Call_RESERVE_RESOURCES, + master.Call_UNRESERVE_RESOURCES, + master.Call_CREATE_VOLUMES, + master.Call_DESTROY_VOLUMES, + master.Call_UPDATE_MAINTENANCE_SCHEDULE, + master.Call_START_MAINTENANCE, + master.Call_STOP_MAINTENANCE, + master.Call_SET_QUOTA, + master.Call_REMOVE_QUOTA: + rc = client.ResponseClassAuto + + default: + err = httpcli.ProtocolError(fmt.Sprintf("unsupported master.Call type: %v", t)) + } + return +} diff --git a/api/v1/lib/httpcli/httpsched/httpsched.go b/api/v1/lib/httpcli/httpsched/httpsched.go index 541ca52d..5785823c 100644 --- a/api/v1/lib/httpcli/httpsched/httpsched.go +++ b/api/v1/lib/httpcli/httpsched/httpsched.go @@ -9,6 +9,7 @@ import ( "github.com/mesos/mesos-go/api/v1/lib" "github.com/mesos/mesos-go/api/v1/lib/backoff" + mesosclient "github.com/mesos/mesos-go/api/v1/lib/client" "github.com/mesos/mesos-go/api/v1/lib/encoding" "github.com/mesos/mesos-go/api/v1/lib/httpcli" "github.com/mesos/mesos-go/api/v1/lib/httpcli/apierrors" @@ -168,8 +169,8 @@ func (mre *mesosRedirectionError) Error() string { // header and computing the address of the next endpoint that should be used to replay the failed // HTTP request. func (cli *client) redirectHandler() httpcli.Opt { - return httpcli.HandleResponse(func(hres *http.Response, err error) (mesos.Response, error) { - resp, err := cli.HandleResponse(hres, err) // default response handler + return httpcli.HandleResponse(func(hres *http.Response, rc mesosclient.ResponseClass, err error) (mesos.Response, error) { + resp, err := cli.HandleResponse(hres, rc, err) // default response handler if err == nil || !apierrors.CodeNotLeader.Matches(err) { return resp, err } diff --git a/api/v1/lib/maintenance/maintenance.pb.go b/api/v1/lib/maintenance/maintenance.pb.go new file mode 100644 index 00000000..ef99f038 --- /dev/null +++ b/api/v1/lib/maintenance/maintenance.pb.go @@ -0,0 +1,1519 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: maintenance/maintenance.proto + +/* + Package maintenance is a generated protocol buffer package. + + It is generated from these files: + maintenance/maintenance.proto + + It has these top-level messages: + Window + Schedule + ClusterStatus +*/ +package maintenance + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" +import mesos "github.com/mesos/mesos-go/api/v1/lib" +import mesos_allocator "github.com/mesos/mesos-go/api/v1/lib/allocator" +import _ "github.com/gogo/protobuf/gogoproto" + +import strings "strings" +import reflect "reflect" + +import io "io" +import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +// * +// A set of machines scheduled to go into maintenance +// in the same `unavailability`. +type Window struct { + // Machines affected by this maintenance window. + MachineIDs []mesos.MachineID `protobuf:"bytes,1,rep,name=machine_ids,json=machineIds" json:"machine_ids"` + // Interval during which this set of machines is expected to be down. + Unavailability mesos.Unavailability `protobuf:"bytes,2,req,name=unavailability" json:"unavailability"` +} + +func (m *Window) Reset() { *m = Window{} } +func (*Window) ProtoMessage() {} +func (*Window) Descriptor() ([]byte, []int) { return fileDescriptorMaintenance, []int{0} } + +func (m *Window) GetMachineIDs() []mesos.MachineID { + if m != nil { + return m.MachineIDs + } + return nil +} + +func (m *Window) GetUnavailability() mesos.Unavailability { + if m != nil { + return m.Unavailability + } + return mesos.Unavailability{} +} + +// * +// A list of maintenance windows. +// For example, this may represent a rolling restart of agents. +type Schedule struct { + Windows []Window `protobuf:"bytes,1,rep,name=windows" json:"windows"` +} + +func (m *Schedule) Reset() { *m = Schedule{} } +func (*Schedule) ProtoMessage() {} +func (*Schedule) Descriptor() ([]byte, []int) { return fileDescriptorMaintenance, []int{1} } + +func (m *Schedule) GetWindows() []Window { + if m != nil { + return m.Windows + } + return nil +} + +// * +// Represents the maintenance status of each machine in the cluster. +// The lists correspond to the `MachineInfo.Mode` enumeration. +type ClusterStatus struct { + DrainingMachines []ClusterStatus_DrainingMachine `protobuf:"bytes,1,rep,name=draining_machines,json=drainingMachines" json:"draining_machines"` + DownMachines []mesos.MachineID `protobuf:"bytes,2,rep,name=down_machines,json=downMachines" json:"down_machines"` +} + +func (m *ClusterStatus) Reset() { *m = ClusterStatus{} } +func (*ClusterStatus) ProtoMessage() {} +func (*ClusterStatus) Descriptor() ([]byte, []int) { return fileDescriptorMaintenance, []int{2} } + +func (m *ClusterStatus) GetDrainingMachines() []ClusterStatus_DrainingMachine { + if m != nil { + return m.DrainingMachines + } + return nil +} + +func (m *ClusterStatus) GetDownMachines() []mesos.MachineID { + if m != nil { + return m.DownMachines + } + return nil +} + +type ClusterStatus_DrainingMachine struct { + ID mesos.MachineID `protobuf:"bytes,1,req,name=id" json:"id"` + // A list of the most recent responses to inverse offers from frameworks + // running on this draining machine. + Statuses []mesos_allocator.InverseOfferStatus `protobuf:"bytes,2,rep,name=statuses" json:"statuses"` +} + +func (m *ClusterStatus_DrainingMachine) Reset() { *m = ClusterStatus_DrainingMachine{} } +func (*ClusterStatus_DrainingMachine) ProtoMessage() {} +func (*ClusterStatus_DrainingMachine) Descriptor() ([]byte, []int) { + return fileDescriptorMaintenance, []int{2, 0} +} + +func (m *ClusterStatus_DrainingMachine) GetID() mesos.MachineID { + if m != nil { + return m.ID + } + return mesos.MachineID{} +} + +func (m *ClusterStatus_DrainingMachine) GetStatuses() []mesos_allocator.InverseOfferStatus { + if m != nil { + return m.Statuses + } + return nil +} + +func init() { + proto.RegisterType((*Window)(nil), "mesos.maintenance.Window") + proto.RegisterType((*Schedule)(nil), "mesos.maintenance.Schedule") + proto.RegisterType((*ClusterStatus)(nil), "mesos.maintenance.ClusterStatus") + proto.RegisterType((*ClusterStatus_DrainingMachine)(nil), "mesos.maintenance.ClusterStatus.DrainingMachine") +} +func (this *Window) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Window) + if !ok { + that2, ok := that.(Window) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Window") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Window but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Window but is not nil && this == nil") + } + if len(this.MachineIDs) != len(that1.MachineIDs) { + return fmt.Errorf("MachineIDs this(%v) Not Equal that(%v)", len(this.MachineIDs), len(that1.MachineIDs)) + } + for i := range this.MachineIDs { + if !this.MachineIDs[i].Equal(&that1.MachineIDs[i]) { + return fmt.Errorf("MachineIDs this[%v](%v) Not Equal that[%v](%v)", i, this.MachineIDs[i], i, that1.MachineIDs[i]) + } + } + if !this.Unavailability.Equal(&that1.Unavailability) { + return fmt.Errorf("Unavailability this(%v) Not Equal that(%v)", this.Unavailability, that1.Unavailability) + } + return nil +} +func (this *Window) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Window) + if !ok { + that2, ok := that.(Window) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.MachineIDs) != len(that1.MachineIDs) { + return false + } + for i := range this.MachineIDs { + if !this.MachineIDs[i].Equal(&that1.MachineIDs[i]) { + return false + } + } + if !this.Unavailability.Equal(&that1.Unavailability) { + return false + } + return true +} +func (this *Schedule) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Schedule) + if !ok { + that2, ok := that.(Schedule) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Schedule") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Schedule but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Schedule but is not nil && this == nil") + } + if len(this.Windows) != len(that1.Windows) { + return fmt.Errorf("Windows this(%v) Not Equal that(%v)", len(this.Windows), len(that1.Windows)) + } + for i := range this.Windows { + if !this.Windows[i].Equal(&that1.Windows[i]) { + return fmt.Errorf("Windows this[%v](%v) Not Equal that[%v](%v)", i, this.Windows[i], i, that1.Windows[i]) + } + } + return nil +} +func (this *Schedule) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Schedule) + if !ok { + that2, ok := that.(Schedule) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Windows) != len(that1.Windows) { + return false + } + for i := range this.Windows { + if !this.Windows[i].Equal(&that1.Windows[i]) { + return false + } + } + return true +} +func (this *ClusterStatus) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*ClusterStatus) + if !ok { + that2, ok := that.(ClusterStatus) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *ClusterStatus") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *ClusterStatus but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *ClusterStatus but is not nil && this == nil") + } + if len(this.DrainingMachines) != len(that1.DrainingMachines) { + return fmt.Errorf("DrainingMachines this(%v) Not Equal that(%v)", len(this.DrainingMachines), len(that1.DrainingMachines)) + } + for i := range this.DrainingMachines { + if !this.DrainingMachines[i].Equal(&that1.DrainingMachines[i]) { + return fmt.Errorf("DrainingMachines this[%v](%v) Not Equal that[%v](%v)", i, this.DrainingMachines[i], i, that1.DrainingMachines[i]) + } + } + if len(this.DownMachines) != len(that1.DownMachines) { + return fmt.Errorf("DownMachines this(%v) Not Equal that(%v)", len(this.DownMachines), len(that1.DownMachines)) + } + for i := range this.DownMachines { + if !this.DownMachines[i].Equal(&that1.DownMachines[i]) { + return fmt.Errorf("DownMachines this[%v](%v) Not Equal that[%v](%v)", i, this.DownMachines[i], i, that1.DownMachines[i]) + } + } + return nil +} +func (this *ClusterStatus) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*ClusterStatus) + if !ok { + that2, ok := that.(ClusterStatus) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.DrainingMachines) != len(that1.DrainingMachines) { + return false + } + for i := range this.DrainingMachines { + if !this.DrainingMachines[i].Equal(&that1.DrainingMachines[i]) { + return false + } + } + if len(this.DownMachines) != len(that1.DownMachines) { + return false + } + for i := range this.DownMachines { + if !this.DownMachines[i].Equal(&that1.DownMachines[i]) { + return false + } + } + return true +} +func (this *ClusterStatus_DrainingMachine) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*ClusterStatus_DrainingMachine) + if !ok { + that2, ok := that.(ClusterStatus_DrainingMachine) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *ClusterStatus_DrainingMachine") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *ClusterStatus_DrainingMachine but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *ClusterStatus_DrainingMachine but is not nil && this == nil") + } + if !this.ID.Equal(&that1.ID) { + return fmt.Errorf("ID this(%v) Not Equal that(%v)", this.ID, that1.ID) + } + if len(this.Statuses) != len(that1.Statuses) { + return fmt.Errorf("Statuses this(%v) Not Equal that(%v)", len(this.Statuses), len(that1.Statuses)) + } + for i := range this.Statuses { + if !this.Statuses[i].Equal(&that1.Statuses[i]) { + return fmt.Errorf("Statuses this[%v](%v) Not Equal that[%v](%v)", i, this.Statuses[i], i, that1.Statuses[i]) + } + } + return nil +} +func (this *ClusterStatus_DrainingMachine) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*ClusterStatus_DrainingMachine) + if !ok { + that2, ok := that.(ClusterStatus_DrainingMachine) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.ID.Equal(&that1.ID) { + return false + } + if len(this.Statuses) != len(that1.Statuses) { + return false + } + for i := range this.Statuses { + if !this.Statuses[i].Equal(&that1.Statuses[i]) { + return false + } + } + return true +} +func (this *Window) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&maintenance.Window{") + if this.MachineIDs != nil { + s = append(s, "MachineIDs: "+fmt.Sprintf("%#v", this.MachineIDs)+",\n") + } + s = append(s, "Unavailability: "+strings.Replace(this.Unavailability.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Schedule) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&maintenance.Schedule{") + if this.Windows != nil { + s = append(s, "Windows: "+fmt.Sprintf("%#v", this.Windows)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ClusterStatus) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&maintenance.ClusterStatus{") + if this.DrainingMachines != nil { + s = append(s, "DrainingMachines: "+fmt.Sprintf("%#v", this.DrainingMachines)+",\n") + } + if this.DownMachines != nil { + s = append(s, "DownMachines: "+fmt.Sprintf("%#v", this.DownMachines)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ClusterStatus_DrainingMachine) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&maintenance.ClusterStatus_DrainingMachine{") + s = append(s, "ID: "+strings.Replace(this.ID.GoString(), `&`, ``, 1)+",\n") + if this.Statuses != nil { + s = append(s, "Statuses: "+fmt.Sprintf("%#v", this.Statuses)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringMaintenance(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} +func (m *Window) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Window) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.MachineIDs) > 0 { + for _, msg := range m.MachineIDs { + dAtA[i] = 0xa + i++ + i = encodeVarintMaintenance(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + dAtA[i] = 0x12 + i++ + i = encodeVarintMaintenance(dAtA, i, uint64(m.Unavailability.ProtoSize())) + n1, err := m.Unavailability.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n1 + return i, nil +} + +func (m *Schedule) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Schedule) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Windows) > 0 { + for _, msg := range m.Windows { + dAtA[i] = 0xa + i++ + i = encodeVarintMaintenance(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *ClusterStatus) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClusterStatus) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.DrainingMachines) > 0 { + for _, msg := range m.DrainingMachines { + dAtA[i] = 0xa + i++ + i = encodeVarintMaintenance(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.DownMachines) > 0 { + for _, msg := range m.DownMachines { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaintenance(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *ClusterStatus_DrainingMachine) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClusterStatus_DrainingMachine) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaintenance(dAtA, i, uint64(m.ID.ProtoSize())) + n2, err := m.ID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + if len(m.Statuses) > 0 { + for _, msg := range m.Statuses { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaintenance(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func encodeFixed64Maintenance(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 encodeFixed32Maintenance(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 encodeVarintMaintenance(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func NewPopulatedWindow(r randyMaintenance, easy bool) *Window { + this := &Window{} + if r.Intn(10) != 0 { + v1 := r.Intn(5) + this.MachineIDs = make([]mesos.MachineID, v1) + for i := 0; i < v1; i++ { + v2 := mesos.NewPopulatedMachineID(r, easy) + this.MachineIDs[i] = *v2 + } + } + v3 := mesos.NewPopulatedUnavailability(r, easy) + this.Unavailability = *v3 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedSchedule(r randyMaintenance, easy bool) *Schedule { + this := &Schedule{} + if r.Intn(10) != 0 { + v4 := r.Intn(5) + this.Windows = make([]Window, v4) + for i := 0; i < v4; i++ { + v5 := NewPopulatedWindow(r, easy) + this.Windows[i] = *v5 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedClusterStatus(r randyMaintenance, easy bool) *ClusterStatus { + this := &ClusterStatus{} + if r.Intn(10) != 0 { + v6 := r.Intn(5) + this.DrainingMachines = make([]ClusterStatus_DrainingMachine, v6) + for i := 0; i < v6; i++ { + v7 := NewPopulatedClusterStatus_DrainingMachine(r, easy) + this.DrainingMachines[i] = *v7 + } + } + if r.Intn(10) != 0 { + v8 := r.Intn(5) + this.DownMachines = make([]mesos.MachineID, v8) + for i := 0; i < v8; i++ { + v9 := mesos.NewPopulatedMachineID(r, easy) + this.DownMachines[i] = *v9 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedClusterStatus_DrainingMachine(r randyMaintenance, easy bool) *ClusterStatus_DrainingMachine { + this := &ClusterStatus_DrainingMachine{} + v10 := mesos.NewPopulatedMachineID(r, easy) + this.ID = *v10 + if r.Intn(10) != 0 { + v11 := r.Intn(5) + this.Statuses = make([]mesos_allocator.InverseOfferStatus, v11) + for i := 0; i < v11; i++ { + v12 := mesos_allocator.NewPopulatedInverseOfferStatus(r, easy) + this.Statuses[i] = *v12 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +type randyMaintenance interface { + Float32() float32 + Float64() float64 + Int63() int64 + Int31() int32 + Uint32() uint32 + Intn(n int) int +} + +func randUTF8RuneMaintenance(r randyMaintenance) rune { + ru := r.Intn(62) + if ru < 10 { + return rune(ru + 48) + } else if ru < 36 { + return rune(ru + 55) + } + return rune(ru + 61) +} +func randStringMaintenance(r randyMaintenance) string { + v13 := r.Intn(100) + tmps := make([]rune, v13) + for i := 0; i < v13; i++ { + tmps[i] = randUTF8RuneMaintenance(r) + } + return string(tmps) +} +func randUnrecognizedMaintenance(r randyMaintenance, maxFieldNumber int) (dAtA []byte) { + l := r.Intn(5) + for i := 0; i < l; i++ { + wire := r.Intn(4) + if wire == 3 { + wire = 5 + } + fieldNumber := maxFieldNumber + r.Intn(100) + dAtA = randFieldMaintenance(dAtA, r, fieldNumber, wire) + } + return dAtA +} +func randFieldMaintenance(dAtA []byte, r randyMaintenance, fieldNumber int, wire int) []byte { + key := uint32(fieldNumber)<<3 | uint32(wire) + switch wire { + case 0: + dAtA = encodeVarintPopulateMaintenance(dAtA, uint64(key)) + v14 := r.Int63() + if r.Intn(2) == 0 { + v14 *= -1 + } + dAtA = encodeVarintPopulateMaintenance(dAtA, uint64(v14)) + case 1: + dAtA = encodeVarintPopulateMaintenance(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + case 2: + dAtA = encodeVarintPopulateMaintenance(dAtA, uint64(key)) + ll := r.Intn(100) + dAtA = encodeVarintPopulateMaintenance(dAtA, uint64(ll)) + for j := 0; j < ll; j++ { + dAtA = append(dAtA, byte(r.Intn(256))) + } + default: + dAtA = encodeVarintPopulateMaintenance(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + } + return dAtA +} +func encodeVarintPopulateMaintenance(dAtA []byte, v uint64) []byte { + for v >= 1<<7 { + dAtA = append(dAtA, uint8(uint64(v)&0x7f|0x80)) + v >>= 7 + } + dAtA = append(dAtA, uint8(v)) + return dAtA +} +func (m *Window) ProtoSize() (n int) { + var l int + _ = l + if len(m.MachineIDs) > 0 { + for _, e := range m.MachineIDs { + l = e.ProtoSize() + n += 1 + l + sovMaintenance(uint64(l)) + } + } + l = m.Unavailability.ProtoSize() + n += 1 + l + sovMaintenance(uint64(l)) + return n +} + +func (m *Schedule) ProtoSize() (n int) { + var l int + _ = l + if len(m.Windows) > 0 { + for _, e := range m.Windows { + l = e.ProtoSize() + n += 1 + l + sovMaintenance(uint64(l)) + } + } + return n +} + +func (m *ClusterStatus) ProtoSize() (n int) { + var l int + _ = l + if len(m.DrainingMachines) > 0 { + for _, e := range m.DrainingMachines { + l = e.ProtoSize() + n += 1 + l + sovMaintenance(uint64(l)) + } + } + if len(m.DownMachines) > 0 { + for _, e := range m.DownMachines { + l = e.ProtoSize() + n += 1 + l + sovMaintenance(uint64(l)) + } + } + return n +} + +func (m *ClusterStatus_DrainingMachine) ProtoSize() (n int) { + var l int + _ = l + l = m.ID.ProtoSize() + n += 1 + l + sovMaintenance(uint64(l)) + if len(m.Statuses) > 0 { + for _, e := range m.Statuses { + l = e.ProtoSize() + n += 1 + l + sovMaintenance(uint64(l)) + } + } + return n +} + +func sovMaintenance(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozMaintenance(x uint64) (n int) { + return sovMaintenance(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *Window) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Window{`, + `MachineIDs:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.MachineIDs), "MachineID", "mesos.MachineID", 1), `&`, ``, 1) + `,`, + `Unavailability:` + strings.Replace(strings.Replace(this.Unavailability.String(), "Unavailability", "mesos.Unavailability", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Schedule) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Schedule{`, + `Windows:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Windows), "Window", "Window", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *ClusterStatus) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ClusterStatus{`, + `DrainingMachines:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.DrainingMachines), "ClusterStatus_DrainingMachine", "ClusterStatus_DrainingMachine", 1), `&`, ``, 1) + `,`, + `DownMachines:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.DownMachines), "MachineID", "mesos.MachineID", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *ClusterStatus_DrainingMachine) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ClusterStatus_DrainingMachine{`, + `ID:` + strings.Replace(strings.Replace(this.ID.String(), "MachineID", "mesos.MachineID", 1), `&`, ``, 1) + `,`, + `Statuses:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Statuses), "InverseOfferStatus", "mesos_allocator.InverseOfferStatus", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func valueToStringMaintenance(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *Window) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Window: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Window: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MachineIDs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaintenance + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MachineIDs = append(m.MachineIDs, mesos.MachineID{}) + if err := m.MachineIDs[len(m.MachineIDs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Unavailability", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaintenance + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Unavailability.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaintenance(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaintenance + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("unavailability") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Schedule) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Schedule: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Schedule: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Windows", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaintenance + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Windows = append(m.Windows, Window{}) + if err := m.Windows[len(m.Windows)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaintenance(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaintenance + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClusterStatus) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClusterStatus: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClusterStatus: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DrainingMachines", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaintenance + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DrainingMachines = append(m.DrainingMachines, ClusterStatus_DrainingMachine{}) + if err := m.DrainingMachines[len(m.DrainingMachines)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DownMachines", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaintenance + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DownMachines = append(m.DownMachines, mesos.MachineID{}) + if err := m.DownMachines[len(m.DownMachines)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaintenance(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaintenance + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClusterStatus_DrainingMachine) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DrainingMachine: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DrainingMachine: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaintenance + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Statuses", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaintenance + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaintenance + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Statuses = append(m.Statuses, mesos_allocator.InverseOfferStatus{}) + if err := m.Statuses[len(m.Statuses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaintenance(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaintenance + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("id") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipMaintenance(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowMaintenance + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowMaintenance + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowMaintenance + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthMaintenance + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowMaintenance + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipMaintenance(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthMaintenance = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowMaintenance = fmt.Errorf("proto: integer overflow") +) + +func init() { proto.RegisterFile("maintenance/maintenance.proto", fileDescriptorMaintenance) } + +var fileDescriptorMaintenance = []byte{ + // 462 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0x31, 0x6f, 0x13, 0x31, + 0x14, 0xc7, 0xcf, 0x07, 0x82, 0xca, 0xa1, 0xd0, 0x5a, 0x42, 0x0a, 0x91, 0x70, 0xab, 0xb0, 0x74, + 0xa0, 0x77, 0xa5, 0x1b, 0x42, 0x62, 0x48, 0x93, 0xe1, 0x06, 0x84, 0x94, 0x0a, 0x21, 0xb1, 0x54, + 0xce, 0xd9, 0xb9, 0x58, 0xba, 0xd8, 0xd5, 0xd9, 0x97, 0x88, 0x8d, 0x89, 0x99, 0x81, 0x0f, 0xc0, + 0xc8, 0x47, 0x40, 0x4c, 0x8c, 0x1d, 0x3b, 0x32, 0x55, 0x9c, 0xbb, 0x30, 0x76, 0x64, 0x44, 0xb5, + 0x7d, 0xc9, 0xa5, 0x05, 0xa9, 0x4b, 0xf4, 0xfc, 0xde, 0xfb, 0xfd, 0xdf, 0xff, 0xbd, 0x1c, 0x7c, + 0x3c, 0x25, 0x5c, 0x68, 0x26, 0x88, 0x48, 0x59, 0xdc, 0x88, 0xa3, 0xe3, 0x42, 0x6a, 0x89, 0x36, + 0xa7, 0x4c, 0x49, 0x15, 0x35, 0x0a, 0x9d, 0xbd, 0x8c, 0xeb, 0x49, 0x39, 0x8a, 0x52, 0x39, 0x8d, + 0x6d, 0xd5, 0xfd, 0xee, 0x66, 0x32, 0x26, 0xc7, 0x3c, 0x9e, 0x3d, 0x8b, 0x73, 0x3e, 0x72, 0x39, + 0x27, 0xd2, 0x79, 0x79, 0x23, 0x82, 0xe4, 0xb9, 0x4c, 0x89, 0x96, 0xc5, 0x32, 0xf2, 0xfc, 0x6e, + 0x83, 0xcf, 0x64, 0x26, 0x63, 0x9b, 0x1e, 0x95, 0x63, 0xfb, 0xb2, 0x0f, 0x1b, 0xb9, 0xf6, 0xee, + 0x67, 0x00, 0xef, 0xbc, 0xe5, 0x82, 0xca, 0x39, 0x1a, 0xc0, 0xd6, 0x94, 0xa4, 0x13, 0x2e, 0xd8, + 0x11, 0xa7, 0xaa, 0x0d, 0xb6, 0x6f, 0xed, 0xb4, 0xf6, 0x37, 0x22, 0x67, 0xee, 0x95, 0xab, 0x24, + 0xfd, 0x1e, 0x3a, 0x39, 0xdb, 0x0a, 0xcc, 0xd9, 0x16, 0x5c, 0xa4, 0xd4, 0x10, 0x7a, 0x30, 0xa1, + 0x0a, 0x1d, 0xc0, 0xfb, 0xa5, 0x20, 0x33, 0xc2, 0x73, 0x32, 0xe2, 0x39, 0xd7, 0xef, 0xdb, 0xe1, + 0x76, 0xb8, 0xd3, 0xda, 0x7f, 0xe8, 0x95, 0xde, 0xac, 0x14, 0x7b, 0xb7, 0x2f, 0xe5, 0x86, 0x57, + 0x90, 0xee, 0x00, 0xae, 0x1d, 0xa6, 0x13, 0x46, 0xcb, 0x9c, 0xa1, 0xe7, 0xf0, 0xee, 0xdc, 0x3a, + 0xac, 0x3d, 0x3d, 0x8a, 0xae, 0x1d, 0x3a, 0x72, 0x3b, 0x78, 0xb5, 0xba, 0xbf, 0xfb, 0x3d, 0x84, + 0xeb, 0x07, 0x79, 0xa9, 0x34, 0x2b, 0x0e, 0x35, 0xd1, 0xa5, 0x42, 0x29, 0xdc, 0xa4, 0x05, 0xe1, + 0x82, 0x8b, 0xec, 0xc8, 0x9b, 0xae, 0x65, 0xf7, 0xfe, 0x21, 0xbb, 0x02, 0x47, 0x7d, 0x4f, 0xfa, + 0xcd, 0xfd, 0xb4, 0x0d, 0xba, 0x9a, 0x56, 0xe8, 0x05, 0x5c, 0xa7, 0x72, 0x2e, 0x96, 0x03, 0xc2, + 0xff, 0xdc, 0xd2, 0x09, 0xdc, 0xbb, 0x6c, 0xae, 0xe1, 0xce, 0x47, 0x00, 0x1f, 0x5c, 0x19, 0x84, + 0x9e, 0xc2, 0x90, 0xd3, 0x36, 0xb0, 0x77, 0xbc, 0xae, 0x02, 0xfd, 0x3f, 0x12, 0x26, 0xfd, 0x61, + 0xc8, 0x29, 0x1a, 0xc0, 0x35, 0x65, 0x0d, 0x2f, 0x26, 0x3f, 0xf1, 0xcc, 0xf2, 0x63, 0x49, 0xc4, + 0x8c, 0x15, 0x8a, 0xbd, 0x1e, 0x8f, 0xeb, 0xed, 0xbc, 0x99, 0x05, 0xda, 0x4b, 0x4e, 0x2b, 0x1c, + 0xfc, 0xac, 0x70, 0xf0, 0xab, 0xc2, 0xe0, 0xa2, 0xc2, 0xe0, 0x4f, 0x85, 0xc1, 0x07, 0x83, 0xc1, + 0x57, 0x83, 0xc1, 0x37, 0x83, 0xc1, 0x0f, 0x83, 0xc1, 0x89, 0xc1, 0xe0, 0xd4, 0x60, 0xf0, 0xdb, + 0xe0, 0xe0, 0xc2, 0x60, 0xf0, 0xe9, 0x1c, 0x07, 0x5f, 0xce, 0x31, 0x78, 0xd7, 0x6a, 0x9c, 0xf1, + 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xbf, 0xff, 0xb9, 0xd9, 0x39, 0x03, 0x00, 0x00, +} diff --git a/api/v1/lib/maintenance/maintenance.pb_ffjson.go b/api/v1/lib/maintenance/maintenance.pb_ffjson.go new file mode 100644 index 00000000..9829d216 --- /dev/null +++ b/api/v1/lib/maintenance/maintenance.pb_ffjson.go @@ -0,0 +1,1230 @@ +// DO NOT EDIT! +// Code generated by ffjson +// source: maintenance/maintenance.pb.go +// DO NOT EDIT! + +package maintenance + +import ( + "bytes" + "fmt" + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/allocator" + fflib "github.com/pquerna/ffjson/fflib/v1" +) + +func (mj *ClusterStatus) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *ClusterStatus) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"draining_machines":`) + if mj.DrainingMachines != nil { + buf.WriteString(`[`) + for i, v := range mj.DrainingMachines { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"down_machines":`) + if mj.DownMachines != nil { + buf.WriteString(`[`) + for i, v := range mj.DownMachines { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_ClusterStatusbase = iota + ffj_t_ClusterStatusno_such_key + + ffj_t_ClusterStatus_DrainingMachines + + ffj_t_ClusterStatus_DownMachines +) + +var ffj_key_ClusterStatus_DrainingMachines = []byte("draining_machines") + +var ffj_key_ClusterStatus_DownMachines = []byte("down_machines") + +func (uj *ClusterStatus) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *ClusterStatus) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_ClusterStatusbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_ClusterStatusno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'd': + + if bytes.Equal(ffj_key_ClusterStatus_DrainingMachines, kn) { + currentKey = ffj_t_ClusterStatus_DrainingMachines + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_ClusterStatus_DownMachines, kn) { + currentKey = ffj_t_ClusterStatus_DownMachines + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_ClusterStatus_DownMachines, kn) { + currentKey = ffj_t_ClusterStatus_DownMachines + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_ClusterStatus_DrainingMachines, kn) { + currentKey = ffj_t_ClusterStatus_DrainingMachines + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_ClusterStatusno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_ClusterStatus_DrainingMachines: + goto handle_DrainingMachines + + case ffj_t_ClusterStatus_DownMachines: + goto handle_DownMachines + + case ffj_t_ClusterStatusno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_DrainingMachines: + + /* handler: uj.DrainingMachines type=[]maintenance.ClusterStatus_DrainingMachine kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.DrainingMachines = nil + } else { + + uj.DrainingMachines = []ClusterStatus_DrainingMachine{} + + wantVal := true + + for { + + var tmp_uj__DrainingMachines ClusterStatus_DrainingMachine + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__DrainingMachines type=maintenance.ClusterStatus_DrainingMachine kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__DrainingMachines.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.DrainingMachines = append(uj.DrainingMachines, tmp_uj__DrainingMachines) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_DownMachines: + + /* handler: uj.DownMachines type=[]mesos.MachineID kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.DownMachines = nil + } else { + + uj.DownMachines = []mesos.MachineID{} + + wantVal := true + + for { + + var tmp_uj__DownMachines mesos.MachineID + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__DownMachines type=mesos.MachineID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__DownMachines.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.DownMachines = append(uj.DownMachines, tmp_uj__DownMachines) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *ClusterStatus_DrainingMachine) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *ClusterStatus_DrainingMachine) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"id":`) + + { + + err = mj.ID.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteString(`,"statuses":`) + if mj.Statuses != nil { + buf.WriteString(`[`) + for i, v := range mj.Statuses { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_ClusterStatus_DrainingMachinebase = iota + ffj_t_ClusterStatus_DrainingMachineno_such_key + + ffj_t_ClusterStatus_DrainingMachine_ID + + ffj_t_ClusterStatus_DrainingMachine_Statuses +) + +var ffj_key_ClusterStatus_DrainingMachine_ID = []byte("id") + +var ffj_key_ClusterStatus_DrainingMachine_Statuses = []byte("statuses") + +func (uj *ClusterStatus_DrainingMachine) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *ClusterStatus_DrainingMachine) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_ClusterStatus_DrainingMachinebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_ClusterStatus_DrainingMachineno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'i': + + if bytes.Equal(ffj_key_ClusterStatus_DrainingMachine_ID, kn) { + currentKey = ffj_t_ClusterStatus_DrainingMachine_ID + state = fflib.FFParse_want_colon + goto mainparse + } + + case 's': + + if bytes.Equal(ffj_key_ClusterStatus_DrainingMachine_Statuses, kn) { + currentKey = ffj_t_ClusterStatus_DrainingMachine_Statuses + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_ClusterStatus_DrainingMachine_Statuses, kn) { + currentKey = ffj_t_ClusterStatus_DrainingMachine_Statuses + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_ClusterStatus_DrainingMachine_ID, kn) { + currentKey = ffj_t_ClusterStatus_DrainingMachine_ID + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_ClusterStatus_DrainingMachineno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_ClusterStatus_DrainingMachine_ID: + goto handle_ID + + case ffj_t_ClusterStatus_DrainingMachine_Statuses: + goto handle_Statuses + + case ffj_t_ClusterStatus_DrainingMachineno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_ID: + + /* handler: uj.ID type=mesos.MachineID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.ID.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Statuses: + + /* handler: uj.Statuses type=[]allocator.InverseOfferStatus kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Statuses = nil + } else { + + uj.Statuses = []allocator.InverseOfferStatus{} + + wantVal := true + + for { + + var tmp_uj__Statuses allocator.InverseOfferStatus + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Statuses type=allocator.InverseOfferStatus kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Statuses.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Statuses = append(uj.Statuses, tmp_uj__Statuses) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Schedule) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Schedule) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"windows":`) + if mj.Windows != nil { + buf.WriteString(`[`) + for i, v := range mj.Windows { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Schedulebase = iota + ffj_t_Scheduleno_such_key + + ffj_t_Schedule_Windows +) + +var ffj_key_Schedule_Windows = []byte("windows") + +func (uj *Schedule) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Schedule) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Schedulebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Scheduleno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'w': + + if bytes.Equal(ffj_key_Schedule_Windows, kn) { + currentKey = ffj_t_Schedule_Windows + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Schedule_Windows, kn) { + currentKey = ffj_t_Schedule_Windows + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Scheduleno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Schedule_Windows: + goto handle_Windows + + case ffj_t_Scheduleno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Windows: + + /* handler: uj.Windows type=[]maintenance.Window kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Windows = nil + } else { + + uj.Windows = []Window{} + + wantVal := true + + for { + + var tmp_uj__Windows Window + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Windows type=maintenance.Window kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Windows.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Windows = append(uj.Windows, tmp_uj__Windows) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Window) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Window) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"machine_ids":`) + if mj.MachineIDs != nil { + buf.WriteString(`[`) + for i, v := range mj.MachineIDs { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"unavailability":`) + + { + + err = mj.Unavailability.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Windowbase = iota + ffj_t_Windowno_such_key + + ffj_t_Window_MachineIDs + + ffj_t_Window_Unavailability +) + +var ffj_key_Window_MachineIDs = []byte("machine_ids") + +var ffj_key_Window_Unavailability = []byte("unavailability") + +func (uj *Window) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Window) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Windowbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Windowno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'm': + + if bytes.Equal(ffj_key_Window_MachineIDs, kn) { + currentKey = ffj_t_Window_MachineIDs + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'u': + + if bytes.Equal(ffj_key_Window_Unavailability, kn) { + currentKey = ffj_t_Window_Unavailability + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Window_Unavailability, kn) { + currentKey = ffj_t_Window_Unavailability + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Window_MachineIDs, kn) { + currentKey = ffj_t_Window_MachineIDs + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Windowno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Window_MachineIDs: + goto handle_MachineIDs + + case ffj_t_Window_Unavailability: + goto handle_Unavailability + + case ffj_t_Windowno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_MachineIDs: + + /* handler: uj.MachineIDs type=[]mesos.MachineID kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.MachineIDs = nil + } else { + + uj.MachineIDs = []mesos.MachineID{} + + wantVal := true + + for { + + var tmp_uj__MachineIDs mesos.MachineID + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__MachineIDs type=mesos.MachineID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__MachineIDs.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.MachineIDs = append(uj.MachineIDs, tmp_uj__MachineIDs) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Unavailability: + + /* handler: uj.Unavailability type=mesos.Unavailability kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Unavailability.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} diff --git a/api/v1/lib/maintenance/maintenance.proto b/api/v1/lib/maintenance/maintenance.proto new file mode 100644 index 00000000..217c9cc6 --- /dev/null +++ b/api/v1/lib/maintenance/maintenance.proto @@ -0,0 +1,102 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto2"; + +package mesos.maintenance; + +import "github.com/mesos/mesos-go/api/v1/lib/mesos.proto"; +import "github.com/mesos/mesos-go/api/v1/lib/allocator/allocator.proto"; +import "github.com/gogo/protobuf/gogoproto/gogo.proto"; + +option go_package = "maintenance"; +option (gogoproto.benchgen_all) = true; +option (gogoproto.enum_stringer_all) = true; +option (gogoproto.equal_all) = true; +option (gogoproto.goproto_enum_prefix_all) = false; +option (gogoproto.goproto_enum_stringer_all) = false; +option (gogoproto.goproto_stringer_all) = false; +option (gogoproto.goproto_unrecognized_all) = false; +option (gogoproto.gostring_all) = true; +option (gogoproto.marshaler_all) = true; +option (gogoproto.populate_all) = true; +option (gogoproto.protosizer_all) = true; +option (gogoproto.stringer_all) = true; +option (gogoproto.testgen_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.verbose_equal_all) = true; + +// This is an illustration of a maintenance `Schedule`: +// +// This is a `Window`. +// Machine ^ | +// ... | v +// 12 | +----------------+ +// 11 | | | +// 10 | +----------------+ +// 9 | +----------------+ +// 8 | | | +// 7 | +----------------+ +// 6 | +----------------+ +// 5 | | | +// 4 | +----------------+ +// 3 | +-----------+ +// 2 | | | +// 1 | +-----------+ +// | +// +-----------------------------------------------------~~~~-> +// Downtime for maintenance + + +/** + * A set of machines scheduled to go into maintenance + * in the same `unavailability`. + */ +message Window { + // Machines affected by this maintenance window. + repeated MachineID machine_ids = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "MachineIDs"]; + + // Interval during which this set of machines is expected to be down. + required Unavailability unavailability = 2 [(gogoproto.nullable) = false]; +} + + +/** + * A list of maintenance windows. + * For example, this may represent a rolling restart of agents. + */ +message Schedule { + repeated Window windows = 1 [(gogoproto.nullable) = false]; +} + + +/** + * Represents the maintenance status of each machine in the cluster. + * The lists correspond to the `MachineInfo.Mode` enumeration. + */ +message ClusterStatus { + message DrainingMachine { + required MachineID id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "ID"]; + + // A list of the most recent responses to inverse offers from frameworks + // running on this draining machine. + repeated allocator.InverseOfferStatus statuses = 2 [(gogoproto.nullable) = false]; + } + + repeated DrainingMachine draining_machines = 1 [(gogoproto.nullable) = false]; + repeated MachineID down_machines = 2 [(gogoproto.nullable) = false]; +} diff --git a/api/v1/lib/maintenance/maintenancepb_test.go b/api/v1/lib/maintenance/maintenancepb_test.go new file mode 100644 index 00000000..a9f77b9d --- /dev/null +++ b/api/v1/lib/maintenance/maintenancepb_test.go @@ -0,0 +1,956 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: maintenance/maintenance.proto + +/* +Package maintenance is a generated protocol buffer package. + +It is generated from these files: + maintenance/maintenance.proto + +It has these top-level messages: + Window + Schedule + ClusterStatus +*/ +package maintenance + +import testing "testing" +import math_rand "math/rand" +import time "time" +import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" +import github_com_gogo_protobuf_jsonpb "github.com/gogo/protobuf/jsonpb" +import fmt "fmt" +import go_parser "go/parser" +import proto "github.com/gogo/protobuf/proto" +import math "math" +import _ "github.com/mesos/mesos-go/api/v1/lib" +import _ "github.com/mesos/mesos-go/api/v1/lib/allocator" +import _ "github.com/gogo/protobuf/gogoproto" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +func TestWindowProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedWindow(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Window{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestWindowMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedWindow(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Window{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkWindowProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Window, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedWindow(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkWindowProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedWindow(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Window{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestScheduleProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedSchedule(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Schedule{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestScheduleMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedSchedule(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Schedule{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkScheduleProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Schedule, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedSchedule(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkScheduleProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedSchedule(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Schedule{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestClusterStatusProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &ClusterStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestClusterStatusMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &ClusterStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkClusterStatusProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*ClusterStatus, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedClusterStatus(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkClusterStatusProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedClusterStatus(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &ClusterStatus{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestClusterStatus_DrainingMachineProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus_DrainingMachine(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &ClusterStatus_DrainingMachine{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestClusterStatus_DrainingMachineMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus_DrainingMachine(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &ClusterStatus_DrainingMachine{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkClusterStatus_DrainingMachineProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*ClusterStatus_DrainingMachine, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedClusterStatus_DrainingMachine(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkClusterStatus_DrainingMachineProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedClusterStatus_DrainingMachine(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &ClusterStatus_DrainingMachine{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestWindowJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedWindow(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Window{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestScheduleJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedSchedule(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Schedule{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestClusterStatusJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &ClusterStatus{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestClusterStatus_DrainingMachineJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus_DrainingMachine(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &ClusterStatus_DrainingMachine{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestWindowProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedWindow(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Window{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestWindowProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedWindow(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Window{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestScheduleProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedSchedule(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Schedule{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestScheduleProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedSchedule(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Schedule{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestClusterStatusProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &ClusterStatus{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestClusterStatusProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &ClusterStatus{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestClusterStatus_DrainingMachineProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus_DrainingMachine(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &ClusterStatus_DrainingMachine{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestClusterStatus_DrainingMachineProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus_DrainingMachine(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &ClusterStatus_DrainingMachine{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestWindowVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedWindow(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Window{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestScheduleVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedSchedule(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Schedule{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestClusterStatusVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedClusterStatus(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &ClusterStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestClusterStatus_DrainingMachineVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedClusterStatus_DrainingMachine(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &ClusterStatus_DrainingMachine{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestWindowGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedWindow(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestScheduleGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedSchedule(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestClusterStatusGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedClusterStatus(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestClusterStatus_DrainingMachineGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedClusterStatus_DrainingMachine(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestWindowProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedWindow(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkWindowProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Window, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedWindow(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestScheduleProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedSchedule(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkScheduleProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Schedule, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedSchedule(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestClusterStatusProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkClusterStatusProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*ClusterStatus, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedClusterStatus(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestClusterStatus_DrainingMachineProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedClusterStatus_DrainingMachine(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkClusterStatus_DrainingMachineProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*ClusterStatus_DrainingMachine, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedClusterStatus_DrainingMachine(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestWindowStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedWindow(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestScheduleStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedSchedule(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestClusterStatusStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedClusterStatus(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestClusterStatus_DrainingMachineStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedClusterStatus_DrainingMachine(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} + +//These tests are generated by github.com/gogo/protobuf/plugin/testgen diff --git a/api/v1/lib/master/calls/calls.go b/api/v1/lib/master/calls/calls.go new file mode 100644 index 00000000..fda37289 --- /dev/null +++ b/api/v1/lib/master/calls/calls.go @@ -0,0 +1,293 @@ +package calls + +import ( + "time" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/maintenance" + "github.com/mesos/mesos-go/api/v1/lib/master" + "github.com/mesos/mesos-go/api/v1/lib/quota" +) + +// GetHealth retrieves the health status of master. +func GetHealth() *master.Call { + return &master.Call{ + Type: master.Call_GET_HEALTH, + } +} + +// GetFlags retrieves the master's overall flag configuration. +func GetFlags() *master.Call { + return &master.Call{ + Type: master.Call_GET_FLAGS, + } +} + +// GetVersion retrieves the master's version information. +func GetVersion() *master.Call { + return &master.Call{ + Type: master.Call_GET_VERSION, + } +} + +// GetMetrics gives the snapshot of current metrics to the end user. If timeout is set in the call, it will be used to +// determine the maximum amount of time the API will take to respond. If the timeout is exceeded, some metrics may not +// be included in the response. +func GetMetrics(d *time.Duration) (call *master.Call) { + call = &master.Call{ + Type: master.Call_GET_METRICS, + GetMetrics: &master.Call_GetMetrics{}, + } + if d != nil { + call.GetMetrics.Timeout = &mesos.DurationInfo{ + Nanoseconds: d.Nanoseconds(), + } + } + return +} + +// GetLoggingLevel retrieves the master's logging level. +func GetLoggingLevel() *master.Call { + return &master.Call{ + Type: master.Call_GET_LOGGING_LEVEL, + } +} + +// SetLoggingLevel sets the logging verbosity level for a specified duration for master. Mesos uses glog for logging. +// The library only uses verbose logging which means nothing will be output unless the verbosity level is set (by +// default it's 0, libprocess uses levels 1, 2, and 3). +func SetLoggingLevel(level uint32, d time.Duration) *master.Call { + return &master.Call{ + Type: master.Call_SET_LOGGING_LEVEL, + SetLoggingLevel: &master.Call_SetLoggingLevel{ + Level: level, + Duration: mesos.DurationInfo{ + Nanoseconds: d.Nanoseconds(), + }, + }, + } +} + +// ListFiles retrieves the file listing for a directory in master. +func ListFiles(path string) *master.Call { + return &master.Call{ + Type: master.Call_LIST_FILES, + ListFiles: &master.Call_ListFiles{ + Path: path, + }, + } +} + +// ReadFile reads data from a file on the master. This call takes the path of the file to be read and the offset to +// start reading. +func ReadFile(path string, offset uint64) *master.Call { + return &master.Call{ + Type: master.Call_READ_FILE, + ReadFile: &master.Call_ReadFile{ + Path: path, + Offset: offset, + }, + } +} + +// ReadFileWithLength is similar to ReadFile but species the maximum number of bytes to read from the file. +func ReadFileWithLength(path string, offset, length uint64) *master.Call { + call := ReadFile(path, offset) + call.ReadFile.Length = &length + return call +} + +// GetState retrieves the overall cluster state. +func GetState() *master.Call { + return &master.Call{ + Type: master.Call_GET_STATE, + } +} + +// GetAgents retrieves information about all the agents known to the master. +func GetAgents() *master.Call { + return &master.Call{ + Type: master.Call_GET_AGENTS, + } +} + +// GetFrameworks retrieves information about all the frameworks known to the master. +func GetFrameworks() *master.Call { + return &master.Call{ + Type: master.Call_GET_FRAMEWORKS, + } +} + +// GetExecutors retrieves information about all the executors known to the master. +func GetExecutors() *master.Call { + return &master.Call{ + Type: master.Call_GET_EXECUTORS, + } +} + +// GetTasks retrieves information about all the tasks known to the master. +func GetTasks() *master.Call { + return &master.Call{ + Type: master.Call_GET_TASKS, + } +} + +// GetRoles retrieves information about all the roles known to the master. +func GetRoles() *master.Call { + return &master.Call{ + Type: master.Call_GET_ROLES, + } +} + +// GetWeights retrieves the information about role weights. +func GetWeights() *master.Call { + return &master.Call{ + Type: master.Call_GET_WEIGHTS, + } +} + +// UpdateWeights updates weights for specific roles. +func UpdateWeights(weights ...mesos.WeightInfo) *master.Call { + return &master.Call{ + Type: master.Call_UPDATE_WEIGHTS, + UpdateWeights: &master.Call_UpdateWeights{ + WeightInfos: weights, + }, + } +} + +// GetMaster retrieves information about the master. +func GetMaster() *master.Call { + return &master.Call{ + Type: master.Call_GET_MASTER, + } +} + +// Subscribe results in a streaming response. The client is expected to keep a persistent connection open to the +// endpoint even after getting a SUBSCRIBED HTTP Response event. This is indicated by "Connection: keep-alive" and +// "Transfer-Encoding: chunked" headers with no "Content-Length" header set. All subsequent events generated by Mesos +// are streamed on this connection. The master encodes each Event in RecordIO format, i.e., string representation of +// length of the event in bytes followed by JSON or binary Protobuf encoded event. +func Subscribe() *master.Call { + return &master.Call{ + Type: master.Call_SUBSCRIBE, + } +} + +// ReserveResources reserves resources dynamically on a specific agent. +func ReserveResources(a mesos.AgentID, r ...mesos.Resource) *master.Call { + return &master.Call{ + Type: master.Call_RESERVE_RESOURCES, + ReserveResources: &master.Call_ReserveResources{ + AgentID: a, + Resources: r, + }, + } +} + +// UnreserveResources unreserves resources dynamically on a specific agent. +func UnreserveResources(a mesos.AgentID, r ...mesos.Resource) *master.Call { + return &master.Call{ + Type: master.Call_UNRESERVE_RESOURCES, + UnreserveResources: &master.Call_UnreserveResources{ + AgentID: a, + Resources: r, + }, + } +} + +// CreateVolumes creates persistent volumes on reserved resources. The request is forwarded asynchronously to the Mesos +// agent where the reserved resources are located. That asynchronous message may not be delivered or creating the +// volumes at the agent might fail. +func CreateVolumes(a mesos.AgentID, v ...mesos.Resource) *master.Call { + return &master.Call{ + Type: master.Call_CREATE_VOLUMES, + CreateVolumes: &master.Call_CreateVolumes{ + AgentID: a, + Volumes: v, + }, + } +} + +// DestroyVolumes destroys persistent volumes. The request is forwarded asynchronously to the Mesos agent where the +// reserved resources are located. +func DestroyVolumes(a mesos.AgentID, v ...mesos.Resource) *master.Call { + return &master.Call{ + Type: master.Call_DESTROY_VOLUMES, + DestroyVolumes: &master.Call_DestroyVolumes{ + AgentID: a, + Volumes: v, + }, + } +} + +// GetMaintenanceStatus retrieves the cluster's maintenance status. +func GetMaintenanceStatus() *master.Call { + return &master.Call{ + Type: master.Call_GET_MAINTENANCE_STATUS, + } +} + +//GetMaintenanceSchedule retrieves the cluster's maintenance schedule. +func GetMaintenanceSchedule() *master.Call { + return &master.Call{ + Type: master.Call_GET_MAINTENANCE_SCHEDULE, + } +} + +// UpdateMaintenanceSchedule updates the cluster's maintenance schedule. +func UpdateMaintenanceSchedule(s maintenance.Schedule) *master.Call { + return &master.Call{ + Type: master.Call_UPDATE_MAINTENANCE_SCHEDULE, + UpdateMaintenanceSchedule: &master.Call_UpdateMaintenanceSchedule{ + Schedule: s, + }, + } +} + +// StartMaintenance starts the maintenance of the cluster, this would bring a set of machines down. +func StartMaintenance(m ...mesos.MachineID) *master.Call { + return &master.Call{ + Type: master.Call_START_MAINTENANCE, + StartMaintenance: &master.Call_StartMaintenance{ + Machines: m, + }, + } +} + +// StopMaintenance stops the maintenance of the cluster, this would bring a set of machines back up. +func StopMaintenance(m ...mesos.MachineID) *master.Call { + return &master.Call{ + Type: master.Call_STOP_MAINTENANCE, + StopMaintenance: &master.Call_StopMaintenance{ + Machines: m, + }, + } +} + +// GetQuota retrieves the cluster's configured quotas. +func GetQuota() *master.Call { + return &master.Call{ + Type: master.Call_GET_QUOTA, + } +} + +// SetQuota sets the quota for resources to be used by a particular role. +func SetQuota(qr quota.QuotaRequest) *master.Call { + return &master.Call{ + Type: master.Call_SET_QUOTA, + SetQuota: &master.Call_SetQuota{ + QuotaRequest: qr, + }, + } +} + +// RemoveQuota removes the quota for a particular role. +func RemoveQuota(role string) *master.Call { + return &master.Call{ + Type: master.Call_REMOVE_QUOTA, + RemoveQuota: &master.Call_RemoveQuota{ + Role: role, + }, + } +} diff --git a/api/v1/lib/master/calls/calls_generated.go b/api/v1/lib/master/calls/calls_generated.go new file mode 100644 index 00000000..5fcb255d --- /dev/null +++ b/api/v1/lib/master/calls/calls_generated.go @@ -0,0 +1,129 @@ +package calls + +// go generate -import github.com/mesos/mesos-go/api/v1/lib/master -type C:master.Call +// GENERATED CODE FOLLOWS; DO NOT EDIT. + +import ( + "context" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/encoding" + + "github.com/mesos/mesos-go/api/v1/lib/master" +) + +type ( + // Request generates a Call that's sent to a Mesos agent. Subsequent invocations are expected to + // yield equivalent calls. Intended for use w/ non-streaming requests to an agent. + Request interface { + Call() *master.Call + } + + // RequestFunc is the functional adaptation of Request. + RequestFunc func() *master.Call + + // RequestStreaming generates a Call that's send to a Mesos agent. Subsequent invocations MAY generate + // different Call objects. No more Call objects are expected once a nil is returned to signal the end of + // of the request stream. + RequestStreaming interface { + Request + IsStreaming() + } + + // RequestStreamingFunc is the functional adaptation of RequestStreaming. + RequestStreamingFunc func() *master.Call + + // Send issues a Request to a Mesos agent and properly manages Call-specific mechanics. + Sender interface { + Send(context.Context, Request) (mesos.Response, error) + } + + // SenderFunc is the functional adaptation of the Sender interface + SenderFunc func(context.Context, Request) (mesos.Response, error) +) + +func (f RequestFunc) Call() *master.Call { return f() } + +func (f RequestFunc) Marshaler() encoding.Marshaler { + // avoid returning (*master.Call)(nil) for interface type + if call := f(); call != nil { + return call + } + return nil +} + +func (f RequestStreamingFunc) Push(c ...*master.Call) RequestStreamingFunc { return Push(f, c...) } + +func (f RequestStreamingFunc) Marshaler() encoding.Marshaler { + // avoid returning (*master.Call)(nil) for interface type + if call := f(); call != nil { + return call + } + return nil +} + +func (f RequestStreamingFunc) IsStreaming() {} + +func (f RequestStreamingFunc) Call() *master.Call { return f() } + +// Push prepends one or more calls onto a request stream. If no calls are given then the original stream is returned. +func Push(r RequestStreaming, c ...*master.Call) RequestStreamingFunc { + return func() *master.Call { + if len(c) == 0 { + return r.Call() + } + head := c[0] + c = c[1:] + return head + } +} + +// Empty generates a stream that always returns nil. +func Empty() RequestStreamingFunc { return func() *master.Call { return nil } } + +var ( + _ = Request(RequestFunc(nil)) + _ = RequestStreaming(RequestStreamingFunc(nil)) + _ = Sender(SenderFunc(nil)) +) + +// NonStreaming returns a RequestFunc that always generates the same Call. +func NonStreaming(c *master.Call) RequestFunc { return func() *master.Call { return c } } + +// FromChan returns a streaming request that fetches calls from the given channel until it closes. +// If a nil chan is specified then the returned func will always generate nil. +func FromChan(ch <-chan *master.Call) RequestStreamingFunc { + if ch == nil { + // avoid blocking forever if we're handed a nil chan + return func() *master.Call { return nil } + } + return func() *master.Call { + if m, ok := <-ch; ok { + return m + } + return nil + } +} + +// Send implements the Sender interface for SenderFunc +func (f SenderFunc) Send(ctx context.Context, r Request) (mesos.Response, error) { + return f(ctx, r) +} + +// IgnoreResponse generates a sender that closes any non-nil response received by Mesos. +func IgnoreResponse(s Sender) SenderFunc { + return func(ctx context.Context, r Request) (mesos.Response, error) { + resp, err := s.Send(ctx, r) + if resp != nil { + resp.Close() + } + return nil, err + } +} + +// SendNoData is a convenience func that executes the given Call using the provided Sender +// and always drops the response data. +func SendNoData(ctx context.Context, sender Sender, r Request) (err error) { + _, err = IgnoreResponse(sender).Send(ctx, r) + return +} diff --git a/api/v1/lib/master/calls/calls_generated_test.go b/api/v1/lib/master/calls/calls_generated_test.go new file mode 100644 index 00000000..113b6012 --- /dev/null +++ b/api/v1/lib/master/calls/calls_generated_test.go @@ -0,0 +1,99 @@ +package calls + +// go generate -import github.com/mesos/mesos-go/api/v1/lib/master -type C:master.Call +// GENERATED CODE FOLLOWS; DO NOT EDIT. + +import ( + "context" + "testing" + + "github.com/mesos/mesos-go/api/v1/lib" + + "github.com/mesos/mesos-go/api/v1/lib/master" +) + +func TestNonStreaming(t *testing.T) { + c := new(master.Call) + f := NonStreaming(c) + if x := f.Call(); x != c { + t.Fatalf("expected %#v instead of %#v", c, x) + } + if x := f.Marshaler(); x == nil { + t.Fatal("expected non-nil Marshaler") + } + f = NonStreaming(nil) + if x := f.Marshaler(); x != nil { + t.Fatalf("expected nil Marshaler instead of %#v", x) + } +} + +func TestStreaming(t *testing.T) { + f := Empty() + + f.IsStreaming() + + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + if x := f.Marshaler(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + + c := new(master.Call) + + f = f.Push(c) + if x := f.Marshaler(); x == nil { + t.Fatal("expected non-nil Marshaler") + } + if x := f.Marshaler(); x != nil { + t.Fatalf("expected nil Marshaler instead of %#v", x) + } + + c2 := new(master.Call) + + f = Empty().Push(c, c2) + if x := f.Call(); x != c { + t.Fatalf("expected %#v instead of %#v", c, x) + } + if x := f.Call(); x != c2 { + t.Fatalf("expected %#v instead of %#v", c2, x) + } + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + + ch := make(chan *master.Call, 2) + ch <- c + ch <- c2 + close(ch) + f = FromChan(ch) + if x := f.Call(); x != c { + t.Fatalf("expected %#v instead of %#v", c, x) + } + if x := f.Call(); x != c2 { + t.Fatalf("expected %#v instead of %#v", c2, x) + } + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } + + f = FromChan(nil) + if x := f.Call(); x != nil { + t.Fatalf("expected nil Call instead of %#v", x) + } +} + +func TestIgnoreResponse(t *testing.T) { + var closed bool + + IgnoreResponse(SenderFunc(func(_ context.Context, _ Request) (mesos.Response, error) { + return &mesos.ResponseWrapper{Closer: mesos.CloseFunc(func() error { + closed = true + return nil + })}, nil + })).Send(nil, nil) + + if !closed { + t.Fatal("expected response to be closed") + } +} diff --git a/api/v1/lib/master/calls/calls_test.go b/api/v1/lib/master/calls/calls_test.go new file mode 100644 index 00000000..02409efc --- /dev/null +++ b/api/v1/lib/master/calls/calls_test.go @@ -0,0 +1,67 @@ +package calls_test + +import ( + "context" + "time" + + "github.com/mesos/mesos-go/api/v1/lib" + "github.com/mesos/mesos-go/api/v1/lib/maintenance" + "github.com/mesos/mesos-go/api/v1/lib/master" + . "github.com/mesos/mesos-go/api/v1/lib/master/calls" + "github.com/mesos/mesos-go/api/v1/lib/quota" +) + +func Example() { + var ( + check = func(err error) { + if err != nil { + panic(err) + } + } + swallow = func(_ mesos.Response, err error) { check(err) } + + ctx = context.Background() + sender = SenderFunc(func(_ context.Context, _ Request) (_ mesos.Response, _ error) { return }) + blackhole = func(c *master.Call) { swallow(sender.Send(ctx, NonStreaming(c))) } + + d = time.Duration(0) + ) + blackhole(GetHealth()) + blackhole(GetFlags()) + blackhole(GetVersion()) + blackhole(GetMetrics(nil)) + blackhole(GetMetrics(&d)) + blackhole(GetLoggingLevel()) + blackhole(ListFiles("")) + blackhole(ReadFile("", 0)) + blackhole(ReadFileWithLength("", 0, 0)) + blackhole(GetState()) + blackhole(GetAgents()) + blackhole(GetFrameworks()) + blackhole(GetExecutors()) + blackhole(GetTasks()) + blackhole(GetRoles()) + blackhole(GetWeights()) + blackhole(GetMaster()) + blackhole(GetMaintenanceStatus()) + blackhole(GetMaintenanceSchedule()) + blackhole(GetQuota()) + blackhole(Subscribe()) + + blackhole = func(c *master.Call) { + check(SendNoData(ctx, sender, NonStreaming(c))) + } + blackhole(SetLoggingLevel(0, 0)) + blackhole(UpdateWeights()) + blackhole(ReserveResources(mesos.AgentID{})) + blackhole(UnreserveResources(mesos.AgentID{})) + blackhole(CreateVolumes(mesos.AgentID{})) + blackhole(DestroyVolumes(mesos.AgentID{})) + blackhole(UpdateMaintenanceSchedule(maintenance.Schedule{})) + blackhole(StartMaintenance()) + blackhole(StopMaintenance()) + blackhole(SetQuota(quota.QuotaRequest{})) + blackhole(RemoveQuota("")) + + // Output: +} diff --git a/api/v1/lib/master/calls/gen.go b/api/v1/lib/master/calls/gen.go new file mode 100644 index 00000000..83f561da --- /dev/null +++ b/api/v1/lib/master/calls/gen.go @@ -0,0 +1,3 @@ +package calls + +//go:generate go run ../../extras/gen/sender.go ../../extras/gen/gen.go -import github.com/mesos/mesos-go/api/v1/lib/master -type C:master.Call diff --git a/api/v1/lib/master/master.pb.go b/api/v1/lib/master/master.pb.go new file mode 100644 index 00000000..74cd5efd --- /dev/null +++ b/api/v1/lib/master/master.pb.go @@ -0,0 +1,16108 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: master/master.proto + +/* + Package master is a generated protocol buffer package. + + It is generated from these files: + master/master.proto + + It has these top-level messages: + Call + Response + Event +*/ +package master + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" +import mesos "github.com/mesos/mesos-go/api/v1/lib" +import _ "github.com/mesos/mesos-go/api/v1/lib/allocator" +import mesos_maintenance "github.com/mesos/mesos-go/api/v1/lib/maintenance" +import mesos_quota "github.com/mesos/mesos-go/api/v1/lib/quota" +import _ "github.com/gogo/protobuf/gogoproto" + +import strconv "strconv" + +import bytes "bytes" + +import strings "strings" +import reflect "reflect" + +import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" + +import io "io" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +type Call_Type int32 + +const ( + // If a call of type `Call::FOO` requires additional parameters they can be + // included in the corresponding `Call::Foo` message. Similarly, if a call + // receives a synchronous response it will be returned as a `Response` + // message of type `Response::FOO`. Currently all calls except + // `Call::SUBSCRIBE` receive synchronous responses; `Call::SUBSCRIBE` returns + // a streaming response of `Event`. + Call_UNKNOWN Call_Type = 0 + Call_GET_HEALTH Call_Type = 1 + Call_GET_FLAGS Call_Type = 2 + Call_GET_VERSION Call_Type = 3 + Call_GET_METRICS Call_Type = 4 + Call_GET_LOGGING_LEVEL Call_Type = 5 + Call_SET_LOGGING_LEVEL Call_Type = 6 + Call_LIST_FILES Call_Type = 7 + Call_READ_FILE Call_Type = 8 + Call_GET_STATE Call_Type = 9 + Call_GET_AGENTS Call_Type = 10 + Call_GET_FRAMEWORKS Call_Type = 11 + Call_GET_EXECUTORS Call_Type = 12 + Call_GET_TASKS Call_Type = 13 + Call_GET_ROLES Call_Type = 14 + Call_GET_WEIGHTS Call_Type = 15 + Call_UPDATE_WEIGHTS Call_Type = 16 + Call_GET_MASTER Call_Type = 17 + Call_SUBSCRIBE Call_Type = 18 + Call_RESERVE_RESOURCES Call_Type = 19 + Call_UNRESERVE_RESOURCES Call_Type = 20 + Call_CREATE_VOLUMES Call_Type = 21 + Call_DESTROY_VOLUMES Call_Type = 22 + // Retrieves the cluster's maintenance status. + Call_GET_MAINTENANCE_STATUS Call_Type = 23 + // Retrieves the cluster's maintenance schedule. + Call_GET_MAINTENANCE_SCHEDULE Call_Type = 24 + Call_UPDATE_MAINTENANCE_SCHEDULE Call_Type = 25 + Call_START_MAINTENANCE Call_Type = 26 + Call_STOP_MAINTENANCE Call_Type = 27 + Call_GET_QUOTA Call_Type = 28 + Call_SET_QUOTA Call_Type = 29 + Call_REMOVE_QUOTA Call_Type = 30 +) + +var Call_Type_name = map[int32]string{ + 0: "UNKNOWN", + 1: "GET_HEALTH", + 2: "GET_FLAGS", + 3: "GET_VERSION", + 4: "GET_METRICS", + 5: "GET_LOGGING_LEVEL", + 6: "SET_LOGGING_LEVEL", + 7: "LIST_FILES", + 8: "READ_FILE", + 9: "GET_STATE", + 10: "GET_AGENTS", + 11: "GET_FRAMEWORKS", + 12: "GET_EXECUTORS", + 13: "GET_TASKS", + 14: "GET_ROLES", + 15: "GET_WEIGHTS", + 16: "UPDATE_WEIGHTS", + 17: "GET_MASTER", + 18: "SUBSCRIBE", + 19: "RESERVE_RESOURCES", + 20: "UNRESERVE_RESOURCES", + 21: "CREATE_VOLUMES", + 22: "DESTROY_VOLUMES", + 23: "GET_MAINTENANCE_STATUS", + 24: "GET_MAINTENANCE_SCHEDULE", + 25: "UPDATE_MAINTENANCE_SCHEDULE", + 26: "START_MAINTENANCE", + 27: "STOP_MAINTENANCE", + 28: "GET_QUOTA", + 29: "SET_QUOTA", + 30: "REMOVE_QUOTA", +} +var Call_Type_value = map[string]int32{ + "UNKNOWN": 0, + "GET_HEALTH": 1, + "GET_FLAGS": 2, + "GET_VERSION": 3, + "GET_METRICS": 4, + "GET_LOGGING_LEVEL": 5, + "SET_LOGGING_LEVEL": 6, + "LIST_FILES": 7, + "READ_FILE": 8, + "GET_STATE": 9, + "GET_AGENTS": 10, + "GET_FRAMEWORKS": 11, + "GET_EXECUTORS": 12, + "GET_TASKS": 13, + "GET_ROLES": 14, + "GET_WEIGHTS": 15, + "UPDATE_WEIGHTS": 16, + "GET_MASTER": 17, + "SUBSCRIBE": 18, + "RESERVE_RESOURCES": 19, + "UNRESERVE_RESOURCES": 20, + "CREATE_VOLUMES": 21, + "DESTROY_VOLUMES": 22, + "GET_MAINTENANCE_STATUS": 23, + "GET_MAINTENANCE_SCHEDULE": 24, + "UPDATE_MAINTENANCE_SCHEDULE": 25, + "START_MAINTENANCE": 26, + "STOP_MAINTENANCE": 27, + "GET_QUOTA": 28, + "SET_QUOTA": 29, + "REMOVE_QUOTA": 30, +} + +func (x Call_Type) Enum() *Call_Type { + p := new(Call_Type) + *p = x + return p +} +func (x Call_Type) MarshalJSON() ([]byte, error) { + return proto.MarshalJSONEnum(Call_Type_name, int32(x)) +} +func (x *Call_Type) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(Call_Type_value, data, "Call_Type") + if err != nil { + return err + } + *x = Call_Type(value) + return nil +} +func (Call_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 0} } + +// Each of the responses of type `FOO` corresponds to `Foo` message below. +type Response_Type int32 + +const ( + Response_UNKNOWN Response_Type = 0 + Response_GET_HEALTH Response_Type = 1 + Response_GET_FLAGS Response_Type = 2 + Response_GET_VERSION Response_Type = 3 + Response_GET_METRICS Response_Type = 4 + Response_GET_LOGGING_LEVEL Response_Type = 5 + Response_LIST_FILES Response_Type = 6 + Response_READ_FILE Response_Type = 7 + Response_GET_STATE Response_Type = 8 + Response_GET_AGENTS Response_Type = 9 + Response_GET_FRAMEWORKS Response_Type = 10 + Response_GET_EXECUTORS Response_Type = 11 + Response_GET_TASKS Response_Type = 12 + Response_GET_ROLES Response_Type = 13 + Response_GET_WEIGHTS Response_Type = 14 + Response_GET_MASTER Response_Type = 15 + Response_GET_MAINTENANCE_STATUS Response_Type = 16 + Response_GET_MAINTENANCE_SCHEDULE Response_Type = 17 + Response_GET_QUOTA Response_Type = 18 +) + +var Response_Type_name = map[int32]string{ + 0: "UNKNOWN", + 1: "GET_HEALTH", + 2: "GET_FLAGS", + 3: "GET_VERSION", + 4: "GET_METRICS", + 5: "GET_LOGGING_LEVEL", + 6: "LIST_FILES", + 7: "READ_FILE", + 8: "GET_STATE", + 9: "GET_AGENTS", + 10: "GET_FRAMEWORKS", + 11: "GET_EXECUTORS", + 12: "GET_TASKS", + 13: "GET_ROLES", + 14: "GET_WEIGHTS", + 15: "GET_MASTER", + 16: "GET_MAINTENANCE_STATUS", + 17: "GET_MAINTENANCE_SCHEDULE", + 18: "GET_QUOTA", +} +var Response_Type_value = map[string]int32{ + "UNKNOWN": 0, + "GET_HEALTH": 1, + "GET_FLAGS": 2, + "GET_VERSION": 3, + "GET_METRICS": 4, + "GET_LOGGING_LEVEL": 5, + "LIST_FILES": 6, + "READ_FILE": 7, + "GET_STATE": 8, + "GET_AGENTS": 9, + "GET_FRAMEWORKS": 10, + "GET_EXECUTORS": 11, + "GET_TASKS": 12, + "GET_ROLES": 13, + "GET_WEIGHTS": 14, + "GET_MASTER": 15, + "GET_MAINTENANCE_STATUS": 16, + "GET_MAINTENANCE_SCHEDULE": 17, + "GET_QUOTA": 18, +} + +func (x Response_Type) Enum() *Response_Type { + p := new(Response_Type) + *p = x + return p +} +func (x Response_Type) MarshalJSON() ([]byte, error) { + return proto.MarshalJSONEnum(Response_Type_name, int32(x)) +} +func (x *Response_Type) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(Response_Type_value, data, "Response_Type") + if err != nil { + return err + } + *x = Response_Type(value) + return nil +} +func (Response_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 0} } + +type Event_Type int32 + +const ( + Event_UNKNOWN Event_Type = 0 + Event_SUBSCRIBED Event_Type = 1 + Event_TASK_ADDED Event_Type = 2 + Event_TASK_UPDATED Event_Type = 3 + Event_AGENT_ADDED Event_Type = 4 + Event_AGENT_REMOVED Event_Type = 5 +) + +var Event_Type_name = map[int32]string{ + 0: "UNKNOWN", + 1: "SUBSCRIBED", + 2: "TASK_ADDED", + 3: "TASK_UPDATED", + 4: "AGENT_ADDED", + 5: "AGENT_REMOVED", +} +var Event_Type_value = map[string]int32{ + "UNKNOWN": 0, + "SUBSCRIBED": 1, + "TASK_ADDED": 2, + "TASK_UPDATED": 3, + "AGENT_ADDED": 4, + "AGENT_REMOVED": 5, +} + +func (x Event_Type) Enum() *Event_Type { + p := new(Event_Type) + *p = x + return p +} +func (x Event_Type) MarshalJSON() ([]byte, error) { + return proto.MarshalJSONEnum(Event_Type_name, int32(x)) +} +func (x *Event_Type) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(Event_Type_value, data, "Event_Type") + if err != nil { + return err + } + *x = Event_Type(value) + return nil +} +func (Event_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorMaster, []int{2, 0} } + +// * +// Calls that can be sent to the v1 master API. +// +// A call is described using the standard protocol buffer "union" +// trick, see +// https://developers.google.com/protocol-buffers/docs/techniques#union. +type Call struct { + Type Call_Type `protobuf:"varint,1,opt,name=type,enum=mesos.master.Call_Type" json:"type"` + GetMetrics *Call_GetMetrics `protobuf:"bytes,2,opt,name=get_metrics,json=getMetrics" json:"get_metrics,omitempty"` + SetLoggingLevel *Call_SetLoggingLevel `protobuf:"bytes,3,opt,name=set_logging_level,json=setLoggingLevel" json:"set_logging_level,omitempty"` + ListFiles *Call_ListFiles `protobuf:"bytes,4,opt,name=list_files,json=listFiles" json:"list_files,omitempty"` + ReadFile *Call_ReadFile `protobuf:"bytes,5,opt,name=read_file,json=readFile" json:"read_file,omitempty"` + UpdateWeights *Call_UpdateWeights `protobuf:"bytes,6,opt,name=update_weights,json=updateWeights" json:"update_weights,omitempty"` + ReserveResources *Call_ReserveResources `protobuf:"bytes,7,opt,name=reserve_resources,json=reserveResources" json:"reserve_resources,omitempty"` + UnreserveResources *Call_UnreserveResources `protobuf:"bytes,8,opt,name=unreserve_resources,json=unreserveResources" json:"unreserve_resources,omitempty"` + CreateVolumes *Call_CreateVolumes `protobuf:"bytes,9,opt,name=create_volumes,json=createVolumes" json:"create_volumes,omitempty"` + DestroyVolumes *Call_DestroyVolumes `protobuf:"bytes,10,opt,name=destroy_volumes,json=destroyVolumes" json:"destroy_volumes,omitempty"` + UpdateMaintenanceSchedule *Call_UpdateMaintenanceSchedule `protobuf:"bytes,11,opt,name=update_maintenance_schedule,json=updateMaintenanceSchedule" json:"update_maintenance_schedule,omitempty"` + StartMaintenance *Call_StartMaintenance `protobuf:"bytes,12,opt,name=start_maintenance,json=startMaintenance" json:"start_maintenance,omitempty"` + StopMaintenance *Call_StopMaintenance `protobuf:"bytes,13,opt,name=stop_maintenance,json=stopMaintenance" json:"stop_maintenance,omitempty"` + SetQuota *Call_SetQuota `protobuf:"bytes,14,opt,name=set_quota,json=setQuota" json:"set_quota,omitempty"` + RemoveQuota *Call_RemoveQuota `protobuf:"bytes,15,opt,name=remove_quota,json=removeQuota" json:"remove_quota,omitempty"` +} + +func (m *Call) Reset() { *m = Call{} } +func (*Call) ProtoMessage() {} +func (*Call) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0} } + +func (m *Call) GetType() Call_Type { + if m != nil { + return m.Type + } + return Call_UNKNOWN +} + +func (m *Call) GetGetMetrics() *Call_GetMetrics { + if m != nil { + return m.GetMetrics + } + return nil +} + +func (m *Call) GetSetLoggingLevel() *Call_SetLoggingLevel { + if m != nil { + return m.SetLoggingLevel + } + return nil +} + +func (m *Call) GetListFiles() *Call_ListFiles { + if m != nil { + return m.ListFiles + } + return nil +} + +func (m *Call) GetReadFile() *Call_ReadFile { + if m != nil { + return m.ReadFile + } + return nil +} + +func (m *Call) GetUpdateWeights() *Call_UpdateWeights { + if m != nil { + return m.UpdateWeights + } + return nil +} + +func (m *Call) GetReserveResources() *Call_ReserveResources { + if m != nil { + return m.ReserveResources + } + return nil +} + +func (m *Call) GetUnreserveResources() *Call_UnreserveResources { + if m != nil { + return m.UnreserveResources + } + return nil +} + +func (m *Call) GetCreateVolumes() *Call_CreateVolumes { + if m != nil { + return m.CreateVolumes + } + return nil +} + +func (m *Call) GetDestroyVolumes() *Call_DestroyVolumes { + if m != nil { + return m.DestroyVolumes + } + return nil +} + +func (m *Call) GetUpdateMaintenanceSchedule() *Call_UpdateMaintenanceSchedule { + if m != nil { + return m.UpdateMaintenanceSchedule + } + return nil +} + +func (m *Call) GetStartMaintenance() *Call_StartMaintenance { + if m != nil { + return m.StartMaintenance + } + return nil +} + +func (m *Call) GetStopMaintenance() *Call_StopMaintenance { + if m != nil { + return m.StopMaintenance + } + return nil +} + +func (m *Call) GetSetQuota() *Call_SetQuota { + if m != nil { + return m.SetQuota + } + return nil +} + +func (m *Call) GetRemoveQuota() *Call_RemoveQuota { + if m != nil { + return m.RemoveQuota + } + return nil +} + +// Provides a snapshot of the current metrics tracked by the master. +type Call_GetMetrics struct { + // If set, `timeout` would be used to determines the maximum amount of time + // the API will take to respond. If the timeout is exceeded, some metrics + // may not be included in the response. + Timeout *mesos.DurationInfo `protobuf:"bytes,1,opt,name=timeout" json:"timeout,omitempty"` +} + +func (m *Call_GetMetrics) Reset() { *m = Call_GetMetrics{} } +func (*Call_GetMetrics) ProtoMessage() {} +func (*Call_GetMetrics) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 0} } + +func (m *Call_GetMetrics) GetTimeout() *mesos.DurationInfo { + if m != nil { + return m.Timeout + } + return nil +} + +// Sets the logging verbosity level for a specified duration. Mesos uses +// [glog](https://github.com/google/glog) for logging. The library only uses +// verbose logging which means nothing will be output unless the verbosity +// level is set (by default it's 0, libprocess uses levels 1, 2, and 3). +type Call_SetLoggingLevel struct { + // The verbosity level. + Level uint32 `protobuf:"varint,1,req,name=level" json:"level"` + // The duration to keep verbosity level toggled. After this duration, the + // verbosity level of log would revert to the original level. + Duration mesos.DurationInfo `protobuf:"bytes,2,req,name=duration" json:"duration"` +} + +func (m *Call_SetLoggingLevel) Reset() { *m = Call_SetLoggingLevel{} } +func (*Call_SetLoggingLevel) ProtoMessage() {} +func (*Call_SetLoggingLevel) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 1} } + +func (m *Call_SetLoggingLevel) GetLevel() uint32 { + if m != nil { + return m.Level + } + return 0 +} + +func (m *Call_SetLoggingLevel) GetDuration() mesos.DurationInfo { + if m != nil { + return m.Duration + } + return mesos.DurationInfo{} +} + +// Provides the file listing for a directory. +type Call_ListFiles struct { + Path string `protobuf:"bytes,1,req,name=path" json:"path"` +} + +func (m *Call_ListFiles) Reset() { *m = Call_ListFiles{} } +func (*Call_ListFiles) ProtoMessage() {} +func (*Call_ListFiles) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 2} } + +func (m *Call_ListFiles) GetPath() string { + if m != nil { + return m.Path + } + return "" +} + +// Reads data from a file. +type Call_ReadFile struct { + // The path of file. + Path string `protobuf:"bytes,1,req,name=path" json:"path"` + // Initial offset in file to start reading from. + Offset uint64 `protobuf:"varint,2,req,name=offset" json:"offset"` + // The maximum number of bytes to read. The read length is capped at 16 + // memory pages. + Length *uint64 `protobuf:"varint,3,opt,name=length" json:"length,omitempty"` +} + +func (m *Call_ReadFile) Reset() { *m = Call_ReadFile{} } +func (*Call_ReadFile) ProtoMessage() {} +func (*Call_ReadFile) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 3} } + +func (m *Call_ReadFile) GetPath() string { + if m != nil { + return m.Path + } + return "" +} + +func (m *Call_ReadFile) GetOffset() uint64 { + if m != nil { + return m.Offset + } + return 0 +} + +func (m *Call_ReadFile) GetLength() uint64 { + if m != nil && m.Length != nil { + return *m.Length + } + return 0 +} + +type Call_UpdateWeights struct { + WeightInfos []mesos.WeightInfo `protobuf:"bytes,1,rep,name=weight_infos,json=weightInfos" json:"weight_infos"` +} + +func (m *Call_UpdateWeights) Reset() { *m = Call_UpdateWeights{} } +func (*Call_UpdateWeights) ProtoMessage() {} +func (*Call_UpdateWeights) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 4} } + +func (m *Call_UpdateWeights) GetWeightInfos() []mesos.WeightInfo { + if m != nil { + return m.WeightInfos + } + return nil +} + +// Reserve resources dynamically on a specific agent. +type Call_ReserveResources struct { + AgentID mesos.AgentID `protobuf:"bytes,1,req,name=agent_id,json=agentId" json:"agent_id"` + Resources []mesos.Resource `protobuf:"bytes,2,rep,name=resources" json:"resources"` +} + +func (m *Call_ReserveResources) Reset() { *m = Call_ReserveResources{} } +func (*Call_ReserveResources) ProtoMessage() {} +func (*Call_ReserveResources) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 5} } + +func (m *Call_ReserveResources) GetAgentID() mesos.AgentID { + if m != nil { + return m.AgentID + } + return mesos.AgentID{} +} + +func (m *Call_ReserveResources) GetResources() []mesos.Resource { + if m != nil { + return m.Resources + } + return nil +} + +// Unreserve resources dynamically on a specific agent. +type Call_UnreserveResources struct { + AgentID mesos.AgentID `protobuf:"bytes,1,req,name=agent_id,json=agentId" json:"agent_id"` + Resources []mesos.Resource `protobuf:"bytes,2,rep,name=resources" json:"resources"` +} + +func (m *Call_UnreserveResources) Reset() { *m = Call_UnreserveResources{} } +func (*Call_UnreserveResources) ProtoMessage() {} +func (*Call_UnreserveResources) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 6} } + +func (m *Call_UnreserveResources) GetAgentID() mesos.AgentID { + if m != nil { + return m.AgentID + } + return mesos.AgentID{} +} + +func (m *Call_UnreserveResources) GetResources() []mesos.Resource { + if m != nil { + return m.Resources + } + return nil +} + +// Create persistent volumes on reserved resources. The request is forwarded +// asynchronously to the Mesos agent where the reserved resources are located. +// That asynchronous message may not be delivered or creating the volumes at +// the agent might fail. Volume creation can be verified by sending a +// `GET_VOLUMES` call. +type Call_CreateVolumes struct { + AgentID mesos.AgentID `protobuf:"bytes,1,req,name=agent_id,json=agentId" json:"agent_id"` + Volumes []mesos.Resource `protobuf:"bytes,2,rep,name=volumes" json:"volumes"` +} + +func (m *Call_CreateVolumes) Reset() { *m = Call_CreateVolumes{} } +func (*Call_CreateVolumes) ProtoMessage() {} +func (*Call_CreateVolumes) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 7} } + +func (m *Call_CreateVolumes) GetAgentID() mesos.AgentID { + if m != nil { + return m.AgentID + } + return mesos.AgentID{} +} + +func (m *Call_CreateVolumes) GetVolumes() []mesos.Resource { + if m != nil { + return m.Volumes + } + return nil +} + +// Destroy persistent volumes. The request is forwarded asynchronously to the +// Mesos agent where the reserved resources are located. That asynchronous +// message may not be delivered or destroying the volumes at the agent might +// fail. Volume deletion can be verified by sending a `GET_VOLUMES` call. +type Call_DestroyVolumes struct { + AgentID mesos.AgentID `protobuf:"bytes,1,req,name=agent_id,json=agentId" json:"agent_id"` + Volumes []mesos.Resource `protobuf:"bytes,2,rep,name=volumes" json:"volumes"` +} + +func (m *Call_DestroyVolumes) Reset() { *m = Call_DestroyVolumes{} } +func (*Call_DestroyVolumes) ProtoMessage() {} +func (*Call_DestroyVolumes) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 8} } + +func (m *Call_DestroyVolumes) GetAgentID() mesos.AgentID { + if m != nil { + return m.AgentID + } + return mesos.AgentID{} +} + +func (m *Call_DestroyVolumes) GetVolumes() []mesos.Resource { + if m != nil { + return m.Volumes + } + return nil +} + +// Updates the cluster's maintenance schedule. +type Call_UpdateMaintenanceSchedule struct { + Schedule mesos_maintenance.Schedule `protobuf:"bytes,1,req,name=schedule" json:"schedule"` +} + +func (m *Call_UpdateMaintenanceSchedule) Reset() { *m = Call_UpdateMaintenanceSchedule{} } +func (*Call_UpdateMaintenanceSchedule) ProtoMessage() {} +func (*Call_UpdateMaintenanceSchedule) Descriptor() ([]byte, []int) { + return fileDescriptorMaster, []int{0, 9} +} + +func (m *Call_UpdateMaintenanceSchedule) GetSchedule() mesos_maintenance.Schedule { + if m != nil { + return m.Schedule + } + return mesos_maintenance.Schedule{} +} + +// Starts the maintenance of the cluster, this would bring a set of machines +// down. +type Call_StartMaintenance struct { + Machines []mesos.MachineID `protobuf:"bytes,1,rep,name=machines" json:"machines"` +} + +func (m *Call_StartMaintenance) Reset() { *m = Call_StartMaintenance{} } +func (*Call_StartMaintenance) ProtoMessage() {} +func (*Call_StartMaintenance) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 10} } + +func (m *Call_StartMaintenance) GetMachines() []mesos.MachineID { + if m != nil { + return m.Machines + } + return nil +} + +// Stops the maintenance of the cluster, this would bring a set of machines +// back up. +type Call_StopMaintenance struct { + Machines []mesos.MachineID `protobuf:"bytes,1,rep,name=machines" json:"machines"` +} + +func (m *Call_StopMaintenance) Reset() { *m = Call_StopMaintenance{} } +func (*Call_StopMaintenance) ProtoMessage() {} +func (*Call_StopMaintenance) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 11} } + +func (m *Call_StopMaintenance) GetMachines() []mesos.MachineID { + if m != nil { + return m.Machines + } + return nil +} + +// Sets the quota for resources to be used by a particular role. +type Call_SetQuota struct { + QuotaRequest mesos_quota.QuotaRequest `protobuf:"bytes,1,req,name=quota_request,json=quotaRequest" json:"quota_request"` +} + +func (m *Call_SetQuota) Reset() { *m = Call_SetQuota{} } +func (*Call_SetQuota) ProtoMessage() {} +func (*Call_SetQuota) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 12} } + +func (m *Call_SetQuota) GetQuotaRequest() mesos_quota.QuotaRequest { + if m != nil { + return m.QuotaRequest + } + return mesos_quota.QuotaRequest{} +} + +type Call_RemoveQuota struct { + Role string `protobuf:"bytes,1,req,name=role" json:"role"` +} + +func (m *Call_RemoveQuota) Reset() { *m = Call_RemoveQuota{} } +func (*Call_RemoveQuota) ProtoMessage() {} +func (*Call_RemoveQuota) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{0, 13} } + +func (m *Call_RemoveQuota) GetRole() string { + if m != nil { + return m.Role + } + return "" +} + +// * +// Synchronous responses for all calls (except Call::SUBSCRIBE) made to +// the v1 master API. +type Response struct { + Type Response_Type `protobuf:"varint,1,opt,name=type,enum=mesos.master.Response_Type" json:"type"` + GetHealth *Response_GetHealth `protobuf:"bytes,2,opt,name=get_health,json=getHealth" json:"get_health,omitempty"` + GetFlags *Response_GetFlags `protobuf:"bytes,3,opt,name=get_flags,json=getFlags" json:"get_flags,omitempty"` + GetVersion *Response_GetVersion `protobuf:"bytes,4,opt,name=get_version,json=getVersion" json:"get_version,omitempty"` + GetMetrics *Response_GetMetrics `protobuf:"bytes,5,opt,name=get_metrics,json=getMetrics" json:"get_metrics,omitempty"` + GetLoggingLevel *Response_GetLoggingLevel `protobuf:"bytes,6,opt,name=get_logging_level,json=getLoggingLevel" json:"get_logging_level,omitempty"` + ListFiles *Response_ListFiles `protobuf:"bytes,7,opt,name=list_files,json=listFiles" json:"list_files,omitempty"` + ReadFile *Response_ReadFile `protobuf:"bytes,8,opt,name=read_file,json=readFile" json:"read_file,omitempty"` + GetState *Response_GetState `protobuf:"bytes,9,opt,name=get_state,json=getState" json:"get_state,omitempty"` + GetAgents *Response_GetAgents `protobuf:"bytes,10,opt,name=get_agents,json=getAgents" json:"get_agents,omitempty"` + GetFrameworks *Response_GetFrameworks `protobuf:"bytes,11,opt,name=get_frameworks,json=getFrameworks" json:"get_frameworks,omitempty"` + GetExecutors *Response_GetExecutors `protobuf:"bytes,12,opt,name=get_executors,json=getExecutors" json:"get_executors,omitempty"` + GetTasks *Response_GetTasks `protobuf:"bytes,13,opt,name=get_tasks,json=getTasks" json:"get_tasks,omitempty"` + GetRoles *Response_GetRoles `protobuf:"bytes,14,opt,name=get_roles,json=getRoles" json:"get_roles,omitempty"` + GetWeights *Response_GetWeights `protobuf:"bytes,15,opt,name=get_weights,json=getWeights" json:"get_weights,omitempty"` + GetMaster *Response_GetMaster `protobuf:"bytes,16,opt,name=get_master,json=getMaster" json:"get_master,omitempty"` + GetMaintenanceStatus *Response_GetMaintenanceStatus `protobuf:"bytes,17,opt,name=get_maintenance_status,json=getMaintenanceStatus" json:"get_maintenance_status,omitempty"` + GetMaintenanceSchedule *Response_GetMaintenanceSchedule `protobuf:"bytes,18,opt,name=get_maintenance_schedule,json=getMaintenanceSchedule" json:"get_maintenance_schedule,omitempty"` + GetQuota *Response_GetQuota `protobuf:"bytes,19,opt,name=get_quota,json=getQuota" json:"get_quota,omitempty"` +} + +func (m *Response) Reset() { *m = Response{} } +func (*Response) ProtoMessage() {} +func (*Response) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1} } + +func (m *Response) GetType() Response_Type { + if m != nil { + return m.Type + } + return Response_UNKNOWN +} + +func (m *Response) GetGetHealth() *Response_GetHealth { + if m != nil { + return m.GetHealth + } + return nil +} + +func (m *Response) GetGetFlags() *Response_GetFlags { + if m != nil { + return m.GetFlags + } + return nil +} + +func (m *Response) GetGetVersion() *Response_GetVersion { + if m != nil { + return m.GetVersion + } + return nil +} + +func (m *Response) GetGetMetrics() *Response_GetMetrics { + if m != nil { + return m.GetMetrics + } + return nil +} + +func (m *Response) GetGetLoggingLevel() *Response_GetLoggingLevel { + if m != nil { + return m.GetLoggingLevel + } + return nil +} + +func (m *Response) GetListFiles() *Response_ListFiles { + if m != nil { + return m.ListFiles + } + return nil +} + +func (m *Response) GetReadFile() *Response_ReadFile { + if m != nil { + return m.ReadFile + } + return nil +} + +func (m *Response) GetGetState() *Response_GetState { + if m != nil { + return m.GetState + } + return nil +} + +func (m *Response) GetGetAgents() *Response_GetAgents { + if m != nil { + return m.GetAgents + } + return nil +} + +func (m *Response) GetGetFrameworks() *Response_GetFrameworks { + if m != nil { + return m.GetFrameworks + } + return nil +} + +func (m *Response) GetGetExecutors() *Response_GetExecutors { + if m != nil { + return m.GetExecutors + } + return nil +} + +func (m *Response) GetGetTasks() *Response_GetTasks { + if m != nil { + return m.GetTasks + } + return nil +} + +func (m *Response) GetGetRoles() *Response_GetRoles { + if m != nil { + return m.GetRoles + } + return nil +} + +func (m *Response) GetGetWeights() *Response_GetWeights { + if m != nil { + return m.GetWeights + } + return nil +} + +func (m *Response) GetGetMaster() *Response_GetMaster { + if m != nil { + return m.GetMaster + } + return nil +} + +func (m *Response) GetGetMaintenanceStatus() *Response_GetMaintenanceStatus { + if m != nil { + return m.GetMaintenanceStatus + } + return nil +} + +func (m *Response) GetGetMaintenanceSchedule() *Response_GetMaintenanceSchedule { + if m != nil { + return m.GetMaintenanceSchedule + } + return nil +} + +func (m *Response) GetGetQuota() *Response_GetQuota { + if m != nil { + return m.GetQuota + } + return nil +} + +// `healthy` would be true if the master is healthy. Delayed responses are +// also indicative of the poor health of the master. +type Response_GetHealth struct { + Healthy bool `protobuf:"varint,1,req,name=healthy" json:"healthy"` +} + +func (m *Response_GetHealth) Reset() { *m = Response_GetHealth{} } +func (*Response_GetHealth) ProtoMessage() {} +func (*Response_GetHealth) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 0} } + +func (m *Response_GetHealth) GetHealthy() bool { + if m != nil { + return m.Healthy + } + return false +} + +// Contains the flag configuration of the master. +type Response_GetFlags struct { + Flags []mesos.Flag `protobuf:"bytes,1,rep,name=flags" json:"flags"` +} + +func (m *Response_GetFlags) Reset() { *m = Response_GetFlags{} } +func (*Response_GetFlags) ProtoMessage() {} +func (*Response_GetFlags) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 1} } + +func (m *Response_GetFlags) GetFlags() []mesos.Flag { + if m != nil { + return m.Flags + } + return nil +} + +// Contains the version information of the master. +type Response_GetVersion struct { + VersionInfo mesos.VersionInfo `protobuf:"bytes,1,req,name=version_info,json=versionInfo" json:"version_info"` +} + +func (m *Response_GetVersion) Reset() { *m = Response_GetVersion{} } +func (*Response_GetVersion) ProtoMessage() {} +func (*Response_GetVersion) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 2} } + +func (m *Response_GetVersion) GetVersionInfo() mesos.VersionInfo { + if m != nil { + return m.VersionInfo + } + return mesos.VersionInfo{} +} + +// Contains a snapshot of the current metrics. +type Response_GetMetrics struct { + Metrics []mesos.Metric `protobuf:"bytes,1,rep,name=metrics" json:"metrics"` +} + +func (m *Response_GetMetrics) Reset() { *m = Response_GetMetrics{} } +func (*Response_GetMetrics) ProtoMessage() {} +func (*Response_GetMetrics) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 3} } + +func (m *Response_GetMetrics) GetMetrics() []mesos.Metric { + if m != nil { + return m.Metrics + } + return nil +} + +// Contains the logging level of the master. +type Response_GetLoggingLevel struct { + Level uint32 `protobuf:"varint,1,req,name=level" json:"level"` +} + +func (m *Response_GetLoggingLevel) Reset() { *m = Response_GetLoggingLevel{} } +func (*Response_GetLoggingLevel) ProtoMessage() {} +func (*Response_GetLoggingLevel) Descriptor() ([]byte, []int) { + return fileDescriptorMaster, []int{1, 4} +} + +func (m *Response_GetLoggingLevel) GetLevel() uint32 { + if m != nil { + return m.Level + } + return 0 +} + +// Contains the file listing(similar to `ls -l`) for a directory. +type Response_ListFiles struct { + FileInfos []mesos.FileInfo `protobuf:"bytes,1,rep,name=file_infos,json=fileInfos" json:"file_infos"` +} + +func (m *Response_ListFiles) Reset() { *m = Response_ListFiles{} } +func (*Response_ListFiles) ProtoMessage() {} +func (*Response_ListFiles) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 5} } + +func (m *Response_ListFiles) GetFileInfos() []mesos.FileInfo { + if m != nil { + return m.FileInfos + } + return nil +} + +// Contains the file data. +type Response_ReadFile struct { + // The size of file (in bytes). + Size uint64 `protobuf:"varint,1,req,name=size" json:"size"` + Data []byte `protobuf:"bytes,2,req,name=data" json:"data,omitempty"` +} + +func (m *Response_ReadFile) Reset() { *m = Response_ReadFile{} } +func (*Response_ReadFile) ProtoMessage() {} +func (*Response_ReadFile) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 6} } + +func (m *Response_ReadFile) GetSize() uint64 { + if m != nil { + return m.Size + } + return 0 +} + +func (m *Response_ReadFile) GetData() []byte { + if m != nil { + return m.Data + } + return nil +} + +// Contains full state of the master i.e. information about the tasks, +// agents, frameworks and executors running in the cluster. +type Response_GetState struct { + GetTasks *Response_GetTasks `protobuf:"bytes,1,opt,name=get_tasks,json=getTasks" json:"get_tasks,omitempty"` + GetExecutors *Response_GetExecutors `protobuf:"bytes,2,opt,name=get_executors,json=getExecutors" json:"get_executors,omitempty"` + GetFrameworks *Response_GetFrameworks `protobuf:"bytes,3,opt,name=get_frameworks,json=getFrameworks" json:"get_frameworks,omitempty"` + GetAgents *Response_GetAgents `protobuf:"bytes,4,opt,name=get_agents,json=getAgents" json:"get_agents,omitempty"` +} + +func (m *Response_GetState) Reset() { *m = Response_GetState{} } +func (*Response_GetState) ProtoMessage() {} +func (*Response_GetState) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 7} } + +func (m *Response_GetState) GetGetTasks() *Response_GetTasks { + if m != nil { + return m.GetTasks + } + return nil +} + +func (m *Response_GetState) GetGetExecutors() *Response_GetExecutors { + if m != nil { + return m.GetExecutors + } + return nil +} + +func (m *Response_GetState) GetGetFrameworks() *Response_GetFrameworks { + if m != nil { + return m.GetFrameworks + } + return nil +} + +func (m *Response_GetState) GetGetAgents() *Response_GetAgents { + if m != nil { + return m.GetAgents + } + return nil +} + +type Response_GetAgents struct { + // Registered agents. + Agents []Response_GetAgents_Agent `protobuf:"bytes,1,rep,name=agents" json:"agents"` + // Agents which are recovered from registry but not reregistered yet. + RecoveredAgents []mesos.AgentInfo `protobuf:"bytes,2,rep,name=recovered_agents,json=recoveredAgents" json:"recovered_agents"` +} + +func (m *Response_GetAgents) Reset() { *m = Response_GetAgents{} } +func (*Response_GetAgents) ProtoMessage() {} +func (*Response_GetAgents) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 8} } + +func (m *Response_GetAgents) GetAgents() []Response_GetAgents_Agent { + if m != nil { + return m.Agents + } + return nil +} + +func (m *Response_GetAgents) GetRecoveredAgents() []mesos.AgentInfo { + if m != nil { + return m.RecoveredAgents + } + return nil +} + +type Response_GetAgents_Agent struct { + AgentInfo mesos.AgentInfo `protobuf:"bytes,1,req,name=agent_info,json=agentInfo" json:"agent_info"` + Active bool `protobuf:"varint,2,req,name=active" json:"active"` + Version string `protobuf:"bytes,3,req,name=version" json:"version"` + PID *string `protobuf:"bytes,4,opt,name=pid" json:"pid,omitempty"` + RegisteredTime *mesos.TimeInfo `protobuf:"bytes,5,opt,name=registered_time,json=registeredTime" json:"registered_time,omitempty"` + ReregisteredTime *mesos.TimeInfo `protobuf:"bytes,6,opt,name=reregistered_time,json=reregisteredTime" json:"reregistered_time,omitempty"` + // Total resources (including oversubscribed resources) the agent + // provides. + TotalResources []mesos.Resource `protobuf:"bytes,7,rep,name=total_resources,json=totalResources" json:"total_resources"` + AllocatedResources []mesos.Resource `protobuf:"bytes,8,rep,name=allocated_resources,json=allocatedResources" json:"allocated_resources"` + OfferedResources []mesos.Resource `protobuf:"bytes,9,rep,name=offered_resources,json=offeredResources" json:"offered_resources"` +} + +func (m *Response_GetAgents_Agent) Reset() { *m = Response_GetAgents_Agent{} } +func (*Response_GetAgents_Agent) ProtoMessage() {} +func (*Response_GetAgents_Agent) Descriptor() ([]byte, []int) { + return fileDescriptorMaster, []int{1, 8, 0} +} + +func (m *Response_GetAgents_Agent) GetAgentInfo() mesos.AgentInfo { + if m != nil { + return m.AgentInfo + } + return mesos.AgentInfo{} +} + +func (m *Response_GetAgents_Agent) GetActive() bool { + if m != nil { + return m.Active + } + return false +} + +func (m *Response_GetAgents_Agent) GetVersion() string { + if m != nil { + return m.Version + } + return "" +} + +func (m *Response_GetAgents_Agent) GetPID() string { + if m != nil && m.PID != nil { + return *m.PID + } + return "" +} + +func (m *Response_GetAgents_Agent) GetRegisteredTime() *mesos.TimeInfo { + if m != nil { + return m.RegisteredTime + } + return nil +} + +func (m *Response_GetAgents_Agent) GetReregisteredTime() *mesos.TimeInfo { + if m != nil { + return m.ReregisteredTime + } + return nil +} + +func (m *Response_GetAgents_Agent) GetTotalResources() []mesos.Resource { + if m != nil { + return m.TotalResources + } + return nil +} + +func (m *Response_GetAgents_Agent) GetAllocatedResources() []mesos.Resource { + if m != nil { + return m.AllocatedResources + } + return nil +} + +func (m *Response_GetAgents_Agent) GetOfferedResources() []mesos.Resource { + if m != nil { + return m.OfferedResources + } + return nil +} + +// Information about all the frameworks known to the master at the current +// time. Note that there might be frameworks unknown to the master running +// on partitioned or unsubscribed agents. +type Response_GetFrameworks struct { + // Frameworks that have subscribed with the master. Note that this includes + // frameworks that are disconnected and in the process of re-subscribing. + Frameworks []Response_GetFrameworks_Framework `protobuf:"bytes,1,rep,name=frameworks" json:"frameworks"` + // Frameworks that have been teared down. + CompletedFrameworks []Response_GetFrameworks_Framework `protobuf:"bytes,2,rep,name=completed_frameworks,json=completedFrameworks" json:"completed_frameworks"` + // This field previously contained frameworks that previously + // subscribed but haven't yet re-subscribed after a master failover. + // As of Mesos 1.2, this field will always be empty; recovered + // frameworks are now reported in the `frameworks` list with the + // `recovered` field set to true. + // + // TODO(neilc): Remove this field after a deprecation cycle starting + // in Mesos 1.2. + RecoveredFrameworks []mesos.FrameworkInfo `protobuf:"bytes,3,rep,name=recovered_frameworks,json=recoveredFrameworks" json:"recovered_frameworks"` +} + +func (m *Response_GetFrameworks) Reset() { *m = Response_GetFrameworks{} } +func (*Response_GetFrameworks) ProtoMessage() {} +func (*Response_GetFrameworks) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 9} } + +func (m *Response_GetFrameworks) GetFrameworks() []Response_GetFrameworks_Framework { + if m != nil { + return m.Frameworks + } + return nil +} + +func (m *Response_GetFrameworks) GetCompletedFrameworks() []Response_GetFrameworks_Framework { + if m != nil { + return m.CompletedFrameworks + } + return nil +} + +func (m *Response_GetFrameworks) GetRecoveredFrameworks() []mesos.FrameworkInfo { + if m != nil { + return m.RecoveredFrameworks + } + return nil +} + +type Response_GetFrameworks_Framework struct { + FrameworkInfo mesos.FrameworkInfo `protobuf:"bytes,1,req,name=framework_info,json=frameworkInfo" json:"framework_info"` + Active bool `protobuf:"varint,2,req,name=active" json:"active"` + Connected bool `protobuf:"varint,3,req,name=connected" json:"connected"` + // If true, this framework was previously subscribed but hasn't + // yet re-subscribed after a master failover. Recovered frameworks + // are only reported if one or more agents running a task or + // executor for the framework have re-registered after master + // failover. + Recovered bool `protobuf:"varint,11,req,name=recovered" json:"recovered"` + RegisteredTime *mesos.TimeInfo `protobuf:"bytes,4,opt,name=registered_time,json=registeredTime" json:"registered_time,omitempty"` + ReregisteredTime *mesos.TimeInfo `protobuf:"bytes,5,opt,name=reregistered_time,json=reregisteredTime" json:"reregistered_time,omitempty"` + UnregisteredTime *mesos.TimeInfo `protobuf:"bytes,6,opt,name=unregistered_time,json=unregisteredTime" json:"unregistered_time,omitempty"` + Offers []mesos.Offer `protobuf:"bytes,7,rep,name=offers" json:"offers"` + InverseOffers []mesos.InverseOffer `protobuf:"bytes,8,rep,name=inverse_offers,json=inverseOffers" json:"inverse_offers"` + AllocatedResources []mesos.Resource `protobuf:"bytes,9,rep,name=allocated_resources,json=allocatedResources" json:"allocated_resources"` + OfferedResources []mesos.Resource `protobuf:"bytes,10,rep,name=offered_resources,json=offeredResources" json:"offered_resources"` +} + +func (m *Response_GetFrameworks_Framework) Reset() { *m = Response_GetFrameworks_Framework{} } +func (*Response_GetFrameworks_Framework) ProtoMessage() {} +func (*Response_GetFrameworks_Framework) Descriptor() ([]byte, []int) { + return fileDescriptorMaster, []int{1, 9, 0} +} + +func (m *Response_GetFrameworks_Framework) GetFrameworkInfo() mesos.FrameworkInfo { + if m != nil { + return m.FrameworkInfo + } + return mesos.FrameworkInfo{} +} + +func (m *Response_GetFrameworks_Framework) GetActive() bool { + if m != nil { + return m.Active + } + return false +} + +func (m *Response_GetFrameworks_Framework) GetConnected() bool { + if m != nil { + return m.Connected + } + return false +} + +func (m *Response_GetFrameworks_Framework) GetRecovered() bool { + if m != nil { + return m.Recovered + } + return false +} + +func (m *Response_GetFrameworks_Framework) GetRegisteredTime() *mesos.TimeInfo { + if m != nil { + return m.RegisteredTime + } + return nil +} + +func (m *Response_GetFrameworks_Framework) GetReregisteredTime() *mesos.TimeInfo { + if m != nil { + return m.ReregisteredTime + } + return nil +} + +func (m *Response_GetFrameworks_Framework) GetUnregisteredTime() *mesos.TimeInfo { + if m != nil { + return m.UnregisteredTime + } + return nil +} + +func (m *Response_GetFrameworks_Framework) GetOffers() []mesos.Offer { + if m != nil { + return m.Offers + } + return nil +} + +func (m *Response_GetFrameworks_Framework) GetInverseOffers() []mesos.InverseOffer { + if m != nil { + return m.InverseOffers + } + return nil +} + +func (m *Response_GetFrameworks_Framework) GetAllocatedResources() []mesos.Resource { + if m != nil { + return m.AllocatedResources + } + return nil +} + +func (m *Response_GetFrameworks_Framework) GetOfferedResources() []mesos.Resource { + if m != nil { + return m.OfferedResources + } + return nil +} + +// Lists information about all the executors known to the master at the +// current time. Note that there might be executors unknown to the master +// running on partitioned or unsubscribed agents. +type Response_GetExecutors struct { + Executors []Response_GetExecutors_Executor `protobuf:"bytes,1,rep,name=executors" json:"executors"` + // As of Mesos 1.2, this field will always be empty. + // + // TODO(neilc): Remove this field after a deprecation cycle starting + // in Mesos 1.2. + OrphanExecutors []Response_GetExecutors_Executor `protobuf:"bytes,2,rep,name=orphan_executors,json=orphanExecutors" json:"orphan_executors"` +} + +func (m *Response_GetExecutors) Reset() { *m = Response_GetExecutors{} } +func (*Response_GetExecutors) ProtoMessage() {} +func (*Response_GetExecutors) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 10} } + +func (m *Response_GetExecutors) GetExecutors() []Response_GetExecutors_Executor { + if m != nil { + return m.Executors + } + return nil +} + +func (m *Response_GetExecutors) GetOrphanExecutors() []Response_GetExecutors_Executor { + if m != nil { + return m.OrphanExecutors + } + return nil +} + +type Response_GetExecutors_Executor struct { + ExecutorInfo mesos.ExecutorInfo `protobuf:"bytes,1,req,name=executor_info,json=executorInfo" json:"executor_info"` + AgentID mesos.AgentID `protobuf:"bytes,2,req,name=agent_id,json=agentId" json:"agent_id"` +} + +func (m *Response_GetExecutors_Executor) Reset() { *m = Response_GetExecutors_Executor{} } +func (*Response_GetExecutors_Executor) ProtoMessage() {} +func (*Response_GetExecutors_Executor) Descriptor() ([]byte, []int) { + return fileDescriptorMaster, []int{1, 10, 0} +} + +func (m *Response_GetExecutors_Executor) GetExecutorInfo() mesos.ExecutorInfo { + if m != nil { + return m.ExecutorInfo + } + return mesos.ExecutorInfo{} +} + +func (m *Response_GetExecutors_Executor) GetAgentID() mesos.AgentID { + if m != nil { + return m.AgentID + } + return mesos.AgentID{} +} + +// Lists information about all the tasks known to the master at the current +// time. Note that there might be tasks unknown to the master running on +// partitioned or unsubscribed agents. +type Response_GetTasks struct { + // Tasks that are enqueued on the master waiting (e.g., authorizing) + // to be launched. + PendingTasks []mesos.Task `protobuf:"bytes,1,rep,name=pending_tasks,json=pendingTasks" json:"pending_tasks"` + // Tasks that have been forwarded to the agent for launch. This + // includes tasks that are staging or running; it also includes + // tasks that have reached a terminal state but the terminal status + // update has not yet been acknowledged by the scheduler. + Tasks []mesos.Task `protobuf:"bytes,2,rep,name=tasks" json:"tasks"` + // Tasks that were running on agents that have become partitioned + // from the master. If/when the agent is no longer partitioned, + // tasks running on that agent will no longer be unreachable (they + // will either be running or completed). Note that the master only + // stores a limited number of unreachable tasks; information about + // unreachable tasks is also not preserved across master failover. + UnreachableTasks []mesos.Task `protobuf:"bytes,5,rep,name=unreachable_tasks,json=unreachableTasks" json:"unreachable_tasks"` + // Tasks that have reached terminal state and have all their updates + // acknowledged by the scheduler. + CompletedTasks []mesos.Task `protobuf:"bytes,3,rep,name=completed_tasks,json=completedTasks" json:"completed_tasks"` + // As of Mesos 1.2, this field will always be empty. + // + // TODO(neilc): Remove this field after a deprecation cycle starting + // in Mesos 1.2. + OrphanTasks []mesos.Task `protobuf:"bytes,4,rep,name=orphan_tasks,json=orphanTasks" json:"orphan_tasks"` +} + +func (m *Response_GetTasks) Reset() { *m = Response_GetTasks{} } +func (*Response_GetTasks) ProtoMessage() {} +func (*Response_GetTasks) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 11} } + +func (m *Response_GetTasks) GetPendingTasks() []mesos.Task { + if m != nil { + return m.PendingTasks + } + return nil +} + +func (m *Response_GetTasks) GetTasks() []mesos.Task { + if m != nil { + return m.Tasks + } + return nil +} + +func (m *Response_GetTasks) GetUnreachableTasks() []mesos.Task { + if m != nil { + return m.UnreachableTasks + } + return nil +} + +func (m *Response_GetTasks) GetCompletedTasks() []mesos.Task { + if m != nil { + return m.CompletedTasks + } + return nil +} + +func (m *Response_GetTasks) GetOrphanTasks() []mesos.Task { + if m != nil { + return m.OrphanTasks + } + return nil +} + +// Provides information about every role that is on the role whitelist (if +// enabled), has one or more registered frameworks or has a non-default weight +// or quota. +type Response_GetRoles struct { + Roles []mesos.Role `protobuf:"bytes,1,rep,name=roles" json:"roles"` +} + +func (m *Response_GetRoles) Reset() { *m = Response_GetRoles{} } +func (*Response_GetRoles) ProtoMessage() {} +func (*Response_GetRoles) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 12} } + +func (m *Response_GetRoles) GetRoles() []mesos.Role { + if m != nil { + return m.Roles + } + return nil +} + +// Provides the weight information about every role. +type Response_GetWeights struct { + WeightInfos []mesos.WeightInfo `protobuf:"bytes,1,rep,name=weight_infos,json=weightInfos" json:"weight_infos"` +} + +func (m *Response_GetWeights) Reset() { *m = Response_GetWeights{} } +func (*Response_GetWeights) ProtoMessage() {} +func (*Response_GetWeights) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 13} } + +func (m *Response_GetWeights) GetWeightInfos() []mesos.WeightInfo { + if m != nil { + return m.WeightInfos + } + return nil +} + +// Contains the master's information. +type Response_GetMaster struct { + MasterInfo *mesos.MasterInfo `protobuf:"bytes,1,opt,name=master_info,json=masterInfo" json:"master_info,omitempty"` +} + +func (m *Response_GetMaster) Reset() { *m = Response_GetMaster{} } +func (*Response_GetMaster) ProtoMessage() {} +func (*Response_GetMaster) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 14} } + +func (m *Response_GetMaster) GetMasterInfo() *mesos.MasterInfo { + if m != nil { + return m.MasterInfo + } + return nil +} + +// Contains the cluster's maintenance status. +type Response_GetMaintenanceStatus struct { + Status mesos_maintenance.ClusterStatus `protobuf:"bytes,1,req,name=status" json:"status"` +} + +func (m *Response_GetMaintenanceStatus) Reset() { *m = Response_GetMaintenanceStatus{} } +func (*Response_GetMaintenanceStatus) ProtoMessage() {} +func (*Response_GetMaintenanceStatus) Descriptor() ([]byte, []int) { + return fileDescriptorMaster, []int{1, 15} +} + +func (m *Response_GetMaintenanceStatus) GetStatus() mesos_maintenance.ClusterStatus { + if m != nil { + return m.Status + } + return mesos_maintenance.ClusterStatus{} +} + +// Contains the cluster's maintenance schedule. +type Response_GetMaintenanceSchedule struct { + Schedule mesos_maintenance.Schedule `protobuf:"bytes,1,req,name=schedule" json:"schedule"` +} + +func (m *Response_GetMaintenanceSchedule) Reset() { *m = Response_GetMaintenanceSchedule{} } +func (*Response_GetMaintenanceSchedule) ProtoMessage() {} +func (*Response_GetMaintenanceSchedule) Descriptor() ([]byte, []int) { + return fileDescriptorMaster, []int{1, 16} +} + +func (m *Response_GetMaintenanceSchedule) GetSchedule() mesos_maintenance.Schedule { + if m != nil { + return m.Schedule + } + return mesos_maintenance.Schedule{} +} + +// Contains the cluster's configured quotas. +type Response_GetQuota struct { + Status mesos_quota.QuotaStatus `protobuf:"bytes,1,req,name=status" json:"status"` +} + +func (m *Response_GetQuota) Reset() { *m = Response_GetQuota{} } +func (*Response_GetQuota) ProtoMessage() {} +func (*Response_GetQuota) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{1, 17} } + +func (m *Response_GetQuota) GetStatus() mesos_quota.QuotaStatus { + if m != nil { + return m.Status + } + return mesos_quota.QuotaStatus{} +} + +// * +// Streaming response to `Call::SUBSCRIBE` made to the master. +type Event struct { + Type Event_Type `protobuf:"varint,1,opt,name=type,enum=mesos.master.Event_Type" json:"type"` + Subscribed *Event_Subscribed `protobuf:"bytes,2,opt,name=subscribed" json:"subscribed,omitempty"` + TaskAdded *Event_TaskAdded `protobuf:"bytes,3,opt,name=task_added,json=taskAdded" json:"task_added,omitempty"` + TaskUpdated *Event_TaskUpdated `protobuf:"bytes,4,opt,name=task_updated,json=taskUpdated" json:"task_updated,omitempty"` + AgentAdded *Event_AgentAdded `protobuf:"bytes,5,opt,name=agent_added,json=agentAdded" json:"agent_added,omitempty"` + AgentRemoved *Event_AgentRemoved `protobuf:"bytes,6,opt,name=agent_removed,json=agentRemoved" json:"agent_removed,omitempty"` +} + +func (m *Event) Reset() { *m = Event{} } +func (*Event) ProtoMessage() {} +func (*Event) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{2} } + +func (m *Event) GetType() Event_Type { + if m != nil { + return m.Type + } + return Event_UNKNOWN +} + +func (m *Event) GetSubscribed() *Event_Subscribed { + if m != nil { + return m.Subscribed + } + return nil +} + +func (m *Event) GetTaskAdded() *Event_TaskAdded { + if m != nil { + return m.TaskAdded + } + return nil +} + +func (m *Event) GetTaskUpdated() *Event_TaskUpdated { + if m != nil { + return m.TaskUpdated + } + return nil +} + +func (m *Event) GetAgentAdded() *Event_AgentAdded { + if m != nil { + return m.AgentAdded + } + return nil +} + +func (m *Event) GetAgentRemoved() *Event_AgentRemoved { + if m != nil { + return m.AgentRemoved + } + return nil +} + +// First event received when a client subscribes. +type Event_Subscribed struct { + // Snapshot of the entire cluster state. Further updates to the + // cluster state are sent as separate events on the stream. + GetState *Response_GetState `protobuf:"bytes,1,opt,name=get_state,json=getState" json:"get_state,omitempty"` +} + +func (m *Event_Subscribed) Reset() { *m = Event_Subscribed{} } +func (*Event_Subscribed) ProtoMessage() {} +func (*Event_Subscribed) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{2, 0} } + +func (m *Event_Subscribed) GetGetState() *Response_GetState { + if m != nil { + return m.GetState + } + return nil +} + +// Forwarded by the master when a task becomes known to it. This can happen +// when a new task is launched by the scheduler or when the task becomes +// known to the master upon an agent (re-)registration after a failover. +type Event_TaskAdded struct { + Task mesos.Task `protobuf:"bytes,1,req,name=task" json:"task"` +} + +func (m *Event_TaskAdded) Reset() { *m = Event_TaskAdded{} } +func (*Event_TaskAdded) ProtoMessage() {} +func (*Event_TaskAdded) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{2, 1} } + +func (m *Event_TaskAdded) GetTask() mesos.Task { + if m != nil { + return m.Task + } + return mesos.Task{} +} + +// Forwarded by the master when an existing task transitions to a new state. +type Event_TaskUpdated struct { + FrameworkID mesos.FrameworkID `protobuf:"bytes,1,req,name=framework_id,json=frameworkId" json:"framework_id"` + // This is the status of the task corresponding to the last + // status update acknowledged by the scheduler. + Status mesos.TaskStatus `protobuf:"bytes,2,req,name=status" json:"status"` + // This is the latest state of the task according to the agent. + State *mesos.TaskState `protobuf:"varint,3,req,name=state,enum=mesos.TaskState" json:"state,omitempty"` +} + +func (m *Event_TaskUpdated) Reset() { *m = Event_TaskUpdated{} } +func (*Event_TaskUpdated) ProtoMessage() {} +func (*Event_TaskUpdated) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{2, 2} } + +func (m *Event_TaskUpdated) GetFrameworkID() mesos.FrameworkID { + if m != nil { + return m.FrameworkID + } + return mesos.FrameworkID{} +} + +func (m *Event_TaskUpdated) GetStatus() mesos.TaskStatus { + if m != nil { + return m.Status + } + return mesos.TaskStatus{} +} + +func (m *Event_TaskUpdated) GetState() mesos.TaskState { + if m != nil && m.State != nil { + return *m.State + } + return mesos.TASK_STAGING +} + +// Forwarded by the master when an agent becomes known to it. +// This can happen when an agent registered for the first +// time, or reregistered after a master failover. +type Event_AgentAdded struct { + Agent Response_GetAgents_Agent `protobuf:"bytes,1,req,name=agent" json:"agent"` +} + +func (m *Event_AgentAdded) Reset() { *m = Event_AgentAdded{} } +func (*Event_AgentAdded) ProtoMessage() {} +func (*Event_AgentAdded) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{2, 3} } + +func (m *Event_AgentAdded) GetAgent() Response_GetAgents_Agent { + if m != nil { + return m.Agent + } + return Response_GetAgents_Agent{} +} + +// Forwarded by the master when an agent is removed. +// This can happen when an agent does not re-register +// within `--agent_reregister_timeout` upon a master failover, +// or when the agent is scheduled for maintenance. +// +// NOTE: It's possible that an agent might become +// active once it has been removed, i.e. if the master +// has gc'ed its list of known "dead" agents. +// See MESOS-5965 for context. +type Event_AgentRemoved struct { + AgentID mesos.AgentID `protobuf:"bytes,1,req,name=agent_id,json=agentId" json:"agent_id"` +} + +func (m *Event_AgentRemoved) Reset() { *m = Event_AgentRemoved{} } +func (*Event_AgentRemoved) ProtoMessage() {} +func (*Event_AgentRemoved) Descriptor() ([]byte, []int) { return fileDescriptorMaster, []int{2, 4} } + +func (m *Event_AgentRemoved) GetAgentID() mesos.AgentID { + if m != nil { + return m.AgentID + } + return mesos.AgentID{} +} + +func init() { + proto.RegisterType((*Call)(nil), "mesos.master.Call") + proto.RegisterType((*Call_GetMetrics)(nil), "mesos.master.Call.GetMetrics") + proto.RegisterType((*Call_SetLoggingLevel)(nil), "mesos.master.Call.SetLoggingLevel") + proto.RegisterType((*Call_ListFiles)(nil), "mesos.master.Call.ListFiles") + proto.RegisterType((*Call_ReadFile)(nil), "mesos.master.Call.ReadFile") + proto.RegisterType((*Call_UpdateWeights)(nil), "mesos.master.Call.UpdateWeights") + proto.RegisterType((*Call_ReserveResources)(nil), "mesos.master.Call.ReserveResources") + proto.RegisterType((*Call_UnreserveResources)(nil), "mesos.master.Call.UnreserveResources") + proto.RegisterType((*Call_CreateVolumes)(nil), "mesos.master.Call.CreateVolumes") + proto.RegisterType((*Call_DestroyVolumes)(nil), "mesos.master.Call.DestroyVolumes") + proto.RegisterType((*Call_UpdateMaintenanceSchedule)(nil), "mesos.master.Call.UpdateMaintenanceSchedule") + proto.RegisterType((*Call_StartMaintenance)(nil), "mesos.master.Call.StartMaintenance") + proto.RegisterType((*Call_StopMaintenance)(nil), "mesos.master.Call.StopMaintenance") + proto.RegisterType((*Call_SetQuota)(nil), "mesos.master.Call.SetQuota") + proto.RegisterType((*Call_RemoveQuota)(nil), "mesos.master.Call.RemoveQuota") + proto.RegisterType((*Response)(nil), "mesos.master.Response") + proto.RegisterType((*Response_GetHealth)(nil), "mesos.master.Response.GetHealth") + proto.RegisterType((*Response_GetFlags)(nil), "mesos.master.Response.GetFlags") + proto.RegisterType((*Response_GetVersion)(nil), "mesos.master.Response.GetVersion") + proto.RegisterType((*Response_GetMetrics)(nil), "mesos.master.Response.GetMetrics") + proto.RegisterType((*Response_GetLoggingLevel)(nil), "mesos.master.Response.GetLoggingLevel") + proto.RegisterType((*Response_ListFiles)(nil), "mesos.master.Response.ListFiles") + proto.RegisterType((*Response_ReadFile)(nil), "mesos.master.Response.ReadFile") + proto.RegisterType((*Response_GetState)(nil), "mesos.master.Response.GetState") + proto.RegisterType((*Response_GetAgents)(nil), "mesos.master.Response.GetAgents") + proto.RegisterType((*Response_GetAgents_Agent)(nil), "mesos.master.Response.GetAgents.Agent") + proto.RegisterType((*Response_GetFrameworks)(nil), "mesos.master.Response.GetFrameworks") + proto.RegisterType((*Response_GetFrameworks_Framework)(nil), "mesos.master.Response.GetFrameworks.Framework") + proto.RegisterType((*Response_GetExecutors)(nil), "mesos.master.Response.GetExecutors") + proto.RegisterType((*Response_GetExecutors_Executor)(nil), "mesos.master.Response.GetExecutors.Executor") + proto.RegisterType((*Response_GetTasks)(nil), "mesos.master.Response.GetTasks") + proto.RegisterType((*Response_GetRoles)(nil), "mesos.master.Response.GetRoles") + proto.RegisterType((*Response_GetWeights)(nil), "mesos.master.Response.GetWeights") + proto.RegisterType((*Response_GetMaster)(nil), "mesos.master.Response.GetMaster") + proto.RegisterType((*Response_GetMaintenanceStatus)(nil), "mesos.master.Response.GetMaintenanceStatus") + proto.RegisterType((*Response_GetMaintenanceSchedule)(nil), "mesos.master.Response.GetMaintenanceSchedule") + proto.RegisterType((*Response_GetQuota)(nil), "mesos.master.Response.GetQuota") + proto.RegisterType((*Event)(nil), "mesos.master.Event") + proto.RegisterType((*Event_Subscribed)(nil), "mesos.master.Event.Subscribed") + proto.RegisterType((*Event_TaskAdded)(nil), "mesos.master.Event.TaskAdded") + proto.RegisterType((*Event_TaskUpdated)(nil), "mesos.master.Event.TaskUpdated") + proto.RegisterType((*Event_AgentAdded)(nil), "mesos.master.Event.AgentAdded") + proto.RegisterType((*Event_AgentRemoved)(nil), "mesos.master.Event.AgentRemoved") + proto.RegisterEnum("mesos.master.Call_Type", Call_Type_name, Call_Type_value) + proto.RegisterEnum("mesos.master.Response_Type", Response_Type_name, Response_Type_value) + proto.RegisterEnum("mesos.master.Event_Type", Event_Type_name, Event_Type_value) +} +func (x Call_Type) String() string { + s, ok := Call_Type_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (x Response_Type) String() string { + s, ok := Response_Type_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (x Event_Type) String() string { + s, ok := Event_Type_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (this *Call) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call) + if !ok { + that2, ok := that.(Call) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call but is not nil && this == nil") + } + if this.Type != that1.Type { + return fmt.Errorf("Type this(%v) Not Equal that(%v)", this.Type, that1.Type) + } + if !this.GetMetrics.Equal(that1.GetMetrics) { + return fmt.Errorf("GetMetrics this(%v) Not Equal that(%v)", this.GetMetrics, that1.GetMetrics) + } + if !this.SetLoggingLevel.Equal(that1.SetLoggingLevel) { + return fmt.Errorf("SetLoggingLevel this(%v) Not Equal that(%v)", this.SetLoggingLevel, that1.SetLoggingLevel) + } + if !this.ListFiles.Equal(that1.ListFiles) { + return fmt.Errorf("ListFiles this(%v) Not Equal that(%v)", this.ListFiles, that1.ListFiles) + } + if !this.ReadFile.Equal(that1.ReadFile) { + return fmt.Errorf("ReadFile this(%v) Not Equal that(%v)", this.ReadFile, that1.ReadFile) + } + if !this.UpdateWeights.Equal(that1.UpdateWeights) { + return fmt.Errorf("UpdateWeights this(%v) Not Equal that(%v)", this.UpdateWeights, that1.UpdateWeights) + } + if !this.ReserveResources.Equal(that1.ReserveResources) { + return fmt.Errorf("ReserveResources this(%v) Not Equal that(%v)", this.ReserveResources, that1.ReserveResources) + } + if !this.UnreserveResources.Equal(that1.UnreserveResources) { + return fmt.Errorf("UnreserveResources this(%v) Not Equal that(%v)", this.UnreserveResources, that1.UnreserveResources) + } + if !this.CreateVolumes.Equal(that1.CreateVolumes) { + return fmt.Errorf("CreateVolumes this(%v) Not Equal that(%v)", this.CreateVolumes, that1.CreateVolumes) + } + if !this.DestroyVolumes.Equal(that1.DestroyVolumes) { + return fmt.Errorf("DestroyVolumes this(%v) Not Equal that(%v)", this.DestroyVolumes, that1.DestroyVolumes) + } + if !this.UpdateMaintenanceSchedule.Equal(that1.UpdateMaintenanceSchedule) { + return fmt.Errorf("UpdateMaintenanceSchedule this(%v) Not Equal that(%v)", this.UpdateMaintenanceSchedule, that1.UpdateMaintenanceSchedule) + } + if !this.StartMaintenance.Equal(that1.StartMaintenance) { + return fmt.Errorf("StartMaintenance this(%v) Not Equal that(%v)", this.StartMaintenance, that1.StartMaintenance) + } + if !this.StopMaintenance.Equal(that1.StopMaintenance) { + return fmt.Errorf("StopMaintenance this(%v) Not Equal that(%v)", this.StopMaintenance, that1.StopMaintenance) + } + if !this.SetQuota.Equal(that1.SetQuota) { + return fmt.Errorf("SetQuota this(%v) Not Equal that(%v)", this.SetQuota, that1.SetQuota) + } + if !this.RemoveQuota.Equal(that1.RemoveQuota) { + return fmt.Errorf("RemoveQuota this(%v) Not Equal that(%v)", this.RemoveQuota, that1.RemoveQuota) + } + return nil +} +func (this *Call) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call) + if !ok { + that2, ok := that.(Call) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Type != that1.Type { + return false + } + if !this.GetMetrics.Equal(that1.GetMetrics) { + return false + } + if !this.SetLoggingLevel.Equal(that1.SetLoggingLevel) { + return false + } + if !this.ListFiles.Equal(that1.ListFiles) { + return false + } + if !this.ReadFile.Equal(that1.ReadFile) { + return false + } + if !this.UpdateWeights.Equal(that1.UpdateWeights) { + return false + } + if !this.ReserveResources.Equal(that1.ReserveResources) { + return false + } + if !this.UnreserveResources.Equal(that1.UnreserveResources) { + return false + } + if !this.CreateVolumes.Equal(that1.CreateVolumes) { + return false + } + if !this.DestroyVolumes.Equal(that1.DestroyVolumes) { + return false + } + if !this.UpdateMaintenanceSchedule.Equal(that1.UpdateMaintenanceSchedule) { + return false + } + if !this.StartMaintenance.Equal(that1.StartMaintenance) { + return false + } + if !this.StopMaintenance.Equal(that1.StopMaintenance) { + return false + } + if !this.SetQuota.Equal(that1.SetQuota) { + return false + } + if !this.RemoveQuota.Equal(that1.RemoveQuota) { + return false + } + return true +} +func (this *Call_GetMetrics) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_GetMetrics) + if !ok { + that2, ok := that.(Call_GetMetrics) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_GetMetrics") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_GetMetrics but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_GetMetrics but is not nil && this == nil") + } + if !this.Timeout.Equal(that1.Timeout) { + return fmt.Errorf("Timeout this(%v) Not Equal that(%v)", this.Timeout, that1.Timeout) + } + return nil +} +func (this *Call_GetMetrics) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_GetMetrics) + if !ok { + that2, ok := that.(Call_GetMetrics) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.Timeout.Equal(that1.Timeout) { + return false + } + return true +} +func (this *Call_SetLoggingLevel) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_SetLoggingLevel) + if !ok { + that2, ok := that.(Call_SetLoggingLevel) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_SetLoggingLevel") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_SetLoggingLevel but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_SetLoggingLevel but is not nil && this == nil") + } + if this.Level != that1.Level { + return fmt.Errorf("Level this(%v) Not Equal that(%v)", this.Level, that1.Level) + } + if !this.Duration.Equal(&that1.Duration) { + return fmt.Errorf("Duration this(%v) Not Equal that(%v)", this.Duration, that1.Duration) + } + return nil +} +func (this *Call_SetLoggingLevel) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_SetLoggingLevel) + if !ok { + that2, ok := that.(Call_SetLoggingLevel) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Level != that1.Level { + return false + } + if !this.Duration.Equal(&that1.Duration) { + return false + } + return true +} +func (this *Call_ListFiles) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_ListFiles) + if !ok { + that2, ok := that.(Call_ListFiles) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_ListFiles") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_ListFiles but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_ListFiles but is not nil && this == nil") + } + if this.Path != that1.Path { + return fmt.Errorf("Path this(%v) Not Equal that(%v)", this.Path, that1.Path) + } + return nil +} +func (this *Call_ListFiles) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_ListFiles) + if !ok { + that2, ok := that.(Call_ListFiles) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Path != that1.Path { + return false + } + return true +} +func (this *Call_ReadFile) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_ReadFile) + if !ok { + that2, ok := that.(Call_ReadFile) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_ReadFile") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_ReadFile but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_ReadFile but is not nil && this == nil") + } + if this.Path != that1.Path { + return fmt.Errorf("Path this(%v) Not Equal that(%v)", this.Path, that1.Path) + } + if this.Offset != that1.Offset { + return fmt.Errorf("Offset this(%v) Not Equal that(%v)", this.Offset, that1.Offset) + } + if this.Length != nil && that1.Length != nil { + if *this.Length != *that1.Length { + return fmt.Errorf("Length this(%v) Not Equal that(%v)", *this.Length, *that1.Length) + } + } else if this.Length != nil { + return fmt.Errorf("this.Length == nil && that.Length != nil") + } else if that1.Length != nil { + return fmt.Errorf("Length this(%v) Not Equal that(%v)", this.Length, that1.Length) + } + return nil +} +func (this *Call_ReadFile) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_ReadFile) + if !ok { + that2, ok := that.(Call_ReadFile) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Path != that1.Path { + return false + } + if this.Offset != that1.Offset { + return false + } + if this.Length != nil && that1.Length != nil { + if *this.Length != *that1.Length { + return false + } + } else if this.Length != nil { + return false + } else if that1.Length != nil { + return false + } + return true +} +func (this *Call_UpdateWeights) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_UpdateWeights) + if !ok { + that2, ok := that.(Call_UpdateWeights) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_UpdateWeights") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_UpdateWeights but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_UpdateWeights but is not nil && this == nil") + } + if len(this.WeightInfos) != len(that1.WeightInfos) { + return fmt.Errorf("WeightInfos this(%v) Not Equal that(%v)", len(this.WeightInfos), len(that1.WeightInfos)) + } + for i := range this.WeightInfos { + if !this.WeightInfos[i].Equal(&that1.WeightInfos[i]) { + return fmt.Errorf("WeightInfos this[%v](%v) Not Equal that[%v](%v)", i, this.WeightInfos[i], i, that1.WeightInfos[i]) + } + } + return nil +} +func (this *Call_UpdateWeights) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_UpdateWeights) + if !ok { + that2, ok := that.(Call_UpdateWeights) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.WeightInfos) != len(that1.WeightInfos) { + return false + } + for i := range this.WeightInfos { + if !this.WeightInfos[i].Equal(&that1.WeightInfos[i]) { + return false + } + } + return true +} +func (this *Call_ReserveResources) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_ReserveResources) + if !ok { + that2, ok := that.(Call_ReserveResources) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_ReserveResources") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_ReserveResources but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_ReserveResources but is not nil && this == nil") + } + if !this.AgentID.Equal(&that1.AgentID) { + return fmt.Errorf("AgentID this(%v) Not Equal that(%v)", this.AgentID, that1.AgentID) + } + if len(this.Resources) != len(that1.Resources) { + return fmt.Errorf("Resources this(%v) Not Equal that(%v)", len(this.Resources), len(that1.Resources)) + } + for i := range this.Resources { + if !this.Resources[i].Equal(&that1.Resources[i]) { + return fmt.Errorf("Resources this[%v](%v) Not Equal that[%v](%v)", i, this.Resources[i], i, that1.Resources[i]) + } + } + return nil +} +func (this *Call_ReserveResources) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_ReserveResources) + if !ok { + that2, ok := that.(Call_ReserveResources) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.AgentID.Equal(&that1.AgentID) { + return false + } + if len(this.Resources) != len(that1.Resources) { + return false + } + for i := range this.Resources { + if !this.Resources[i].Equal(&that1.Resources[i]) { + return false + } + } + return true +} +func (this *Call_UnreserveResources) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_UnreserveResources) + if !ok { + that2, ok := that.(Call_UnreserveResources) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_UnreserveResources") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_UnreserveResources but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_UnreserveResources but is not nil && this == nil") + } + if !this.AgentID.Equal(&that1.AgentID) { + return fmt.Errorf("AgentID this(%v) Not Equal that(%v)", this.AgentID, that1.AgentID) + } + if len(this.Resources) != len(that1.Resources) { + return fmt.Errorf("Resources this(%v) Not Equal that(%v)", len(this.Resources), len(that1.Resources)) + } + for i := range this.Resources { + if !this.Resources[i].Equal(&that1.Resources[i]) { + return fmt.Errorf("Resources this[%v](%v) Not Equal that[%v](%v)", i, this.Resources[i], i, that1.Resources[i]) + } + } + return nil +} +func (this *Call_UnreserveResources) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_UnreserveResources) + if !ok { + that2, ok := that.(Call_UnreserveResources) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.AgentID.Equal(&that1.AgentID) { + return false + } + if len(this.Resources) != len(that1.Resources) { + return false + } + for i := range this.Resources { + if !this.Resources[i].Equal(&that1.Resources[i]) { + return false + } + } + return true +} +func (this *Call_CreateVolumes) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_CreateVolumes) + if !ok { + that2, ok := that.(Call_CreateVolumes) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_CreateVolumes") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_CreateVolumes but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_CreateVolumes but is not nil && this == nil") + } + if !this.AgentID.Equal(&that1.AgentID) { + return fmt.Errorf("AgentID this(%v) Not Equal that(%v)", this.AgentID, that1.AgentID) + } + if len(this.Volumes) != len(that1.Volumes) { + return fmt.Errorf("Volumes this(%v) Not Equal that(%v)", len(this.Volumes), len(that1.Volumes)) + } + for i := range this.Volumes { + if !this.Volumes[i].Equal(&that1.Volumes[i]) { + return fmt.Errorf("Volumes this[%v](%v) Not Equal that[%v](%v)", i, this.Volumes[i], i, that1.Volumes[i]) + } + } + return nil +} +func (this *Call_CreateVolumes) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_CreateVolumes) + if !ok { + that2, ok := that.(Call_CreateVolumes) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.AgentID.Equal(&that1.AgentID) { + return false + } + if len(this.Volumes) != len(that1.Volumes) { + return false + } + for i := range this.Volumes { + if !this.Volumes[i].Equal(&that1.Volumes[i]) { + return false + } + } + return true +} +func (this *Call_DestroyVolumes) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_DestroyVolumes) + if !ok { + that2, ok := that.(Call_DestroyVolumes) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_DestroyVolumes") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_DestroyVolumes but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_DestroyVolumes but is not nil && this == nil") + } + if !this.AgentID.Equal(&that1.AgentID) { + return fmt.Errorf("AgentID this(%v) Not Equal that(%v)", this.AgentID, that1.AgentID) + } + if len(this.Volumes) != len(that1.Volumes) { + return fmt.Errorf("Volumes this(%v) Not Equal that(%v)", len(this.Volumes), len(that1.Volumes)) + } + for i := range this.Volumes { + if !this.Volumes[i].Equal(&that1.Volumes[i]) { + return fmt.Errorf("Volumes this[%v](%v) Not Equal that[%v](%v)", i, this.Volumes[i], i, that1.Volumes[i]) + } + } + return nil +} +func (this *Call_DestroyVolumes) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_DestroyVolumes) + if !ok { + that2, ok := that.(Call_DestroyVolumes) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.AgentID.Equal(&that1.AgentID) { + return false + } + if len(this.Volumes) != len(that1.Volumes) { + return false + } + for i := range this.Volumes { + if !this.Volumes[i].Equal(&that1.Volumes[i]) { + return false + } + } + return true +} +func (this *Call_UpdateMaintenanceSchedule) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_UpdateMaintenanceSchedule) + if !ok { + that2, ok := that.(Call_UpdateMaintenanceSchedule) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_UpdateMaintenanceSchedule") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_UpdateMaintenanceSchedule but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_UpdateMaintenanceSchedule but is not nil && this == nil") + } + if !this.Schedule.Equal(&that1.Schedule) { + return fmt.Errorf("Schedule this(%v) Not Equal that(%v)", this.Schedule, that1.Schedule) + } + return nil +} +func (this *Call_UpdateMaintenanceSchedule) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_UpdateMaintenanceSchedule) + if !ok { + that2, ok := that.(Call_UpdateMaintenanceSchedule) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.Schedule.Equal(&that1.Schedule) { + return false + } + return true +} +func (this *Call_StartMaintenance) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_StartMaintenance) + if !ok { + that2, ok := that.(Call_StartMaintenance) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_StartMaintenance") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_StartMaintenance but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_StartMaintenance but is not nil && this == nil") + } + if len(this.Machines) != len(that1.Machines) { + return fmt.Errorf("Machines this(%v) Not Equal that(%v)", len(this.Machines), len(that1.Machines)) + } + for i := range this.Machines { + if !this.Machines[i].Equal(&that1.Machines[i]) { + return fmt.Errorf("Machines this[%v](%v) Not Equal that[%v](%v)", i, this.Machines[i], i, that1.Machines[i]) + } + } + return nil +} +func (this *Call_StartMaintenance) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_StartMaintenance) + if !ok { + that2, ok := that.(Call_StartMaintenance) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Machines) != len(that1.Machines) { + return false + } + for i := range this.Machines { + if !this.Machines[i].Equal(&that1.Machines[i]) { + return false + } + } + return true +} +func (this *Call_StopMaintenance) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_StopMaintenance) + if !ok { + that2, ok := that.(Call_StopMaintenance) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_StopMaintenance") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_StopMaintenance but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_StopMaintenance but is not nil && this == nil") + } + if len(this.Machines) != len(that1.Machines) { + return fmt.Errorf("Machines this(%v) Not Equal that(%v)", len(this.Machines), len(that1.Machines)) + } + for i := range this.Machines { + if !this.Machines[i].Equal(&that1.Machines[i]) { + return fmt.Errorf("Machines this[%v](%v) Not Equal that[%v](%v)", i, this.Machines[i], i, that1.Machines[i]) + } + } + return nil +} +func (this *Call_StopMaintenance) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_StopMaintenance) + if !ok { + that2, ok := that.(Call_StopMaintenance) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Machines) != len(that1.Machines) { + return false + } + for i := range this.Machines { + if !this.Machines[i].Equal(&that1.Machines[i]) { + return false + } + } + return true +} +func (this *Call_SetQuota) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_SetQuota) + if !ok { + that2, ok := that.(Call_SetQuota) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_SetQuota") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_SetQuota but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_SetQuota but is not nil && this == nil") + } + if !this.QuotaRequest.Equal(&that1.QuotaRequest) { + return fmt.Errorf("QuotaRequest this(%v) Not Equal that(%v)", this.QuotaRequest, that1.QuotaRequest) + } + return nil +} +func (this *Call_SetQuota) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_SetQuota) + if !ok { + that2, ok := that.(Call_SetQuota) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.QuotaRequest.Equal(&that1.QuotaRequest) { + return false + } + return true +} +func (this *Call_RemoveQuota) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Call_RemoveQuota) + if !ok { + that2, ok := that.(Call_RemoveQuota) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Call_RemoveQuota") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Call_RemoveQuota but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Call_RemoveQuota but is not nil && this == nil") + } + if this.Role != that1.Role { + return fmt.Errorf("Role this(%v) Not Equal that(%v)", this.Role, that1.Role) + } + return nil +} +func (this *Call_RemoveQuota) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Call_RemoveQuota) + if !ok { + that2, ok := that.(Call_RemoveQuota) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Role != that1.Role { + return false + } + return true +} +func (this *Response) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response) + if !ok { + that2, ok := that.(Response) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response but is not nil && this == nil") + } + if this.Type != that1.Type { + return fmt.Errorf("Type this(%v) Not Equal that(%v)", this.Type, that1.Type) + } + if !this.GetHealth.Equal(that1.GetHealth) { + return fmt.Errorf("GetHealth this(%v) Not Equal that(%v)", this.GetHealth, that1.GetHealth) + } + if !this.GetFlags.Equal(that1.GetFlags) { + return fmt.Errorf("GetFlags this(%v) Not Equal that(%v)", this.GetFlags, that1.GetFlags) + } + if !this.GetVersion.Equal(that1.GetVersion) { + return fmt.Errorf("GetVersion this(%v) Not Equal that(%v)", this.GetVersion, that1.GetVersion) + } + if !this.GetMetrics.Equal(that1.GetMetrics) { + return fmt.Errorf("GetMetrics this(%v) Not Equal that(%v)", this.GetMetrics, that1.GetMetrics) + } + if !this.GetLoggingLevel.Equal(that1.GetLoggingLevel) { + return fmt.Errorf("GetLoggingLevel this(%v) Not Equal that(%v)", this.GetLoggingLevel, that1.GetLoggingLevel) + } + if !this.ListFiles.Equal(that1.ListFiles) { + return fmt.Errorf("ListFiles this(%v) Not Equal that(%v)", this.ListFiles, that1.ListFiles) + } + if !this.ReadFile.Equal(that1.ReadFile) { + return fmt.Errorf("ReadFile this(%v) Not Equal that(%v)", this.ReadFile, that1.ReadFile) + } + if !this.GetState.Equal(that1.GetState) { + return fmt.Errorf("GetState this(%v) Not Equal that(%v)", this.GetState, that1.GetState) + } + if !this.GetAgents.Equal(that1.GetAgents) { + return fmt.Errorf("GetAgents this(%v) Not Equal that(%v)", this.GetAgents, that1.GetAgents) + } + if !this.GetFrameworks.Equal(that1.GetFrameworks) { + return fmt.Errorf("GetFrameworks this(%v) Not Equal that(%v)", this.GetFrameworks, that1.GetFrameworks) + } + if !this.GetExecutors.Equal(that1.GetExecutors) { + return fmt.Errorf("GetExecutors this(%v) Not Equal that(%v)", this.GetExecutors, that1.GetExecutors) + } + if !this.GetTasks.Equal(that1.GetTasks) { + return fmt.Errorf("GetTasks this(%v) Not Equal that(%v)", this.GetTasks, that1.GetTasks) + } + if !this.GetRoles.Equal(that1.GetRoles) { + return fmt.Errorf("GetRoles this(%v) Not Equal that(%v)", this.GetRoles, that1.GetRoles) + } + if !this.GetWeights.Equal(that1.GetWeights) { + return fmt.Errorf("GetWeights this(%v) Not Equal that(%v)", this.GetWeights, that1.GetWeights) + } + if !this.GetMaster.Equal(that1.GetMaster) { + return fmt.Errorf("GetMaster this(%v) Not Equal that(%v)", this.GetMaster, that1.GetMaster) + } + if !this.GetMaintenanceStatus.Equal(that1.GetMaintenanceStatus) { + return fmt.Errorf("GetMaintenanceStatus this(%v) Not Equal that(%v)", this.GetMaintenanceStatus, that1.GetMaintenanceStatus) + } + if !this.GetMaintenanceSchedule.Equal(that1.GetMaintenanceSchedule) { + return fmt.Errorf("GetMaintenanceSchedule this(%v) Not Equal that(%v)", this.GetMaintenanceSchedule, that1.GetMaintenanceSchedule) + } + if !this.GetQuota.Equal(that1.GetQuota) { + return fmt.Errorf("GetQuota this(%v) Not Equal that(%v)", this.GetQuota, that1.GetQuota) + } + return nil +} +func (this *Response) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response) + if !ok { + that2, ok := that.(Response) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Type != that1.Type { + return false + } + if !this.GetHealth.Equal(that1.GetHealth) { + return false + } + if !this.GetFlags.Equal(that1.GetFlags) { + return false + } + if !this.GetVersion.Equal(that1.GetVersion) { + return false + } + if !this.GetMetrics.Equal(that1.GetMetrics) { + return false + } + if !this.GetLoggingLevel.Equal(that1.GetLoggingLevel) { + return false + } + if !this.ListFiles.Equal(that1.ListFiles) { + return false + } + if !this.ReadFile.Equal(that1.ReadFile) { + return false + } + if !this.GetState.Equal(that1.GetState) { + return false + } + if !this.GetAgents.Equal(that1.GetAgents) { + return false + } + if !this.GetFrameworks.Equal(that1.GetFrameworks) { + return false + } + if !this.GetExecutors.Equal(that1.GetExecutors) { + return false + } + if !this.GetTasks.Equal(that1.GetTasks) { + return false + } + if !this.GetRoles.Equal(that1.GetRoles) { + return false + } + if !this.GetWeights.Equal(that1.GetWeights) { + return false + } + if !this.GetMaster.Equal(that1.GetMaster) { + return false + } + if !this.GetMaintenanceStatus.Equal(that1.GetMaintenanceStatus) { + return false + } + if !this.GetMaintenanceSchedule.Equal(that1.GetMaintenanceSchedule) { + return false + } + if !this.GetQuota.Equal(that1.GetQuota) { + return false + } + return true +} +func (this *Response_GetHealth) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetHealth) + if !ok { + that2, ok := that.(Response_GetHealth) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetHealth") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetHealth but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetHealth but is not nil && this == nil") + } + if this.Healthy != that1.Healthy { + return fmt.Errorf("Healthy this(%v) Not Equal that(%v)", this.Healthy, that1.Healthy) + } + return nil +} +func (this *Response_GetHealth) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetHealth) + if !ok { + that2, ok := that.(Response_GetHealth) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Healthy != that1.Healthy { + return false + } + return true +} +func (this *Response_GetFlags) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetFlags) + if !ok { + that2, ok := that.(Response_GetFlags) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetFlags") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetFlags but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetFlags but is not nil && this == nil") + } + if len(this.Flags) != len(that1.Flags) { + return fmt.Errorf("Flags this(%v) Not Equal that(%v)", len(this.Flags), len(that1.Flags)) + } + for i := range this.Flags { + if !this.Flags[i].Equal(&that1.Flags[i]) { + return fmt.Errorf("Flags this[%v](%v) Not Equal that[%v](%v)", i, this.Flags[i], i, that1.Flags[i]) + } + } + return nil +} +func (this *Response_GetFlags) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetFlags) + if !ok { + that2, ok := that.(Response_GetFlags) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Flags) != len(that1.Flags) { + return false + } + for i := range this.Flags { + if !this.Flags[i].Equal(&that1.Flags[i]) { + return false + } + } + return true +} +func (this *Response_GetVersion) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetVersion) + if !ok { + that2, ok := that.(Response_GetVersion) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetVersion") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetVersion but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetVersion but is not nil && this == nil") + } + if !this.VersionInfo.Equal(&that1.VersionInfo) { + return fmt.Errorf("VersionInfo this(%v) Not Equal that(%v)", this.VersionInfo, that1.VersionInfo) + } + return nil +} +func (this *Response_GetVersion) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetVersion) + if !ok { + that2, ok := that.(Response_GetVersion) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.VersionInfo.Equal(&that1.VersionInfo) { + return false + } + return true +} +func (this *Response_GetMetrics) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetMetrics) + if !ok { + that2, ok := that.(Response_GetMetrics) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetMetrics") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetMetrics but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetMetrics but is not nil && this == nil") + } + if len(this.Metrics) != len(that1.Metrics) { + return fmt.Errorf("Metrics this(%v) Not Equal that(%v)", len(this.Metrics), len(that1.Metrics)) + } + for i := range this.Metrics { + if !this.Metrics[i].Equal(&that1.Metrics[i]) { + return fmt.Errorf("Metrics this[%v](%v) Not Equal that[%v](%v)", i, this.Metrics[i], i, that1.Metrics[i]) + } + } + return nil +} +func (this *Response_GetMetrics) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetMetrics) + if !ok { + that2, ok := that.(Response_GetMetrics) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Metrics) != len(that1.Metrics) { + return false + } + for i := range this.Metrics { + if !this.Metrics[i].Equal(&that1.Metrics[i]) { + return false + } + } + return true +} +func (this *Response_GetLoggingLevel) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetLoggingLevel) + if !ok { + that2, ok := that.(Response_GetLoggingLevel) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetLoggingLevel") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetLoggingLevel but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetLoggingLevel but is not nil && this == nil") + } + if this.Level != that1.Level { + return fmt.Errorf("Level this(%v) Not Equal that(%v)", this.Level, that1.Level) + } + return nil +} +func (this *Response_GetLoggingLevel) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetLoggingLevel) + if !ok { + that2, ok := that.(Response_GetLoggingLevel) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Level != that1.Level { + return false + } + return true +} +func (this *Response_ListFiles) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_ListFiles) + if !ok { + that2, ok := that.(Response_ListFiles) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_ListFiles") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_ListFiles but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_ListFiles but is not nil && this == nil") + } + if len(this.FileInfos) != len(that1.FileInfos) { + return fmt.Errorf("FileInfos this(%v) Not Equal that(%v)", len(this.FileInfos), len(that1.FileInfos)) + } + for i := range this.FileInfos { + if !this.FileInfos[i].Equal(&that1.FileInfos[i]) { + return fmt.Errorf("FileInfos this[%v](%v) Not Equal that[%v](%v)", i, this.FileInfos[i], i, that1.FileInfos[i]) + } + } + return nil +} +func (this *Response_ListFiles) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_ListFiles) + if !ok { + that2, ok := that.(Response_ListFiles) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.FileInfos) != len(that1.FileInfos) { + return false + } + for i := range this.FileInfos { + if !this.FileInfos[i].Equal(&that1.FileInfos[i]) { + return false + } + } + return true +} +func (this *Response_ReadFile) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_ReadFile) + if !ok { + that2, ok := that.(Response_ReadFile) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_ReadFile") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_ReadFile but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_ReadFile but is not nil && this == nil") + } + if this.Size != that1.Size { + return fmt.Errorf("Size this(%v) Not Equal that(%v)", this.Size, that1.Size) + } + if !bytes.Equal(this.Data, that1.Data) { + return fmt.Errorf("Data this(%v) Not Equal that(%v)", this.Data, that1.Data) + } + return nil +} +func (this *Response_ReadFile) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_ReadFile) + if !ok { + that2, ok := that.(Response_ReadFile) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Size != that1.Size { + return false + } + if !bytes.Equal(this.Data, that1.Data) { + return false + } + return true +} +func (this *Response_GetState) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetState) + if !ok { + that2, ok := that.(Response_GetState) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetState") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetState but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetState but is not nil && this == nil") + } + if !this.GetTasks.Equal(that1.GetTasks) { + return fmt.Errorf("GetTasks this(%v) Not Equal that(%v)", this.GetTasks, that1.GetTasks) + } + if !this.GetExecutors.Equal(that1.GetExecutors) { + return fmt.Errorf("GetExecutors this(%v) Not Equal that(%v)", this.GetExecutors, that1.GetExecutors) + } + if !this.GetFrameworks.Equal(that1.GetFrameworks) { + return fmt.Errorf("GetFrameworks this(%v) Not Equal that(%v)", this.GetFrameworks, that1.GetFrameworks) + } + if !this.GetAgents.Equal(that1.GetAgents) { + return fmt.Errorf("GetAgents this(%v) Not Equal that(%v)", this.GetAgents, that1.GetAgents) + } + return nil +} +func (this *Response_GetState) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetState) + if !ok { + that2, ok := that.(Response_GetState) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.GetTasks.Equal(that1.GetTasks) { + return false + } + if !this.GetExecutors.Equal(that1.GetExecutors) { + return false + } + if !this.GetFrameworks.Equal(that1.GetFrameworks) { + return false + } + if !this.GetAgents.Equal(that1.GetAgents) { + return false + } + return true +} +func (this *Response_GetAgents) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetAgents) + if !ok { + that2, ok := that.(Response_GetAgents) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetAgents") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetAgents but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetAgents but is not nil && this == nil") + } + if len(this.Agents) != len(that1.Agents) { + return fmt.Errorf("Agents this(%v) Not Equal that(%v)", len(this.Agents), len(that1.Agents)) + } + for i := range this.Agents { + if !this.Agents[i].Equal(&that1.Agents[i]) { + return fmt.Errorf("Agents this[%v](%v) Not Equal that[%v](%v)", i, this.Agents[i], i, that1.Agents[i]) + } + } + if len(this.RecoveredAgents) != len(that1.RecoveredAgents) { + return fmt.Errorf("RecoveredAgents this(%v) Not Equal that(%v)", len(this.RecoveredAgents), len(that1.RecoveredAgents)) + } + for i := range this.RecoveredAgents { + if !this.RecoveredAgents[i].Equal(&that1.RecoveredAgents[i]) { + return fmt.Errorf("RecoveredAgents this[%v](%v) Not Equal that[%v](%v)", i, this.RecoveredAgents[i], i, that1.RecoveredAgents[i]) + } + } + return nil +} +func (this *Response_GetAgents) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetAgents) + if !ok { + that2, ok := that.(Response_GetAgents) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Agents) != len(that1.Agents) { + return false + } + for i := range this.Agents { + if !this.Agents[i].Equal(&that1.Agents[i]) { + return false + } + } + if len(this.RecoveredAgents) != len(that1.RecoveredAgents) { + return false + } + for i := range this.RecoveredAgents { + if !this.RecoveredAgents[i].Equal(&that1.RecoveredAgents[i]) { + return false + } + } + return true +} +func (this *Response_GetAgents_Agent) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetAgents_Agent) + if !ok { + that2, ok := that.(Response_GetAgents_Agent) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetAgents_Agent") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetAgents_Agent but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetAgents_Agent but is not nil && this == nil") + } + if !this.AgentInfo.Equal(&that1.AgentInfo) { + return fmt.Errorf("AgentInfo this(%v) Not Equal that(%v)", this.AgentInfo, that1.AgentInfo) + } + if this.Active != that1.Active { + return fmt.Errorf("Active this(%v) Not Equal that(%v)", this.Active, that1.Active) + } + if this.Version != that1.Version { + return fmt.Errorf("Version this(%v) Not Equal that(%v)", this.Version, that1.Version) + } + if this.PID != nil && that1.PID != nil { + if *this.PID != *that1.PID { + return fmt.Errorf("PID this(%v) Not Equal that(%v)", *this.PID, *that1.PID) + } + } else if this.PID != nil { + return fmt.Errorf("this.PID == nil && that.PID != nil") + } else if that1.PID != nil { + return fmt.Errorf("PID this(%v) Not Equal that(%v)", this.PID, that1.PID) + } + if !this.RegisteredTime.Equal(that1.RegisteredTime) { + return fmt.Errorf("RegisteredTime this(%v) Not Equal that(%v)", this.RegisteredTime, that1.RegisteredTime) + } + if !this.ReregisteredTime.Equal(that1.ReregisteredTime) { + return fmt.Errorf("ReregisteredTime this(%v) Not Equal that(%v)", this.ReregisteredTime, that1.ReregisteredTime) + } + if len(this.TotalResources) != len(that1.TotalResources) { + return fmt.Errorf("TotalResources this(%v) Not Equal that(%v)", len(this.TotalResources), len(that1.TotalResources)) + } + for i := range this.TotalResources { + if !this.TotalResources[i].Equal(&that1.TotalResources[i]) { + return fmt.Errorf("TotalResources this[%v](%v) Not Equal that[%v](%v)", i, this.TotalResources[i], i, that1.TotalResources[i]) + } + } + if len(this.AllocatedResources) != len(that1.AllocatedResources) { + return fmt.Errorf("AllocatedResources this(%v) Not Equal that(%v)", len(this.AllocatedResources), len(that1.AllocatedResources)) + } + for i := range this.AllocatedResources { + if !this.AllocatedResources[i].Equal(&that1.AllocatedResources[i]) { + return fmt.Errorf("AllocatedResources this[%v](%v) Not Equal that[%v](%v)", i, this.AllocatedResources[i], i, that1.AllocatedResources[i]) + } + } + if len(this.OfferedResources) != len(that1.OfferedResources) { + return fmt.Errorf("OfferedResources this(%v) Not Equal that(%v)", len(this.OfferedResources), len(that1.OfferedResources)) + } + for i := range this.OfferedResources { + if !this.OfferedResources[i].Equal(&that1.OfferedResources[i]) { + return fmt.Errorf("OfferedResources this[%v](%v) Not Equal that[%v](%v)", i, this.OfferedResources[i], i, that1.OfferedResources[i]) + } + } + return nil +} +func (this *Response_GetAgents_Agent) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetAgents_Agent) + if !ok { + that2, ok := that.(Response_GetAgents_Agent) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.AgentInfo.Equal(&that1.AgentInfo) { + return false + } + if this.Active != that1.Active { + return false + } + if this.Version != that1.Version { + return false + } + if this.PID != nil && that1.PID != nil { + if *this.PID != *that1.PID { + return false + } + } else if this.PID != nil { + return false + } else if that1.PID != nil { + return false + } + if !this.RegisteredTime.Equal(that1.RegisteredTime) { + return false + } + if !this.ReregisteredTime.Equal(that1.ReregisteredTime) { + return false + } + if len(this.TotalResources) != len(that1.TotalResources) { + return false + } + for i := range this.TotalResources { + if !this.TotalResources[i].Equal(&that1.TotalResources[i]) { + return false + } + } + if len(this.AllocatedResources) != len(that1.AllocatedResources) { + return false + } + for i := range this.AllocatedResources { + if !this.AllocatedResources[i].Equal(&that1.AllocatedResources[i]) { + return false + } + } + if len(this.OfferedResources) != len(that1.OfferedResources) { + return false + } + for i := range this.OfferedResources { + if !this.OfferedResources[i].Equal(&that1.OfferedResources[i]) { + return false + } + } + return true +} +func (this *Response_GetFrameworks) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetFrameworks) + if !ok { + that2, ok := that.(Response_GetFrameworks) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetFrameworks") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetFrameworks but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetFrameworks but is not nil && this == nil") + } + if len(this.Frameworks) != len(that1.Frameworks) { + return fmt.Errorf("Frameworks this(%v) Not Equal that(%v)", len(this.Frameworks), len(that1.Frameworks)) + } + for i := range this.Frameworks { + if !this.Frameworks[i].Equal(&that1.Frameworks[i]) { + return fmt.Errorf("Frameworks this[%v](%v) Not Equal that[%v](%v)", i, this.Frameworks[i], i, that1.Frameworks[i]) + } + } + if len(this.CompletedFrameworks) != len(that1.CompletedFrameworks) { + return fmt.Errorf("CompletedFrameworks this(%v) Not Equal that(%v)", len(this.CompletedFrameworks), len(that1.CompletedFrameworks)) + } + for i := range this.CompletedFrameworks { + if !this.CompletedFrameworks[i].Equal(&that1.CompletedFrameworks[i]) { + return fmt.Errorf("CompletedFrameworks this[%v](%v) Not Equal that[%v](%v)", i, this.CompletedFrameworks[i], i, that1.CompletedFrameworks[i]) + } + } + if len(this.RecoveredFrameworks) != len(that1.RecoveredFrameworks) { + return fmt.Errorf("RecoveredFrameworks this(%v) Not Equal that(%v)", len(this.RecoveredFrameworks), len(that1.RecoveredFrameworks)) + } + for i := range this.RecoveredFrameworks { + if !this.RecoveredFrameworks[i].Equal(&that1.RecoveredFrameworks[i]) { + return fmt.Errorf("RecoveredFrameworks this[%v](%v) Not Equal that[%v](%v)", i, this.RecoveredFrameworks[i], i, that1.RecoveredFrameworks[i]) + } + } + return nil +} +func (this *Response_GetFrameworks) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetFrameworks) + if !ok { + that2, ok := that.(Response_GetFrameworks) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Frameworks) != len(that1.Frameworks) { + return false + } + for i := range this.Frameworks { + if !this.Frameworks[i].Equal(&that1.Frameworks[i]) { + return false + } + } + if len(this.CompletedFrameworks) != len(that1.CompletedFrameworks) { + return false + } + for i := range this.CompletedFrameworks { + if !this.CompletedFrameworks[i].Equal(&that1.CompletedFrameworks[i]) { + return false + } + } + if len(this.RecoveredFrameworks) != len(that1.RecoveredFrameworks) { + return false + } + for i := range this.RecoveredFrameworks { + if !this.RecoveredFrameworks[i].Equal(&that1.RecoveredFrameworks[i]) { + return false + } + } + return true +} +func (this *Response_GetFrameworks_Framework) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetFrameworks_Framework) + if !ok { + that2, ok := that.(Response_GetFrameworks_Framework) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetFrameworks_Framework") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetFrameworks_Framework but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetFrameworks_Framework but is not nil && this == nil") + } + if !this.FrameworkInfo.Equal(&that1.FrameworkInfo) { + return fmt.Errorf("FrameworkInfo this(%v) Not Equal that(%v)", this.FrameworkInfo, that1.FrameworkInfo) + } + if this.Active != that1.Active { + return fmt.Errorf("Active this(%v) Not Equal that(%v)", this.Active, that1.Active) + } + if this.Connected != that1.Connected { + return fmt.Errorf("Connected this(%v) Not Equal that(%v)", this.Connected, that1.Connected) + } + if this.Recovered != that1.Recovered { + return fmt.Errorf("Recovered this(%v) Not Equal that(%v)", this.Recovered, that1.Recovered) + } + if !this.RegisteredTime.Equal(that1.RegisteredTime) { + return fmt.Errorf("RegisteredTime this(%v) Not Equal that(%v)", this.RegisteredTime, that1.RegisteredTime) + } + if !this.ReregisteredTime.Equal(that1.ReregisteredTime) { + return fmt.Errorf("ReregisteredTime this(%v) Not Equal that(%v)", this.ReregisteredTime, that1.ReregisteredTime) + } + if !this.UnregisteredTime.Equal(that1.UnregisteredTime) { + return fmt.Errorf("UnregisteredTime this(%v) Not Equal that(%v)", this.UnregisteredTime, that1.UnregisteredTime) + } + if len(this.Offers) != len(that1.Offers) { + return fmt.Errorf("Offers this(%v) Not Equal that(%v)", len(this.Offers), len(that1.Offers)) + } + for i := range this.Offers { + if !this.Offers[i].Equal(&that1.Offers[i]) { + return fmt.Errorf("Offers this[%v](%v) Not Equal that[%v](%v)", i, this.Offers[i], i, that1.Offers[i]) + } + } + if len(this.InverseOffers) != len(that1.InverseOffers) { + return fmt.Errorf("InverseOffers this(%v) Not Equal that(%v)", len(this.InverseOffers), len(that1.InverseOffers)) + } + for i := range this.InverseOffers { + if !this.InverseOffers[i].Equal(&that1.InverseOffers[i]) { + return fmt.Errorf("InverseOffers this[%v](%v) Not Equal that[%v](%v)", i, this.InverseOffers[i], i, that1.InverseOffers[i]) + } + } + if len(this.AllocatedResources) != len(that1.AllocatedResources) { + return fmt.Errorf("AllocatedResources this(%v) Not Equal that(%v)", len(this.AllocatedResources), len(that1.AllocatedResources)) + } + for i := range this.AllocatedResources { + if !this.AllocatedResources[i].Equal(&that1.AllocatedResources[i]) { + return fmt.Errorf("AllocatedResources this[%v](%v) Not Equal that[%v](%v)", i, this.AllocatedResources[i], i, that1.AllocatedResources[i]) + } + } + if len(this.OfferedResources) != len(that1.OfferedResources) { + return fmt.Errorf("OfferedResources this(%v) Not Equal that(%v)", len(this.OfferedResources), len(that1.OfferedResources)) + } + for i := range this.OfferedResources { + if !this.OfferedResources[i].Equal(&that1.OfferedResources[i]) { + return fmt.Errorf("OfferedResources this[%v](%v) Not Equal that[%v](%v)", i, this.OfferedResources[i], i, that1.OfferedResources[i]) + } + } + return nil +} +func (this *Response_GetFrameworks_Framework) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetFrameworks_Framework) + if !ok { + that2, ok := that.(Response_GetFrameworks_Framework) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.FrameworkInfo.Equal(&that1.FrameworkInfo) { + return false + } + if this.Active != that1.Active { + return false + } + if this.Connected != that1.Connected { + return false + } + if this.Recovered != that1.Recovered { + return false + } + if !this.RegisteredTime.Equal(that1.RegisteredTime) { + return false + } + if !this.ReregisteredTime.Equal(that1.ReregisteredTime) { + return false + } + if !this.UnregisteredTime.Equal(that1.UnregisteredTime) { + return false + } + if len(this.Offers) != len(that1.Offers) { + return false + } + for i := range this.Offers { + if !this.Offers[i].Equal(&that1.Offers[i]) { + return false + } + } + if len(this.InverseOffers) != len(that1.InverseOffers) { + return false + } + for i := range this.InverseOffers { + if !this.InverseOffers[i].Equal(&that1.InverseOffers[i]) { + return false + } + } + if len(this.AllocatedResources) != len(that1.AllocatedResources) { + return false + } + for i := range this.AllocatedResources { + if !this.AllocatedResources[i].Equal(&that1.AllocatedResources[i]) { + return false + } + } + if len(this.OfferedResources) != len(that1.OfferedResources) { + return false + } + for i := range this.OfferedResources { + if !this.OfferedResources[i].Equal(&that1.OfferedResources[i]) { + return false + } + } + return true +} +func (this *Response_GetExecutors) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetExecutors) + if !ok { + that2, ok := that.(Response_GetExecutors) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetExecutors") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetExecutors but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetExecutors but is not nil && this == nil") + } + if len(this.Executors) != len(that1.Executors) { + return fmt.Errorf("Executors this(%v) Not Equal that(%v)", len(this.Executors), len(that1.Executors)) + } + for i := range this.Executors { + if !this.Executors[i].Equal(&that1.Executors[i]) { + return fmt.Errorf("Executors this[%v](%v) Not Equal that[%v](%v)", i, this.Executors[i], i, that1.Executors[i]) + } + } + if len(this.OrphanExecutors) != len(that1.OrphanExecutors) { + return fmt.Errorf("OrphanExecutors this(%v) Not Equal that(%v)", len(this.OrphanExecutors), len(that1.OrphanExecutors)) + } + for i := range this.OrphanExecutors { + if !this.OrphanExecutors[i].Equal(&that1.OrphanExecutors[i]) { + return fmt.Errorf("OrphanExecutors this[%v](%v) Not Equal that[%v](%v)", i, this.OrphanExecutors[i], i, that1.OrphanExecutors[i]) + } + } + return nil +} +func (this *Response_GetExecutors) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetExecutors) + if !ok { + that2, ok := that.(Response_GetExecutors) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Executors) != len(that1.Executors) { + return false + } + for i := range this.Executors { + if !this.Executors[i].Equal(&that1.Executors[i]) { + return false + } + } + if len(this.OrphanExecutors) != len(that1.OrphanExecutors) { + return false + } + for i := range this.OrphanExecutors { + if !this.OrphanExecutors[i].Equal(&that1.OrphanExecutors[i]) { + return false + } + } + return true +} +func (this *Response_GetExecutors_Executor) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetExecutors_Executor) + if !ok { + that2, ok := that.(Response_GetExecutors_Executor) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetExecutors_Executor") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetExecutors_Executor but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetExecutors_Executor but is not nil && this == nil") + } + if !this.ExecutorInfo.Equal(&that1.ExecutorInfo) { + return fmt.Errorf("ExecutorInfo this(%v) Not Equal that(%v)", this.ExecutorInfo, that1.ExecutorInfo) + } + if !this.AgentID.Equal(&that1.AgentID) { + return fmt.Errorf("AgentID this(%v) Not Equal that(%v)", this.AgentID, that1.AgentID) + } + return nil +} +func (this *Response_GetExecutors_Executor) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetExecutors_Executor) + if !ok { + that2, ok := that.(Response_GetExecutors_Executor) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.ExecutorInfo.Equal(&that1.ExecutorInfo) { + return false + } + if !this.AgentID.Equal(&that1.AgentID) { + return false + } + return true +} +func (this *Response_GetTasks) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetTasks) + if !ok { + that2, ok := that.(Response_GetTasks) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetTasks") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetTasks but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetTasks but is not nil && this == nil") + } + if len(this.PendingTasks) != len(that1.PendingTasks) { + return fmt.Errorf("PendingTasks this(%v) Not Equal that(%v)", len(this.PendingTasks), len(that1.PendingTasks)) + } + for i := range this.PendingTasks { + if !this.PendingTasks[i].Equal(&that1.PendingTasks[i]) { + return fmt.Errorf("PendingTasks this[%v](%v) Not Equal that[%v](%v)", i, this.PendingTasks[i], i, that1.PendingTasks[i]) + } + } + if len(this.Tasks) != len(that1.Tasks) { + return fmt.Errorf("Tasks this(%v) Not Equal that(%v)", len(this.Tasks), len(that1.Tasks)) + } + for i := range this.Tasks { + if !this.Tasks[i].Equal(&that1.Tasks[i]) { + return fmt.Errorf("Tasks this[%v](%v) Not Equal that[%v](%v)", i, this.Tasks[i], i, that1.Tasks[i]) + } + } + if len(this.UnreachableTasks) != len(that1.UnreachableTasks) { + return fmt.Errorf("UnreachableTasks this(%v) Not Equal that(%v)", len(this.UnreachableTasks), len(that1.UnreachableTasks)) + } + for i := range this.UnreachableTasks { + if !this.UnreachableTasks[i].Equal(&that1.UnreachableTasks[i]) { + return fmt.Errorf("UnreachableTasks this[%v](%v) Not Equal that[%v](%v)", i, this.UnreachableTasks[i], i, that1.UnreachableTasks[i]) + } + } + if len(this.CompletedTasks) != len(that1.CompletedTasks) { + return fmt.Errorf("CompletedTasks this(%v) Not Equal that(%v)", len(this.CompletedTasks), len(that1.CompletedTasks)) + } + for i := range this.CompletedTasks { + if !this.CompletedTasks[i].Equal(&that1.CompletedTasks[i]) { + return fmt.Errorf("CompletedTasks this[%v](%v) Not Equal that[%v](%v)", i, this.CompletedTasks[i], i, that1.CompletedTasks[i]) + } + } + if len(this.OrphanTasks) != len(that1.OrphanTasks) { + return fmt.Errorf("OrphanTasks this(%v) Not Equal that(%v)", len(this.OrphanTasks), len(that1.OrphanTasks)) + } + for i := range this.OrphanTasks { + if !this.OrphanTasks[i].Equal(&that1.OrphanTasks[i]) { + return fmt.Errorf("OrphanTasks this[%v](%v) Not Equal that[%v](%v)", i, this.OrphanTasks[i], i, that1.OrphanTasks[i]) + } + } + return nil +} +func (this *Response_GetTasks) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetTasks) + if !ok { + that2, ok := that.(Response_GetTasks) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.PendingTasks) != len(that1.PendingTasks) { + return false + } + for i := range this.PendingTasks { + if !this.PendingTasks[i].Equal(&that1.PendingTasks[i]) { + return false + } + } + if len(this.Tasks) != len(that1.Tasks) { + return false + } + for i := range this.Tasks { + if !this.Tasks[i].Equal(&that1.Tasks[i]) { + return false + } + } + if len(this.UnreachableTasks) != len(that1.UnreachableTasks) { + return false + } + for i := range this.UnreachableTasks { + if !this.UnreachableTasks[i].Equal(&that1.UnreachableTasks[i]) { + return false + } + } + if len(this.CompletedTasks) != len(that1.CompletedTasks) { + return false + } + for i := range this.CompletedTasks { + if !this.CompletedTasks[i].Equal(&that1.CompletedTasks[i]) { + return false + } + } + if len(this.OrphanTasks) != len(that1.OrphanTasks) { + return false + } + for i := range this.OrphanTasks { + if !this.OrphanTasks[i].Equal(&that1.OrphanTasks[i]) { + return false + } + } + return true +} +func (this *Response_GetRoles) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetRoles) + if !ok { + that2, ok := that.(Response_GetRoles) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetRoles") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetRoles but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetRoles but is not nil && this == nil") + } + if len(this.Roles) != len(that1.Roles) { + return fmt.Errorf("Roles this(%v) Not Equal that(%v)", len(this.Roles), len(that1.Roles)) + } + for i := range this.Roles { + if !this.Roles[i].Equal(&that1.Roles[i]) { + return fmt.Errorf("Roles this[%v](%v) Not Equal that[%v](%v)", i, this.Roles[i], i, that1.Roles[i]) + } + } + return nil +} +func (this *Response_GetRoles) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetRoles) + if !ok { + that2, ok := that.(Response_GetRoles) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Roles) != len(that1.Roles) { + return false + } + for i := range this.Roles { + if !this.Roles[i].Equal(&that1.Roles[i]) { + return false + } + } + return true +} +func (this *Response_GetWeights) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetWeights) + if !ok { + that2, ok := that.(Response_GetWeights) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetWeights") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetWeights but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetWeights but is not nil && this == nil") + } + if len(this.WeightInfos) != len(that1.WeightInfos) { + return fmt.Errorf("WeightInfos this(%v) Not Equal that(%v)", len(this.WeightInfos), len(that1.WeightInfos)) + } + for i := range this.WeightInfos { + if !this.WeightInfos[i].Equal(&that1.WeightInfos[i]) { + return fmt.Errorf("WeightInfos this[%v](%v) Not Equal that[%v](%v)", i, this.WeightInfos[i], i, that1.WeightInfos[i]) + } + } + return nil +} +func (this *Response_GetWeights) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetWeights) + if !ok { + that2, ok := that.(Response_GetWeights) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.WeightInfos) != len(that1.WeightInfos) { + return false + } + for i := range this.WeightInfos { + if !this.WeightInfos[i].Equal(&that1.WeightInfos[i]) { + return false + } + } + return true +} +func (this *Response_GetMaster) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetMaster) + if !ok { + that2, ok := that.(Response_GetMaster) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetMaster") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetMaster but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetMaster but is not nil && this == nil") + } + if !this.MasterInfo.Equal(that1.MasterInfo) { + return fmt.Errorf("MasterInfo this(%v) Not Equal that(%v)", this.MasterInfo, that1.MasterInfo) + } + return nil +} +func (this *Response_GetMaster) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetMaster) + if !ok { + that2, ok := that.(Response_GetMaster) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.MasterInfo.Equal(that1.MasterInfo) { + return false + } + return true +} +func (this *Response_GetMaintenanceStatus) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetMaintenanceStatus) + if !ok { + that2, ok := that.(Response_GetMaintenanceStatus) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetMaintenanceStatus") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetMaintenanceStatus but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetMaintenanceStatus but is not nil && this == nil") + } + if !this.Status.Equal(&that1.Status) { + return fmt.Errorf("Status this(%v) Not Equal that(%v)", this.Status, that1.Status) + } + return nil +} +func (this *Response_GetMaintenanceStatus) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetMaintenanceStatus) + if !ok { + that2, ok := that.(Response_GetMaintenanceStatus) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.Status.Equal(&that1.Status) { + return false + } + return true +} +func (this *Response_GetMaintenanceSchedule) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetMaintenanceSchedule) + if !ok { + that2, ok := that.(Response_GetMaintenanceSchedule) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetMaintenanceSchedule") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetMaintenanceSchedule but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetMaintenanceSchedule but is not nil && this == nil") + } + if !this.Schedule.Equal(&that1.Schedule) { + return fmt.Errorf("Schedule this(%v) Not Equal that(%v)", this.Schedule, that1.Schedule) + } + return nil +} +func (this *Response_GetMaintenanceSchedule) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetMaintenanceSchedule) + if !ok { + that2, ok := that.(Response_GetMaintenanceSchedule) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.Schedule.Equal(&that1.Schedule) { + return false + } + return true +} +func (this *Response_GetQuota) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Response_GetQuota) + if !ok { + that2, ok := that.(Response_GetQuota) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Response_GetQuota") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Response_GetQuota but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Response_GetQuota but is not nil && this == nil") + } + if !this.Status.Equal(&that1.Status) { + return fmt.Errorf("Status this(%v) Not Equal that(%v)", this.Status, that1.Status) + } + return nil +} +func (this *Response_GetQuota) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Response_GetQuota) + if !ok { + that2, ok := that.(Response_GetQuota) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.Status.Equal(&that1.Status) { + return false + } + return true +} +func (this *Event) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Event) + if !ok { + that2, ok := that.(Event) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Event") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Event but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Event but is not nil && this == nil") + } + if this.Type != that1.Type { + return fmt.Errorf("Type this(%v) Not Equal that(%v)", this.Type, that1.Type) + } + if !this.Subscribed.Equal(that1.Subscribed) { + return fmt.Errorf("Subscribed this(%v) Not Equal that(%v)", this.Subscribed, that1.Subscribed) + } + if !this.TaskAdded.Equal(that1.TaskAdded) { + return fmt.Errorf("TaskAdded this(%v) Not Equal that(%v)", this.TaskAdded, that1.TaskAdded) + } + if !this.TaskUpdated.Equal(that1.TaskUpdated) { + return fmt.Errorf("TaskUpdated this(%v) Not Equal that(%v)", this.TaskUpdated, that1.TaskUpdated) + } + if !this.AgentAdded.Equal(that1.AgentAdded) { + return fmt.Errorf("AgentAdded this(%v) Not Equal that(%v)", this.AgentAdded, that1.AgentAdded) + } + if !this.AgentRemoved.Equal(that1.AgentRemoved) { + return fmt.Errorf("AgentRemoved this(%v) Not Equal that(%v)", this.AgentRemoved, that1.AgentRemoved) + } + return nil +} +func (this *Event) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Event) + if !ok { + that2, ok := that.(Event) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Type != that1.Type { + return false + } + if !this.Subscribed.Equal(that1.Subscribed) { + return false + } + if !this.TaskAdded.Equal(that1.TaskAdded) { + return false + } + if !this.TaskUpdated.Equal(that1.TaskUpdated) { + return false + } + if !this.AgentAdded.Equal(that1.AgentAdded) { + return false + } + if !this.AgentRemoved.Equal(that1.AgentRemoved) { + return false + } + return true +} +func (this *Event_Subscribed) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Event_Subscribed) + if !ok { + that2, ok := that.(Event_Subscribed) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Event_Subscribed") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Event_Subscribed but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Event_Subscribed but is not nil && this == nil") + } + if !this.GetState.Equal(that1.GetState) { + return fmt.Errorf("GetState this(%v) Not Equal that(%v)", this.GetState, that1.GetState) + } + return nil +} +func (this *Event_Subscribed) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Event_Subscribed) + if !ok { + that2, ok := that.(Event_Subscribed) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.GetState.Equal(that1.GetState) { + return false + } + return true +} +func (this *Event_TaskAdded) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Event_TaskAdded) + if !ok { + that2, ok := that.(Event_TaskAdded) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Event_TaskAdded") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Event_TaskAdded but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Event_TaskAdded but is not nil && this == nil") + } + if !this.Task.Equal(&that1.Task) { + return fmt.Errorf("Task this(%v) Not Equal that(%v)", this.Task, that1.Task) + } + return nil +} +func (this *Event_TaskAdded) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Event_TaskAdded) + if !ok { + that2, ok := that.(Event_TaskAdded) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.Task.Equal(&that1.Task) { + return false + } + return true +} +func (this *Event_TaskUpdated) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Event_TaskUpdated) + if !ok { + that2, ok := that.(Event_TaskUpdated) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Event_TaskUpdated") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Event_TaskUpdated but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Event_TaskUpdated but is not nil && this == nil") + } + if !this.FrameworkID.Equal(&that1.FrameworkID) { + return fmt.Errorf("FrameworkID this(%v) Not Equal that(%v)", this.FrameworkID, that1.FrameworkID) + } + if !this.Status.Equal(&that1.Status) { + return fmt.Errorf("Status this(%v) Not Equal that(%v)", this.Status, that1.Status) + } + if this.State != nil && that1.State != nil { + if *this.State != *that1.State { + return fmt.Errorf("State this(%v) Not Equal that(%v)", *this.State, *that1.State) + } + } else if this.State != nil { + return fmt.Errorf("this.State == nil && that.State != nil") + } else if that1.State != nil { + return fmt.Errorf("State this(%v) Not Equal that(%v)", this.State, that1.State) + } + return nil +} +func (this *Event_TaskUpdated) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Event_TaskUpdated) + if !ok { + that2, ok := that.(Event_TaskUpdated) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.FrameworkID.Equal(&that1.FrameworkID) { + return false + } + if !this.Status.Equal(&that1.Status) { + return false + } + if this.State != nil && that1.State != nil { + if *this.State != *that1.State { + return false + } + } else if this.State != nil { + return false + } else if that1.State != nil { + return false + } + return true +} +func (this *Event_AgentAdded) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Event_AgentAdded) + if !ok { + that2, ok := that.(Event_AgentAdded) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Event_AgentAdded") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Event_AgentAdded but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Event_AgentAdded but is not nil && this == nil") + } + if !this.Agent.Equal(&that1.Agent) { + return fmt.Errorf("Agent this(%v) Not Equal that(%v)", this.Agent, that1.Agent) + } + return nil +} +func (this *Event_AgentAdded) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Event_AgentAdded) + if !ok { + that2, ok := that.(Event_AgentAdded) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.Agent.Equal(&that1.Agent) { + return false + } + return true +} +func (this *Event_AgentRemoved) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*Event_AgentRemoved) + if !ok { + that2, ok := that.(Event_AgentRemoved) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *Event_AgentRemoved") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *Event_AgentRemoved but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *Event_AgentRemoved but is not nil && this == nil") + } + if !this.AgentID.Equal(&that1.AgentID) { + return fmt.Errorf("AgentID this(%v) Not Equal that(%v)", this.AgentID, that1.AgentID) + } + return nil +} +func (this *Event_AgentRemoved) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*Event_AgentRemoved) + if !ok { + that2, ok := that.(Event_AgentRemoved) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if !this.AgentID.Equal(&that1.AgentID) { + return false + } + return true +} +func (this *Call) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 19) + s = append(s, "&master.Call{") + s = append(s, "Type: "+fmt.Sprintf("%#v", this.Type)+",\n") + if this.GetMetrics != nil { + s = append(s, "GetMetrics: "+fmt.Sprintf("%#v", this.GetMetrics)+",\n") + } + if this.SetLoggingLevel != nil { + s = append(s, "SetLoggingLevel: "+fmt.Sprintf("%#v", this.SetLoggingLevel)+",\n") + } + if this.ListFiles != nil { + s = append(s, "ListFiles: "+fmt.Sprintf("%#v", this.ListFiles)+",\n") + } + if this.ReadFile != nil { + s = append(s, "ReadFile: "+fmt.Sprintf("%#v", this.ReadFile)+",\n") + } + if this.UpdateWeights != nil { + s = append(s, "UpdateWeights: "+fmt.Sprintf("%#v", this.UpdateWeights)+",\n") + } + if this.ReserveResources != nil { + s = append(s, "ReserveResources: "+fmt.Sprintf("%#v", this.ReserveResources)+",\n") + } + if this.UnreserveResources != nil { + s = append(s, "UnreserveResources: "+fmt.Sprintf("%#v", this.UnreserveResources)+",\n") + } + if this.CreateVolumes != nil { + s = append(s, "CreateVolumes: "+fmt.Sprintf("%#v", this.CreateVolumes)+",\n") + } + if this.DestroyVolumes != nil { + s = append(s, "DestroyVolumes: "+fmt.Sprintf("%#v", this.DestroyVolumes)+",\n") + } + if this.UpdateMaintenanceSchedule != nil { + s = append(s, "UpdateMaintenanceSchedule: "+fmt.Sprintf("%#v", this.UpdateMaintenanceSchedule)+",\n") + } + if this.StartMaintenance != nil { + s = append(s, "StartMaintenance: "+fmt.Sprintf("%#v", this.StartMaintenance)+",\n") + } + if this.StopMaintenance != nil { + s = append(s, "StopMaintenance: "+fmt.Sprintf("%#v", this.StopMaintenance)+",\n") + } + if this.SetQuota != nil { + s = append(s, "SetQuota: "+fmt.Sprintf("%#v", this.SetQuota)+",\n") + } + if this.RemoveQuota != nil { + s = append(s, "RemoveQuota: "+fmt.Sprintf("%#v", this.RemoveQuota)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_GetMetrics) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Call_GetMetrics{") + if this.Timeout != nil { + s = append(s, "Timeout: "+fmt.Sprintf("%#v", this.Timeout)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_SetLoggingLevel) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&master.Call_SetLoggingLevel{") + s = append(s, "Level: "+fmt.Sprintf("%#v", this.Level)+",\n") + s = append(s, "Duration: "+strings.Replace(this.Duration.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_ListFiles) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Call_ListFiles{") + s = append(s, "Path: "+fmt.Sprintf("%#v", this.Path)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_ReadFile) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&master.Call_ReadFile{") + s = append(s, "Path: "+fmt.Sprintf("%#v", this.Path)+",\n") + s = append(s, "Offset: "+fmt.Sprintf("%#v", this.Offset)+",\n") + if this.Length != nil { + s = append(s, "Length: "+valueToGoStringMaster(this.Length, "uint64")+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_UpdateWeights) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Call_UpdateWeights{") + if this.WeightInfos != nil { + s = append(s, "WeightInfos: "+fmt.Sprintf("%#v", this.WeightInfos)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_ReserveResources) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&master.Call_ReserveResources{") + s = append(s, "AgentID: "+strings.Replace(this.AgentID.GoString(), `&`, ``, 1)+",\n") + if this.Resources != nil { + s = append(s, "Resources: "+fmt.Sprintf("%#v", this.Resources)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_UnreserveResources) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&master.Call_UnreserveResources{") + s = append(s, "AgentID: "+strings.Replace(this.AgentID.GoString(), `&`, ``, 1)+",\n") + if this.Resources != nil { + s = append(s, "Resources: "+fmt.Sprintf("%#v", this.Resources)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_CreateVolumes) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&master.Call_CreateVolumes{") + s = append(s, "AgentID: "+strings.Replace(this.AgentID.GoString(), `&`, ``, 1)+",\n") + if this.Volumes != nil { + s = append(s, "Volumes: "+fmt.Sprintf("%#v", this.Volumes)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_DestroyVolumes) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&master.Call_DestroyVolumes{") + s = append(s, "AgentID: "+strings.Replace(this.AgentID.GoString(), `&`, ``, 1)+",\n") + if this.Volumes != nil { + s = append(s, "Volumes: "+fmt.Sprintf("%#v", this.Volumes)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_UpdateMaintenanceSchedule) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Call_UpdateMaintenanceSchedule{") + s = append(s, "Schedule: "+strings.Replace(this.Schedule.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_StartMaintenance) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Call_StartMaintenance{") + if this.Machines != nil { + s = append(s, "Machines: "+fmt.Sprintf("%#v", this.Machines)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_StopMaintenance) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Call_StopMaintenance{") + if this.Machines != nil { + s = append(s, "Machines: "+fmt.Sprintf("%#v", this.Machines)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_SetQuota) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Call_SetQuota{") + s = append(s, "QuotaRequest: "+strings.Replace(this.QuotaRequest.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Call_RemoveQuota) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Call_RemoveQuota{") + s = append(s, "Role: "+fmt.Sprintf("%#v", this.Role)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 23) + s = append(s, "&master.Response{") + s = append(s, "Type: "+fmt.Sprintf("%#v", this.Type)+",\n") + if this.GetHealth != nil { + s = append(s, "GetHealth: "+fmt.Sprintf("%#v", this.GetHealth)+",\n") + } + if this.GetFlags != nil { + s = append(s, "GetFlags: "+fmt.Sprintf("%#v", this.GetFlags)+",\n") + } + if this.GetVersion != nil { + s = append(s, "GetVersion: "+fmt.Sprintf("%#v", this.GetVersion)+",\n") + } + if this.GetMetrics != nil { + s = append(s, "GetMetrics: "+fmt.Sprintf("%#v", this.GetMetrics)+",\n") + } + if this.GetLoggingLevel != nil { + s = append(s, "GetLoggingLevel: "+fmt.Sprintf("%#v", this.GetLoggingLevel)+",\n") + } + if this.ListFiles != nil { + s = append(s, "ListFiles: "+fmt.Sprintf("%#v", this.ListFiles)+",\n") + } + if this.ReadFile != nil { + s = append(s, "ReadFile: "+fmt.Sprintf("%#v", this.ReadFile)+",\n") + } + if this.GetState != nil { + s = append(s, "GetState: "+fmt.Sprintf("%#v", this.GetState)+",\n") + } + if this.GetAgents != nil { + s = append(s, "GetAgents: "+fmt.Sprintf("%#v", this.GetAgents)+",\n") + } + if this.GetFrameworks != nil { + s = append(s, "GetFrameworks: "+fmt.Sprintf("%#v", this.GetFrameworks)+",\n") + } + if this.GetExecutors != nil { + s = append(s, "GetExecutors: "+fmt.Sprintf("%#v", this.GetExecutors)+",\n") + } + if this.GetTasks != nil { + s = append(s, "GetTasks: "+fmt.Sprintf("%#v", this.GetTasks)+",\n") + } + if this.GetRoles != nil { + s = append(s, "GetRoles: "+fmt.Sprintf("%#v", this.GetRoles)+",\n") + } + if this.GetWeights != nil { + s = append(s, "GetWeights: "+fmt.Sprintf("%#v", this.GetWeights)+",\n") + } + if this.GetMaster != nil { + s = append(s, "GetMaster: "+fmt.Sprintf("%#v", this.GetMaster)+",\n") + } + if this.GetMaintenanceStatus != nil { + s = append(s, "GetMaintenanceStatus: "+fmt.Sprintf("%#v", this.GetMaintenanceStatus)+",\n") + } + if this.GetMaintenanceSchedule != nil { + s = append(s, "GetMaintenanceSchedule: "+fmt.Sprintf("%#v", this.GetMaintenanceSchedule)+",\n") + } + if this.GetQuota != nil { + s = append(s, "GetQuota: "+fmt.Sprintf("%#v", this.GetQuota)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetHealth) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetHealth{") + s = append(s, "Healthy: "+fmt.Sprintf("%#v", this.Healthy)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetFlags) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetFlags{") + if this.Flags != nil { + s = append(s, "Flags: "+fmt.Sprintf("%#v", this.Flags)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetVersion) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetVersion{") + s = append(s, "VersionInfo: "+strings.Replace(this.VersionInfo.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetMetrics) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetMetrics{") + if this.Metrics != nil { + s = append(s, "Metrics: "+fmt.Sprintf("%#v", this.Metrics)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetLoggingLevel) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetLoggingLevel{") + s = append(s, "Level: "+fmt.Sprintf("%#v", this.Level)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_ListFiles) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_ListFiles{") + if this.FileInfos != nil { + s = append(s, "FileInfos: "+fmt.Sprintf("%#v", this.FileInfos)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_ReadFile) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&master.Response_ReadFile{") + s = append(s, "Size: "+fmt.Sprintf("%#v", this.Size)+",\n") + if this.Data != nil { + s = append(s, "Data: "+valueToGoStringMaster(this.Data, "byte")+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetState) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 8) + s = append(s, "&master.Response_GetState{") + if this.GetTasks != nil { + s = append(s, "GetTasks: "+fmt.Sprintf("%#v", this.GetTasks)+",\n") + } + if this.GetExecutors != nil { + s = append(s, "GetExecutors: "+fmt.Sprintf("%#v", this.GetExecutors)+",\n") + } + if this.GetFrameworks != nil { + s = append(s, "GetFrameworks: "+fmt.Sprintf("%#v", this.GetFrameworks)+",\n") + } + if this.GetAgents != nil { + s = append(s, "GetAgents: "+fmt.Sprintf("%#v", this.GetAgents)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetAgents) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&master.Response_GetAgents{") + if this.Agents != nil { + s = append(s, "Agents: "+fmt.Sprintf("%#v", this.Agents)+",\n") + } + if this.RecoveredAgents != nil { + s = append(s, "RecoveredAgents: "+fmt.Sprintf("%#v", this.RecoveredAgents)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetAgents_Agent) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 13) + s = append(s, "&master.Response_GetAgents_Agent{") + s = append(s, "AgentInfo: "+strings.Replace(this.AgentInfo.GoString(), `&`, ``, 1)+",\n") + s = append(s, "Active: "+fmt.Sprintf("%#v", this.Active)+",\n") + s = append(s, "Version: "+fmt.Sprintf("%#v", this.Version)+",\n") + if this.PID != nil { + s = append(s, "PID: "+valueToGoStringMaster(this.PID, "string")+",\n") + } + if this.RegisteredTime != nil { + s = append(s, "RegisteredTime: "+fmt.Sprintf("%#v", this.RegisteredTime)+",\n") + } + if this.ReregisteredTime != nil { + s = append(s, "ReregisteredTime: "+fmt.Sprintf("%#v", this.ReregisteredTime)+",\n") + } + if this.TotalResources != nil { + s = append(s, "TotalResources: "+fmt.Sprintf("%#v", this.TotalResources)+",\n") + } + if this.AllocatedResources != nil { + s = append(s, "AllocatedResources: "+fmt.Sprintf("%#v", this.AllocatedResources)+",\n") + } + if this.OfferedResources != nil { + s = append(s, "OfferedResources: "+fmt.Sprintf("%#v", this.OfferedResources)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetFrameworks) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&master.Response_GetFrameworks{") + if this.Frameworks != nil { + s = append(s, "Frameworks: "+fmt.Sprintf("%#v", this.Frameworks)+",\n") + } + if this.CompletedFrameworks != nil { + s = append(s, "CompletedFrameworks: "+fmt.Sprintf("%#v", this.CompletedFrameworks)+",\n") + } + if this.RecoveredFrameworks != nil { + s = append(s, "RecoveredFrameworks: "+fmt.Sprintf("%#v", this.RecoveredFrameworks)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetFrameworks_Framework) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 15) + s = append(s, "&master.Response_GetFrameworks_Framework{") + s = append(s, "FrameworkInfo: "+strings.Replace(this.FrameworkInfo.GoString(), `&`, ``, 1)+",\n") + s = append(s, "Active: "+fmt.Sprintf("%#v", this.Active)+",\n") + s = append(s, "Connected: "+fmt.Sprintf("%#v", this.Connected)+",\n") + s = append(s, "Recovered: "+fmt.Sprintf("%#v", this.Recovered)+",\n") + if this.RegisteredTime != nil { + s = append(s, "RegisteredTime: "+fmt.Sprintf("%#v", this.RegisteredTime)+",\n") + } + if this.ReregisteredTime != nil { + s = append(s, "ReregisteredTime: "+fmt.Sprintf("%#v", this.ReregisteredTime)+",\n") + } + if this.UnregisteredTime != nil { + s = append(s, "UnregisteredTime: "+fmt.Sprintf("%#v", this.UnregisteredTime)+",\n") + } + if this.Offers != nil { + s = append(s, "Offers: "+fmt.Sprintf("%#v", this.Offers)+",\n") + } + if this.InverseOffers != nil { + s = append(s, "InverseOffers: "+fmt.Sprintf("%#v", this.InverseOffers)+",\n") + } + if this.AllocatedResources != nil { + s = append(s, "AllocatedResources: "+fmt.Sprintf("%#v", this.AllocatedResources)+",\n") + } + if this.OfferedResources != nil { + s = append(s, "OfferedResources: "+fmt.Sprintf("%#v", this.OfferedResources)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetExecutors) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&master.Response_GetExecutors{") + if this.Executors != nil { + s = append(s, "Executors: "+fmt.Sprintf("%#v", this.Executors)+",\n") + } + if this.OrphanExecutors != nil { + s = append(s, "OrphanExecutors: "+fmt.Sprintf("%#v", this.OrphanExecutors)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetExecutors_Executor) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&master.Response_GetExecutors_Executor{") + s = append(s, "ExecutorInfo: "+strings.Replace(this.ExecutorInfo.GoString(), `&`, ``, 1)+",\n") + s = append(s, "AgentID: "+strings.Replace(this.AgentID.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetTasks) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 9) + s = append(s, "&master.Response_GetTasks{") + if this.PendingTasks != nil { + s = append(s, "PendingTasks: "+fmt.Sprintf("%#v", this.PendingTasks)+",\n") + } + if this.Tasks != nil { + s = append(s, "Tasks: "+fmt.Sprintf("%#v", this.Tasks)+",\n") + } + if this.UnreachableTasks != nil { + s = append(s, "UnreachableTasks: "+fmt.Sprintf("%#v", this.UnreachableTasks)+",\n") + } + if this.CompletedTasks != nil { + s = append(s, "CompletedTasks: "+fmt.Sprintf("%#v", this.CompletedTasks)+",\n") + } + if this.OrphanTasks != nil { + s = append(s, "OrphanTasks: "+fmt.Sprintf("%#v", this.OrphanTasks)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetRoles) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetRoles{") + if this.Roles != nil { + s = append(s, "Roles: "+fmt.Sprintf("%#v", this.Roles)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetWeights) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetWeights{") + if this.WeightInfos != nil { + s = append(s, "WeightInfos: "+fmt.Sprintf("%#v", this.WeightInfos)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetMaster) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetMaster{") + if this.MasterInfo != nil { + s = append(s, "MasterInfo: "+fmt.Sprintf("%#v", this.MasterInfo)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetMaintenanceStatus) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetMaintenanceStatus{") + s = append(s, "Status: "+strings.Replace(this.Status.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetMaintenanceSchedule) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetMaintenanceSchedule{") + s = append(s, "Schedule: "+strings.Replace(this.Schedule.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Response_GetQuota) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Response_GetQuota{") + s = append(s, "Status: "+strings.Replace(this.Status.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Event) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 10) + s = append(s, "&master.Event{") + s = append(s, "Type: "+fmt.Sprintf("%#v", this.Type)+",\n") + if this.Subscribed != nil { + s = append(s, "Subscribed: "+fmt.Sprintf("%#v", this.Subscribed)+",\n") + } + if this.TaskAdded != nil { + s = append(s, "TaskAdded: "+fmt.Sprintf("%#v", this.TaskAdded)+",\n") + } + if this.TaskUpdated != nil { + s = append(s, "TaskUpdated: "+fmt.Sprintf("%#v", this.TaskUpdated)+",\n") + } + if this.AgentAdded != nil { + s = append(s, "AgentAdded: "+fmt.Sprintf("%#v", this.AgentAdded)+",\n") + } + if this.AgentRemoved != nil { + s = append(s, "AgentRemoved: "+fmt.Sprintf("%#v", this.AgentRemoved)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Event_Subscribed) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Event_Subscribed{") + if this.GetState != nil { + s = append(s, "GetState: "+fmt.Sprintf("%#v", this.GetState)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Event_TaskAdded) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Event_TaskAdded{") + s = append(s, "Task: "+strings.Replace(this.Task.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Event_TaskUpdated) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&master.Event_TaskUpdated{") + s = append(s, "FrameworkID: "+strings.Replace(this.FrameworkID.GoString(), `&`, ``, 1)+",\n") + s = append(s, "Status: "+strings.Replace(this.Status.GoString(), `&`, ``, 1)+",\n") + if this.State != nil { + s = append(s, "State: "+valueToGoStringMaster(this.State, "mesos.TaskState")+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Event_AgentAdded) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Event_AgentAdded{") + s = append(s, "Agent: "+strings.Replace(this.Agent.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Event_AgentRemoved) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&master.Event_AgentRemoved{") + s = append(s, "AgentID: "+strings.Replace(this.AgentID.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringMaster(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} +func (m *Call) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Type)) + if m.GetMetrics != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetMetrics.ProtoSize())) + n1, err := m.GetMetrics.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n1 + } + if m.SetLoggingLevel != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.SetLoggingLevel.ProtoSize())) + n2, err := m.SetLoggingLevel.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + } + if m.ListFiles != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.ListFiles.ProtoSize())) + n3, err := m.ListFiles.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + } + if m.ReadFile != nil { + dAtA[i] = 0x2a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.ReadFile.ProtoSize())) + n4, err := m.ReadFile.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n4 + } + if m.UpdateWeights != nil { + dAtA[i] = 0x32 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.UpdateWeights.ProtoSize())) + n5, err := m.UpdateWeights.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n5 + } + if m.ReserveResources != nil { + dAtA[i] = 0x3a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.ReserveResources.ProtoSize())) + n6, err := m.ReserveResources.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n6 + } + if m.UnreserveResources != nil { + dAtA[i] = 0x42 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.UnreserveResources.ProtoSize())) + n7, err := m.UnreserveResources.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n7 + } + if m.CreateVolumes != nil { + dAtA[i] = 0x4a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.CreateVolumes.ProtoSize())) + n8, err := m.CreateVolumes.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n8 + } + if m.DestroyVolumes != nil { + dAtA[i] = 0x52 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.DestroyVolumes.ProtoSize())) + n9, err := m.DestroyVolumes.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n9 + } + if m.UpdateMaintenanceSchedule != nil { + dAtA[i] = 0x5a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.UpdateMaintenanceSchedule.ProtoSize())) + n10, err := m.UpdateMaintenanceSchedule.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n10 + } + if m.StartMaintenance != nil { + dAtA[i] = 0x62 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.StartMaintenance.ProtoSize())) + n11, err := m.StartMaintenance.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n11 + } + if m.StopMaintenance != nil { + dAtA[i] = 0x6a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.StopMaintenance.ProtoSize())) + n12, err := m.StopMaintenance.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n12 + } + if m.SetQuota != nil { + dAtA[i] = 0x72 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.SetQuota.ProtoSize())) + n13, err := m.SetQuota.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n13 + } + if m.RemoveQuota != nil { + dAtA[i] = 0x7a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.RemoveQuota.ProtoSize())) + n14, err := m.RemoveQuota.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n14 + } + return i, nil +} + +func (m *Call_GetMetrics) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_GetMetrics) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Timeout != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Timeout.ProtoSize())) + n15, err := m.Timeout.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n15 + } + return i, nil +} + +func (m *Call_SetLoggingLevel) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_SetLoggingLevel) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Level)) + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Duration.ProtoSize())) + n16, err := m.Duration.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n16 + return i, nil +} + +func (m *Call_ListFiles) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_ListFiles) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(len(m.Path))) + i += copy(dAtA[i:], m.Path) + return i, nil +} + +func (m *Call_ReadFile) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_ReadFile) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(len(m.Path))) + i += copy(dAtA[i:], m.Path) + dAtA[i] = 0x10 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Offset)) + if m.Length != nil { + dAtA[i] = 0x18 + i++ + i = encodeVarintMaster(dAtA, i, uint64(*m.Length)) + } + return i, nil +} + +func (m *Call_UpdateWeights) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_UpdateWeights) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.WeightInfos) > 0 { + for _, msg := range m.WeightInfos { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Call_ReserveResources) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_ReserveResources) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.AgentID.ProtoSize())) + n17, err := m.AgentID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n17 + if len(m.Resources) > 0 { + for _, msg := range m.Resources { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Call_UnreserveResources) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_UnreserveResources) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.AgentID.ProtoSize())) + n18, err := m.AgentID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n18 + if len(m.Resources) > 0 { + for _, msg := range m.Resources { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Call_CreateVolumes) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_CreateVolumes) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.AgentID.ProtoSize())) + n19, err := m.AgentID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n19 + if len(m.Volumes) > 0 { + for _, msg := range m.Volumes { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Call_DestroyVolumes) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_DestroyVolumes) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.AgentID.ProtoSize())) + n20, err := m.AgentID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n20 + if len(m.Volumes) > 0 { + for _, msg := range m.Volumes { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Call_UpdateMaintenanceSchedule) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_UpdateMaintenanceSchedule) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Schedule.ProtoSize())) + n21, err := m.Schedule.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n21 + return i, nil +} + +func (m *Call_StartMaintenance) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_StartMaintenance) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Machines) > 0 { + for _, msg := range m.Machines { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Call_StopMaintenance) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_StopMaintenance) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Machines) > 0 { + for _, msg := range m.Machines { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Call_SetQuota) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_SetQuota) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.QuotaRequest.ProtoSize())) + n22, err := m.QuotaRequest.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n22 + return i, nil +} + +func (m *Call_RemoveQuota) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Call_RemoveQuota) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(len(m.Role))) + i += copy(dAtA[i:], m.Role) + return i, nil +} + +func (m *Response) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Type)) + if m.GetHealth != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetHealth.ProtoSize())) + n23, err := m.GetHealth.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n23 + } + if m.GetFlags != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetFlags.ProtoSize())) + n24, err := m.GetFlags.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n24 + } + if m.GetVersion != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetVersion.ProtoSize())) + n25, err := m.GetVersion.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n25 + } + if m.GetMetrics != nil { + dAtA[i] = 0x2a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetMetrics.ProtoSize())) + n26, err := m.GetMetrics.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n26 + } + if m.GetLoggingLevel != nil { + dAtA[i] = 0x32 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetLoggingLevel.ProtoSize())) + n27, err := m.GetLoggingLevel.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n27 + } + if m.ListFiles != nil { + dAtA[i] = 0x3a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.ListFiles.ProtoSize())) + n28, err := m.ListFiles.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n28 + } + if m.ReadFile != nil { + dAtA[i] = 0x42 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.ReadFile.ProtoSize())) + n29, err := m.ReadFile.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n29 + } + if m.GetState != nil { + dAtA[i] = 0x4a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetState.ProtoSize())) + n30, err := m.GetState.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n30 + } + if m.GetAgents != nil { + dAtA[i] = 0x52 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetAgents.ProtoSize())) + n31, err := m.GetAgents.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n31 + } + if m.GetFrameworks != nil { + dAtA[i] = 0x5a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetFrameworks.ProtoSize())) + n32, err := m.GetFrameworks.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n32 + } + if m.GetExecutors != nil { + dAtA[i] = 0x62 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetExecutors.ProtoSize())) + n33, err := m.GetExecutors.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n33 + } + if m.GetTasks != nil { + dAtA[i] = 0x6a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetTasks.ProtoSize())) + n34, err := m.GetTasks.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n34 + } + if m.GetRoles != nil { + dAtA[i] = 0x72 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetRoles.ProtoSize())) + n35, err := m.GetRoles.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n35 + } + if m.GetWeights != nil { + dAtA[i] = 0x7a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetWeights.ProtoSize())) + n36, err := m.GetWeights.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n36 + } + if m.GetMaster != nil { + dAtA[i] = 0x82 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetMaster.ProtoSize())) + n37, err := m.GetMaster.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n37 + } + if m.GetMaintenanceStatus != nil { + dAtA[i] = 0x8a + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetMaintenanceStatus.ProtoSize())) + n38, err := m.GetMaintenanceStatus.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n38 + } + if m.GetMaintenanceSchedule != nil { + dAtA[i] = 0x92 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetMaintenanceSchedule.ProtoSize())) + n39, err := m.GetMaintenanceSchedule.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n39 + } + if m.GetQuota != nil { + dAtA[i] = 0x9a + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetQuota.ProtoSize())) + n40, err := m.GetQuota.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n40 + } + return i, nil +} + +func (m *Response_GetHealth) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetHealth) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + if m.Healthy { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + return i, nil +} + +func (m *Response_GetFlags) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetFlags) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Flags) > 0 { + for _, msg := range m.Flags { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_GetVersion) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetVersion) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.VersionInfo.ProtoSize())) + n41, err := m.VersionInfo.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n41 + return i, nil +} + +func (m *Response_GetMetrics) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetMetrics) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Metrics) > 0 { + for _, msg := range m.Metrics { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_GetLoggingLevel) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetLoggingLevel) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Level)) + return i, nil +} + +func (m *Response_ListFiles) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_ListFiles) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.FileInfos) > 0 { + for _, msg := range m.FileInfos { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_ReadFile) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_ReadFile) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Size)) + if m.Data == nil { + return 0, github_com_gogo_protobuf_proto.NewRequiredNotSetError("data") + } else { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(len(m.Data))) + i += copy(dAtA[i:], m.Data) + } + return i, nil +} + +func (m *Response_GetState) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetState) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.GetTasks != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetTasks.ProtoSize())) + n42, err := m.GetTasks.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n42 + } + if m.GetExecutors != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetExecutors.ProtoSize())) + n43, err := m.GetExecutors.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n43 + } + if m.GetFrameworks != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetFrameworks.ProtoSize())) + n44, err := m.GetFrameworks.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n44 + } + if m.GetAgents != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetAgents.ProtoSize())) + n45, err := m.GetAgents.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n45 + } + return i, nil +} + +func (m *Response_GetAgents) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetAgents) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Agents) > 0 { + for _, msg := range m.Agents { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.RecoveredAgents) > 0 { + for _, msg := range m.RecoveredAgents { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_GetAgents_Agent) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetAgents_Agent) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.AgentInfo.ProtoSize())) + n46, err := m.AgentInfo.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n46 + dAtA[i] = 0x10 + i++ + if m.Active { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + dAtA[i] = 0x1a + i++ + i = encodeVarintMaster(dAtA, i, uint64(len(m.Version))) + i += copy(dAtA[i:], m.Version) + if m.PID != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintMaster(dAtA, i, uint64(len(*m.PID))) + i += copy(dAtA[i:], *m.PID) + } + if m.RegisteredTime != nil { + dAtA[i] = 0x2a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.RegisteredTime.ProtoSize())) + n47, err := m.RegisteredTime.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n47 + } + if m.ReregisteredTime != nil { + dAtA[i] = 0x32 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.ReregisteredTime.ProtoSize())) + n48, err := m.ReregisteredTime.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n48 + } + if len(m.TotalResources) > 0 { + for _, msg := range m.TotalResources { + dAtA[i] = 0x3a + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.AllocatedResources) > 0 { + for _, msg := range m.AllocatedResources { + dAtA[i] = 0x42 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.OfferedResources) > 0 { + for _, msg := range m.OfferedResources { + dAtA[i] = 0x4a + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_GetFrameworks) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetFrameworks) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Frameworks) > 0 { + for _, msg := range m.Frameworks { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.CompletedFrameworks) > 0 { + for _, msg := range m.CompletedFrameworks { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.RecoveredFrameworks) > 0 { + for _, msg := range m.RecoveredFrameworks { + dAtA[i] = 0x1a + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_GetFrameworks_Framework) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetFrameworks_Framework) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.FrameworkInfo.ProtoSize())) + n49, err := m.FrameworkInfo.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n49 + dAtA[i] = 0x10 + i++ + if m.Active { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + dAtA[i] = 0x18 + i++ + if m.Connected { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + if m.RegisteredTime != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.RegisteredTime.ProtoSize())) + n50, err := m.RegisteredTime.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n50 + } + if m.ReregisteredTime != nil { + dAtA[i] = 0x2a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.ReregisteredTime.ProtoSize())) + n51, err := m.ReregisteredTime.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n51 + } + if m.UnregisteredTime != nil { + dAtA[i] = 0x32 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.UnregisteredTime.ProtoSize())) + n52, err := m.UnregisteredTime.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n52 + } + if len(m.Offers) > 0 { + for _, msg := range m.Offers { + dAtA[i] = 0x3a + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.InverseOffers) > 0 { + for _, msg := range m.InverseOffers { + dAtA[i] = 0x42 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.AllocatedResources) > 0 { + for _, msg := range m.AllocatedResources { + dAtA[i] = 0x4a + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.OfferedResources) > 0 { + for _, msg := range m.OfferedResources { + dAtA[i] = 0x52 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + dAtA[i] = 0x58 + i++ + if m.Recovered { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + return i, nil +} + +func (m *Response_GetExecutors) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetExecutors) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Executors) > 0 { + for _, msg := range m.Executors { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.OrphanExecutors) > 0 { + for _, msg := range m.OrphanExecutors { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_GetExecutors_Executor) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetExecutors_Executor) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.ExecutorInfo.ProtoSize())) + n53, err := m.ExecutorInfo.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n53 + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.AgentID.ProtoSize())) + n54, err := m.AgentID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n54 + return i, nil +} + +func (m *Response_GetTasks) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetTasks) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.PendingTasks) > 0 { + for _, msg := range m.PendingTasks { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.Tasks) > 0 { + for _, msg := range m.Tasks { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.CompletedTasks) > 0 { + for _, msg := range m.CompletedTasks { + dAtA[i] = 0x1a + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.OrphanTasks) > 0 { + for _, msg := range m.OrphanTasks { + dAtA[i] = 0x22 + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.UnreachableTasks) > 0 { + for _, msg := range m.UnreachableTasks { + dAtA[i] = 0x2a + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_GetRoles) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetRoles) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Roles) > 0 { + for _, msg := range m.Roles { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_GetWeights) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetWeights) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.WeightInfos) > 0 { + for _, msg := range m.WeightInfos { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *Response_GetMaster) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetMaster) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.MasterInfo != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.MasterInfo.ProtoSize())) + n55, err := m.MasterInfo.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n55 + } + return i, nil +} + +func (m *Response_GetMaintenanceStatus) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetMaintenanceStatus) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Status.ProtoSize())) + n56, err := m.Status.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n56 + return i, nil +} + +func (m *Response_GetMaintenanceSchedule) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetMaintenanceSchedule) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Schedule.ProtoSize())) + n57, err := m.Schedule.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n57 + return i, nil +} + +func (m *Response_GetQuota) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response_GetQuota) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Status.ProtoSize())) + n58, err := m.Status.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n58 + return i, nil +} + +func (m *Event) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Event) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Type)) + if m.Subscribed != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Subscribed.ProtoSize())) + n59, err := m.Subscribed.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n59 + } + if m.TaskAdded != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.TaskAdded.ProtoSize())) + n60, err := m.TaskAdded.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n60 + } + if m.TaskUpdated != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.TaskUpdated.ProtoSize())) + n61, err := m.TaskUpdated.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n61 + } + if m.AgentAdded != nil { + dAtA[i] = 0x2a + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.AgentAdded.ProtoSize())) + n62, err := m.AgentAdded.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n62 + } + if m.AgentRemoved != nil { + dAtA[i] = 0x32 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.AgentRemoved.ProtoSize())) + n63, err := m.AgentRemoved.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n63 + } + return i, nil +} + +func (m *Event_Subscribed) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Event_Subscribed) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.GetState != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.GetState.ProtoSize())) + n64, err := m.GetState.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n64 + } + return i, nil +} + +func (m *Event_TaskAdded) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Event_TaskAdded) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Task.ProtoSize())) + n65, err := m.Task.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n65 + return i, nil +} + +func (m *Event_TaskUpdated) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Event_TaskUpdated) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.FrameworkID.ProtoSize())) + n66, err := m.FrameworkID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n66 + dAtA[i] = 0x12 + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Status.ProtoSize())) + n67, err := m.Status.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n67 + if m.State == nil { + return 0, github_com_gogo_protobuf_proto.NewRequiredNotSetError("state") + } else { + dAtA[i] = 0x18 + i++ + i = encodeVarintMaster(dAtA, i, uint64(*m.State)) + } + return i, nil +} + +func (m *Event_AgentAdded) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Event_AgentAdded) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.Agent.ProtoSize())) + n68, err := m.Agent.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n68 + return i, nil +} + +func (m *Event_AgentRemoved) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Event_AgentRemoved) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintMaster(dAtA, i, uint64(m.AgentID.ProtoSize())) + n69, err := m.AgentID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n69 + return i, nil +} + +func encodeFixed64Master(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 encodeFixed32Master(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 encodeVarintMaster(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func NewPopulatedCall(r randyMaster, easy bool) *Call { + this := &Call{} + this.Type = Call_Type([]int32{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30}[r.Intn(31)]) + if r.Intn(10) != 0 { + this.GetMetrics = NewPopulatedCall_GetMetrics(r, easy) + } + if r.Intn(10) != 0 { + this.SetLoggingLevel = NewPopulatedCall_SetLoggingLevel(r, easy) + } + if r.Intn(10) != 0 { + this.ListFiles = NewPopulatedCall_ListFiles(r, easy) + } + if r.Intn(10) != 0 { + this.ReadFile = NewPopulatedCall_ReadFile(r, easy) + } + if r.Intn(10) != 0 { + this.UpdateWeights = NewPopulatedCall_UpdateWeights(r, easy) + } + if r.Intn(10) != 0 { + this.ReserveResources = NewPopulatedCall_ReserveResources(r, easy) + } + if r.Intn(10) != 0 { + this.UnreserveResources = NewPopulatedCall_UnreserveResources(r, easy) + } + if r.Intn(10) != 0 { + this.CreateVolumes = NewPopulatedCall_CreateVolumes(r, easy) + } + if r.Intn(10) != 0 { + this.DestroyVolumes = NewPopulatedCall_DestroyVolumes(r, easy) + } + if r.Intn(10) != 0 { + this.UpdateMaintenanceSchedule = NewPopulatedCall_UpdateMaintenanceSchedule(r, easy) + } + if r.Intn(10) != 0 { + this.StartMaintenance = NewPopulatedCall_StartMaintenance(r, easy) + } + if r.Intn(10) != 0 { + this.StopMaintenance = NewPopulatedCall_StopMaintenance(r, easy) + } + if r.Intn(10) != 0 { + this.SetQuota = NewPopulatedCall_SetQuota(r, easy) + } + if r.Intn(10) != 0 { + this.RemoveQuota = NewPopulatedCall_RemoveQuota(r, easy) + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_GetMetrics(r randyMaster, easy bool) *Call_GetMetrics { + this := &Call_GetMetrics{} + if r.Intn(10) != 0 { + this.Timeout = mesos.NewPopulatedDurationInfo(r, easy) + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_SetLoggingLevel(r randyMaster, easy bool) *Call_SetLoggingLevel { + this := &Call_SetLoggingLevel{} + this.Level = uint32(r.Uint32()) + v1 := mesos.NewPopulatedDurationInfo(r, easy) + this.Duration = *v1 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_ListFiles(r randyMaster, easy bool) *Call_ListFiles { + this := &Call_ListFiles{} + this.Path = string(randStringMaster(r)) + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_ReadFile(r randyMaster, easy bool) *Call_ReadFile { + this := &Call_ReadFile{} + this.Path = string(randStringMaster(r)) + this.Offset = uint64(uint64(r.Uint32())) + if r.Intn(10) != 0 { + v2 := uint64(uint64(r.Uint32())) + this.Length = &v2 + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_UpdateWeights(r randyMaster, easy bool) *Call_UpdateWeights { + this := &Call_UpdateWeights{} + if r.Intn(10) != 0 { + v3 := r.Intn(5) + this.WeightInfos = make([]mesos.WeightInfo, v3) + for i := 0; i < v3; i++ { + v4 := mesos.NewPopulatedWeightInfo(r, easy) + this.WeightInfos[i] = *v4 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_ReserveResources(r randyMaster, easy bool) *Call_ReserveResources { + this := &Call_ReserveResources{} + v5 := mesos.NewPopulatedAgentID(r, easy) + this.AgentID = *v5 + if r.Intn(10) != 0 { + v6 := r.Intn(5) + this.Resources = make([]mesos.Resource, v6) + for i := 0; i < v6; i++ { + v7 := mesos.NewPopulatedResource(r, easy) + this.Resources[i] = *v7 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_UnreserveResources(r randyMaster, easy bool) *Call_UnreserveResources { + this := &Call_UnreserveResources{} + v8 := mesos.NewPopulatedAgentID(r, easy) + this.AgentID = *v8 + if r.Intn(10) != 0 { + v9 := r.Intn(5) + this.Resources = make([]mesos.Resource, v9) + for i := 0; i < v9; i++ { + v10 := mesos.NewPopulatedResource(r, easy) + this.Resources[i] = *v10 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_CreateVolumes(r randyMaster, easy bool) *Call_CreateVolumes { + this := &Call_CreateVolumes{} + v11 := mesos.NewPopulatedAgentID(r, easy) + this.AgentID = *v11 + if r.Intn(10) != 0 { + v12 := r.Intn(5) + this.Volumes = make([]mesos.Resource, v12) + for i := 0; i < v12; i++ { + v13 := mesos.NewPopulatedResource(r, easy) + this.Volumes[i] = *v13 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_DestroyVolumes(r randyMaster, easy bool) *Call_DestroyVolumes { + this := &Call_DestroyVolumes{} + v14 := mesos.NewPopulatedAgentID(r, easy) + this.AgentID = *v14 + if r.Intn(10) != 0 { + v15 := r.Intn(5) + this.Volumes = make([]mesos.Resource, v15) + for i := 0; i < v15; i++ { + v16 := mesos.NewPopulatedResource(r, easy) + this.Volumes[i] = *v16 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_UpdateMaintenanceSchedule(r randyMaster, easy bool) *Call_UpdateMaintenanceSchedule { + this := &Call_UpdateMaintenanceSchedule{} + v17 := mesos_maintenance.NewPopulatedSchedule(r, easy) + this.Schedule = *v17 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_StartMaintenance(r randyMaster, easy bool) *Call_StartMaintenance { + this := &Call_StartMaintenance{} + if r.Intn(10) != 0 { + v18 := r.Intn(5) + this.Machines = make([]mesos.MachineID, v18) + for i := 0; i < v18; i++ { + v19 := mesos.NewPopulatedMachineID(r, easy) + this.Machines[i] = *v19 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_StopMaintenance(r randyMaster, easy bool) *Call_StopMaintenance { + this := &Call_StopMaintenance{} + if r.Intn(10) != 0 { + v20 := r.Intn(5) + this.Machines = make([]mesos.MachineID, v20) + for i := 0; i < v20; i++ { + v21 := mesos.NewPopulatedMachineID(r, easy) + this.Machines[i] = *v21 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_SetQuota(r randyMaster, easy bool) *Call_SetQuota { + this := &Call_SetQuota{} + v22 := mesos_quota.NewPopulatedQuotaRequest(r, easy) + this.QuotaRequest = *v22 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedCall_RemoveQuota(r randyMaster, easy bool) *Call_RemoveQuota { + this := &Call_RemoveQuota{} + this.Role = string(randStringMaster(r)) + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse(r randyMaster, easy bool) *Response { + this := &Response{} + this.Type = Response_Type([]int32{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18}[r.Intn(19)]) + if r.Intn(10) != 0 { + this.GetHealth = NewPopulatedResponse_GetHealth(r, easy) + } + if r.Intn(10) != 0 { + this.GetFlags = NewPopulatedResponse_GetFlags(r, easy) + } + if r.Intn(10) != 0 { + this.GetVersion = NewPopulatedResponse_GetVersion(r, easy) + } + if r.Intn(10) != 0 { + this.GetMetrics = NewPopulatedResponse_GetMetrics(r, easy) + } + if r.Intn(10) != 0 { + this.GetLoggingLevel = NewPopulatedResponse_GetLoggingLevel(r, easy) + } + if r.Intn(10) != 0 { + this.ListFiles = NewPopulatedResponse_ListFiles(r, easy) + } + if r.Intn(10) != 0 { + this.ReadFile = NewPopulatedResponse_ReadFile(r, easy) + } + if r.Intn(10) == 0 { + this.GetState = NewPopulatedResponse_GetState(r, easy) + } + if r.Intn(10) != 0 { + this.GetAgents = NewPopulatedResponse_GetAgents(r, easy) + } + if r.Intn(10) != 0 { + this.GetFrameworks = NewPopulatedResponse_GetFrameworks(r, easy) + } + if r.Intn(10) != 0 { + this.GetExecutors = NewPopulatedResponse_GetExecutors(r, easy) + } + if r.Intn(10) == 0 { + this.GetTasks = NewPopulatedResponse_GetTasks(r, easy) + } + if r.Intn(10) != 0 { + this.GetRoles = NewPopulatedResponse_GetRoles(r, easy) + } + if r.Intn(10) != 0 { + this.GetWeights = NewPopulatedResponse_GetWeights(r, easy) + } + if r.Intn(10) != 0 { + this.GetMaster = NewPopulatedResponse_GetMaster(r, easy) + } + if r.Intn(10) != 0 { + this.GetMaintenanceStatus = NewPopulatedResponse_GetMaintenanceStatus(r, easy) + } + if r.Intn(10) != 0 { + this.GetMaintenanceSchedule = NewPopulatedResponse_GetMaintenanceSchedule(r, easy) + } + if r.Intn(10) != 0 { + this.GetQuota = NewPopulatedResponse_GetQuota(r, easy) + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetHealth(r randyMaster, easy bool) *Response_GetHealth { + this := &Response_GetHealth{} + this.Healthy = bool(bool(r.Intn(2) == 0)) + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetFlags(r randyMaster, easy bool) *Response_GetFlags { + this := &Response_GetFlags{} + if r.Intn(10) != 0 { + v23 := r.Intn(5) + this.Flags = make([]mesos.Flag, v23) + for i := 0; i < v23; i++ { + v24 := mesos.NewPopulatedFlag(r, easy) + this.Flags[i] = *v24 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetVersion(r randyMaster, easy bool) *Response_GetVersion { + this := &Response_GetVersion{} + v25 := mesos.NewPopulatedVersionInfo(r, easy) + this.VersionInfo = *v25 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetMetrics(r randyMaster, easy bool) *Response_GetMetrics { + this := &Response_GetMetrics{} + if r.Intn(10) != 0 { + v26 := r.Intn(5) + this.Metrics = make([]mesos.Metric, v26) + for i := 0; i < v26; i++ { + v27 := mesos.NewPopulatedMetric(r, easy) + this.Metrics[i] = *v27 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetLoggingLevel(r randyMaster, easy bool) *Response_GetLoggingLevel { + this := &Response_GetLoggingLevel{} + this.Level = uint32(r.Uint32()) + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_ListFiles(r randyMaster, easy bool) *Response_ListFiles { + this := &Response_ListFiles{} + if r.Intn(10) != 0 { + v28 := r.Intn(5) + this.FileInfos = make([]mesos.FileInfo, v28) + for i := 0; i < v28; i++ { + v29 := mesos.NewPopulatedFileInfo(r, easy) + this.FileInfos[i] = *v29 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_ReadFile(r randyMaster, easy bool) *Response_ReadFile { + this := &Response_ReadFile{} + this.Size = uint64(uint64(r.Uint32())) + v30 := r.Intn(100) + this.Data = make([]byte, v30) + for i := 0; i < v30; i++ { + this.Data[i] = byte(r.Intn(256)) + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetState(r randyMaster, easy bool) *Response_GetState { + this := &Response_GetState{} + if r.Intn(10) == 0 { + this.GetTasks = NewPopulatedResponse_GetTasks(r, easy) + } + if r.Intn(10) != 0 { + this.GetExecutors = NewPopulatedResponse_GetExecutors(r, easy) + } + if r.Intn(10) != 0 { + this.GetFrameworks = NewPopulatedResponse_GetFrameworks(r, easy) + } + if r.Intn(10) != 0 { + this.GetAgents = NewPopulatedResponse_GetAgents(r, easy) + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetAgents(r randyMaster, easy bool) *Response_GetAgents { + this := &Response_GetAgents{} + if r.Intn(10) != 0 { + v31 := r.Intn(5) + this.Agents = make([]Response_GetAgents_Agent, v31) + for i := 0; i < v31; i++ { + v32 := NewPopulatedResponse_GetAgents_Agent(r, easy) + this.Agents[i] = *v32 + } + } + if r.Intn(10) != 0 { + v33 := r.Intn(5) + this.RecoveredAgents = make([]mesos.AgentInfo, v33) + for i := 0; i < v33; i++ { + v34 := mesos.NewPopulatedAgentInfo(r, easy) + this.RecoveredAgents[i] = *v34 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetAgents_Agent(r randyMaster, easy bool) *Response_GetAgents_Agent { + this := &Response_GetAgents_Agent{} + v35 := mesos.NewPopulatedAgentInfo(r, easy) + this.AgentInfo = *v35 + this.Active = bool(bool(r.Intn(2) == 0)) + this.Version = string(randStringMaster(r)) + if r.Intn(10) != 0 { + v36 := string(randStringMaster(r)) + this.PID = &v36 + } + if r.Intn(10) != 0 { + this.RegisteredTime = mesos.NewPopulatedTimeInfo(r, easy) + } + if r.Intn(10) != 0 { + this.ReregisteredTime = mesos.NewPopulatedTimeInfo(r, easy) + } + if r.Intn(10) != 0 { + v37 := r.Intn(5) + this.TotalResources = make([]mesos.Resource, v37) + for i := 0; i < v37; i++ { + v38 := mesos.NewPopulatedResource(r, easy) + this.TotalResources[i] = *v38 + } + } + if r.Intn(10) != 0 { + v39 := r.Intn(5) + this.AllocatedResources = make([]mesos.Resource, v39) + for i := 0; i < v39; i++ { + v40 := mesos.NewPopulatedResource(r, easy) + this.AllocatedResources[i] = *v40 + } + } + if r.Intn(10) != 0 { + v41 := r.Intn(5) + this.OfferedResources = make([]mesos.Resource, v41) + for i := 0; i < v41; i++ { + v42 := mesos.NewPopulatedResource(r, easy) + this.OfferedResources[i] = *v42 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetFrameworks(r randyMaster, easy bool) *Response_GetFrameworks { + this := &Response_GetFrameworks{} + if r.Intn(10) != 0 { + v43 := r.Intn(5) + this.Frameworks = make([]Response_GetFrameworks_Framework, v43) + for i := 0; i < v43; i++ { + v44 := NewPopulatedResponse_GetFrameworks_Framework(r, easy) + this.Frameworks[i] = *v44 + } + } + if r.Intn(10) != 0 { + v45 := r.Intn(5) + this.CompletedFrameworks = make([]Response_GetFrameworks_Framework, v45) + for i := 0; i < v45; i++ { + v46 := NewPopulatedResponse_GetFrameworks_Framework(r, easy) + this.CompletedFrameworks[i] = *v46 + } + } + if r.Intn(10) != 0 { + v47 := r.Intn(5) + this.RecoveredFrameworks = make([]mesos.FrameworkInfo, v47) + for i := 0; i < v47; i++ { + v48 := mesos.NewPopulatedFrameworkInfo(r, easy) + this.RecoveredFrameworks[i] = *v48 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetFrameworks_Framework(r randyMaster, easy bool) *Response_GetFrameworks_Framework { + this := &Response_GetFrameworks_Framework{} + v49 := mesos.NewPopulatedFrameworkInfo(r, easy) + this.FrameworkInfo = *v49 + this.Active = bool(bool(r.Intn(2) == 0)) + this.Connected = bool(bool(r.Intn(2) == 0)) + if r.Intn(10) != 0 { + this.RegisteredTime = mesos.NewPopulatedTimeInfo(r, easy) + } + if r.Intn(10) != 0 { + this.ReregisteredTime = mesos.NewPopulatedTimeInfo(r, easy) + } + if r.Intn(10) != 0 { + this.UnregisteredTime = mesos.NewPopulatedTimeInfo(r, easy) + } + if r.Intn(10) != 0 { + v50 := r.Intn(5) + this.Offers = make([]mesos.Offer, v50) + for i := 0; i < v50; i++ { + v51 := mesos.NewPopulatedOffer(r, easy) + this.Offers[i] = *v51 + } + } + if r.Intn(10) != 0 { + v52 := r.Intn(5) + this.InverseOffers = make([]mesos.InverseOffer, v52) + for i := 0; i < v52; i++ { + v53 := mesos.NewPopulatedInverseOffer(r, easy) + this.InverseOffers[i] = *v53 + } + } + if r.Intn(10) != 0 { + v54 := r.Intn(5) + this.AllocatedResources = make([]mesos.Resource, v54) + for i := 0; i < v54; i++ { + v55 := mesos.NewPopulatedResource(r, easy) + this.AllocatedResources[i] = *v55 + } + } + if r.Intn(10) != 0 { + v56 := r.Intn(5) + this.OfferedResources = make([]mesos.Resource, v56) + for i := 0; i < v56; i++ { + v57 := mesos.NewPopulatedResource(r, easy) + this.OfferedResources[i] = *v57 + } + } + this.Recovered = bool(bool(r.Intn(2) == 0)) + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetExecutors(r randyMaster, easy bool) *Response_GetExecutors { + this := &Response_GetExecutors{} + if r.Intn(10) != 0 { + v58 := r.Intn(5) + this.Executors = make([]Response_GetExecutors_Executor, v58) + for i := 0; i < v58; i++ { + v59 := NewPopulatedResponse_GetExecutors_Executor(r, easy) + this.Executors[i] = *v59 + } + } + if r.Intn(10) != 0 { + v60 := r.Intn(5) + this.OrphanExecutors = make([]Response_GetExecutors_Executor, v60) + for i := 0; i < v60; i++ { + v61 := NewPopulatedResponse_GetExecutors_Executor(r, easy) + this.OrphanExecutors[i] = *v61 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetExecutors_Executor(r randyMaster, easy bool) *Response_GetExecutors_Executor { + this := &Response_GetExecutors_Executor{} + v62 := mesos.NewPopulatedExecutorInfo(r, easy) + this.ExecutorInfo = *v62 + v63 := mesos.NewPopulatedAgentID(r, easy) + this.AgentID = *v63 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetTasks(r randyMaster, easy bool) *Response_GetTasks { + this := &Response_GetTasks{} + if r.Intn(10) == 0 { + v64 := r.Intn(5) + this.PendingTasks = make([]mesos.Task, v64) + for i := 0; i < v64; i++ { + v65 := mesos.NewPopulatedTask(r, easy) + this.PendingTasks[i] = *v65 + } + } + if r.Intn(10) == 0 { + v66 := r.Intn(5) + this.Tasks = make([]mesos.Task, v66) + for i := 0; i < v66; i++ { + v67 := mesos.NewPopulatedTask(r, easy) + this.Tasks[i] = *v67 + } + } + if r.Intn(10) == 0 { + v68 := r.Intn(5) + this.CompletedTasks = make([]mesos.Task, v68) + for i := 0; i < v68; i++ { + v69 := mesos.NewPopulatedTask(r, easy) + this.CompletedTasks[i] = *v69 + } + } + if r.Intn(10) == 0 { + v70 := r.Intn(5) + this.OrphanTasks = make([]mesos.Task, v70) + for i := 0; i < v70; i++ { + v71 := mesos.NewPopulatedTask(r, easy) + this.OrphanTasks[i] = *v71 + } + } + if r.Intn(10) == 0 { + v72 := r.Intn(5) + this.UnreachableTasks = make([]mesos.Task, v72) + for i := 0; i < v72; i++ { + v73 := mesos.NewPopulatedTask(r, easy) + this.UnreachableTasks[i] = *v73 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetRoles(r randyMaster, easy bool) *Response_GetRoles { + this := &Response_GetRoles{} + if r.Intn(10) != 0 { + v74 := r.Intn(5) + this.Roles = make([]mesos.Role, v74) + for i := 0; i < v74; i++ { + v75 := mesos.NewPopulatedRole(r, easy) + this.Roles[i] = *v75 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetWeights(r randyMaster, easy bool) *Response_GetWeights { + this := &Response_GetWeights{} + if r.Intn(10) != 0 { + v76 := r.Intn(5) + this.WeightInfos = make([]mesos.WeightInfo, v76) + for i := 0; i < v76; i++ { + v77 := mesos.NewPopulatedWeightInfo(r, easy) + this.WeightInfos[i] = *v77 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetMaster(r randyMaster, easy bool) *Response_GetMaster { + this := &Response_GetMaster{} + if r.Intn(10) != 0 { + this.MasterInfo = mesos.NewPopulatedMasterInfo(r, easy) + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetMaintenanceStatus(r randyMaster, easy bool) *Response_GetMaintenanceStatus { + this := &Response_GetMaintenanceStatus{} + v78 := mesos_maintenance.NewPopulatedClusterStatus(r, easy) + this.Status = *v78 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetMaintenanceSchedule(r randyMaster, easy bool) *Response_GetMaintenanceSchedule { + this := &Response_GetMaintenanceSchedule{} + v79 := mesos_maintenance.NewPopulatedSchedule(r, easy) + this.Schedule = *v79 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedResponse_GetQuota(r randyMaster, easy bool) *Response_GetQuota { + this := &Response_GetQuota{} + v80 := mesos_quota.NewPopulatedQuotaStatus(r, easy) + this.Status = *v80 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedEvent(r randyMaster, easy bool) *Event { + this := &Event{} + this.Type = Event_Type([]int32{0, 1, 2, 3, 4, 5}[r.Intn(6)]) + if r.Intn(10) == 0 { + this.Subscribed = NewPopulatedEvent_Subscribed(r, easy) + } + if r.Intn(10) == 0 { + this.TaskAdded = NewPopulatedEvent_TaskAdded(r, easy) + } + if r.Intn(10) == 0 { + this.TaskUpdated = NewPopulatedEvent_TaskUpdated(r, easy) + } + if r.Intn(10) != 0 { + this.AgentAdded = NewPopulatedEvent_AgentAdded(r, easy) + } + if r.Intn(10) != 0 { + this.AgentRemoved = NewPopulatedEvent_AgentRemoved(r, easy) + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedEvent_Subscribed(r randyMaster, easy bool) *Event_Subscribed { + this := &Event_Subscribed{} + if r.Intn(10) == 0 { + this.GetState = NewPopulatedResponse_GetState(r, easy) + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedEvent_TaskAdded(r randyMaster, easy bool) *Event_TaskAdded { + this := &Event_TaskAdded{} + v81 := mesos.NewPopulatedTask(r, easy) + this.Task = *v81 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedEvent_TaskUpdated(r randyMaster, easy bool) *Event_TaskUpdated { + this := &Event_TaskUpdated{} + v82 := mesos.NewPopulatedFrameworkID(r, easy) + this.FrameworkID = *v82 + v83 := mesos.NewPopulatedTaskStatus(r, easy) + this.Status = *v83 + v84 := mesos.TaskState([]int32{6, 0, 1, 8, 2, 3, 4, 7, 5, 9, 10, 11, 12, 13}[r.Intn(14)]) + this.State = &v84 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedEvent_AgentAdded(r randyMaster, easy bool) *Event_AgentAdded { + this := &Event_AgentAdded{} + v85 := NewPopulatedResponse_GetAgents_Agent(r, easy) + this.Agent = *v85 + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedEvent_AgentRemoved(r randyMaster, easy bool) *Event_AgentRemoved { + this := &Event_AgentRemoved{} + v86 := mesos.NewPopulatedAgentID(r, easy) + this.AgentID = *v86 + if !easy && r.Intn(10) != 0 { + } + return this +} + +type randyMaster interface { + Float32() float32 + Float64() float64 + Int63() int64 + Int31() int32 + Uint32() uint32 + Intn(n int) int +} + +func randUTF8RuneMaster(r randyMaster) rune { + ru := r.Intn(62) + if ru < 10 { + return rune(ru + 48) + } else if ru < 36 { + return rune(ru + 55) + } + return rune(ru + 61) +} +func randStringMaster(r randyMaster) string { + v87 := r.Intn(100) + tmps := make([]rune, v87) + for i := 0; i < v87; i++ { + tmps[i] = randUTF8RuneMaster(r) + } + return string(tmps) +} +func randUnrecognizedMaster(r randyMaster, maxFieldNumber int) (dAtA []byte) { + l := r.Intn(5) + for i := 0; i < l; i++ { + wire := r.Intn(4) + if wire == 3 { + wire = 5 + } + fieldNumber := maxFieldNumber + r.Intn(100) + dAtA = randFieldMaster(dAtA, r, fieldNumber, wire) + } + return dAtA +} +func randFieldMaster(dAtA []byte, r randyMaster, fieldNumber int, wire int) []byte { + key := uint32(fieldNumber)<<3 | uint32(wire) + switch wire { + case 0: + dAtA = encodeVarintPopulateMaster(dAtA, uint64(key)) + v88 := r.Int63() + if r.Intn(2) == 0 { + v88 *= -1 + } + dAtA = encodeVarintPopulateMaster(dAtA, uint64(v88)) + case 1: + dAtA = encodeVarintPopulateMaster(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + case 2: + dAtA = encodeVarintPopulateMaster(dAtA, uint64(key)) + ll := r.Intn(100) + dAtA = encodeVarintPopulateMaster(dAtA, uint64(ll)) + for j := 0; j < ll; j++ { + dAtA = append(dAtA, byte(r.Intn(256))) + } + default: + dAtA = encodeVarintPopulateMaster(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + } + return dAtA +} +func encodeVarintPopulateMaster(dAtA []byte, v uint64) []byte { + for v >= 1<<7 { + dAtA = append(dAtA, uint8(uint64(v)&0x7f|0x80)) + v >>= 7 + } + dAtA = append(dAtA, uint8(v)) + return dAtA +} +func (m *Call) ProtoSize() (n int) { + var l int + _ = l + n += 1 + sovMaster(uint64(m.Type)) + if m.GetMetrics != nil { + l = m.GetMetrics.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.SetLoggingLevel != nil { + l = m.SetLoggingLevel.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.ListFiles != nil { + l = m.ListFiles.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.ReadFile != nil { + l = m.ReadFile.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.UpdateWeights != nil { + l = m.UpdateWeights.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.ReserveResources != nil { + l = m.ReserveResources.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.UnreserveResources != nil { + l = m.UnreserveResources.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.CreateVolumes != nil { + l = m.CreateVolumes.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.DestroyVolumes != nil { + l = m.DestroyVolumes.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.UpdateMaintenanceSchedule != nil { + l = m.UpdateMaintenanceSchedule.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.StartMaintenance != nil { + l = m.StartMaintenance.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.StopMaintenance != nil { + l = m.StopMaintenance.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.SetQuota != nil { + l = m.SetQuota.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.RemoveQuota != nil { + l = m.RemoveQuota.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + return n +} + +func (m *Call_GetMetrics) ProtoSize() (n int) { + var l int + _ = l + if m.Timeout != nil { + l = m.Timeout.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + return n +} + +func (m *Call_SetLoggingLevel) ProtoSize() (n int) { + var l int + _ = l + n += 1 + sovMaster(uint64(m.Level)) + l = m.Duration.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Call_ListFiles) ProtoSize() (n int) { + var l int + _ = l + l = len(m.Path) + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Call_ReadFile) ProtoSize() (n int) { + var l int + _ = l + l = len(m.Path) + n += 1 + l + sovMaster(uint64(l)) + n += 1 + sovMaster(uint64(m.Offset)) + if m.Length != nil { + n += 1 + sovMaster(uint64(*m.Length)) + } + return n +} + +func (m *Call_UpdateWeights) ProtoSize() (n int) { + var l int + _ = l + if len(m.WeightInfos) > 0 { + for _, e := range m.WeightInfos { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Call_ReserveResources) ProtoSize() (n int) { + var l int + _ = l + l = m.AgentID.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + if len(m.Resources) > 0 { + for _, e := range m.Resources { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Call_UnreserveResources) ProtoSize() (n int) { + var l int + _ = l + l = m.AgentID.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + if len(m.Resources) > 0 { + for _, e := range m.Resources { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Call_CreateVolumes) ProtoSize() (n int) { + var l int + _ = l + l = m.AgentID.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + if len(m.Volumes) > 0 { + for _, e := range m.Volumes { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Call_DestroyVolumes) ProtoSize() (n int) { + var l int + _ = l + l = m.AgentID.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + if len(m.Volumes) > 0 { + for _, e := range m.Volumes { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Call_UpdateMaintenanceSchedule) ProtoSize() (n int) { + var l int + _ = l + l = m.Schedule.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Call_StartMaintenance) ProtoSize() (n int) { + var l int + _ = l + if len(m.Machines) > 0 { + for _, e := range m.Machines { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Call_StopMaintenance) ProtoSize() (n int) { + var l int + _ = l + if len(m.Machines) > 0 { + for _, e := range m.Machines { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Call_SetQuota) ProtoSize() (n int) { + var l int + _ = l + l = m.QuotaRequest.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Call_RemoveQuota) ProtoSize() (n int) { + var l int + _ = l + l = len(m.Role) + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Response) ProtoSize() (n int) { + var l int + _ = l + n += 1 + sovMaster(uint64(m.Type)) + if m.GetHealth != nil { + l = m.GetHealth.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetFlags != nil { + l = m.GetFlags.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetVersion != nil { + l = m.GetVersion.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetMetrics != nil { + l = m.GetMetrics.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetLoggingLevel != nil { + l = m.GetLoggingLevel.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.ListFiles != nil { + l = m.ListFiles.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.ReadFile != nil { + l = m.ReadFile.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetState != nil { + l = m.GetState.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetAgents != nil { + l = m.GetAgents.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetFrameworks != nil { + l = m.GetFrameworks.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetExecutors != nil { + l = m.GetExecutors.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetTasks != nil { + l = m.GetTasks.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetRoles != nil { + l = m.GetRoles.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetWeights != nil { + l = m.GetWeights.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetMaster != nil { + l = m.GetMaster.ProtoSize() + n += 2 + l + sovMaster(uint64(l)) + } + if m.GetMaintenanceStatus != nil { + l = m.GetMaintenanceStatus.ProtoSize() + n += 2 + l + sovMaster(uint64(l)) + } + if m.GetMaintenanceSchedule != nil { + l = m.GetMaintenanceSchedule.ProtoSize() + n += 2 + l + sovMaster(uint64(l)) + } + if m.GetQuota != nil { + l = m.GetQuota.ProtoSize() + n += 2 + l + sovMaster(uint64(l)) + } + return n +} + +func (m *Response_GetHealth) ProtoSize() (n int) { + var l int + _ = l + n += 2 + return n +} + +func (m *Response_GetFlags) ProtoSize() (n int) { + var l int + _ = l + if len(m.Flags) > 0 { + for _, e := range m.Flags { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_GetVersion) ProtoSize() (n int) { + var l int + _ = l + l = m.VersionInfo.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Response_GetMetrics) ProtoSize() (n int) { + var l int + _ = l + if len(m.Metrics) > 0 { + for _, e := range m.Metrics { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_GetLoggingLevel) ProtoSize() (n int) { + var l int + _ = l + n += 1 + sovMaster(uint64(m.Level)) + return n +} + +func (m *Response_ListFiles) ProtoSize() (n int) { + var l int + _ = l + if len(m.FileInfos) > 0 { + for _, e := range m.FileInfos { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_ReadFile) ProtoSize() (n int) { + var l int + _ = l + n += 1 + sovMaster(uint64(m.Size)) + if m.Data != nil { + l = len(m.Data) + n += 1 + l + sovMaster(uint64(l)) + } + return n +} + +func (m *Response_GetState) ProtoSize() (n int) { + var l int + _ = l + if m.GetTasks != nil { + l = m.GetTasks.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetExecutors != nil { + l = m.GetExecutors.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetFrameworks != nil { + l = m.GetFrameworks.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.GetAgents != nil { + l = m.GetAgents.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + return n +} + +func (m *Response_GetAgents) ProtoSize() (n int) { + var l int + _ = l + if len(m.Agents) > 0 { + for _, e := range m.Agents { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.RecoveredAgents) > 0 { + for _, e := range m.RecoveredAgents { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_GetAgents_Agent) ProtoSize() (n int) { + var l int + _ = l + l = m.AgentInfo.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + n += 2 + l = len(m.Version) + n += 1 + l + sovMaster(uint64(l)) + if m.PID != nil { + l = len(*m.PID) + n += 1 + l + sovMaster(uint64(l)) + } + if m.RegisteredTime != nil { + l = m.RegisteredTime.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.ReregisteredTime != nil { + l = m.ReregisteredTime.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if len(m.TotalResources) > 0 { + for _, e := range m.TotalResources { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.AllocatedResources) > 0 { + for _, e := range m.AllocatedResources { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.OfferedResources) > 0 { + for _, e := range m.OfferedResources { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_GetFrameworks) ProtoSize() (n int) { + var l int + _ = l + if len(m.Frameworks) > 0 { + for _, e := range m.Frameworks { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.CompletedFrameworks) > 0 { + for _, e := range m.CompletedFrameworks { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.RecoveredFrameworks) > 0 { + for _, e := range m.RecoveredFrameworks { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_GetFrameworks_Framework) ProtoSize() (n int) { + var l int + _ = l + l = m.FrameworkInfo.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + n += 2 + n += 2 + if m.RegisteredTime != nil { + l = m.RegisteredTime.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.ReregisteredTime != nil { + l = m.ReregisteredTime.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.UnregisteredTime != nil { + l = m.UnregisteredTime.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if len(m.Offers) > 0 { + for _, e := range m.Offers { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.InverseOffers) > 0 { + for _, e := range m.InverseOffers { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.AllocatedResources) > 0 { + for _, e := range m.AllocatedResources { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.OfferedResources) > 0 { + for _, e := range m.OfferedResources { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + n += 2 + return n +} + +func (m *Response_GetExecutors) ProtoSize() (n int) { + var l int + _ = l + if len(m.Executors) > 0 { + for _, e := range m.Executors { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.OrphanExecutors) > 0 { + for _, e := range m.OrphanExecutors { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_GetExecutors_Executor) ProtoSize() (n int) { + var l int + _ = l + l = m.ExecutorInfo.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + l = m.AgentID.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Response_GetTasks) ProtoSize() (n int) { + var l int + _ = l + if len(m.PendingTasks) > 0 { + for _, e := range m.PendingTasks { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.Tasks) > 0 { + for _, e := range m.Tasks { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.CompletedTasks) > 0 { + for _, e := range m.CompletedTasks { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.OrphanTasks) > 0 { + for _, e := range m.OrphanTasks { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + if len(m.UnreachableTasks) > 0 { + for _, e := range m.UnreachableTasks { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_GetRoles) ProtoSize() (n int) { + var l int + _ = l + if len(m.Roles) > 0 { + for _, e := range m.Roles { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_GetWeights) ProtoSize() (n int) { + var l int + _ = l + if len(m.WeightInfos) > 0 { + for _, e := range m.WeightInfos { + l = e.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + } + return n +} + +func (m *Response_GetMaster) ProtoSize() (n int) { + var l int + _ = l + if m.MasterInfo != nil { + l = m.MasterInfo.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + return n +} + +func (m *Response_GetMaintenanceStatus) ProtoSize() (n int) { + var l int + _ = l + l = m.Status.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Response_GetMaintenanceSchedule) ProtoSize() (n int) { + var l int + _ = l + l = m.Schedule.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Response_GetQuota) ProtoSize() (n int) { + var l int + _ = l + l = m.Status.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Event) ProtoSize() (n int) { + var l int + _ = l + n += 1 + sovMaster(uint64(m.Type)) + if m.Subscribed != nil { + l = m.Subscribed.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.TaskAdded != nil { + l = m.TaskAdded.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.TaskUpdated != nil { + l = m.TaskUpdated.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.AgentAdded != nil { + l = m.AgentAdded.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + if m.AgentRemoved != nil { + l = m.AgentRemoved.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + return n +} + +func (m *Event_Subscribed) ProtoSize() (n int) { + var l int + _ = l + if m.GetState != nil { + l = m.GetState.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + } + return n +} + +func (m *Event_TaskAdded) ProtoSize() (n int) { + var l int + _ = l + l = m.Task.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Event_TaskUpdated) ProtoSize() (n int) { + var l int + _ = l + l = m.FrameworkID.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + l = m.Status.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + if m.State != nil { + n += 1 + sovMaster(uint64(*m.State)) + } + return n +} + +func (m *Event_AgentAdded) ProtoSize() (n int) { + var l int + _ = l + l = m.Agent.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func (m *Event_AgentRemoved) ProtoSize() (n int) { + var l int + _ = l + l = m.AgentID.ProtoSize() + n += 1 + l + sovMaster(uint64(l)) + return n +} + +func sovMaster(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozMaster(x uint64) (n int) { + return sovMaster(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *Call) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call{`, + `Type:` + fmt.Sprintf("%v", this.Type) + `,`, + `GetMetrics:` + strings.Replace(fmt.Sprintf("%v", this.GetMetrics), "Call_GetMetrics", "Call_GetMetrics", 1) + `,`, + `SetLoggingLevel:` + strings.Replace(fmt.Sprintf("%v", this.SetLoggingLevel), "Call_SetLoggingLevel", "Call_SetLoggingLevel", 1) + `,`, + `ListFiles:` + strings.Replace(fmt.Sprintf("%v", this.ListFiles), "Call_ListFiles", "Call_ListFiles", 1) + `,`, + `ReadFile:` + strings.Replace(fmt.Sprintf("%v", this.ReadFile), "Call_ReadFile", "Call_ReadFile", 1) + `,`, + `UpdateWeights:` + strings.Replace(fmt.Sprintf("%v", this.UpdateWeights), "Call_UpdateWeights", "Call_UpdateWeights", 1) + `,`, + `ReserveResources:` + strings.Replace(fmt.Sprintf("%v", this.ReserveResources), "Call_ReserveResources", "Call_ReserveResources", 1) + `,`, + `UnreserveResources:` + strings.Replace(fmt.Sprintf("%v", this.UnreserveResources), "Call_UnreserveResources", "Call_UnreserveResources", 1) + `,`, + `CreateVolumes:` + strings.Replace(fmt.Sprintf("%v", this.CreateVolumes), "Call_CreateVolumes", "Call_CreateVolumes", 1) + `,`, + `DestroyVolumes:` + strings.Replace(fmt.Sprintf("%v", this.DestroyVolumes), "Call_DestroyVolumes", "Call_DestroyVolumes", 1) + `,`, + `UpdateMaintenanceSchedule:` + strings.Replace(fmt.Sprintf("%v", this.UpdateMaintenanceSchedule), "Call_UpdateMaintenanceSchedule", "Call_UpdateMaintenanceSchedule", 1) + `,`, + `StartMaintenance:` + strings.Replace(fmt.Sprintf("%v", this.StartMaintenance), "Call_StartMaintenance", "Call_StartMaintenance", 1) + `,`, + `StopMaintenance:` + strings.Replace(fmt.Sprintf("%v", this.StopMaintenance), "Call_StopMaintenance", "Call_StopMaintenance", 1) + `,`, + `SetQuota:` + strings.Replace(fmt.Sprintf("%v", this.SetQuota), "Call_SetQuota", "Call_SetQuota", 1) + `,`, + `RemoveQuota:` + strings.Replace(fmt.Sprintf("%v", this.RemoveQuota), "Call_RemoveQuota", "Call_RemoveQuota", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_GetMetrics) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_GetMetrics{`, + `Timeout:` + strings.Replace(fmt.Sprintf("%v", this.Timeout), "DurationInfo", "mesos.DurationInfo", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_SetLoggingLevel) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_SetLoggingLevel{`, + `Level:` + fmt.Sprintf("%v", this.Level) + `,`, + `Duration:` + strings.Replace(strings.Replace(this.Duration.String(), "DurationInfo", "mesos.DurationInfo", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_ListFiles) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_ListFiles{`, + `Path:` + fmt.Sprintf("%v", this.Path) + `,`, + `}`, + }, "") + return s +} +func (this *Call_ReadFile) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_ReadFile{`, + `Path:` + fmt.Sprintf("%v", this.Path) + `,`, + `Offset:` + fmt.Sprintf("%v", this.Offset) + `,`, + `Length:` + valueToStringMaster(this.Length) + `,`, + `}`, + }, "") + return s +} +func (this *Call_UpdateWeights) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_UpdateWeights{`, + `WeightInfos:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.WeightInfos), "WeightInfo", "mesos.WeightInfo", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_ReserveResources) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_ReserveResources{`, + `AgentID:` + strings.Replace(strings.Replace(this.AgentID.String(), "AgentID", "mesos.AgentID", 1), `&`, ``, 1) + `,`, + `Resources:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Resources), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_UnreserveResources) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_UnreserveResources{`, + `AgentID:` + strings.Replace(strings.Replace(this.AgentID.String(), "AgentID", "mesos.AgentID", 1), `&`, ``, 1) + `,`, + `Resources:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Resources), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_CreateVolumes) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_CreateVolumes{`, + `AgentID:` + strings.Replace(strings.Replace(this.AgentID.String(), "AgentID", "mesos.AgentID", 1), `&`, ``, 1) + `,`, + `Volumes:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Volumes), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_DestroyVolumes) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_DestroyVolumes{`, + `AgentID:` + strings.Replace(strings.Replace(this.AgentID.String(), "AgentID", "mesos.AgentID", 1), `&`, ``, 1) + `,`, + `Volumes:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Volumes), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_UpdateMaintenanceSchedule) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_UpdateMaintenanceSchedule{`, + `Schedule:` + strings.Replace(strings.Replace(this.Schedule.String(), "Schedule", "mesos_maintenance.Schedule", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_StartMaintenance) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_StartMaintenance{`, + `Machines:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Machines), "MachineID", "mesos.MachineID", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_StopMaintenance) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_StopMaintenance{`, + `Machines:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Machines), "MachineID", "mesos.MachineID", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_SetQuota) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_SetQuota{`, + `QuotaRequest:` + strings.Replace(strings.Replace(this.QuotaRequest.String(), "QuotaRequest", "mesos_quota.QuotaRequest", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Call_RemoveQuota) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Call_RemoveQuota{`, + `Role:` + fmt.Sprintf("%v", this.Role) + `,`, + `}`, + }, "") + return s +} +func (this *Response) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response{`, + `Type:` + fmt.Sprintf("%v", this.Type) + `,`, + `GetHealth:` + strings.Replace(fmt.Sprintf("%v", this.GetHealth), "Response_GetHealth", "Response_GetHealth", 1) + `,`, + `GetFlags:` + strings.Replace(fmt.Sprintf("%v", this.GetFlags), "Response_GetFlags", "Response_GetFlags", 1) + `,`, + `GetVersion:` + strings.Replace(fmt.Sprintf("%v", this.GetVersion), "Response_GetVersion", "Response_GetVersion", 1) + `,`, + `GetMetrics:` + strings.Replace(fmt.Sprintf("%v", this.GetMetrics), "Response_GetMetrics", "Response_GetMetrics", 1) + `,`, + `GetLoggingLevel:` + strings.Replace(fmt.Sprintf("%v", this.GetLoggingLevel), "Response_GetLoggingLevel", "Response_GetLoggingLevel", 1) + `,`, + `ListFiles:` + strings.Replace(fmt.Sprintf("%v", this.ListFiles), "Response_ListFiles", "Response_ListFiles", 1) + `,`, + `ReadFile:` + strings.Replace(fmt.Sprintf("%v", this.ReadFile), "Response_ReadFile", "Response_ReadFile", 1) + `,`, + `GetState:` + strings.Replace(fmt.Sprintf("%v", this.GetState), "Response_GetState", "Response_GetState", 1) + `,`, + `GetAgents:` + strings.Replace(fmt.Sprintf("%v", this.GetAgents), "Response_GetAgents", "Response_GetAgents", 1) + `,`, + `GetFrameworks:` + strings.Replace(fmt.Sprintf("%v", this.GetFrameworks), "Response_GetFrameworks", "Response_GetFrameworks", 1) + `,`, + `GetExecutors:` + strings.Replace(fmt.Sprintf("%v", this.GetExecutors), "Response_GetExecutors", "Response_GetExecutors", 1) + `,`, + `GetTasks:` + strings.Replace(fmt.Sprintf("%v", this.GetTasks), "Response_GetTasks", "Response_GetTasks", 1) + `,`, + `GetRoles:` + strings.Replace(fmt.Sprintf("%v", this.GetRoles), "Response_GetRoles", "Response_GetRoles", 1) + `,`, + `GetWeights:` + strings.Replace(fmt.Sprintf("%v", this.GetWeights), "Response_GetWeights", "Response_GetWeights", 1) + `,`, + `GetMaster:` + strings.Replace(fmt.Sprintf("%v", this.GetMaster), "Response_GetMaster", "Response_GetMaster", 1) + `,`, + `GetMaintenanceStatus:` + strings.Replace(fmt.Sprintf("%v", this.GetMaintenanceStatus), "Response_GetMaintenanceStatus", "Response_GetMaintenanceStatus", 1) + `,`, + `GetMaintenanceSchedule:` + strings.Replace(fmt.Sprintf("%v", this.GetMaintenanceSchedule), "Response_GetMaintenanceSchedule", "Response_GetMaintenanceSchedule", 1) + `,`, + `GetQuota:` + strings.Replace(fmt.Sprintf("%v", this.GetQuota), "Response_GetQuota", "Response_GetQuota", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetHealth) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetHealth{`, + `Healthy:` + fmt.Sprintf("%v", this.Healthy) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetFlags) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetFlags{`, + `Flags:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Flags), "Flag", "mesos.Flag", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetVersion) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetVersion{`, + `VersionInfo:` + strings.Replace(strings.Replace(this.VersionInfo.String(), "VersionInfo", "mesos.VersionInfo", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetMetrics) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetMetrics{`, + `Metrics:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Metrics), "Metric", "mesos.Metric", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetLoggingLevel) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetLoggingLevel{`, + `Level:` + fmt.Sprintf("%v", this.Level) + `,`, + `}`, + }, "") + return s +} +func (this *Response_ListFiles) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_ListFiles{`, + `FileInfos:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.FileInfos), "FileInfo", "mesos.FileInfo", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_ReadFile) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_ReadFile{`, + `Size:` + fmt.Sprintf("%v", this.Size) + `,`, + `Data:` + valueToStringMaster(this.Data) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetState) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetState{`, + `GetTasks:` + strings.Replace(fmt.Sprintf("%v", this.GetTasks), "Response_GetTasks", "Response_GetTasks", 1) + `,`, + `GetExecutors:` + strings.Replace(fmt.Sprintf("%v", this.GetExecutors), "Response_GetExecutors", "Response_GetExecutors", 1) + `,`, + `GetFrameworks:` + strings.Replace(fmt.Sprintf("%v", this.GetFrameworks), "Response_GetFrameworks", "Response_GetFrameworks", 1) + `,`, + `GetAgents:` + strings.Replace(fmt.Sprintf("%v", this.GetAgents), "Response_GetAgents", "Response_GetAgents", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetAgents) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetAgents{`, + `Agents:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Agents), "Response_GetAgents_Agent", "Response_GetAgents_Agent", 1), `&`, ``, 1) + `,`, + `RecoveredAgents:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.RecoveredAgents), "AgentInfo", "mesos.AgentInfo", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetAgents_Agent) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetAgents_Agent{`, + `AgentInfo:` + strings.Replace(strings.Replace(this.AgentInfo.String(), "AgentInfo", "mesos.AgentInfo", 1), `&`, ``, 1) + `,`, + `Active:` + fmt.Sprintf("%v", this.Active) + `,`, + `Version:` + fmt.Sprintf("%v", this.Version) + `,`, + `PID:` + valueToStringMaster(this.PID) + `,`, + `RegisteredTime:` + strings.Replace(fmt.Sprintf("%v", this.RegisteredTime), "TimeInfo", "mesos.TimeInfo", 1) + `,`, + `ReregisteredTime:` + strings.Replace(fmt.Sprintf("%v", this.ReregisteredTime), "TimeInfo", "mesos.TimeInfo", 1) + `,`, + `TotalResources:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.TotalResources), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `AllocatedResources:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.AllocatedResources), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `OfferedResources:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.OfferedResources), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetFrameworks) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetFrameworks{`, + `Frameworks:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Frameworks), "Response_GetFrameworks_Framework", "Response_GetFrameworks_Framework", 1), `&`, ``, 1) + `,`, + `CompletedFrameworks:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.CompletedFrameworks), "Response_GetFrameworks_Framework", "Response_GetFrameworks_Framework", 1), `&`, ``, 1) + `,`, + `RecoveredFrameworks:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.RecoveredFrameworks), "FrameworkInfo", "mesos.FrameworkInfo", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetFrameworks_Framework) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetFrameworks_Framework{`, + `FrameworkInfo:` + strings.Replace(strings.Replace(this.FrameworkInfo.String(), "FrameworkInfo", "mesos.FrameworkInfo", 1), `&`, ``, 1) + `,`, + `Active:` + fmt.Sprintf("%v", this.Active) + `,`, + `Connected:` + fmt.Sprintf("%v", this.Connected) + `,`, + `RegisteredTime:` + strings.Replace(fmt.Sprintf("%v", this.RegisteredTime), "TimeInfo", "mesos.TimeInfo", 1) + `,`, + `ReregisteredTime:` + strings.Replace(fmt.Sprintf("%v", this.ReregisteredTime), "TimeInfo", "mesos.TimeInfo", 1) + `,`, + `UnregisteredTime:` + strings.Replace(fmt.Sprintf("%v", this.UnregisteredTime), "TimeInfo", "mesos.TimeInfo", 1) + `,`, + `Offers:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Offers), "Offer", "mesos.Offer", 1), `&`, ``, 1) + `,`, + `InverseOffers:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.InverseOffers), "InverseOffer", "mesos.InverseOffer", 1), `&`, ``, 1) + `,`, + `AllocatedResources:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.AllocatedResources), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `OfferedResources:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.OfferedResources), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `Recovered:` + fmt.Sprintf("%v", this.Recovered) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetExecutors) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetExecutors{`, + `Executors:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Executors), "Response_GetExecutors_Executor", "Response_GetExecutors_Executor", 1), `&`, ``, 1) + `,`, + `OrphanExecutors:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.OrphanExecutors), "Response_GetExecutors_Executor", "Response_GetExecutors_Executor", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetExecutors_Executor) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetExecutors_Executor{`, + `ExecutorInfo:` + strings.Replace(strings.Replace(this.ExecutorInfo.String(), "ExecutorInfo", "mesos.ExecutorInfo", 1), `&`, ``, 1) + `,`, + `AgentID:` + strings.Replace(strings.Replace(this.AgentID.String(), "AgentID", "mesos.AgentID", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetTasks) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetTasks{`, + `PendingTasks:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.PendingTasks), "Task", "mesos.Task", 1), `&`, ``, 1) + `,`, + `Tasks:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Tasks), "Task", "mesos.Task", 1), `&`, ``, 1) + `,`, + `CompletedTasks:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.CompletedTasks), "Task", "mesos.Task", 1), `&`, ``, 1) + `,`, + `OrphanTasks:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.OrphanTasks), "Task", "mesos.Task", 1), `&`, ``, 1) + `,`, + `UnreachableTasks:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.UnreachableTasks), "Task", "mesos.Task", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetRoles) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetRoles{`, + `Roles:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Roles), "Role", "mesos.Role", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetWeights) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetWeights{`, + `WeightInfos:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.WeightInfos), "WeightInfo", "mesos.WeightInfo", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetMaster) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetMaster{`, + `MasterInfo:` + strings.Replace(fmt.Sprintf("%v", this.MasterInfo), "MasterInfo", "mesos.MasterInfo", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetMaintenanceStatus) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetMaintenanceStatus{`, + `Status:` + strings.Replace(strings.Replace(this.Status.String(), "ClusterStatus", "mesos_maintenance.ClusterStatus", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetMaintenanceSchedule) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetMaintenanceSchedule{`, + `Schedule:` + strings.Replace(strings.Replace(this.Schedule.String(), "Schedule", "mesos_maintenance.Schedule", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Response_GetQuota) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Response_GetQuota{`, + `Status:` + strings.Replace(strings.Replace(this.Status.String(), "QuotaStatus", "mesos_quota.QuotaStatus", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Event) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Event{`, + `Type:` + fmt.Sprintf("%v", this.Type) + `,`, + `Subscribed:` + strings.Replace(fmt.Sprintf("%v", this.Subscribed), "Event_Subscribed", "Event_Subscribed", 1) + `,`, + `TaskAdded:` + strings.Replace(fmt.Sprintf("%v", this.TaskAdded), "Event_TaskAdded", "Event_TaskAdded", 1) + `,`, + `TaskUpdated:` + strings.Replace(fmt.Sprintf("%v", this.TaskUpdated), "Event_TaskUpdated", "Event_TaskUpdated", 1) + `,`, + `AgentAdded:` + strings.Replace(fmt.Sprintf("%v", this.AgentAdded), "Event_AgentAdded", "Event_AgentAdded", 1) + `,`, + `AgentRemoved:` + strings.Replace(fmt.Sprintf("%v", this.AgentRemoved), "Event_AgentRemoved", "Event_AgentRemoved", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Event_Subscribed) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Event_Subscribed{`, + `GetState:` + strings.Replace(fmt.Sprintf("%v", this.GetState), "Response_GetState", "Response_GetState", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Event_TaskAdded) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Event_TaskAdded{`, + `Task:` + strings.Replace(strings.Replace(this.Task.String(), "Task", "mesos.Task", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Event_TaskUpdated) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Event_TaskUpdated{`, + `FrameworkID:` + strings.Replace(strings.Replace(this.FrameworkID.String(), "FrameworkID", "mesos.FrameworkID", 1), `&`, ``, 1) + `,`, + `Status:` + strings.Replace(strings.Replace(this.Status.String(), "TaskStatus", "mesos.TaskStatus", 1), `&`, ``, 1) + `,`, + `State:` + valueToStringMaster(this.State) + `,`, + `}`, + }, "") + return s +} +func (this *Event_AgentAdded) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Event_AgentAdded{`, + `Agent:` + strings.Replace(strings.Replace(this.Agent.String(), "Response_GetAgents_Agent", "Response_GetAgents_Agent", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Event_AgentRemoved) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Event_AgentRemoved{`, + `AgentID:` + strings.Replace(strings.Replace(this.AgentID.String(), "AgentID", "mesos.AgentID", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func valueToStringMaster(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *Call) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Call: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Call: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= (Call_Type(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetMetrics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetMetrics == nil { + m.GetMetrics = &Call_GetMetrics{} + } + if err := m.GetMetrics.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SetLoggingLevel", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SetLoggingLevel == nil { + m.SetLoggingLevel = &Call_SetLoggingLevel{} + } + if err := m.SetLoggingLevel.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ListFiles", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ListFiles == nil { + m.ListFiles = &Call_ListFiles{} + } + if err := m.ListFiles.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReadFile", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ReadFile == nil { + m.ReadFile = &Call_ReadFile{} + } + if err := m.ReadFile.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UpdateWeights", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.UpdateWeights == nil { + m.UpdateWeights = &Call_UpdateWeights{} + } + if err := m.UpdateWeights.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReserveResources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ReserveResources == nil { + m.ReserveResources = &Call_ReserveResources{} + } + if err := m.ReserveResources.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UnreserveResources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.UnreserveResources == nil { + m.UnreserveResources = &Call_UnreserveResources{} + } + if err := m.UnreserveResources.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CreateVolumes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CreateVolumes == nil { + m.CreateVolumes = &Call_CreateVolumes{} + } + if err := m.CreateVolumes.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DestroyVolumes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DestroyVolumes == nil { + m.DestroyVolumes = &Call_DestroyVolumes{} + } + if err := m.DestroyVolumes.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UpdateMaintenanceSchedule", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.UpdateMaintenanceSchedule == nil { + m.UpdateMaintenanceSchedule = &Call_UpdateMaintenanceSchedule{} + } + if err := m.UpdateMaintenanceSchedule.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartMaintenance", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StartMaintenance == nil { + m.StartMaintenance = &Call_StartMaintenance{} + } + if err := m.StartMaintenance.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StopMaintenance", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StopMaintenance == nil { + m.StopMaintenance = &Call_StopMaintenance{} + } + if err := m.StopMaintenance.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SetQuota", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SetQuota == nil { + m.SetQuota = &Call_SetQuota{} + } + if err := m.SetQuota.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemoveQuota", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RemoveQuota == nil { + m.RemoveQuota = &Call_RemoveQuota{} + } + if err := m.RemoveQuota.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_GetMetrics) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetMetrics: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetMetrics: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timeout", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Timeout == nil { + m.Timeout = &mesos.DurationInfo{} + } + if err := m.Timeout.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_SetLoggingLevel) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SetLoggingLevel: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SetLoggingLevel: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Level", wireType) + } + m.Level = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Level |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Duration", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Duration.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000002) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("level") + } + if hasFields[0]&uint64(0x00000002) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("duration") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_ListFiles) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ListFiles: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ListFiles: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Path", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + 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 ErrInvalidLengthMaster + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Path = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("path") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_ReadFile) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ReadFile: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReadFile: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Path", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + 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 ErrInvalidLengthMaster + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Path = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Offset", wireType) + } + m.Offset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Offset |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + hasFields[0] |= uint64(0x00000002) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Length", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Length = &v + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("path") + } + if hasFields[0]&uint64(0x00000002) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("offset") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_UpdateWeights) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UpdateWeights: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UpdateWeights: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WeightInfos", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WeightInfos = append(m.WeightInfos, mesos.WeightInfo{}) + if err := m.WeightInfos[len(m.WeightInfos)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_ReserveResources) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ReserveResources: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReserveResources: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.AgentID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Resources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Resources = append(m.Resources, mesos.Resource{}) + if err := m.Resources[len(m.Resources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("agent_id") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_UnreserveResources) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UnreserveResources: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UnreserveResources: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.AgentID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Resources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Resources = append(m.Resources, mesos.Resource{}) + if err := m.Resources[len(m.Resources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("agent_id") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_CreateVolumes) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CreateVolumes: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CreateVolumes: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.AgentID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Volumes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Volumes = append(m.Volumes, mesos.Resource{}) + if err := m.Volumes[len(m.Volumes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("agent_id") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_DestroyVolumes) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DestroyVolumes: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DestroyVolumes: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.AgentID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Volumes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Volumes = append(m.Volumes, mesos.Resource{}) + if err := m.Volumes[len(m.Volumes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("agent_id") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_UpdateMaintenanceSchedule) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UpdateMaintenanceSchedule: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UpdateMaintenanceSchedule: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Schedule", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Schedule.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("schedule") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_StartMaintenance) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StartMaintenance: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StartMaintenance: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Machines", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Machines = append(m.Machines, mesos.MachineID{}) + if err := m.Machines[len(m.Machines)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_StopMaintenance) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StopMaintenance: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StopMaintenance: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Machines", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Machines = append(m.Machines, mesos.MachineID{}) + if err := m.Machines[len(m.Machines)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_SetQuota) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SetQuota: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SetQuota: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QuotaRequest", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.QuotaRequest.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("quota_request") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Call_RemoveQuota) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RemoveQuota: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RemoveQuota: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + 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 ErrInvalidLengthMaster + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Role = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("role") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Response: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Response: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= (Response_Type(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetHealth", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetHealth == nil { + m.GetHealth = &Response_GetHealth{} + } + if err := m.GetHealth.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetFlags", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetFlags == nil { + m.GetFlags = &Response_GetFlags{} + } + if err := m.GetFlags.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetVersion", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetVersion == nil { + m.GetVersion = &Response_GetVersion{} + } + if err := m.GetVersion.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetMetrics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetMetrics == nil { + m.GetMetrics = &Response_GetMetrics{} + } + if err := m.GetMetrics.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetLoggingLevel", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetLoggingLevel == nil { + m.GetLoggingLevel = &Response_GetLoggingLevel{} + } + if err := m.GetLoggingLevel.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ListFiles", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ListFiles == nil { + m.ListFiles = &Response_ListFiles{} + } + if err := m.ListFiles.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReadFile", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ReadFile == nil { + m.ReadFile = &Response_ReadFile{} + } + if err := m.ReadFile.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetState", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetState == nil { + m.GetState = &Response_GetState{} + } + if err := m.GetState.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetAgents", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetAgents == nil { + m.GetAgents = &Response_GetAgents{} + } + if err := m.GetAgents.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetFrameworks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetFrameworks == nil { + m.GetFrameworks = &Response_GetFrameworks{} + } + if err := m.GetFrameworks.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetExecutors", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetExecutors == nil { + m.GetExecutors = &Response_GetExecutors{} + } + if err := m.GetExecutors.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetTasks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetTasks == nil { + m.GetTasks = &Response_GetTasks{} + } + if err := m.GetTasks.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetRoles", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetRoles == nil { + m.GetRoles = &Response_GetRoles{} + } + if err := m.GetRoles.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetWeights", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetWeights == nil { + m.GetWeights = &Response_GetWeights{} + } + if err := m.GetWeights.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 16: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetMaster", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetMaster == nil { + m.GetMaster = &Response_GetMaster{} + } + if err := m.GetMaster.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetMaintenanceStatus", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetMaintenanceStatus == nil { + m.GetMaintenanceStatus = &Response_GetMaintenanceStatus{} + } + if err := m.GetMaintenanceStatus.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 18: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetMaintenanceSchedule", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetMaintenanceSchedule == nil { + m.GetMaintenanceSchedule = &Response_GetMaintenanceSchedule{} + } + if err := m.GetMaintenanceSchedule.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 19: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetQuota", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetQuota == nil { + m.GetQuota = &Response_GetQuota{} + } + if err := m.GetQuota.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetHealth) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetHealth: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetHealth: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Healthy", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Healthy = bool(v != 0) + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("healthy") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetFlags) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetFlags: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetFlags: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Flags", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Flags = append(m.Flags, mesos.Flag{}) + if err := m.Flags[len(m.Flags)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetVersion) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetVersion: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetVersion: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VersionInfo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.VersionInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("version_info") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetMetrics) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetMetrics: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetMetrics: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metrics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Metrics = append(m.Metrics, mesos.Metric{}) + if err := m.Metrics[len(m.Metrics)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetLoggingLevel) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetLoggingLevel: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetLoggingLevel: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Level", wireType) + } + m.Level = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Level |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("level") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_ListFiles) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ListFiles: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ListFiles: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FileInfos", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FileInfos = append(m.FileInfos, mesos.FileInfo{}) + if err := m.FileInfos[len(m.FileInfos)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_ReadFile) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ReadFile: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReadFile: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Size", wireType) + } + m.Size = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Size |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...) + if m.Data == nil { + m.Data = []byte{} + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000002) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("size") + } + if hasFields[0]&uint64(0x00000002) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("data") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetState) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetState: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetState: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetTasks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetTasks == nil { + m.GetTasks = &Response_GetTasks{} + } + if err := m.GetTasks.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetExecutors", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetExecutors == nil { + m.GetExecutors = &Response_GetExecutors{} + } + if err := m.GetExecutors.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetFrameworks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetFrameworks == nil { + m.GetFrameworks = &Response_GetFrameworks{} + } + if err := m.GetFrameworks.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetAgents", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetAgents == nil { + m.GetAgents = &Response_GetAgents{} + } + if err := m.GetAgents.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetAgents) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetAgents: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetAgents: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Agents", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Agents = append(m.Agents, Response_GetAgents_Agent{}) + if err := m.Agents[len(m.Agents)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RecoveredAgents", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RecoveredAgents = append(m.RecoveredAgents, mesos.AgentInfo{}) + if err := m.RecoveredAgents[len(m.RecoveredAgents)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetAgents_Agent) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Agent: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Agent: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentInfo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.AgentInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Active", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Active = bool(v != 0) + hasFields[0] |= uint64(0x00000002) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + 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 ErrInvalidLengthMaster + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + hasFields[0] |= uint64(0x00000004) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PID", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + 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 ErrInvalidLengthMaster + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + s := string(dAtA[iNdEx:postIndex]) + m.PID = &s + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RegisteredTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RegisteredTime == nil { + m.RegisteredTime = &mesos.TimeInfo{} + } + if err := m.RegisteredTime.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReregisteredTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ReregisteredTime == nil { + m.ReregisteredTime = &mesos.TimeInfo{} + } + if err := m.ReregisteredTime.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalResources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TotalResources = append(m.TotalResources, mesos.Resource{}) + if err := m.TotalResources[len(m.TotalResources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AllocatedResources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AllocatedResources = append(m.AllocatedResources, mesos.Resource{}) + if err := m.AllocatedResources[len(m.AllocatedResources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OfferedResources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OfferedResources = append(m.OfferedResources, mesos.Resource{}) + if err := m.OfferedResources[len(m.OfferedResources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("agent_info") + } + if hasFields[0]&uint64(0x00000002) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("active") + } + if hasFields[0]&uint64(0x00000004) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("version") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetFrameworks) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetFrameworks: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetFrameworks: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Frameworks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Frameworks = append(m.Frameworks, Response_GetFrameworks_Framework{}) + if err := m.Frameworks[len(m.Frameworks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CompletedFrameworks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CompletedFrameworks = append(m.CompletedFrameworks, Response_GetFrameworks_Framework{}) + if err := m.CompletedFrameworks[len(m.CompletedFrameworks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RecoveredFrameworks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RecoveredFrameworks = append(m.RecoveredFrameworks, mesos.FrameworkInfo{}) + if err := m.RecoveredFrameworks[len(m.RecoveredFrameworks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetFrameworks_Framework) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Framework: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Framework: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FrameworkInfo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.FrameworkInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Active", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Active = bool(v != 0) + hasFields[0] |= uint64(0x00000002) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Connected", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Connected = bool(v != 0) + hasFields[0] |= uint64(0x00000004) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RegisteredTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RegisteredTime == nil { + m.RegisteredTime = &mesos.TimeInfo{} + } + if err := m.RegisteredTime.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReregisteredTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ReregisteredTime == nil { + m.ReregisteredTime = &mesos.TimeInfo{} + } + if err := m.ReregisteredTime.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UnregisteredTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.UnregisteredTime == nil { + m.UnregisteredTime = &mesos.TimeInfo{} + } + if err := m.UnregisteredTime.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Offers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Offers = append(m.Offers, mesos.Offer{}) + if err := m.Offers[len(m.Offers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InverseOffers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InverseOffers = append(m.InverseOffers, mesos.InverseOffer{}) + if err := m.InverseOffers[len(m.InverseOffers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AllocatedResources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AllocatedResources = append(m.AllocatedResources, mesos.Resource{}) + if err := m.AllocatedResources[len(m.AllocatedResources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OfferedResources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OfferedResources = append(m.OfferedResources, mesos.Resource{}) + if err := m.OfferedResources[len(m.OfferedResources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Recovered", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Recovered = bool(v != 0) + hasFields[0] |= uint64(0x00000008) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("framework_info") + } + if hasFields[0]&uint64(0x00000002) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("active") + } + if hasFields[0]&uint64(0x00000004) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("connected") + } + if hasFields[0]&uint64(0x00000008) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("recovered") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetExecutors) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetExecutors: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetExecutors: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Executors", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Executors = append(m.Executors, Response_GetExecutors_Executor{}) + if err := m.Executors[len(m.Executors)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OrphanExecutors", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OrphanExecutors = append(m.OrphanExecutors, Response_GetExecutors_Executor{}) + if err := m.OrphanExecutors[len(m.OrphanExecutors)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetExecutors_Executor) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Executor: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Executor: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutorInfo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ExecutorInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.AgentID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000002) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("executor_info") + } + if hasFields[0]&uint64(0x00000002) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("agent_id") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetTasks) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetTasks: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetTasks: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PendingTasks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PendingTasks = append(m.PendingTasks, mesos.Task{}) + if err := m.PendingTasks[len(m.PendingTasks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tasks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Tasks = append(m.Tasks, mesos.Task{}) + if err := m.Tasks[len(m.Tasks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CompletedTasks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CompletedTasks = append(m.CompletedTasks, mesos.Task{}) + if err := m.CompletedTasks[len(m.CompletedTasks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OrphanTasks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OrphanTasks = append(m.OrphanTasks, mesos.Task{}) + if err := m.OrphanTasks[len(m.OrphanTasks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UnreachableTasks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.UnreachableTasks = append(m.UnreachableTasks, mesos.Task{}) + if err := m.UnreachableTasks[len(m.UnreachableTasks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetRoles) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetRoles: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetRoles: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Roles", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Roles = append(m.Roles, mesos.Role{}) + if err := m.Roles[len(m.Roles)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetWeights) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetWeights: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetWeights: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WeightInfos", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WeightInfos = append(m.WeightInfos, mesos.WeightInfo{}) + if err := m.WeightInfos[len(m.WeightInfos)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetMaster) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetMaster: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetMaster: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MasterInfo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MasterInfo == nil { + m.MasterInfo = &mesos.MasterInfo{} + } + if err := m.MasterInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetMaintenanceStatus) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetMaintenanceStatus: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetMaintenanceStatus: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Status.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("status") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetMaintenanceSchedule) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetMaintenanceSchedule: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetMaintenanceSchedule: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Schedule", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Schedule.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("schedule") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response_GetQuota) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetQuota: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetQuota: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Status.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("status") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Event) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Event: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Event: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= (Event_Type(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Subscribed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Subscribed == nil { + m.Subscribed = &Event_Subscribed{} + } + if err := m.Subscribed.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TaskAdded", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TaskAdded == nil { + m.TaskAdded = &Event_TaskAdded{} + } + if err := m.TaskAdded.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TaskUpdated", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TaskUpdated == nil { + m.TaskUpdated = &Event_TaskUpdated{} + } + if err := m.TaskUpdated.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentAdded", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AgentAdded == nil { + m.AgentAdded = &Event_AgentAdded{} + } + if err := m.AgentAdded.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentRemoved", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AgentRemoved == nil { + m.AgentRemoved = &Event_AgentRemoved{} + } + if err := m.AgentRemoved.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Event_Subscribed) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Subscribed: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Subscribed: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetState", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.GetState == nil { + m.GetState = &Response_GetState{} + } + if err := m.GetState.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Event_TaskAdded) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TaskAdded: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TaskAdded: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Task.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("task") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Event_TaskUpdated) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TaskUpdated: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TaskUpdated: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FrameworkID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.FrameworkID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Status.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000002) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + var v mesos.TaskState + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (mesos.TaskState(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.State = &v + hasFields[0] |= uint64(0x00000004) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("framework_id") + } + if hasFields[0]&uint64(0x00000002) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("status") + } + if hasFields[0]&uint64(0x00000004) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("state") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Event_AgentAdded) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AgentAdded: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AgentAdded: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Agent", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Agent.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("agent") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Event_AgentRemoved) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AgentRemoved: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AgentRemoved: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaster + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.AgentID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + hasFields[0] |= uint64(0x00000001) + default: + iNdEx = preIndex + skippy, err := skipMaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("agent_id") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipMaster(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowMaster + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowMaster + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowMaster + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthMaster + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowMaster + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipMaster(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthMaster = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowMaster = fmt.Errorf("proto: integer overflow") +) + +func init() { proto.RegisterFile("master/master.proto", fileDescriptorMaster) } + +var fileDescriptorMaster = []byte{ + // 2887 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0x4b, 0x6c, 0x1b, 0xd7, + 0xd5, 0xf6, 0x50, 0xa4, 0x44, 0x1e, 0xbe, 0x86, 0x57, 0x8a, 0x4d, 0x8f, 0x1d, 0x5a, 0xbf, 0xfe, + 0xc6, 0x36, 0x9a, 0x58, 0x6a, 0x94, 0x07, 0x82, 0xc4, 0xb5, 0x4b, 0x8a, 0x63, 0x89, 0xb6, 0x24, + 0x2a, 0x33, 0x94, 0xdc, 0x06, 0x28, 0x88, 0x31, 0xe7, 0x6a, 0x44, 0x84, 0xe4, 0x30, 0x33, 0x43, + 0xa5, 0x2e, 0x5a, 0xa0, 0x40, 0x1f, 0xe8, 0xb2, 0xfb, 0xa2, 0x40, 0x80, 0x6e, 0xba, 0xee, 0xaa, + 0xcb, 0x2e, 0xd3, 0x5d, 0x96, 0x05, 0x0a, 0x18, 0xb1, 0xd2, 0x45, 0x97, 0x41, 0x56, 0x5d, 0x16, + 0xf7, 0x35, 0xbc, 0x43, 0x0e, 0x69, 0xd9, 0x0e, 0xb2, 0x91, 0xee, 0x3d, 0xf7, 0x3b, 0xdf, 0x7d, + 0x9e, 0x33, 0xe7, 0x1c, 0xc2, 0x72, 0xdf, 0xf2, 0x03, 0xec, 0x6d, 0xb0, 0x7f, 0xeb, 0x43, 0xcf, + 0x0d, 0x5c, 0x94, 0xeb, 0x63, 0xdf, 0xf5, 0xd7, 0x99, 0x4c, 0xfb, 0x81, 0xd3, 0x0d, 0x4e, 0x46, + 0x8f, 0xd6, 0x3b, 0x6e, 0x7f, 0x83, 0x0e, 0xb0, 0xbf, 0xb7, 0x1c, 0x77, 0xc3, 0x1a, 0x76, 0x37, + 0x4e, 0xdf, 0xdc, 0xe8, 0x75, 0x1f, 0x31, 0x19, 0xd3, 0xd7, 0xee, 0x9c, 0x4b, 0xc3, 0xea, 0xf5, + 0xdc, 0x8e, 0x15, 0xb8, 0xde, 0xb8, 0xc5, 0xf5, 0x6b, 0xe7, 0x9b, 0xd1, 0xea, 0x0e, 0x02, 0x3c, + 0xb0, 0x06, 0x1d, 0x2c, 0xb7, 0x39, 0xc7, 0xbb, 0xe7, 0xe2, 0xf8, 0x64, 0xe4, 0x06, 0x16, 0xfb, + 0xcb, 0xf5, 0x6e, 0x49, 0x7a, 0x8e, 0xeb, 0xb8, 0x1b, 0x54, 0xfc, 0x68, 0x74, 0x4c, 0x7b, 0xb4, + 0x43, 0x5b, 0x0c, 0xbe, 0xf6, 0xc7, 0x4b, 0x90, 0xdc, 0xb2, 0x7a, 0x3d, 0xf4, 0x26, 0x24, 0x83, + 0xc7, 0x43, 0x5c, 0x56, 0x56, 0x95, 0x9b, 0x85, 0xcd, 0x4b, 0xeb, 0xf2, 0x11, 0xae, 0x13, 0xc4, + 0x7a, 0xeb, 0xf1, 0x10, 0xd7, 0x92, 0x9f, 0x3f, 0xb9, 0x76, 0xc1, 0xa0, 0x50, 0x74, 0x07, 0xb2, + 0x0e, 0x0e, 0xda, 0x7d, 0x1c, 0x78, 0xdd, 0x8e, 0x5f, 0x4e, 0xac, 0x2a, 0x37, 0xb3, 0x9b, 0xaf, + 0xc6, 0x68, 0x6e, 0xe3, 0x60, 0x8f, 0x81, 0x0c, 0x70, 0xc2, 0x36, 0xda, 0x87, 0x92, 0x8f, 0x83, + 0x76, 0xcf, 0x75, 0x9c, 0xee, 0xc0, 0x69, 0xf7, 0xf0, 0x29, 0xee, 0x95, 0x17, 0x28, 0xcb, 0x5a, + 0x0c, 0x8b, 0x89, 0x83, 0x5d, 0x06, 0xdd, 0x25, 0x48, 0xa3, 0xe8, 0x47, 0x05, 0xe8, 0x03, 0x80, + 0x5e, 0xd7, 0x0f, 0xda, 0xc7, 0xdd, 0x1e, 0xf6, 0xcb, 0x49, 0x4a, 0x74, 0x35, 0x86, 0x68, 0xb7, + 0xeb, 0x07, 0xf7, 0x08, 0xc6, 0xc8, 0xf4, 0x44, 0x13, 0xbd, 0x07, 0x19, 0x0f, 0x5b, 0x36, 0x55, + 0x2e, 0xa7, 0xa8, 0xee, 0x95, 0x18, 0x5d, 0x03, 0x5b, 0x36, 0x51, 0x30, 0xd2, 0x1e, 0x6f, 0xa1, + 0x6d, 0x28, 0x8c, 0x86, 0xb6, 0x15, 0xe0, 0xf6, 0xa7, 0xb8, 0xeb, 0x9c, 0x04, 0x7e, 0x79, 0x91, + 0xaa, 0xaf, 0xc6, 0xa8, 0x1f, 0x52, 0xe0, 0x43, 0x86, 0x33, 0xf2, 0x23, 0xb9, 0x8b, 0x0e, 0xa0, + 0xe4, 0x61, 0x1f, 0x7b, 0xa7, 0xb8, 0xed, 0x61, 0xdf, 0x1d, 0x79, 0x1d, 0xec, 0x97, 0x97, 0x28, + 0xd7, 0xff, 0xc7, 0x2e, 0x85, 0x62, 0x0d, 0x01, 0x35, 0x54, 0x6f, 0x42, 0x82, 0x8e, 0x60, 0x79, + 0x34, 0x98, 0xe6, 0x4c, 0x53, 0xce, 0xd7, 0xe2, 0xd6, 0x37, 0x98, 0xe4, 0x30, 0xd0, 0x68, 0x4a, + 0x46, 0xb6, 0xdc, 0xf1, 0x30, 0xd9, 0xf2, 0xa9, 0xdb, 0x1b, 0xf5, 0xb1, 0x5f, 0xce, 0xcc, 0xdc, + 0xf2, 0x16, 0x05, 0x1e, 0x31, 0x9c, 0x91, 0xef, 0xc8, 0x5d, 0x74, 0x1f, 0x8a, 0x36, 0xf6, 0x03, + 0xcf, 0x7d, 0x1c, 0x32, 0x01, 0x65, 0xfa, 0xbf, 0x18, 0xa6, 0x3a, 0x43, 0x0a, 0xaa, 0x82, 0x1d, + 0xe9, 0xa3, 0x1e, 0x5c, 0xe1, 0xf7, 0x20, 0x59, 0x53, 0xdb, 0xef, 0x9c, 0x60, 0x7b, 0xd4, 0xc3, + 0xe5, 0x2c, 0xe5, 0x7d, 0x63, 0xe6, 0xa5, 0xec, 0x8d, 0x95, 0x4c, 0xae, 0x63, 0x5c, 0x1e, 0xcd, + 0x1a, 0x22, 0x97, 0xe5, 0x07, 0x96, 0x17, 0xc8, 0x93, 0x95, 0x73, 0x33, 0x2f, 0xcb, 0x24, 0x58, + 0x89, 0xc7, 0x50, 0xfd, 0x09, 0x09, 0xda, 0x03, 0xd5, 0x0f, 0xdc, 0x61, 0x84, 0x30, 0x3f, 0xdb, + 0x1a, 0x02, 0x77, 0x28, 0xf3, 0x15, 0xfd, 0xa8, 0x80, 0x3c, 0x68, 0x62, 0x5d, 0xd4, 0x37, 0x94, + 0x0b, 0x33, 0x1f, 0xb4, 0x89, 0x83, 0x0f, 0x09, 0xc4, 0x48, 0xfb, 0xbc, 0x85, 0xaa, 0x90, 0xf3, + 0x70, 0xdf, 0x3d, 0xc5, 0x5c, 0xb9, 0x48, 0x95, 0x2b, 0xb1, 0x4f, 0x90, 0xc0, 0x98, 0x7e, 0xd6, + 0x1b, 0x77, 0xb4, 0x0f, 0x00, 0xc6, 0x46, 0x8f, 0x6e, 0xc1, 0x52, 0xd0, 0xed, 0x63, 0x77, 0x14, + 0x50, 0xf7, 0x92, 0xdd, 0x5c, 0xe6, 0x5c, 0xf5, 0x91, 0x67, 0x05, 0x5d, 0x77, 0xd0, 0x18, 0x1c, + 0xbb, 0x86, 0xc0, 0x68, 0x36, 0x14, 0x27, 0x6c, 0x1d, 0x69, 0x90, 0x62, 0xee, 0x41, 0x59, 0x4d, + 0xdc, 0xcc, 0x73, 0x2f, 0xc4, 0x44, 0xe8, 0x1d, 0x48, 0xdb, 0x9c, 0xa7, 0x9c, 0x58, 0x4d, 0xcc, + 0xa0, 0xe7, 0x3a, 0x21, 0x54, 0x7b, 0x0d, 0x32, 0xa1, 0x23, 0x40, 0x65, 0x48, 0x0e, 0xad, 0xe0, + 0x84, 0xd2, 0x67, 0x84, 0x93, 0x23, 0x12, 0xed, 0x23, 0x48, 0x0b, 0x9b, 0x9f, 0x8d, 0x42, 0x57, + 0x61, 0xd1, 0x3d, 0x3e, 0xf6, 0x71, 0x40, 0x57, 0x90, 0xe4, 0x63, 0x5c, 0x86, 0x2e, 0xc2, 0x62, + 0x0f, 0x0f, 0x9c, 0xe0, 0x84, 0x7a, 0xb7, 0xa4, 0xc1, 0x7b, 0xda, 0x03, 0xc8, 0x47, 0x1c, 0x02, + 0x7a, 0x1f, 0x72, 0xcc, 0x87, 0xb4, 0xbb, 0x83, 0x63, 0xd7, 0x2f, 0x2b, 0xab, 0x0b, 0x37, 0xb3, + 0x9b, 0x25, 0xbe, 0x1d, 0x86, 0x92, 0x36, 0x93, 0xfd, 0x34, 0x94, 0xf8, 0xda, 0xaf, 0x15, 0x50, + 0x27, 0x5d, 0x02, 0x7a, 0x1f, 0xd2, 0x96, 0x83, 0x07, 0x41, 0xbb, 0x6b, 0xd3, 0x55, 0x67, 0x37, + 0x0b, 0x9c, 0xac, 0x4a, 0xc4, 0x8d, 0x7a, 0xad, 0x48, 0x98, 0xce, 0x9e, 0x5c, 0x5b, 0xe2, 0x02, + 0x63, 0x89, 0x2a, 0x34, 0x6c, 0xf4, 0x16, 0xf1, 0x88, 0xc2, 0x65, 0x24, 0xe8, 0x4a, 0x8a, 0x5c, + 0x59, 0x4c, 0xc0, 0xd7, 0x31, 0xc6, 0x69, 0xbf, 0x55, 0x00, 0x4d, 0x3b, 0x91, 0xef, 0x7e, 0x1d, + 0xbf, 0x80, 0x7c, 0xc4, 0xf1, 0xbc, 0xd4, 0x0a, 0x36, 0x60, 0x49, 0x78, 0xa7, 0xb9, 0xf3, 0x0b, + 0x94, 0xf6, 0x4b, 0x28, 0x44, 0x9d, 0xd5, 0x77, 0x3b, 0xfd, 0x47, 0x70, 0x79, 0xa6, 0x4f, 0x43, + 0x3f, 0x84, 0x74, 0xe8, 0x13, 0xd9, 0x4a, 0xc6, 0x6e, 0x61, 0x1c, 0x84, 0x08, 0xb8, 0x30, 0x1b, + 0xa1, 0xa2, 0xdd, 0x03, 0x75, 0xd2, 0x97, 0xa1, 0x4d, 0x48, 0xf7, 0xad, 0xce, 0x49, 0x77, 0x80, + 0xc5, 0x93, 0x55, 0x39, 0xe5, 0x1e, 0x13, 0x37, 0xea, 0x82, 0x47, 0xe0, 0x34, 0x1d, 0x8a, 0x13, + 0x2e, 0xec, 0x85, 0x68, 0x0e, 0x20, 0x2d, 0x3c, 0x18, 0xaa, 0x43, 0x9e, 0x3a, 0xac, 0xb6, 0x87, + 0x3f, 0x19, 0x61, 0x3f, 0xe0, 0xdb, 0xbb, 0xcc, 0x49, 0x58, 0x94, 0xc4, 0x9c, 0x15, 0x03, 0x70, + 0xb6, 0xdc, 0x27, 0x92, 0x4c, 0xbb, 0x01, 0x59, 0xc9, 0xad, 0x11, 0x9b, 0xf7, 0x5c, 0x7e, 0x54, + 0xa1, 0xcd, 0x13, 0xc9, 0xda, 0x97, 0x49, 0x48, 0x92, 0x98, 0x08, 0x65, 0x61, 0xe9, 0x70, 0xff, + 0xc1, 0x7e, 0xf3, 0xe1, 0xbe, 0x7a, 0x01, 0x15, 0x00, 0xb6, 0xf5, 0x56, 0x7b, 0x47, 0xaf, 0xee, + 0xb6, 0x76, 0x54, 0x05, 0xe5, 0x21, 0x43, 0xfa, 0xf7, 0x76, 0xab, 0xdb, 0xa6, 0x9a, 0x40, 0x45, + 0xc8, 0x92, 0xee, 0x91, 0x6e, 0x98, 0x8d, 0xe6, 0xbe, 0xba, 0x20, 0x04, 0x7b, 0x7a, 0xcb, 0x68, + 0x6c, 0x99, 0x6a, 0x12, 0xbd, 0x02, 0x25, 0x22, 0xd8, 0x6d, 0x6e, 0x6f, 0x37, 0xf6, 0xb7, 0xdb, + 0xbb, 0xfa, 0x91, 0xbe, 0xab, 0xa6, 0x88, 0xd8, 0x9c, 0x12, 0x2f, 0x92, 0xe9, 0x76, 0x1b, 0x66, + 0xab, 0x7d, 0xaf, 0xb1, 0xab, 0x9b, 0xea, 0x12, 0x99, 0xce, 0xd0, 0xab, 0x75, 0xda, 0x57, 0xd3, + 0x62, 0x76, 0xb3, 0x55, 0x6d, 0xe9, 0x6a, 0x46, 0x2c, 0xae, 0xba, 0xad, 0xef, 0xb7, 0x4c, 0x15, + 0x10, 0x82, 0x02, 0x5d, 0x9c, 0x51, 0xdd, 0xd3, 0x1f, 0x36, 0x8d, 0x07, 0xa6, 0x9a, 0x45, 0x25, + 0xc8, 0x13, 0x99, 0xfe, 0x63, 0x7d, 0xeb, 0xb0, 0xd5, 0x34, 0x4c, 0x35, 0x27, 0x58, 0x5a, 0x55, + 0xf3, 0x81, 0xa9, 0xe6, 0x45, 0xd7, 0x68, 0x92, 0x29, 0x0b, 0x62, 0x07, 0x0f, 0xf5, 0xc6, 0xf6, + 0x4e, 0xcb, 0x54, 0x8b, 0x84, 0xf5, 0xf0, 0xa0, 0x5e, 0x6d, 0xe9, 0xa1, 0x4c, 0x15, 0x33, 0xef, + 0x55, 0xcd, 0x96, 0x6e, 0xa8, 0x25, 0xc2, 0x61, 0x1e, 0xd6, 0xcc, 0x2d, 0xa3, 0x51, 0xd3, 0x55, + 0x44, 0x76, 0x67, 0xe8, 0xa6, 0x6e, 0x1c, 0xe9, 0x6d, 0x43, 0x37, 0x9b, 0x87, 0xc6, 0x96, 0x6e, + 0xaa, 0xcb, 0xe8, 0x12, 0x2c, 0x1f, 0xee, 0x4f, 0x0f, 0xac, 0x90, 0x29, 0xb6, 0x0c, 0x9d, 0x4c, + 0x71, 0xd4, 0xdc, 0x3d, 0xdc, 0xd3, 0x4d, 0xf5, 0x15, 0xb4, 0x0c, 0xc5, 0xba, 0x6e, 0xb6, 0x8c, + 0xe6, 0x4f, 0x42, 0xe1, 0x45, 0xa4, 0xc1, 0x45, 0x36, 0x6f, 0x63, 0xbf, 0xa5, 0xef, 0x57, 0xf7, + 0xb7, 0x74, 0x7a, 0x18, 0x87, 0xa6, 0x7a, 0x09, 0x5d, 0x85, 0xf2, 0xd4, 0xd8, 0xd6, 0x8e, 0x5e, + 0x3f, 0xdc, 0xd5, 0xd5, 0x32, 0xba, 0x06, 0x57, 0xf8, 0x2e, 0x62, 0x01, 0x97, 0xe9, 0x8d, 0xb4, + 0xaa, 0x46, 0x84, 0x40, 0xd5, 0xd0, 0x0a, 0xa8, 0x66, 0xab, 0x79, 0x10, 0x91, 0x5e, 0x11, 0x67, + 0xf6, 0xe1, 0x61, 0xb3, 0x55, 0x55, 0xaf, 0xd2, 0xed, 0x87, 0xdd, 0x57, 0x91, 0x0a, 0x39, 0x43, + 0xdf, 0x6b, 0x1e, 0xe9, 0x5c, 0x52, 0xd1, 0x92, 0xbf, 0xff, 0x73, 0x45, 0x59, 0xfb, 0xe6, 0x3a, + 0xf9, 0xfa, 0xf8, 0x43, 0x77, 0xe0, 0x63, 0xf4, 0x4e, 0x24, 0x42, 0x9f, 0xf8, 0x96, 0x0b, 0xd4, + 0x74, 0x94, 0x7e, 0x17, 0x48, 0xcc, 0xdd, 0x3e, 0xc1, 0x56, 0x2f, 0x38, 0xe1, 0x41, 0xfa, 0xea, + 0x0c, 0xe5, 0x6d, 0x1c, 0xec, 0x50, 0x9c, 0x91, 0x71, 0x44, 0x13, 0xdd, 0x06, 0xd2, 0x69, 0x1f, + 0xf7, 0x2c, 0xc7, 0xe7, 0xe1, 0xf9, 0xb5, 0xd9, 0xfa, 0xf7, 0x08, 0xcc, 0x48, 0x3b, 0xbc, 0x85, + 0x6a, 0x2c, 0x49, 0x38, 0xc5, 0x9e, 0x4f, 0x3e, 0xd0, 0xc9, 0xb8, 0xe8, 0x4e, 0xd6, 0x3f, 0x62, + 0x40, 0x9a, 0x28, 0xf0, 0xb6, 0xe0, 0x10, 0x89, 0x46, 0xea, 0x59, 0x1c, 0x71, 0xc9, 0x86, 0x01, + 0x25, 0x67, 0x2a, 0xd9, 0x60, 0x81, 0xfa, 0xf5, 0xd9, 0x4c, 0xd1, 0x84, 0xc3, 0x99, 0x88, 0x4a, + 0xee, 0x46, 0x12, 0x8e, 0xa5, 0xb9, 0x47, 0x1b, 0x9b, 0x74, 0xdc, 0x96, 0x93, 0x8e, 0xf4, 0xdc, + 0xa3, 0x8d, 0x49, 0x3c, 0xf8, 0xc5, 0xf8, 0x81, 0x15, 0x60, 0x1e, 0x80, 0xcf, 0xb9, 0x18, 0x93, + 0xc0, 0xe8, 0xc5, 0xd0, 0x96, 0x78, 0x17, 0xf4, 0x23, 0x23, 0xa2, 0xee, 0x39, 0xef, 0x82, 0x7e, + 0x95, 0x7c, 0xfa, 0x2e, 0x58, 0x13, 0x3d, 0x80, 0x02, 0x7d, 0x17, 0x9e, 0xd5, 0xc7, 0x9f, 0xba, + 0xde, 0xc7, 0x3e, 0x0f, 0xb1, 0xbf, 0x37, 0xe7, 0x71, 0x84, 0x58, 0x23, 0xef, 0xc8, 0x5d, 0xb4, + 0x03, 0x44, 0xd0, 0xc6, 0x3f, 0xc3, 0x9d, 0x51, 0xe0, 0x7a, 0x7e, 0x7c, 0x28, 0x2d, 0x73, 0xe9, + 0x02, 0x6a, 0xe4, 0x1c, 0xa9, 0x27, 0x4e, 0x25, 0xb0, 0xfc, 0x8f, 0x7d, 0x1e, 0x3f, 0xcf, 0x39, + 0x95, 0x16, 0x81, 0xd1, 0x53, 0xa1, 0x2d, 0xa1, 0x4d, 0x1c, 0xbc, 0xcf, 0xa3, 0xe6, 0x39, 0xda, + 0x06, 0x81, 0x51, 0x6d, 0xda, 0x12, 0x0f, 0x55, 0xe4, 0x81, 0xc5, 0x67, 0x3d, 0x54, 0x91, 0x08, + 0x92, 0x9b, 0x10, 0x31, 0x20, 0xbf, 0x17, 0x86, 0x2e, 0xab, 0xcf, 0xba, 0x97, 0x3d, 0x2a, 0xa2, + 0xf7, 0xc2, 0x9a, 0xc8, 0x82, 0x8b, 0x8c, 0x40, 0x4a, 0x82, 0x02, 0x2b, 0x18, 0xf9, 0xe5, 0x12, + 0x25, 0x7b, 0x7d, 0x1e, 0xd9, 0x38, 0x5e, 0xa0, 0x2a, 0xc6, 0x8a, 0x13, 0x23, 0x45, 0x0e, 0x94, + 0xa7, 0xa6, 0x10, 0x31, 0x05, 0xa2, 0x93, 0xdc, 0x3a, 0xdf, 0x24, 0x22, 0xd1, 0xba, 0xe8, 0xc4, + 0xca, 0xc5, 0x75, 0xb0, 0x3c, 0x64, 0xf9, 0x59, 0xd7, 0xc1, 0x13, 0x19, 0x87, 0xb7, 0xb4, 0xd7, + 0x21, 0x13, 0x7a, 0x34, 0x54, 0x81, 0x25, 0xe6, 0x03, 0x1f, 0xd3, 0x6f, 0x79, 0x5a, 0x04, 0x4d, + 0x5c, 0xa8, 0xbd, 0x05, 0x69, 0xe1, 0xbe, 0xd0, 0x0d, 0x48, 0x31, 0x77, 0xc7, 0xc2, 0x90, 0x2c, + 0x9f, 0x92, 0x0c, 0x8a, 0xdc, 0x83, 0x8e, 0x6b, 0x0d, 0x9a, 0xe7, 0x08, 0x3f, 0xf5, 0x01, 0xe4, + 0xb8, 0x9f, 0xa3, 0xf1, 0x3b, 0x8f, 0x3f, 0x10, 0xd7, 0xe6, 0x28, 0x39, 0x7e, 0x3f, 0x1d, 0x8b, + 0xa6, 0x53, 0x26, 0xe1, 0xee, 0xd8, 0x1a, 0xf2, 0x22, 0x14, 0xa2, 0x52, 0xb1, 0x78, 0x8e, 0xd1, + 0x6e, 0x41, 0x71, 0xfb, 0xfc, 0x29, 0x93, 0x56, 0x95, 0x73, 0x9f, 0xb7, 0x01, 0x88, 0xff, 0x89, + 0xa4, 0x1c, 0x22, 0xc2, 0x24, 0x08, 0x69, 0xc1, 0x99, 0x63, 0xde, 0xf7, 0xb5, 0xf7, 0xa2, 0x79, + 0x91, 0xdf, 0xfd, 0x39, 0x8b, 0x91, 0x44, 0xee, 0x43, 0x25, 0x08, 0x41, 0xd2, 0xb6, 0x02, 0x8b, + 0x66, 0x45, 0x39, 0x83, 0xb6, 0xb5, 0xcf, 0x12, 0xf4, 0xa4, 0x99, 0x17, 0x8a, 0x58, 0xab, 0xf2, + 0xbc, 0xd6, 0x3a, 0xe5, 0x35, 0x12, 0x2f, 0xea, 0x35, 0xa6, 0x9d, 0xd9, 0xc2, 0x8b, 0x3b, 0xb3, + 0xa8, 0x6b, 0x4d, 0x3e, 0xb7, 0x6b, 0xd5, 0xbe, 0x49, 0xd2, 0x97, 0xcb, 0x1d, 0x6d, 0x1d, 0x16, + 0x39, 0x15, 0xbb, 0x9c, 0xeb, 0xcf, 0xa2, 0x62, 0x39, 0x85, 0x48, 0x42, 0x99, 0x2e, 0xaa, 0x82, + 0xea, 0xe1, 0x8e, 0x7b, 0x8a, 0x3d, 0x6c, 0x8b, 0xa5, 0x25, 0x22, 0x51, 0x36, 0x4b, 0x3c, 0xc6, + 0xb7, 0x5d, 0x0c, 0xf1, 0x7c, 0x59, 0xff, 0x5a, 0x80, 0x14, 0x6d, 0xa2, 0x77, 0x00, 0x78, 0x3a, + 0x33, 0x7e, 0xe7, 0xb3, 0x68, 0x32, 0x96, 0x10, 0x90, 0x34, 0xd9, 0xea, 0x04, 0xdd, 0x53, 0x4c, + 0x1f, 0x44, 0x3a, 0x5c, 0x21, 0x95, 0x11, 0x0b, 0x15, 0x61, 0xc2, 0x82, 0x14, 0x6d, 0x0b, 0x21, + 0xba, 0x0c, 0x0b, 0xc3, 0xae, 0x4d, 0xcf, 0x33, 0x53, 0x5b, 0x3a, 0x7b, 0x72, 0x6d, 0xe1, 0xa0, + 0x51, 0x37, 0x88, 0x0c, 0xbd, 0x07, 0x45, 0x0f, 0x3b, 0x5d, 0x72, 0x1e, 0xd8, 0x6e, 0x07, 0xdd, + 0xbe, 0xa8, 0xe1, 0x89, 0x87, 0xdc, 0xea, 0xf6, 0xe9, 0xc3, 0x35, 0x0a, 0x63, 0x1c, 0x91, 0xa1, + 0xdb, 0x50, 0xf2, 0xf0, 0xa4, 0xee, 0x62, 0xbc, 0xae, 0x2a, 0x23, 0xa9, 0xf6, 0x1d, 0x28, 0x06, + 0x6e, 0x60, 0xf5, 0x22, 0x05, 0xbb, 0x39, 0x29, 0x5a, 0x81, 0xa2, 0xc7, 0x79, 0xf1, 0x3d, 0x58, + 0xe6, 0xc5, 0x63, 0x6c, 0x47, 0x0a, 0x74, 0x73, 0x38, 0x50, 0xa8, 0x31, 0xe6, 0xa9, 0x41, 0xc9, + 0x3d, 0x3e, 0xa6, 0x1b, 0x18, 0xb3, 0x64, 0xe6, 0xb1, 0xa8, 0x1c, 0x1f, 0x72, 0x68, 0xbf, 0x59, + 0x82, 0x7c, 0xe4, 0x59, 0xa3, 0x16, 0x80, 0x64, 0x10, 0xec, 0xf1, 0xad, 0x9f, 0xc7, 0x20, 0xd6, + 0xc3, 0x26, 0x9f, 0x4d, 0xe2, 0x41, 0x0e, 0xac, 0x74, 0xdc, 0xfe, 0xb0, 0x87, 0xc9, 0x9e, 0x25, + 0xfe, 0xc4, 0x4b, 0xf0, 0x2f, 0x87, 0x8c, 0xd2, 0xf2, 0xf7, 0x60, 0x65, 0xfc, 0xe2, 0x23, 0x96, + 0x4d, 0x26, 0x5a, 0x11, 0x2e, 0x4e, 0x0c, 0x48, 0x4f, 0x76, 0x39, 0xd4, 0x1b, 0xd3, 0x69, 0xff, + 0x4e, 0x42, 0x26, 0xec, 0xa2, 0x2a, 0x14, 0x42, 0x4a, 0xd9, 0x0a, 0xe6, 0xd1, 0xe6, 0x8f, 0x65, + 0xe1, 0x33, 0xac, 0x61, 0x0d, 0x32, 0x1d, 0x77, 0x30, 0xc0, 0x9d, 0x00, 0xdb, 0xd4, 0x1e, 0x04, + 0x60, 0x2c, 0x8e, 0x7b, 0xf6, 0xc9, 0x97, 0x78, 0xf6, 0xa9, 0xf3, 0x3e, 0xfb, 0xdb, 0x50, 0x1a, + 0x0d, 0xce, 0x6b, 0x34, 0x32, 0x92, 0x6a, 0x7f, 0x9f, 0x16, 0xcb, 0xb0, 0x27, 0x6c, 0x25, 0xc7, + 0x55, 0x9a, 0x44, 0x28, 0x95, 0xce, 0xb0, 0xe7, 0xa3, 0x1f, 0x41, 0xa1, 0x3b, 0x20, 0x0e, 0x00, + 0xb7, 0xb9, 0x0e, 0xb3, 0x0d, 0x51, 0xe2, 0x6b, 0xb0, 0x41, 0x59, 0x35, 0xdf, 0x95, 0x64, 0x33, + 0x4d, 0x2c, 0xf3, 0xad, 0x98, 0x18, 0x3c, 0x97, 0x89, 0x91, 0x3b, 0x0d, 0x5f, 0x56, 0x39, 0x2b, + 0xdf, 0x69, 0x28, 0xd6, 0x3e, 0x4f, 0x40, 0x4e, 0xfe, 0x50, 0xa1, 0x03, 0xc8, 0x8c, 0x3f, 0x70, + 0xcc, 0x08, 0xdf, 0x38, 0xc7, 0x07, 0x6e, 0x5d, 0xb4, 0xc4, 0x14, 0x21, 0x09, 0xfa, 0x29, 0xa8, + 0xae, 0x37, 0x3c, 0xb1, 0x06, 0x91, 0x2f, 0xe7, 0x8b, 0x12, 0x17, 0x19, 0x57, 0x38, 0xae, 0xfd, + 0x4e, 0x81, 0xb4, 0xe8, 0xa1, 0x3b, 0x90, 0x17, 0x93, 0xc8, 0x66, 0x22, 0xee, 0x4f, 0xe0, 0x24, + 0x2b, 0xc9, 0x61, 0x49, 0x16, 0x29, 0x9c, 0x25, 0x9e, 0xaf, 0x70, 0xa6, 0xfd, 0x89, 0x45, 0x1a, + 0x2c, 0x56, 0x78, 0x17, 0xf2, 0x43, 0x3c, 0xb0, 0x49, 0xf2, 0x27, 0xa2, 0x0d, 0x39, 0xb6, 0x23, + 0x20, 0xb1, 0x00, 0x8e, 0x63, 0x7a, 0x37, 0x20, 0xc5, 0xf0, 0x89, 0x59, 0x78, 0x36, 0x8e, 0xde, + 0x87, 0xe2, 0xd8, 0xaf, 0x31, 0x95, 0x85, 0x59, 0x2a, 0x85, 0x10, 0xc9, 0x26, 0x79, 0x1b, 0x72, + 0xfc, 0x46, 0x98, 0x62, 0x72, 0x96, 0x62, 0x96, 0xc1, 0x98, 0xd6, 0x1d, 0x66, 0x86, 0x56, 0xe7, + 0xc4, 0x7a, 0xd4, 0xc3, 0x5c, 0x35, 0x35, 0x4b, 0x55, 0x95, 0xb0, 0x54, 0x9f, 0x87, 0xbc, 0x2c, + 0x75, 0xb9, 0x01, 0x29, 0x96, 0xf4, 0x44, 0x8f, 0x85, 0x0c, 0x8a, 0x6d, 0xd2, 0x71, 0x6d, 0x87, + 0xc6, 0xa9, 0xdf, 0x46, 0xc5, 0xfa, 0x2e, 0x0d, 0x72, 0x78, 0xd6, 0xb2, 0x09, 0x59, 0xf6, 0xe8, + 0xc4, 0x2b, 0x51, 0x24, 0x1e, 0x86, 0xa1, 0xee, 0x04, 0xfa, 0x61, 0x5b, 0x3b, 0x82, 0x95, 0xb8, + 0xa4, 0x05, 0xdd, 0x81, 0x45, 0x9e, 0xf1, 0xb0, 0xc7, 0xb6, 0x1a, 0x53, 0xe0, 0xdc, 0xea, 0x8d, + 0x08, 0x0f, 0xd3, 0x10, 0x4e, 0x87, 0x69, 0x69, 0x0f, 0xe1, 0x62, 0x7c, 0x9e, 0xf2, 0xb2, 0xc5, + 0xd3, 0x1a, 0x3d, 0x70, 0x56, 0x58, 0x7c, 0x77, 0x62, 0x91, 0xe5, 0xe9, 0x32, 0x65, 0xdc, 0xe2, + 0xd6, 0x9e, 0x24, 0xbe, 0xcb, 0xb2, 0x63, 0xb4, 0xbe, 0xb8, 0x18, 0xad, 0x2f, 0x2e, 0x45, 0xeb, + 0x8b, 0xe9, 0x89, 0xfa, 0x62, 0x26, 0xa6, 0xbe, 0x08, 0xd3, 0xf5, 0xc5, 0x6c, 0xb4, 0xbe, 0x98, + 0x8b, 0xd6, 0x17, 0xf3, 0x93, 0xf5, 0xc5, 0xc2, 0x44, 0x2d, 0xb1, 0x38, 0xa7, 0xc6, 0xa7, 0xce, + 0xad, 0xf1, 0x95, 0xa2, 0x55, 0x39, 0xc4, 0x8b, 0x6e, 0xff, 0x58, 0x82, 0x94, 0x7e, 0x4a, 0xa2, + 0xdc, 0xcd, 0x48, 0xc5, 0xad, 0x1c, 0xf5, 0x8d, 0x14, 0x12, 0xf7, 0xa3, 0x38, 0xf8, 0xa3, 0x47, + 0x7e, 0xc7, 0xeb, 0x3e, 0xc2, 0x36, 0xcf, 0x47, 0x2a, 0x71, 0x9a, 0x66, 0x88, 0x32, 0x24, 0x0d, + 0x74, 0x1b, 0x80, 0xd8, 0x71, 0xdb, 0xb2, 0x6d, 0xfa, 0xdd, 0x8f, 0xf9, 0x4d, 0x9d, 0xcf, 0x6c, + 0xf9, 0x1f, 0x57, 0x09, 0xc8, 0xc8, 0x04, 0xa2, 0x89, 0x6a, 0x90, 0xa3, 0xda, 0xec, 0x77, 0x4b, + 0x9b, 0x47, 0x03, 0xd7, 0x66, 0xe9, 0xb3, 0x5f, 0x09, 0x6c, 0x23, 0x1b, 0x8c, 0x3b, 0xe8, 0x2e, + 0x64, 0x99, 0xc7, 0x65, 0x4b, 0x48, 0xcd, 0xde, 0x02, 0x75, 0xb8, 0x6c, 0x0d, 0x2c, 0x1d, 0x60, + 0x8b, 0xd0, 0x21, 0xcf, 0x08, 0xd8, 0x2f, 0x82, 0x76, 0xfc, 0xef, 0xe1, 0x12, 0x05, 0xab, 0xb7, + 0xdb, 0x46, 0xce, 0x92, 0x7a, 0xda, 0x7d, 0x00, 0x53, 0x3e, 0x17, 0xa9, 0xd8, 0xa5, 0x3c, 0x67, + 0xb1, 0x4b, 0xdb, 0x84, 0x4c, 0x78, 0x5e, 0xe8, 0x35, 0x48, 0x92, 0xfd, 0x72, 0xbb, 0x8b, 0xf1, + 0x94, 0x74, 0x58, 0xfb, 0xab, 0x02, 0x59, 0xe9, 0x90, 0xd0, 0x7d, 0xc8, 0x49, 0x11, 0x9f, 0x3d, + 0x91, 0xdd, 0x8f, 0xe3, 0xbd, 0x7a, 0x6d, 0x99, 0x7f, 0x91, 0xb2, 0x92, 0xd0, 0xc8, 0x8e, 0x83, + 0x3f, 0x1b, 0x6d, 0x84, 0xc6, 0xcf, 0xbe, 0x69, 0x25, 0x69, 0x11, 0x71, 0x56, 0x8f, 0xae, 0x43, + 0x8a, 0x6d, 0x9d, 0x44, 0x82, 0x85, 0x30, 0xd7, 0x12, 0x78, 0x6c, 0xb0, 0x61, 0xed, 0x00, 0x60, + 0x7c, 0x2b, 0xa8, 0x06, 0x29, 0x7a, 0xa4, 0x7c, 0xad, 0xcf, 0x97, 0x38, 0x32, 0x55, 0xed, 0x3e, + 0xe4, 0xe4, 0x4b, 0x7a, 0x99, 0x5f, 0xb2, 0xd6, 0xdc, 0x19, 0xae, 0x2b, 0xfc, 0x29, 0xa0, 0xae, + 0x2a, 0xa4, 0x4f, 0x3c, 0x41, 0xbb, 0x5a, 0xaf, 0xeb, 0x75, 0x35, 0x81, 0x54, 0xc8, 0xd1, 0x3e, + 0xab, 0xc6, 0xd7, 0x99, 0xf3, 0xa2, 0x2e, 0x86, 0x43, 0x92, 0xc4, 0xa7, 0x30, 0x01, 0xab, 0xa2, + 0xd7, 0xd5, 0x14, 0xb3, 0xe5, 0x5a, 0xfd, 0x8b, 0xa7, 0x95, 0x0b, 0xff, 0x7c, 0x5a, 0xb9, 0xf0, + 0xe5, 0xd3, 0x8a, 0xf2, 0xf5, 0xd3, 0x8a, 0xf2, 0xdf, 0xa7, 0x15, 0xe5, 0x57, 0x67, 0x15, 0xe5, + 0x2f, 0x67, 0x15, 0xe5, 0x6f, 0x67, 0x15, 0xe5, 0xef, 0x67, 0x15, 0xe5, 0xf3, 0xb3, 0x8a, 0xf2, + 0xc5, 0x59, 0x45, 0xf9, 0xcf, 0x59, 0xe5, 0xc2, 0xd7, 0x67, 0x15, 0xe5, 0x0f, 0x5f, 0x55, 0x2e, + 0x7c, 0xf6, 0x55, 0x45, 0xf9, 0x68, 0x91, 0x1d, 0xd4, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x76, + 0xd4, 0xc4, 0xdc, 0x65, 0x24, 0x00, 0x00, +} diff --git a/api/v1/lib/master/master.pb_ffjson.go b/api/v1/lib/master/master.pb_ffjson.go new file mode 100644 index 00000000..3b9047d3 --- /dev/null +++ b/api/v1/lib/master/master.pb_ffjson.go @@ -0,0 +1,14659 @@ +// DO NOT EDIT! +// Code generated by ffjson +// source: master/master.pb.go +// DO NOT EDIT! + +package master + +import ( + "bytes" + "encoding/base64" + "errors" + "fmt" + "github.com/mesos/mesos-go/api/v1/lib" + fflib "github.com/pquerna/ffjson/fflib/v1" + "reflect" +) + +func (mj *Call) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ "type":`) + + { + + obj, err = mj.Type.MarshalJSON() + if err != nil { + return err + } + buf.Write(obj) + + } + buf.WriteByte(',') + if mj.GetMetrics != nil { + if true { + buf.WriteString(`"get_metrics":`) + + { + + err = mj.GetMetrics.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.SetLoggingLevel != nil { + if true { + buf.WriteString(`"set_logging_level":`) + + { + + err = mj.SetLoggingLevel.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.ListFiles != nil { + if true { + buf.WriteString(`"list_files":`) + + { + + err = mj.ListFiles.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.ReadFile != nil { + if true { + buf.WriteString(`"read_file":`) + + { + + err = mj.ReadFile.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.UpdateWeights != nil { + if true { + buf.WriteString(`"update_weights":`) + + { + + err = mj.UpdateWeights.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.ReserveResources != nil { + if true { + buf.WriteString(`"reserve_resources":`) + + { + + err = mj.ReserveResources.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.UnreserveResources != nil { + if true { + buf.WriteString(`"unreserve_resources":`) + + { + + err = mj.UnreserveResources.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.CreateVolumes != nil { + if true { + buf.WriteString(`"create_volumes":`) + + { + + err = mj.CreateVolumes.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.DestroyVolumes != nil { + if true { + buf.WriteString(`"destroy_volumes":`) + + { + + err = mj.DestroyVolumes.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.UpdateMaintenanceSchedule != nil { + if true { + buf.WriteString(`"update_maintenance_schedule":`) + + { + + err = mj.UpdateMaintenanceSchedule.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.StartMaintenance != nil { + if true { + buf.WriteString(`"start_maintenance":`) + + { + + err = mj.StartMaintenance.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.StopMaintenance != nil { + if true { + buf.WriteString(`"stop_maintenance":`) + + { + + err = mj.StopMaintenance.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.SetQuota != nil { + if true { + buf.WriteString(`"set_quota":`) + + { + + err = mj.SetQuota.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.RemoveQuota != nil { + if true { + buf.WriteString(`"remove_quota":`) + + { + + err = mj.RemoveQuota.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Callbase = iota + ffj_t_Callno_such_key + + ffj_t_Call_Type + + ffj_t_Call_GetMetrics + + ffj_t_Call_SetLoggingLevel + + ffj_t_Call_ListFiles + + ffj_t_Call_ReadFile + + ffj_t_Call_UpdateWeights + + ffj_t_Call_ReserveResources + + ffj_t_Call_UnreserveResources + + ffj_t_Call_CreateVolumes + + ffj_t_Call_DestroyVolumes + + ffj_t_Call_UpdateMaintenanceSchedule + + ffj_t_Call_StartMaintenance + + ffj_t_Call_StopMaintenance + + ffj_t_Call_SetQuota + + ffj_t_Call_RemoveQuota +) + +var ffj_key_Call_Type = []byte("type") + +var ffj_key_Call_GetMetrics = []byte("get_metrics") + +var ffj_key_Call_SetLoggingLevel = []byte("set_logging_level") + +var ffj_key_Call_ListFiles = []byte("list_files") + +var ffj_key_Call_ReadFile = []byte("read_file") + +var ffj_key_Call_UpdateWeights = []byte("update_weights") + +var ffj_key_Call_ReserveResources = []byte("reserve_resources") + +var ffj_key_Call_UnreserveResources = []byte("unreserve_resources") + +var ffj_key_Call_CreateVolumes = []byte("create_volumes") + +var ffj_key_Call_DestroyVolumes = []byte("destroy_volumes") + +var ffj_key_Call_UpdateMaintenanceSchedule = []byte("update_maintenance_schedule") + +var ffj_key_Call_StartMaintenance = []byte("start_maintenance") + +var ffj_key_Call_StopMaintenance = []byte("stop_maintenance") + +var ffj_key_Call_SetQuota = []byte("set_quota") + +var ffj_key_Call_RemoveQuota = []byte("remove_quota") + +func (uj *Call) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Callbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Callno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'c': + + if bytes.Equal(ffj_key_Call_CreateVolumes, kn) { + currentKey = ffj_t_Call_CreateVolumes + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'd': + + if bytes.Equal(ffj_key_Call_DestroyVolumes, kn) { + currentKey = ffj_t_Call_DestroyVolumes + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'g': + + if bytes.Equal(ffj_key_Call_GetMetrics, kn) { + currentKey = ffj_t_Call_GetMetrics + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'l': + + if bytes.Equal(ffj_key_Call_ListFiles, kn) { + currentKey = ffj_t_Call_ListFiles + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_Call_ReadFile, kn) { + currentKey = ffj_t_Call_ReadFile + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Call_ReserveResources, kn) { + currentKey = ffj_t_Call_ReserveResources + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Call_RemoveQuota, kn) { + currentKey = ffj_t_Call_RemoveQuota + state = fflib.FFParse_want_colon + goto mainparse + } + + case 's': + + if bytes.Equal(ffj_key_Call_SetLoggingLevel, kn) { + currentKey = ffj_t_Call_SetLoggingLevel + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Call_StartMaintenance, kn) { + currentKey = ffj_t_Call_StartMaintenance + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Call_StopMaintenance, kn) { + currentKey = ffj_t_Call_StopMaintenance + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Call_SetQuota, kn) { + currentKey = ffj_t_Call_SetQuota + state = fflib.FFParse_want_colon + goto mainparse + } + + case 't': + + if bytes.Equal(ffj_key_Call_Type, kn) { + currentKey = ffj_t_Call_Type + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'u': + + if bytes.Equal(ffj_key_Call_UpdateWeights, kn) { + currentKey = ffj_t_Call_UpdateWeights + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Call_UnreserveResources, kn) { + currentKey = ffj_t_Call_UnreserveResources + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Call_UpdateMaintenanceSchedule, kn) { + currentKey = ffj_t_Call_UpdateMaintenanceSchedule + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.AsciiEqualFold(ffj_key_Call_RemoveQuota, kn) { + currentKey = ffj_t_Call_RemoveQuota + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_SetQuota, kn) { + currentKey = ffj_t_Call_SetQuota + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_StopMaintenance, kn) { + currentKey = ffj_t_Call_StopMaintenance + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_StartMaintenance, kn) { + currentKey = ffj_t_Call_StartMaintenance + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_UpdateMaintenanceSchedule, kn) { + currentKey = ffj_t_Call_UpdateMaintenanceSchedule + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_DestroyVolumes, kn) { + currentKey = ffj_t_Call_DestroyVolumes + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_CreateVolumes, kn) { + currentKey = ffj_t_Call_CreateVolumes + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_UnreserveResources, kn) { + currentKey = ffj_t_Call_UnreserveResources + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_ReserveResources, kn) { + currentKey = ffj_t_Call_ReserveResources + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_UpdateWeights, kn) { + currentKey = ffj_t_Call_UpdateWeights + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Call_ReadFile, kn) { + currentKey = ffj_t_Call_ReadFile + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_ListFiles, kn) { + currentKey = ffj_t_Call_ListFiles + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_SetLoggingLevel, kn) { + currentKey = ffj_t_Call_SetLoggingLevel + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_GetMetrics, kn) { + currentKey = ffj_t_Call_GetMetrics + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Call_Type, kn) { + currentKey = ffj_t_Call_Type + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Callno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_Type: + goto handle_Type + + case ffj_t_Call_GetMetrics: + goto handle_GetMetrics + + case ffj_t_Call_SetLoggingLevel: + goto handle_SetLoggingLevel + + case ffj_t_Call_ListFiles: + goto handle_ListFiles + + case ffj_t_Call_ReadFile: + goto handle_ReadFile + + case ffj_t_Call_UpdateWeights: + goto handle_UpdateWeights + + case ffj_t_Call_ReserveResources: + goto handle_ReserveResources + + case ffj_t_Call_UnreserveResources: + goto handle_UnreserveResources + + case ffj_t_Call_CreateVolumes: + goto handle_CreateVolumes + + case ffj_t_Call_DestroyVolumes: + goto handle_DestroyVolumes + + case ffj_t_Call_UpdateMaintenanceSchedule: + goto handle_UpdateMaintenanceSchedule + + case ffj_t_Call_StartMaintenance: + goto handle_StartMaintenance + + case ffj_t_Call_StopMaintenance: + goto handle_StopMaintenance + + case ffj_t_Call_SetQuota: + goto handle_SetQuota + + case ffj_t_Call_RemoveQuota: + goto handle_RemoveQuota + + case ffj_t_Callno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Type: + + /* handler: uj.Type type=master.Call_Type kind=int32 quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + tbuf, err := fs.CaptureField(tok) + if err != nil { + return fs.WrapErr(err) + } + + err = uj.Type.UnmarshalJSON(tbuf) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetMetrics: + + /* handler: uj.GetMetrics type=master.Call_GetMetrics kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetMetrics = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetMetrics == nil { + uj.GetMetrics = new(Call_GetMetrics) + } + + err = uj.GetMetrics.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_SetLoggingLevel: + + /* handler: uj.SetLoggingLevel type=master.Call_SetLoggingLevel kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.SetLoggingLevel = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.SetLoggingLevel == nil { + uj.SetLoggingLevel = new(Call_SetLoggingLevel) + } + + err = uj.SetLoggingLevel.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_ListFiles: + + /* handler: uj.ListFiles type=master.Call_ListFiles kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.ListFiles = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.ListFiles == nil { + uj.ListFiles = new(Call_ListFiles) + } + + err = uj.ListFiles.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_ReadFile: + + /* handler: uj.ReadFile type=master.Call_ReadFile kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.ReadFile = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.ReadFile == nil { + uj.ReadFile = new(Call_ReadFile) + } + + err = uj.ReadFile.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_UpdateWeights: + + /* handler: uj.UpdateWeights type=master.Call_UpdateWeights kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.UpdateWeights = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.UpdateWeights == nil { + uj.UpdateWeights = new(Call_UpdateWeights) + } + + err = uj.UpdateWeights.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_ReserveResources: + + /* handler: uj.ReserveResources type=master.Call_ReserveResources kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.ReserveResources = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.ReserveResources == nil { + uj.ReserveResources = new(Call_ReserveResources) + } + + err = uj.ReserveResources.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_UnreserveResources: + + /* handler: uj.UnreserveResources type=master.Call_UnreserveResources kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.UnreserveResources = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.UnreserveResources == nil { + uj.UnreserveResources = new(Call_UnreserveResources) + } + + err = uj.UnreserveResources.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_CreateVolumes: + + /* handler: uj.CreateVolumes type=master.Call_CreateVolumes kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.CreateVolumes = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.CreateVolumes == nil { + uj.CreateVolumes = new(Call_CreateVolumes) + } + + err = uj.CreateVolumes.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_DestroyVolumes: + + /* handler: uj.DestroyVolumes type=master.Call_DestroyVolumes kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.DestroyVolumes = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.DestroyVolumes == nil { + uj.DestroyVolumes = new(Call_DestroyVolumes) + } + + err = uj.DestroyVolumes.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_UpdateMaintenanceSchedule: + + /* handler: uj.UpdateMaintenanceSchedule type=master.Call_UpdateMaintenanceSchedule kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.UpdateMaintenanceSchedule = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.UpdateMaintenanceSchedule == nil { + uj.UpdateMaintenanceSchedule = new(Call_UpdateMaintenanceSchedule) + } + + err = uj.UpdateMaintenanceSchedule.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_StartMaintenance: + + /* handler: uj.StartMaintenance type=master.Call_StartMaintenance kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.StartMaintenance = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.StartMaintenance == nil { + uj.StartMaintenance = new(Call_StartMaintenance) + } + + err = uj.StartMaintenance.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_StopMaintenance: + + /* handler: uj.StopMaintenance type=master.Call_StopMaintenance kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.StopMaintenance = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.StopMaintenance == nil { + uj.StopMaintenance = new(Call_StopMaintenance) + } + + err = uj.StopMaintenance.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_SetQuota: + + /* handler: uj.SetQuota type=master.Call_SetQuota kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.SetQuota = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.SetQuota == nil { + uj.SetQuota = new(Call_SetQuota) + } + + err = uj.SetQuota.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_RemoveQuota: + + /* handler: uj.RemoveQuota type=master.Call_RemoveQuota kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.RemoveQuota = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.RemoveQuota == nil { + uj.RemoveQuota = new(Call_RemoveQuota) + } + + err = uj.RemoveQuota.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_CreateVolumes) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_CreateVolumes) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"agent_id":`) + + { + + err = mj.AgentID.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteString(`,"volumes":`) + if mj.Volumes != nil { + buf.WriteString(`[`) + for i, v := range mj.Volumes { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_CreateVolumesbase = iota + ffj_t_Call_CreateVolumesno_such_key + + ffj_t_Call_CreateVolumes_AgentID + + ffj_t_Call_CreateVolumes_Volumes +) + +var ffj_key_Call_CreateVolumes_AgentID = []byte("agent_id") + +var ffj_key_Call_CreateVolumes_Volumes = []byte("volumes") + +func (uj *Call_CreateVolumes) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_CreateVolumes) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_CreateVolumesbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_CreateVolumesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Call_CreateVolumes_AgentID, kn) { + currentKey = ffj_t_Call_CreateVolumes_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'v': + + if bytes.Equal(ffj_key_Call_CreateVolumes_Volumes, kn) { + currentKey = ffj_t_Call_CreateVolumes_Volumes + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Call_CreateVolumes_Volumes, kn) { + currentKey = ffj_t_Call_CreateVolumes_Volumes + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Call_CreateVolumes_AgentID, kn) { + currentKey = ffj_t_Call_CreateVolumes_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_CreateVolumesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_CreateVolumes_AgentID: + goto handle_AgentID + + case ffj_t_Call_CreateVolumes_Volumes: + goto handle_Volumes + + case ffj_t_Call_CreateVolumesno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_AgentID: + + /* handler: uj.AgentID type=mesos.AgentID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.AgentID.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Volumes: + + /* handler: uj.Volumes type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Volumes = nil + } else { + + uj.Volumes = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__Volumes mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Volumes type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Volumes.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Volumes = append(uj.Volumes, tmp_uj__Volumes) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_DestroyVolumes) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_DestroyVolumes) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"agent_id":`) + + { + + err = mj.AgentID.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteString(`,"volumes":`) + if mj.Volumes != nil { + buf.WriteString(`[`) + for i, v := range mj.Volumes { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_DestroyVolumesbase = iota + ffj_t_Call_DestroyVolumesno_such_key + + ffj_t_Call_DestroyVolumes_AgentID + + ffj_t_Call_DestroyVolumes_Volumes +) + +var ffj_key_Call_DestroyVolumes_AgentID = []byte("agent_id") + +var ffj_key_Call_DestroyVolumes_Volumes = []byte("volumes") + +func (uj *Call_DestroyVolumes) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_DestroyVolumes) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_DestroyVolumesbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_DestroyVolumesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Call_DestroyVolumes_AgentID, kn) { + currentKey = ffj_t_Call_DestroyVolumes_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'v': + + if bytes.Equal(ffj_key_Call_DestroyVolumes_Volumes, kn) { + currentKey = ffj_t_Call_DestroyVolumes_Volumes + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Call_DestroyVolumes_Volumes, kn) { + currentKey = ffj_t_Call_DestroyVolumes_Volumes + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Call_DestroyVolumes_AgentID, kn) { + currentKey = ffj_t_Call_DestroyVolumes_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_DestroyVolumesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_DestroyVolumes_AgentID: + goto handle_AgentID + + case ffj_t_Call_DestroyVolumes_Volumes: + goto handle_Volumes + + case ffj_t_Call_DestroyVolumesno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_AgentID: + + /* handler: uj.AgentID type=mesos.AgentID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.AgentID.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Volumes: + + /* handler: uj.Volumes type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Volumes = nil + } else { + + uj.Volumes = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__Volumes mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Volumes type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Volumes.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Volumes = append(uj.Volumes, tmp_uj__Volumes) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_GetMetrics) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_GetMetrics) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ `) + if mj.Timeout != nil { + if true { + buf.WriteString(`"timeout":`) + + { + + err = mj.Timeout.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_GetMetricsbase = iota + ffj_t_Call_GetMetricsno_such_key + + ffj_t_Call_GetMetrics_Timeout +) + +var ffj_key_Call_GetMetrics_Timeout = []byte("timeout") + +func (uj *Call_GetMetrics) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_GetMetrics) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_GetMetricsbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_GetMetricsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 't': + + if bytes.Equal(ffj_key_Call_GetMetrics_Timeout, kn) { + currentKey = ffj_t_Call_GetMetrics_Timeout + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Call_GetMetrics_Timeout, kn) { + currentKey = ffj_t_Call_GetMetrics_Timeout + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_GetMetricsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_GetMetrics_Timeout: + goto handle_Timeout + + case ffj_t_Call_GetMetricsno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Timeout: + + /* handler: uj.Timeout type=mesos.DurationInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.Timeout = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.Timeout == nil { + uj.Timeout = new(mesos.DurationInfo) + } + + err = uj.Timeout.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_ListFiles) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_ListFiles) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"path":`) + fflib.WriteJsonString(buf, string(mj.Path)) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_ListFilesbase = iota + ffj_t_Call_ListFilesno_such_key + + ffj_t_Call_ListFiles_Path +) + +var ffj_key_Call_ListFiles_Path = []byte("path") + +func (uj *Call_ListFiles) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_ListFiles) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_ListFilesbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_ListFilesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'p': + + if bytes.Equal(ffj_key_Call_ListFiles_Path, kn) { + currentKey = ffj_t_Call_ListFiles_Path + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Call_ListFiles_Path, kn) { + currentKey = ffj_t_Call_ListFiles_Path + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_ListFilesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_ListFiles_Path: + goto handle_Path + + case ffj_t_Call_ListFilesno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Path: + + /* handler: uj.Path type=string kind=string quoted=false*/ + + { + + { + if tok != fflib.FFTok_string && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for string", tok)) + } + } + + if tok == fflib.FFTok_null { + + } else { + + outBuf := fs.Output.Bytes() + + uj.Path = string(string(outBuf)) + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_ReadFile) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_ReadFile) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ "path":`) + fflib.WriteJsonString(buf, string(mj.Path)) + buf.WriteString(`,"offset":`) + fflib.FormatBits2(buf, uint64(mj.Offset), 10, false) + buf.WriteByte(',') + if mj.Length != nil { + if true { + buf.WriteString(`"length":`) + fflib.FormatBits2(buf, uint64(*mj.Length), 10, false) + buf.WriteByte(',') + } + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_ReadFilebase = iota + ffj_t_Call_ReadFileno_such_key + + ffj_t_Call_ReadFile_Path + + ffj_t_Call_ReadFile_Offset + + ffj_t_Call_ReadFile_Length +) + +var ffj_key_Call_ReadFile_Path = []byte("path") + +var ffj_key_Call_ReadFile_Offset = []byte("offset") + +var ffj_key_Call_ReadFile_Length = []byte("length") + +func (uj *Call_ReadFile) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_ReadFile) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_ReadFilebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_ReadFileno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'l': + + if bytes.Equal(ffj_key_Call_ReadFile_Length, kn) { + currentKey = ffj_t_Call_ReadFile_Length + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'o': + + if bytes.Equal(ffj_key_Call_ReadFile_Offset, kn) { + currentKey = ffj_t_Call_ReadFile_Offset + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'p': + + if bytes.Equal(ffj_key_Call_ReadFile_Path, kn) { + currentKey = ffj_t_Call_ReadFile_Path + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Call_ReadFile_Length, kn) { + currentKey = ffj_t_Call_ReadFile_Length + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Call_ReadFile_Offset, kn) { + currentKey = ffj_t_Call_ReadFile_Offset + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Call_ReadFile_Path, kn) { + currentKey = ffj_t_Call_ReadFile_Path + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_ReadFileno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_ReadFile_Path: + goto handle_Path + + case ffj_t_Call_ReadFile_Offset: + goto handle_Offset + + case ffj_t_Call_ReadFile_Length: + goto handle_Length + + case ffj_t_Call_ReadFileno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Path: + + /* handler: uj.Path type=string kind=string quoted=false*/ + + { + + { + if tok != fflib.FFTok_string && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for string", tok)) + } + } + + if tok == fflib.FFTok_null { + + } else { + + outBuf := fs.Output.Bytes() + + uj.Path = string(string(outBuf)) + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Offset: + + /* handler: uj.Offset type=uint64 kind=uint64 quoted=false*/ + + { + if tok != fflib.FFTok_integer && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for uint64", tok)) + } + } + + { + + if tok == fflib.FFTok_null { + + } else { + + tval, err := fflib.ParseUint(fs.Output.Bytes(), 10, 64) + + if err != nil { + return fs.WrapErr(err) + } + + uj.Offset = uint64(tval) + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Length: + + /* handler: uj.Length type=uint64 kind=uint64 quoted=false*/ + + { + if tok != fflib.FFTok_integer && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for uint64", tok)) + } + } + + { + + if tok == fflib.FFTok_null { + + uj.Length = nil + + } else { + + tval, err := fflib.ParseUint(fs.Output.Bytes(), 10, 64) + + if err != nil { + return fs.WrapErr(err) + } + + ttypval := uint64(tval) + uj.Length = &ttypval + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_RemoveQuota) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_RemoveQuota) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"role":`) + fflib.WriteJsonString(buf, string(mj.Role)) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_RemoveQuotabase = iota + ffj_t_Call_RemoveQuotano_such_key + + ffj_t_Call_RemoveQuota_Role +) + +var ffj_key_Call_RemoveQuota_Role = []byte("role") + +func (uj *Call_RemoveQuota) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_RemoveQuota) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_RemoveQuotabase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_RemoveQuotano_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'r': + + if bytes.Equal(ffj_key_Call_RemoveQuota_Role, kn) { + currentKey = ffj_t_Call_RemoveQuota_Role + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Call_RemoveQuota_Role, kn) { + currentKey = ffj_t_Call_RemoveQuota_Role + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_RemoveQuotano_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_RemoveQuota_Role: + goto handle_Role + + case ffj_t_Call_RemoveQuotano_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Role: + + /* handler: uj.Role type=string kind=string quoted=false*/ + + { + + { + if tok != fflib.FFTok_string && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for string", tok)) + } + } + + if tok == fflib.FFTok_null { + + } else { + + outBuf := fs.Output.Bytes() + + uj.Role = string(string(outBuf)) + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_ReserveResources) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_ReserveResources) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"agent_id":`) + + { + + err = mj.AgentID.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteString(`,"resources":`) + if mj.Resources != nil { + buf.WriteString(`[`) + for i, v := range mj.Resources { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_ReserveResourcesbase = iota + ffj_t_Call_ReserveResourcesno_such_key + + ffj_t_Call_ReserveResources_AgentID + + ffj_t_Call_ReserveResources_Resources +) + +var ffj_key_Call_ReserveResources_AgentID = []byte("agent_id") + +var ffj_key_Call_ReserveResources_Resources = []byte("resources") + +func (uj *Call_ReserveResources) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_ReserveResources) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_ReserveResourcesbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_ReserveResourcesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Call_ReserveResources_AgentID, kn) { + currentKey = ffj_t_Call_ReserveResources_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_Call_ReserveResources_Resources, kn) { + currentKey = ffj_t_Call_ReserveResources_Resources + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Call_ReserveResources_Resources, kn) { + currentKey = ffj_t_Call_ReserveResources_Resources + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Call_ReserveResources_AgentID, kn) { + currentKey = ffj_t_Call_ReserveResources_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_ReserveResourcesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_ReserveResources_AgentID: + goto handle_AgentID + + case ffj_t_Call_ReserveResources_Resources: + goto handle_Resources + + case ffj_t_Call_ReserveResourcesno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_AgentID: + + /* handler: uj.AgentID type=mesos.AgentID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.AgentID.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Resources: + + /* handler: uj.Resources type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Resources = nil + } else { + + uj.Resources = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__Resources mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Resources type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Resources.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Resources = append(uj.Resources, tmp_uj__Resources) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_SetLoggingLevel) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_SetLoggingLevel) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"level":`) + fflib.FormatBits2(buf, uint64(mj.Level), 10, false) + buf.WriteString(`,"duration":`) + + { + + err = mj.Duration.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_SetLoggingLevelbase = iota + ffj_t_Call_SetLoggingLevelno_such_key + + ffj_t_Call_SetLoggingLevel_Level + + ffj_t_Call_SetLoggingLevel_Duration +) + +var ffj_key_Call_SetLoggingLevel_Level = []byte("level") + +var ffj_key_Call_SetLoggingLevel_Duration = []byte("duration") + +func (uj *Call_SetLoggingLevel) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_SetLoggingLevel) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_SetLoggingLevelbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_SetLoggingLevelno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'd': + + if bytes.Equal(ffj_key_Call_SetLoggingLevel_Duration, kn) { + currentKey = ffj_t_Call_SetLoggingLevel_Duration + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'l': + + if bytes.Equal(ffj_key_Call_SetLoggingLevel_Level, kn) { + currentKey = ffj_t_Call_SetLoggingLevel_Level + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Call_SetLoggingLevel_Duration, kn) { + currentKey = ffj_t_Call_SetLoggingLevel_Duration + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Call_SetLoggingLevel_Level, kn) { + currentKey = ffj_t_Call_SetLoggingLevel_Level + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_SetLoggingLevelno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_SetLoggingLevel_Level: + goto handle_Level + + case ffj_t_Call_SetLoggingLevel_Duration: + goto handle_Duration + + case ffj_t_Call_SetLoggingLevelno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Level: + + /* handler: uj.Level type=uint32 kind=uint32 quoted=false*/ + + { + if tok != fflib.FFTok_integer && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for uint32", tok)) + } + } + + { + + if tok == fflib.FFTok_null { + + } else { + + tval, err := fflib.ParseUint(fs.Output.Bytes(), 10, 32) + + if err != nil { + return fs.WrapErr(err) + } + + uj.Level = uint32(tval) + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Duration: + + /* handler: uj.Duration type=mesos.DurationInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Duration.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_SetQuota) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_SetQuota) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"quota_request":`) + + { + + err = mj.QuotaRequest.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_SetQuotabase = iota + ffj_t_Call_SetQuotano_such_key + + ffj_t_Call_SetQuota_QuotaRequest +) + +var ffj_key_Call_SetQuota_QuotaRequest = []byte("quota_request") + +func (uj *Call_SetQuota) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_SetQuota) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_SetQuotabase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_SetQuotano_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'q': + + if bytes.Equal(ffj_key_Call_SetQuota_QuotaRequest, kn) { + currentKey = ffj_t_Call_SetQuota_QuotaRequest + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Call_SetQuota_QuotaRequest, kn) { + currentKey = ffj_t_Call_SetQuota_QuotaRequest + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_SetQuotano_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_SetQuota_QuotaRequest: + goto handle_QuotaRequest + + case ffj_t_Call_SetQuotano_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_QuotaRequest: + + /* handler: uj.QuotaRequest type=quota.QuotaRequest kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.QuotaRequest.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_StartMaintenance) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_StartMaintenance) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"machines":`) + if mj.Machines != nil { + buf.WriteString(`[`) + for i, v := range mj.Machines { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_StartMaintenancebase = iota + ffj_t_Call_StartMaintenanceno_such_key + + ffj_t_Call_StartMaintenance_Machines +) + +var ffj_key_Call_StartMaintenance_Machines = []byte("machines") + +func (uj *Call_StartMaintenance) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_StartMaintenance) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_StartMaintenancebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_StartMaintenanceno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'm': + + if bytes.Equal(ffj_key_Call_StartMaintenance_Machines, kn) { + currentKey = ffj_t_Call_StartMaintenance_Machines + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Call_StartMaintenance_Machines, kn) { + currentKey = ffj_t_Call_StartMaintenance_Machines + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_StartMaintenanceno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_StartMaintenance_Machines: + goto handle_Machines + + case ffj_t_Call_StartMaintenanceno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Machines: + + /* handler: uj.Machines type=[]mesos.MachineID kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Machines = nil + } else { + + uj.Machines = []mesos.MachineID{} + + wantVal := true + + for { + + var tmp_uj__Machines mesos.MachineID + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Machines type=mesos.MachineID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Machines.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Machines = append(uj.Machines, tmp_uj__Machines) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_StopMaintenance) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_StopMaintenance) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"machines":`) + if mj.Machines != nil { + buf.WriteString(`[`) + for i, v := range mj.Machines { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_StopMaintenancebase = iota + ffj_t_Call_StopMaintenanceno_such_key + + ffj_t_Call_StopMaintenance_Machines +) + +var ffj_key_Call_StopMaintenance_Machines = []byte("machines") + +func (uj *Call_StopMaintenance) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_StopMaintenance) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_StopMaintenancebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_StopMaintenanceno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'm': + + if bytes.Equal(ffj_key_Call_StopMaintenance_Machines, kn) { + currentKey = ffj_t_Call_StopMaintenance_Machines + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Call_StopMaintenance_Machines, kn) { + currentKey = ffj_t_Call_StopMaintenance_Machines + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_StopMaintenanceno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_StopMaintenance_Machines: + goto handle_Machines + + case ffj_t_Call_StopMaintenanceno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Machines: + + /* handler: uj.Machines type=[]mesos.MachineID kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Machines = nil + } else { + + uj.Machines = []mesos.MachineID{} + + wantVal := true + + for { + + var tmp_uj__Machines mesos.MachineID + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Machines type=mesos.MachineID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Machines.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Machines = append(uj.Machines, tmp_uj__Machines) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_UnreserveResources) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_UnreserveResources) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"agent_id":`) + + { + + err = mj.AgentID.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteString(`,"resources":`) + if mj.Resources != nil { + buf.WriteString(`[`) + for i, v := range mj.Resources { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_UnreserveResourcesbase = iota + ffj_t_Call_UnreserveResourcesno_such_key + + ffj_t_Call_UnreserveResources_AgentID + + ffj_t_Call_UnreserveResources_Resources +) + +var ffj_key_Call_UnreserveResources_AgentID = []byte("agent_id") + +var ffj_key_Call_UnreserveResources_Resources = []byte("resources") + +func (uj *Call_UnreserveResources) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_UnreserveResources) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_UnreserveResourcesbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_UnreserveResourcesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Call_UnreserveResources_AgentID, kn) { + currentKey = ffj_t_Call_UnreserveResources_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_Call_UnreserveResources_Resources, kn) { + currentKey = ffj_t_Call_UnreserveResources_Resources + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Call_UnreserveResources_Resources, kn) { + currentKey = ffj_t_Call_UnreserveResources_Resources + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Call_UnreserveResources_AgentID, kn) { + currentKey = ffj_t_Call_UnreserveResources_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_UnreserveResourcesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_UnreserveResources_AgentID: + goto handle_AgentID + + case ffj_t_Call_UnreserveResources_Resources: + goto handle_Resources + + case ffj_t_Call_UnreserveResourcesno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_AgentID: + + /* handler: uj.AgentID type=mesos.AgentID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.AgentID.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Resources: + + /* handler: uj.Resources type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Resources = nil + } else { + + uj.Resources = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__Resources mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Resources type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Resources.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Resources = append(uj.Resources, tmp_uj__Resources) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_UpdateMaintenanceSchedule) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_UpdateMaintenanceSchedule) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"schedule":`) + + { + + err = mj.Schedule.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_UpdateMaintenanceSchedulebase = iota + ffj_t_Call_UpdateMaintenanceScheduleno_such_key + + ffj_t_Call_UpdateMaintenanceSchedule_Schedule +) + +var ffj_key_Call_UpdateMaintenanceSchedule_Schedule = []byte("schedule") + +func (uj *Call_UpdateMaintenanceSchedule) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_UpdateMaintenanceSchedule) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_UpdateMaintenanceSchedulebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_UpdateMaintenanceScheduleno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 's': + + if bytes.Equal(ffj_key_Call_UpdateMaintenanceSchedule_Schedule, kn) { + currentKey = ffj_t_Call_UpdateMaintenanceSchedule_Schedule + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Call_UpdateMaintenanceSchedule_Schedule, kn) { + currentKey = ffj_t_Call_UpdateMaintenanceSchedule_Schedule + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_UpdateMaintenanceScheduleno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_UpdateMaintenanceSchedule_Schedule: + goto handle_Schedule + + case ffj_t_Call_UpdateMaintenanceScheduleno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Schedule: + + /* handler: uj.Schedule type=maintenance.Schedule kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Schedule.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Call_UpdateWeights) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Call_UpdateWeights) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"weight_infos":`) + if mj.WeightInfos != nil { + buf.WriteString(`[`) + for i, v := range mj.WeightInfos { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Call_UpdateWeightsbase = iota + ffj_t_Call_UpdateWeightsno_such_key + + ffj_t_Call_UpdateWeights_WeightInfos +) + +var ffj_key_Call_UpdateWeights_WeightInfos = []byte("weight_infos") + +func (uj *Call_UpdateWeights) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Call_UpdateWeights) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Call_UpdateWeightsbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Call_UpdateWeightsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'w': + + if bytes.Equal(ffj_key_Call_UpdateWeights_WeightInfos, kn) { + currentKey = ffj_t_Call_UpdateWeights_WeightInfos + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Call_UpdateWeights_WeightInfos, kn) { + currentKey = ffj_t_Call_UpdateWeights_WeightInfos + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Call_UpdateWeightsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Call_UpdateWeights_WeightInfos: + goto handle_WeightInfos + + case ffj_t_Call_UpdateWeightsno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_WeightInfos: + + /* handler: uj.WeightInfos type=[]mesos.WeightInfo kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.WeightInfos = nil + } else { + + uj.WeightInfos = []mesos.WeightInfo{} + + wantVal := true + + for { + + var tmp_uj__WeightInfos mesos.WeightInfo + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__WeightInfos type=mesos.WeightInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__WeightInfos.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.WeightInfos = append(uj.WeightInfos, tmp_uj__WeightInfos) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Event) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Event) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ "type":`) + + { + + obj, err = mj.Type.MarshalJSON() + if err != nil { + return err + } + buf.Write(obj) + + } + buf.WriteByte(',') + if mj.Subscribed != nil { + if true { + buf.WriteString(`"subscribed":`) + + { + + err = mj.Subscribed.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.TaskAdded != nil { + if true { + buf.WriteString(`"task_added":`) + + { + + err = mj.TaskAdded.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.TaskUpdated != nil { + if true { + buf.WriteString(`"task_updated":`) + + { + + err = mj.TaskUpdated.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.AgentAdded != nil { + if true { + buf.WriteString(`"agent_added":`) + + { + + err = mj.AgentAdded.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.AgentRemoved != nil { + if true { + buf.WriteString(`"agent_removed":`) + + { + + err = mj.AgentRemoved.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Eventbase = iota + ffj_t_Eventno_such_key + + ffj_t_Event_Type + + ffj_t_Event_Subscribed + + ffj_t_Event_TaskAdded + + ffj_t_Event_TaskUpdated + + ffj_t_Event_AgentAdded + + ffj_t_Event_AgentRemoved +) + +var ffj_key_Event_Type = []byte("type") + +var ffj_key_Event_Subscribed = []byte("subscribed") + +var ffj_key_Event_TaskAdded = []byte("task_added") + +var ffj_key_Event_TaskUpdated = []byte("task_updated") + +var ffj_key_Event_AgentAdded = []byte("agent_added") + +var ffj_key_Event_AgentRemoved = []byte("agent_removed") + +func (uj *Event) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Event) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Eventbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Eventno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Event_AgentAdded, kn) { + currentKey = ffj_t_Event_AgentAdded + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Event_AgentRemoved, kn) { + currentKey = ffj_t_Event_AgentRemoved + state = fflib.FFParse_want_colon + goto mainparse + } + + case 's': + + if bytes.Equal(ffj_key_Event_Subscribed, kn) { + currentKey = ffj_t_Event_Subscribed + state = fflib.FFParse_want_colon + goto mainparse + } + + case 't': + + if bytes.Equal(ffj_key_Event_Type, kn) { + currentKey = ffj_t_Event_Type + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Event_TaskAdded, kn) { + currentKey = ffj_t_Event_TaskAdded + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Event_TaskUpdated, kn) { + currentKey = ffj_t_Event_TaskUpdated + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.AsciiEqualFold(ffj_key_Event_AgentRemoved, kn) { + currentKey = ffj_t_Event_AgentRemoved + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Event_AgentAdded, kn) { + currentKey = ffj_t_Event_AgentAdded + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Event_TaskUpdated, kn) { + currentKey = ffj_t_Event_TaskUpdated + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Event_TaskAdded, kn) { + currentKey = ffj_t_Event_TaskAdded + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Event_Subscribed, kn) { + currentKey = ffj_t_Event_Subscribed + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Event_Type, kn) { + currentKey = ffj_t_Event_Type + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Eventno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Event_Type: + goto handle_Type + + case ffj_t_Event_Subscribed: + goto handle_Subscribed + + case ffj_t_Event_TaskAdded: + goto handle_TaskAdded + + case ffj_t_Event_TaskUpdated: + goto handle_TaskUpdated + + case ffj_t_Event_AgentAdded: + goto handle_AgentAdded + + case ffj_t_Event_AgentRemoved: + goto handle_AgentRemoved + + case ffj_t_Eventno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Type: + + /* handler: uj.Type type=master.Event_Type kind=int32 quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + tbuf, err := fs.CaptureField(tok) + if err != nil { + return fs.WrapErr(err) + } + + err = uj.Type.UnmarshalJSON(tbuf) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Subscribed: + + /* handler: uj.Subscribed type=master.Event_Subscribed kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.Subscribed = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.Subscribed == nil { + uj.Subscribed = new(Event_Subscribed) + } + + err = uj.Subscribed.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_TaskAdded: + + /* handler: uj.TaskAdded type=master.Event_TaskAdded kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.TaskAdded = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.TaskAdded == nil { + uj.TaskAdded = new(Event_TaskAdded) + } + + err = uj.TaskAdded.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_TaskUpdated: + + /* handler: uj.TaskUpdated type=master.Event_TaskUpdated kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.TaskUpdated = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.TaskUpdated == nil { + uj.TaskUpdated = new(Event_TaskUpdated) + } + + err = uj.TaskUpdated.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_AgentAdded: + + /* handler: uj.AgentAdded type=master.Event_AgentAdded kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.AgentAdded = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.AgentAdded == nil { + uj.AgentAdded = new(Event_AgentAdded) + } + + err = uj.AgentAdded.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_AgentRemoved: + + /* handler: uj.AgentRemoved type=master.Event_AgentRemoved kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.AgentRemoved = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.AgentRemoved == nil { + uj.AgentRemoved = new(Event_AgentRemoved) + } + + err = uj.AgentRemoved.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Event_AgentAdded) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Event_AgentAdded) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"agent":`) + + { + + err = mj.Agent.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Event_AgentAddedbase = iota + ffj_t_Event_AgentAddedno_such_key + + ffj_t_Event_AgentAdded_Agent +) + +var ffj_key_Event_AgentAdded_Agent = []byte("agent") + +func (uj *Event_AgentAdded) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Event_AgentAdded) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Event_AgentAddedbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Event_AgentAddedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Event_AgentAdded_Agent, kn) { + currentKey = ffj_t_Event_AgentAdded_Agent + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Event_AgentAdded_Agent, kn) { + currentKey = ffj_t_Event_AgentAdded_Agent + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Event_AgentAddedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Event_AgentAdded_Agent: + goto handle_Agent + + case ffj_t_Event_AgentAddedno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Agent: + + /* handler: uj.Agent type=master.Response_GetAgents_Agent kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Agent.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Event_AgentRemoved) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Event_AgentRemoved) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"agent_id":`) + + { + + err = mj.AgentID.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Event_AgentRemovedbase = iota + ffj_t_Event_AgentRemovedno_such_key + + ffj_t_Event_AgentRemoved_AgentID +) + +var ffj_key_Event_AgentRemoved_AgentID = []byte("agent_id") + +func (uj *Event_AgentRemoved) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Event_AgentRemoved) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Event_AgentRemovedbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Event_AgentRemovedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Event_AgentRemoved_AgentID, kn) { + currentKey = ffj_t_Event_AgentRemoved_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.AsciiEqualFold(ffj_key_Event_AgentRemoved_AgentID, kn) { + currentKey = ffj_t_Event_AgentRemoved_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Event_AgentRemovedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Event_AgentRemoved_AgentID: + goto handle_AgentID + + case ffj_t_Event_AgentRemovedno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_AgentID: + + /* handler: uj.AgentID type=mesos.AgentID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.AgentID.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Event_Subscribed) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Event_Subscribed) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ `) + if mj.GetState != nil { + if true { + buf.WriteString(`"get_state":`) + + { + + err = mj.GetState.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Event_Subscribedbase = iota + ffj_t_Event_Subscribedno_such_key + + ffj_t_Event_Subscribed_GetState +) + +var ffj_key_Event_Subscribed_GetState = []byte("get_state") + +func (uj *Event_Subscribed) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Event_Subscribed) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Event_Subscribedbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Event_Subscribedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'g': + + if bytes.Equal(ffj_key_Event_Subscribed_GetState, kn) { + currentKey = ffj_t_Event_Subscribed_GetState + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Event_Subscribed_GetState, kn) { + currentKey = ffj_t_Event_Subscribed_GetState + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Event_Subscribedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Event_Subscribed_GetState: + goto handle_GetState + + case ffj_t_Event_Subscribedno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_GetState: + + /* handler: uj.GetState type=master.Response_GetState kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetState = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetState == nil { + uj.GetState = new(Response_GetState) + } + + err = uj.GetState.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Event_TaskAdded) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Event_TaskAdded) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"task":`) + + { + + err = mj.Task.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Event_TaskAddedbase = iota + ffj_t_Event_TaskAddedno_such_key + + ffj_t_Event_TaskAdded_Task +) + +var ffj_key_Event_TaskAdded_Task = []byte("task") + +func (uj *Event_TaskAdded) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Event_TaskAdded) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Event_TaskAddedbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Event_TaskAddedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 't': + + if bytes.Equal(ffj_key_Event_TaskAdded_Task, kn) { + currentKey = ffj_t_Event_TaskAdded_Task + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Event_TaskAdded_Task, kn) { + currentKey = ffj_t_Event_TaskAdded_Task + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Event_TaskAddedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Event_TaskAdded_Task: + goto handle_Task + + case ffj_t_Event_TaskAddedno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Task: + + /* handler: uj.Task type=mesos.Task kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Task.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Event_TaskUpdated) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Event_TaskUpdated) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ "framework_id":`) + + { + + err = mj.FrameworkID.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteString(`,"status":`) + + { + + err = mj.Status.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + if mj.State != nil { + if true { + buf.WriteString(`"state":`) + + { + + obj, err = mj.State.MarshalJSON() + if err != nil { + return err + } + buf.Write(obj) + + } + buf.WriteByte(',') + } + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Event_TaskUpdatedbase = iota + ffj_t_Event_TaskUpdatedno_such_key + + ffj_t_Event_TaskUpdated_FrameworkID + + ffj_t_Event_TaskUpdated_Status + + ffj_t_Event_TaskUpdated_State +) + +var ffj_key_Event_TaskUpdated_FrameworkID = []byte("framework_id") + +var ffj_key_Event_TaskUpdated_Status = []byte("status") + +var ffj_key_Event_TaskUpdated_State = []byte("state") + +func (uj *Event_TaskUpdated) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Event_TaskUpdated) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Event_TaskUpdatedbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Event_TaskUpdatedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'f': + + if bytes.Equal(ffj_key_Event_TaskUpdated_FrameworkID, kn) { + currentKey = ffj_t_Event_TaskUpdated_FrameworkID + state = fflib.FFParse_want_colon + goto mainparse + } + + case 's': + + if bytes.Equal(ffj_key_Event_TaskUpdated_Status, kn) { + currentKey = ffj_t_Event_TaskUpdated_Status + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Event_TaskUpdated_State, kn) { + currentKey = ffj_t_Event_TaskUpdated_State + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Event_TaskUpdated_State, kn) { + currentKey = ffj_t_Event_TaskUpdated_State + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Event_TaskUpdated_Status, kn) { + currentKey = ffj_t_Event_TaskUpdated_Status + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Event_TaskUpdated_FrameworkID, kn) { + currentKey = ffj_t_Event_TaskUpdated_FrameworkID + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Event_TaskUpdatedno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Event_TaskUpdated_FrameworkID: + goto handle_FrameworkID + + case ffj_t_Event_TaskUpdated_Status: + goto handle_Status + + case ffj_t_Event_TaskUpdated_State: + goto handle_State + + case ffj_t_Event_TaskUpdatedno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_FrameworkID: + + /* handler: uj.FrameworkID type=mesos.FrameworkID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.FrameworkID.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Status: + + /* handler: uj.Status type=mesos.TaskStatus kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Status.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_State: + + /* handler: uj.State type=mesos.TaskState kind=int32 quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.State = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + tbuf, err := fs.CaptureField(tok) + if err != nil { + return fs.WrapErr(err) + } + + if uj.State == nil { + uj.State = new(mesos.TaskState) + } + + err = uj.State.UnmarshalJSON(tbuf) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ "type":`) + + { + + obj, err = mj.Type.MarshalJSON() + if err != nil { + return err + } + buf.Write(obj) + + } + buf.WriteByte(',') + if mj.GetHealth != nil { + if true { + buf.WriteString(`"get_health":`) + + { + + err = mj.GetHealth.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetFlags != nil { + if true { + buf.WriteString(`"get_flags":`) + + { + + err = mj.GetFlags.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetVersion != nil { + if true { + buf.WriteString(`"get_version":`) + + { + + err = mj.GetVersion.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetMetrics != nil { + if true { + buf.WriteString(`"get_metrics":`) + + { + + err = mj.GetMetrics.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetLoggingLevel != nil { + if true { + buf.WriteString(`"get_logging_level":`) + + { + + err = mj.GetLoggingLevel.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.ListFiles != nil { + if true { + buf.WriteString(`"list_files":`) + + { + + err = mj.ListFiles.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.ReadFile != nil { + if true { + buf.WriteString(`"read_file":`) + + { + + err = mj.ReadFile.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetState != nil { + if true { + buf.WriteString(`"get_state":`) + + { + + err = mj.GetState.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetAgents != nil { + if true { + buf.WriteString(`"get_agents":`) + + { + + err = mj.GetAgents.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetFrameworks != nil { + if true { + buf.WriteString(`"get_frameworks":`) + + { + + err = mj.GetFrameworks.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetExecutors != nil { + if true { + buf.WriteString(`"get_executors":`) + + { + + err = mj.GetExecutors.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetTasks != nil { + if true { + buf.WriteString(`"get_tasks":`) + + { + + err = mj.GetTasks.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetRoles != nil { + if true { + buf.WriteString(`"get_roles":`) + + { + + err = mj.GetRoles.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetWeights != nil { + if true { + buf.WriteString(`"get_weights":`) + + { + + err = mj.GetWeights.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetMaster != nil { + if true { + buf.WriteString(`"get_master":`) + + { + + err = mj.GetMaster.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetMaintenanceStatus != nil { + if true { + buf.WriteString(`"get_maintenance_status":`) + + { + + err = mj.GetMaintenanceStatus.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetMaintenanceSchedule != nil { + if true { + buf.WriteString(`"get_maintenance_schedule":`) + + { + + err = mj.GetMaintenanceSchedule.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetQuota != nil { + if true { + buf.WriteString(`"get_quota":`) + + { + + err = mj.GetQuota.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Responsebase = iota + ffj_t_Responseno_such_key + + ffj_t_Response_Type + + ffj_t_Response_GetHealth + + ffj_t_Response_GetFlags + + ffj_t_Response_GetVersion + + ffj_t_Response_GetMetrics + + ffj_t_Response_GetLoggingLevel + + ffj_t_Response_ListFiles + + ffj_t_Response_ReadFile + + ffj_t_Response_GetState + + ffj_t_Response_GetAgents + + ffj_t_Response_GetFrameworks + + ffj_t_Response_GetExecutors + + ffj_t_Response_GetTasks + + ffj_t_Response_GetRoles + + ffj_t_Response_GetWeights + + ffj_t_Response_GetMaster + + ffj_t_Response_GetMaintenanceStatus + + ffj_t_Response_GetMaintenanceSchedule + + ffj_t_Response_GetQuota +) + +var ffj_key_Response_Type = []byte("type") + +var ffj_key_Response_GetHealth = []byte("get_health") + +var ffj_key_Response_GetFlags = []byte("get_flags") + +var ffj_key_Response_GetVersion = []byte("get_version") + +var ffj_key_Response_GetMetrics = []byte("get_metrics") + +var ffj_key_Response_GetLoggingLevel = []byte("get_logging_level") + +var ffj_key_Response_ListFiles = []byte("list_files") + +var ffj_key_Response_ReadFile = []byte("read_file") + +var ffj_key_Response_GetState = []byte("get_state") + +var ffj_key_Response_GetAgents = []byte("get_agents") + +var ffj_key_Response_GetFrameworks = []byte("get_frameworks") + +var ffj_key_Response_GetExecutors = []byte("get_executors") + +var ffj_key_Response_GetTasks = []byte("get_tasks") + +var ffj_key_Response_GetRoles = []byte("get_roles") + +var ffj_key_Response_GetWeights = []byte("get_weights") + +var ffj_key_Response_GetMaster = []byte("get_master") + +var ffj_key_Response_GetMaintenanceStatus = []byte("get_maintenance_status") + +var ffj_key_Response_GetMaintenanceSchedule = []byte("get_maintenance_schedule") + +var ffj_key_Response_GetQuota = []byte("get_quota") + +func (uj *Response) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Responsebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Responseno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'g': + + if bytes.Equal(ffj_key_Response_GetHealth, kn) { + currentKey = ffj_t_Response_GetHealth + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetFlags, kn) { + currentKey = ffj_t_Response_GetFlags + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetVersion, kn) { + currentKey = ffj_t_Response_GetVersion + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetMetrics, kn) { + currentKey = ffj_t_Response_GetMetrics + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetLoggingLevel, kn) { + currentKey = ffj_t_Response_GetLoggingLevel + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetState, kn) { + currentKey = ffj_t_Response_GetState + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetAgents, kn) { + currentKey = ffj_t_Response_GetAgents + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetFrameworks, kn) { + currentKey = ffj_t_Response_GetFrameworks + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetExecutors, kn) { + currentKey = ffj_t_Response_GetExecutors + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetTasks, kn) { + currentKey = ffj_t_Response_GetTasks + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetRoles, kn) { + currentKey = ffj_t_Response_GetRoles + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetWeights, kn) { + currentKey = ffj_t_Response_GetWeights + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetMaster, kn) { + currentKey = ffj_t_Response_GetMaster + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetMaintenanceStatus, kn) { + currentKey = ffj_t_Response_GetMaintenanceStatus + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetMaintenanceSchedule, kn) { + currentKey = ffj_t_Response_GetMaintenanceSchedule + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetQuota, kn) { + currentKey = ffj_t_Response_GetQuota + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'l': + + if bytes.Equal(ffj_key_Response_ListFiles, kn) { + currentKey = ffj_t_Response_ListFiles + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_Response_ReadFile, kn) { + currentKey = ffj_t_Response_ReadFile + state = fflib.FFParse_want_colon + goto mainparse + } + + case 't': + + if bytes.Equal(ffj_key_Response_Type, kn) { + currentKey = ffj_t_Response_Type + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.AsciiEqualFold(ffj_key_Response_GetQuota, kn) { + currentKey = ffj_t_Response_GetQuota + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetMaintenanceSchedule, kn) { + currentKey = ffj_t_Response_GetMaintenanceSchedule + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetMaintenanceStatus, kn) { + currentKey = ffj_t_Response_GetMaintenanceStatus + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetMaster, kn) { + currentKey = ffj_t_Response_GetMaster + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetWeights, kn) { + currentKey = ffj_t_Response_GetWeights + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetRoles, kn) { + currentKey = ffj_t_Response_GetRoles + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetTasks, kn) { + currentKey = ffj_t_Response_GetTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetExecutors, kn) { + currentKey = ffj_t_Response_GetExecutors + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks, kn) { + currentKey = ffj_t_Response_GetFrameworks + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetAgents, kn) { + currentKey = ffj_t_Response_GetAgents + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetState, kn) { + currentKey = ffj_t_Response_GetState + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Response_ReadFile, kn) { + currentKey = ffj_t_Response_ReadFile + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_ListFiles, kn) { + currentKey = ffj_t_Response_ListFiles + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Response_GetLoggingLevel, kn) { + currentKey = ffj_t_Response_GetLoggingLevel + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetMetrics, kn) { + currentKey = ffj_t_Response_GetMetrics + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetVersion, kn) { + currentKey = ffj_t_Response_GetVersion + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFlags, kn) { + currentKey = ffj_t_Response_GetFlags + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Response_GetHealth, kn) { + currentKey = ffj_t_Response_GetHealth + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Response_Type, kn) { + currentKey = ffj_t_Response_Type + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Responseno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_Type: + goto handle_Type + + case ffj_t_Response_GetHealth: + goto handle_GetHealth + + case ffj_t_Response_GetFlags: + goto handle_GetFlags + + case ffj_t_Response_GetVersion: + goto handle_GetVersion + + case ffj_t_Response_GetMetrics: + goto handle_GetMetrics + + case ffj_t_Response_GetLoggingLevel: + goto handle_GetLoggingLevel + + case ffj_t_Response_ListFiles: + goto handle_ListFiles + + case ffj_t_Response_ReadFile: + goto handle_ReadFile + + case ffj_t_Response_GetState: + goto handle_GetState + + case ffj_t_Response_GetAgents: + goto handle_GetAgents + + case ffj_t_Response_GetFrameworks: + goto handle_GetFrameworks + + case ffj_t_Response_GetExecutors: + goto handle_GetExecutors + + case ffj_t_Response_GetTasks: + goto handle_GetTasks + + case ffj_t_Response_GetRoles: + goto handle_GetRoles + + case ffj_t_Response_GetWeights: + goto handle_GetWeights + + case ffj_t_Response_GetMaster: + goto handle_GetMaster + + case ffj_t_Response_GetMaintenanceStatus: + goto handle_GetMaintenanceStatus + + case ffj_t_Response_GetMaintenanceSchedule: + goto handle_GetMaintenanceSchedule + + case ffj_t_Response_GetQuota: + goto handle_GetQuota + + case ffj_t_Responseno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Type: + + /* handler: uj.Type type=master.Response_Type kind=int32 quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + tbuf, err := fs.CaptureField(tok) + if err != nil { + return fs.WrapErr(err) + } + + err = uj.Type.UnmarshalJSON(tbuf) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetHealth: + + /* handler: uj.GetHealth type=master.Response_GetHealth kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetHealth = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetHealth == nil { + uj.GetHealth = new(Response_GetHealth) + } + + err = uj.GetHealth.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetFlags: + + /* handler: uj.GetFlags type=master.Response_GetFlags kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetFlags = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetFlags == nil { + uj.GetFlags = new(Response_GetFlags) + } + + err = uj.GetFlags.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetVersion: + + /* handler: uj.GetVersion type=master.Response_GetVersion kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetVersion = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetVersion == nil { + uj.GetVersion = new(Response_GetVersion) + } + + err = uj.GetVersion.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetMetrics: + + /* handler: uj.GetMetrics type=master.Response_GetMetrics kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetMetrics = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetMetrics == nil { + uj.GetMetrics = new(Response_GetMetrics) + } + + err = uj.GetMetrics.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetLoggingLevel: + + /* handler: uj.GetLoggingLevel type=master.Response_GetLoggingLevel kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetLoggingLevel = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetLoggingLevel == nil { + uj.GetLoggingLevel = new(Response_GetLoggingLevel) + } + + err = uj.GetLoggingLevel.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_ListFiles: + + /* handler: uj.ListFiles type=master.Response_ListFiles kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.ListFiles = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.ListFiles == nil { + uj.ListFiles = new(Response_ListFiles) + } + + err = uj.ListFiles.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_ReadFile: + + /* handler: uj.ReadFile type=master.Response_ReadFile kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.ReadFile = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.ReadFile == nil { + uj.ReadFile = new(Response_ReadFile) + } + + err = uj.ReadFile.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetState: + + /* handler: uj.GetState type=master.Response_GetState kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetState = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetState == nil { + uj.GetState = new(Response_GetState) + } + + err = uj.GetState.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetAgents: + + /* handler: uj.GetAgents type=master.Response_GetAgents kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetAgents = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetAgents == nil { + uj.GetAgents = new(Response_GetAgents) + } + + err = uj.GetAgents.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetFrameworks: + + /* handler: uj.GetFrameworks type=master.Response_GetFrameworks kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetFrameworks = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetFrameworks == nil { + uj.GetFrameworks = new(Response_GetFrameworks) + } + + err = uj.GetFrameworks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetExecutors: + + /* handler: uj.GetExecutors type=master.Response_GetExecutors kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetExecutors = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetExecutors == nil { + uj.GetExecutors = new(Response_GetExecutors) + } + + err = uj.GetExecutors.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetTasks: + + /* handler: uj.GetTasks type=master.Response_GetTasks kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetTasks = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetTasks == nil { + uj.GetTasks = new(Response_GetTasks) + } + + err = uj.GetTasks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetRoles: + + /* handler: uj.GetRoles type=master.Response_GetRoles kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetRoles = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetRoles == nil { + uj.GetRoles = new(Response_GetRoles) + } + + err = uj.GetRoles.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetWeights: + + /* handler: uj.GetWeights type=master.Response_GetWeights kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetWeights = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetWeights == nil { + uj.GetWeights = new(Response_GetWeights) + } + + err = uj.GetWeights.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetMaster: + + /* handler: uj.GetMaster type=master.Response_GetMaster kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetMaster = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetMaster == nil { + uj.GetMaster = new(Response_GetMaster) + } + + err = uj.GetMaster.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetMaintenanceStatus: + + /* handler: uj.GetMaintenanceStatus type=master.Response_GetMaintenanceStatus kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetMaintenanceStatus = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetMaintenanceStatus == nil { + uj.GetMaintenanceStatus = new(Response_GetMaintenanceStatus) + } + + err = uj.GetMaintenanceStatus.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetMaintenanceSchedule: + + /* handler: uj.GetMaintenanceSchedule type=master.Response_GetMaintenanceSchedule kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetMaintenanceSchedule = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetMaintenanceSchedule == nil { + uj.GetMaintenanceSchedule = new(Response_GetMaintenanceSchedule) + } + + err = uj.GetMaintenanceSchedule.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetQuota: + + /* handler: uj.GetQuota type=master.Response_GetQuota kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetQuota = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetQuota == nil { + uj.GetQuota = new(Response_GetQuota) + } + + err = uj.GetQuota.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetAgents) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetAgents) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"agents":`) + if mj.Agents != nil { + buf.WriteString(`[`) + for i, v := range mj.Agents { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"recovered_agents":`) + if mj.RecoveredAgents != nil { + buf.WriteString(`[`) + for i, v := range mj.RecoveredAgents { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetAgentsbase = iota + ffj_t_Response_GetAgentsno_such_key + + ffj_t_Response_GetAgents_Agents + + ffj_t_Response_GetAgents_RecoveredAgents +) + +var ffj_key_Response_GetAgents_Agents = []byte("agents") + +var ffj_key_Response_GetAgents_RecoveredAgents = []byte("recovered_agents") + +func (uj *Response_GetAgents) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetAgents) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetAgentsbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetAgentsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Response_GetAgents_Agents, kn) { + currentKey = ffj_t_Response_GetAgents_Agents + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_Response_GetAgents_RecoveredAgents, kn) { + currentKey = ffj_t_Response_GetAgents_RecoveredAgents + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetAgents_RecoveredAgents, kn) { + currentKey = ffj_t_Response_GetAgents_RecoveredAgents + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetAgents_Agents, kn) { + currentKey = ffj_t_Response_GetAgents_Agents + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetAgentsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetAgents_Agents: + goto handle_Agents + + case ffj_t_Response_GetAgents_RecoveredAgents: + goto handle_RecoveredAgents + + case ffj_t_Response_GetAgentsno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Agents: + + /* handler: uj.Agents type=[]master.Response_GetAgents_Agent kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Agents = nil + } else { + + uj.Agents = []Response_GetAgents_Agent{} + + wantVal := true + + for { + + var tmp_uj__Agents Response_GetAgents_Agent + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Agents type=master.Response_GetAgents_Agent kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Agents.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Agents = append(uj.Agents, tmp_uj__Agents) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_RecoveredAgents: + + /* handler: uj.RecoveredAgents type=[]mesos.AgentInfo kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.RecoveredAgents = nil + } else { + + uj.RecoveredAgents = []mesos.AgentInfo{} + + wantVal := true + + for { + + var tmp_uj__RecoveredAgents mesos.AgentInfo + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__RecoveredAgents type=mesos.AgentInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__RecoveredAgents.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.RecoveredAgents = append(uj.RecoveredAgents, tmp_uj__RecoveredAgents) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetAgents_Agent) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetAgents_Agent) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"agent_info":`) + + { + + err = mj.AgentInfo.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + if mj.Active { + buf.WriteString(`,"active":true`) + } else { + buf.WriteString(`,"active":false`) + } + buf.WriteString(`,"version":`) + fflib.WriteJsonString(buf, string(mj.Version)) + buf.WriteByte(',') + if mj.PID != nil { + if true { + buf.WriteString(`"pid":`) + fflib.WriteJsonString(buf, string(*mj.PID)) + buf.WriteByte(',') + } + } + if mj.RegisteredTime != nil { + if true { + buf.WriteString(`"registered_time":`) + + { + + err = mj.RegisteredTime.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.ReregisteredTime != nil { + if true { + buf.WriteString(`"reregistered_time":`) + + { + + err = mj.ReregisteredTime.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + buf.WriteString(`"total_resources":`) + if mj.TotalResources != nil { + buf.WriteString(`[`) + for i, v := range mj.TotalResources { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"allocated_resources":`) + if mj.AllocatedResources != nil { + buf.WriteString(`[`) + for i, v := range mj.AllocatedResources { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"offered_resources":`) + if mj.OfferedResources != nil { + buf.WriteString(`[`) + for i, v := range mj.OfferedResources { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetAgents_Agentbase = iota + ffj_t_Response_GetAgents_Agentno_such_key + + ffj_t_Response_GetAgents_Agent_AgentInfo + + ffj_t_Response_GetAgents_Agent_Active + + ffj_t_Response_GetAgents_Agent_Version + + ffj_t_Response_GetAgents_Agent_PID + + ffj_t_Response_GetAgents_Agent_RegisteredTime + + ffj_t_Response_GetAgents_Agent_ReregisteredTime + + ffj_t_Response_GetAgents_Agent_TotalResources + + ffj_t_Response_GetAgents_Agent_AllocatedResources + + ffj_t_Response_GetAgents_Agent_OfferedResources +) + +var ffj_key_Response_GetAgents_Agent_AgentInfo = []byte("agent_info") + +var ffj_key_Response_GetAgents_Agent_Active = []byte("active") + +var ffj_key_Response_GetAgents_Agent_Version = []byte("version") + +var ffj_key_Response_GetAgents_Agent_PID = []byte("pid") + +var ffj_key_Response_GetAgents_Agent_RegisteredTime = []byte("registered_time") + +var ffj_key_Response_GetAgents_Agent_ReregisteredTime = []byte("reregistered_time") + +var ffj_key_Response_GetAgents_Agent_TotalResources = []byte("total_resources") + +var ffj_key_Response_GetAgents_Agent_AllocatedResources = []byte("allocated_resources") + +var ffj_key_Response_GetAgents_Agent_OfferedResources = []byte("offered_resources") + +func (uj *Response_GetAgents_Agent) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetAgents_Agent) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetAgents_Agentbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetAgents_Agentno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Response_GetAgents_Agent_AgentInfo, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_AgentInfo + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetAgents_Agent_Active, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_Active + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetAgents_Agent_AllocatedResources, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_AllocatedResources + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'o': + + if bytes.Equal(ffj_key_Response_GetAgents_Agent_OfferedResources, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_OfferedResources + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'p': + + if bytes.Equal(ffj_key_Response_GetAgents_Agent_PID, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_PID + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_Response_GetAgents_Agent_RegisteredTime, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_RegisteredTime + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetAgents_Agent_ReregisteredTime, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_ReregisteredTime + state = fflib.FFParse_want_colon + goto mainparse + } + + case 't': + + if bytes.Equal(ffj_key_Response_GetAgents_Agent_TotalResources, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_TotalResources + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'v': + + if bytes.Equal(ffj_key_Response_GetAgents_Agent_Version, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_Version + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetAgents_Agent_OfferedResources, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_OfferedResources + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetAgents_Agent_AllocatedResources, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_AllocatedResources + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetAgents_Agent_TotalResources, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_TotalResources + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetAgents_Agent_ReregisteredTime, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_ReregisteredTime + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetAgents_Agent_RegisteredTime, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_RegisteredTime + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Response_GetAgents_Agent_PID, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_PID + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetAgents_Agent_Version, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_Version + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Response_GetAgents_Agent_Active, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_Active + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Response_GetAgents_Agent_AgentInfo, kn) { + currentKey = ffj_t_Response_GetAgents_Agent_AgentInfo + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetAgents_Agentno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetAgents_Agent_AgentInfo: + goto handle_AgentInfo + + case ffj_t_Response_GetAgents_Agent_Active: + goto handle_Active + + case ffj_t_Response_GetAgents_Agent_Version: + goto handle_Version + + case ffj_t_Response_GetAgents_Agent_PID: + goto handle_PID + + case ffj_t_Response_GetAgents_Agent_RegisteredTime: + goto handle_RegisteredTime + + case ffj_t_Response_GetAgents_Agent_ReregisteredTime: + goto handle_ReregisteredTime + + case ffj_t_Response_GetAgents_Agent_TotalResources: + goto handle_TotalResources + + case ffj_t_Response_GetAgents_Agent_AllocatedResources: + goto handle_AllocatedResources + + case ffj_t_Response_GetAgents_Agent_OfferedResources: + goto handle_OfferedResources + + case ffj_t_Response_GetAgents_Agentno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_AgentInfo: + + /* handler: uj.AgentInfo type=mesos.AgentInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.AgentInfo.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Active: + + /* handler: uj.Active type=bool kind=bool quoted=false*/ + + { + if tok != fflib.FFTok_bool && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for bool", tok)) + } + } + + { + if tok == fflib.FFTok_null { + + } else { + tmpb := fs.Output.Bytes() + + if bytes.Compare([]byte{'t', 'r', 'u', 'e'}, tmpb) == 0 { + + uj.Active = true + + } else if bytes.Compare([]byte{'f', 'a', 'l', 's', 'e'}, tmpb) == 0 { + + uj.Active = false + + } else { + err = errors.New("unexpected bytes for true/false value") + return fs.WrapErr(err) + } + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Version: + + /* handler: uj.Version type=string kind=string quoted=false*/ + + { + + { + if tok != fflib.FFTok_string && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for string", tok)) + } + } + + if tok == fflib.FFTok_null { + + } else { + + outBuf := fs.Output.Bytes() + + uj.Version = string(string(outBuf)) + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_PID: + + /* handler: uj.PID type=string kind=string quoted=false*/ + + { + + { + if tok != fflib.FFTok_string && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for string", tok)) + } + } + + if tok == fflib.FFTok_null { + + uj.PID = nil + + } else { + + var tval string + outBuf := fs.Output.Bytes() + + tval = string(string(outBuf)) + uj.PID = &tval + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_RegisteredTime: + + /* handler: uj.RegisteredTime type=mesos.TimeInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.RegisteredTime = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.RegisteredTime == nil { + uj.RegisteredTime = new(mesos.TimeInfo) + } + + err = uj.RegisteredTime.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_ReregisteredTime: + + /* handler: uj.ReregisteredTime type=mesos.TimeInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.ReregisteredTime = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.ReregisteredTime == nil { + uj.ReregisteredTime = new(mesos.TimeInfo) + } + + err = uj.ReregisteredTime.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_TotalResources: + + /* handler: uj.TotalResources type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.TotalResources = nil + } else { + + uj.TotalResources = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__TotalResources mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__TotalResources type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__TotalResources.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.TotalResources = append(uj.TotalResources, tmp_uj__TotalResources) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_AllocatedResources: + + /* handler: uj.AllocatedResources type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.AllocatedResources = nil + } else { + + uj.AllocatedResources = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__AllocatedResources mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__AllocatedResources type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__AllocatedResources.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.AllocatedResources = append(uj.AllocatedResources, tmp_uj__AllocatedResources) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_OfferedResources: + + /* handler: uj.OfferedResources type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.OfferedResources = nil + } else { + + uj.OfferedResources = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__OfferedResources mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__OfferedResources type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__OfferedResources.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.OfferedResources = append(uj.OfferedResources, tmp_uj__OfferedResources) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetExecutors) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetExecutors) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"executors":`) + if mj.Executors != nil { + buf.WriteString(`[`) + for i, v := range mj.Executors { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"orphan_executors":`) + if mj.OrphanExecutors != nil { + buf.WriteString(`[`) + for i, v := range mj.OrphanExecutors { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetExecutorsbase = iota + ffj_t_Response_GetExecutorsno_such_key + + ffj_t_Response_GetExecutors_Executors + + ffj_t_Response_GetExecutors_OrphanExecutors +) + +var ffj_key_Response_GetExecutors_Executors = []byte("executors") + +var ffj_key_Response_GetExecutors_OrphanExecutors = []byte("orphan_executors") + +func (uj *Response_GetExecutors) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetExecutors) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetExecutorsbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetExecutorsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'e': + + if bytes.Equal(ffj_key_Response_GetExecutors_Executors, kn) { + currentKey = ffj_t_Response_GetExecutors_Executors + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'o': + + if bytes.Equal(ffj_key_Response_GetExecutors_OrphanExecutors, kn) { + currentKey = ffj_t_Response_GetExecutors_OrphanExecutors + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetExecutors_OrphanExecutors, kn) { + currentKey = ffj_t_Response_GetExecutors_OrphanExecutors + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetExecutors_Executors, kn) { + currentKey = ffj_t_Response_GetExecutors_Executors + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetExecutorsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetExecutors_Executors: + goto handle_Executors + + case ffj_t_Response_GetExecutors_OrphanExecutors: + goto handle_OrphanExecutors + + case ffj_t_Response_GetExecutorsno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Executors: + + /* handler: uj.Executors type=[]master.Response_GetExecutors_Executor kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Executors = nil + } else { + + uj.Executors = []Response_GetExecutors_Executor{} + + wantVal := true + + for { + + var tmp_uj__Executors Response_GetExecutors_Executor + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Executors type=master.Response_GetExecutors_Executor kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Executors.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Executors = append(uj.Executors, tmp_uj__Executors) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_OrphanExecutors: + + /* handler: uj.OrphanExecutors type=[]master.Response_GetExecutors_Executor kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.OrphanExecutors = nil + } else { + + uj.OrphanExecutors = []Response_GetExecutors_Executor{} + + wantVal := true + + for { + + var tmp_uj__OrphanExecutors Response_GetExecutors_Executor + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__OrphanExecutors type=master.Response_GetExecutors_Executor kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__OrphanExecutors.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.OrphanExecutors = append(uj.OrphanExecutors, tmp_uj__OrphanExecutors) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetExecutors_Executor) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetExecutors_Executor) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"executor_info":`) + + { + + err = mj.ExecutorInfo.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteString(`,"agent_id":`) + + { + + err = mj.AgentID.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetExecutors_Executorbase = iota + ffj_t_Response_GetExecutors_Executorno_such_key + + ffj_t_Response_GetExecutors_Executor_ExecutorInfo + + ffj_t_Response_GetExecutors_Executor_AgentID +) + +var ffj_key_Response_GetExecutors_Executor_ExecutorInfo = []byte("executor_info") + +var ffj_key_Response_GetExecutors_Executor_AgentID = []byte("agent_id") + +func (uj *Response_GetExecutors_Executor) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetExecutors_Executor) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetExecutors_Executorbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetExecutors_Executorno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Response_GetExecutors_Executor_AgentID, kn) { + currentKey = ffj_t_Response_GetExecutors_Executor_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'e': + + if bytes.Equal(ffj_key_Response_GetExecutors_Executor_ExecutorInfo, kn) { + currentKey = ffj_t_Response_GetExecutors_Executor_ExecutorInfo + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.AsciiEqualFold(ffj_key_Response_GetExecutors_Executor_AgentID, kn) { + currentKey = ffj_t_Response_GetExecutors_Executor_AgentID + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.AsciiEqualFold(ffj_key_Response_GetExecutors_Executor_ExecutorInfo, kn) { + currentKey = ffj_t_Response_GetExecutors_Executor_ExecutorInfo + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetExecutors_Executorno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetExecutors_Executor_ExecutorInfo: + goto handle_ExecutorInfo + + case ffj_t_Response_GetExecutors_Executor_AgentID: + goto handle_AgentID + + case ffj_t_Response_GetExecutors_Executorno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_ExecutorInfo: + + /* handler: uj.ExecutorInfo type=mesos.ExecutorInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.ExecutorInfo.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_AgentID: + + /* handler: uj.AgentID type=mesos.AgentID kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.AgentID.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetFlags) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetFlags) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"flags":`) + if mj.Flags != nil { + buf.WriteString(`[`) + for i, v := range mj.Flags { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetFlagsbase = iota + ffj_t_Response_GetFlagsno_such_key + + ffj_t_Response_GetFlags_Flags +) + +var ffj_key_Response_GetFlags_Flags = []byte("flags") + +func (uj *Response_GetFlags) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetFlags) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetFlagsbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetFlagsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'f': + + if bytes.Equal(ffj_key_Response_GetFlags_Flags, kn) { + currentKey = ffj_t_Response_GetFlags_Flags + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFlags_Flags, kn) { + currentKey = ffj_t_Response_GetFlags_Flags + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetFlagsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetFlags_Flags: + goto handle_Flags + + case ffj_t_Response_GetFlagsno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Flags: + + /* handler: uj.Flags type=[]mesos.Flag kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Flags = nil + } else { + + uj.Flags = []mesos.Flag{} + + wantVal := true + + for { + + var tmp_uj__Flags mesos.Flag + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Flags type=mesos.Flag kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Flags.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Flags = append(uj.Flags, tmp_uj__Flags) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetFrameworks) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetFrameworks) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"frameworks":`) + if mj.Frameworks != nil { + buf.WriteString(`[`) + for i, v := range mj.Frameworks { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"completed_frameworks":`) + if mj.CompletedFrameworks != nil { + buf.WriteString(`[`) + for i, v := range mj.CompletedFrameworks { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"recovered_frameworks":`) + if mj.RecoveredFrameworks != nil { + buf.WriteString(`[`) + for i, v := range mj.RecoveredFrameworks { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetFrameworksbase = iota + ffj_t_Response_GetFrameworksno_such_key + + ffj_t_Response_GetFrameworks_Frameworks + + ffj_t_Response_GetFrameworks_CompletedFrameworks + + ffj_t_Response_GetFrameworks_RecoveredFrameworks +) + +var ffj_key_Response_GetFrameworks_Frameworks = []byte("frameworks") + +var ffj_key_Response_GetFrameworks_CompletedFrameworks = []byte("completed_frameworks") + +var ffj_key_Response_GetFrameworks_RecoveredFrameworks = []byte("recovered_frameworks") + +func (uj *Response_GetFrameworks) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetFrameworks) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetFrameworksbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetFrameworksno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'c': + + if bytes.Equal(ffj_key_Response_GetFrameworks_CompletedFrameworks, kn) { + currentKey = ffj_t_Response_GetFrameworks_CompletedFrameworks + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'f': + + if bytes.Equal(ffj_key_Response_GetFrameworks_Frameworks, kn) { + currentKey = ffj_t_Response_GetFrameworks_Frameworks + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_Response_GetFrameworks_RecoveredFrameworks, kn) { + currentKey = ffj_t_Response_GetFrameworks_RecoveredFrameworks + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_RecoveredFrameworks, kn) { + currentKey = ffj_t_Response_GetFrameworks_RecoveredFrameworks + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_CompletedFrameworks, kn) { + currentKey = ffj_t_Response_GetFrameworks_CompletedFrameworks + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_Frameworks, kn) { + currentKey = ffj_t_Response_GetFrameworks_Frameworks + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetFrameworksno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetFrameworks_Frameworks: + goto handle_Frameworks + + case ffj_t_Response_GetFrameworks_CompletedFrameworks: + goto handle_CompletedFrameworks + + case ffj_t_Response_GetFrameworks_RecoveredFrameworks: + goto handle_RecoveredFrameworks + + case ffj_t_Response_GetFrameworksno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Frameworks: + + /* handler: uj.Frameworks type=[]master.Response_GetFrameworks_Framework kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Frameworks = nil + } else { + + uj.Frameworks = []Response_GetFrameworks_Framework{} + + wantVal := true + + for { + + var tmp_uj__Frameworks Response_GetFrameworks_Framework + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Frameworks type=master.Response_GetFrameworks_Framework kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Frameworks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Frameworks = append(uj.Frameworks, tmp_uj__Frameworks) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_CompletedFrameworks: + + /* handler: uj.CompletedFrameworks type=[]master.Response_GetFrameworks_Framework kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.CompletedFrameworks = nil + } else { + + uj.CompletedFrameworks = []Response_GetFrameworks_Framework{} + + wantVal := true + + for { + + var tmp_uj__CompletedFrameworks Response_GetFrameworks_Framework + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__CompletedFrameworks type=master.Response_GetFrameworks_Framework kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__CompletedFrameworks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.CompletedFrameworks = append(uj.CompletedFrameworks, tmp_uj__CompletedFrameworks) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_RecoveredFrameworks: + + /* handler: uj.RecoveredFrameworks type=[]mesos.FrameworkInfo kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.RecoveredFrameworks = nil + } else { + + uj.RecoveredFrameworks = []mesos.FrameworkInfo{} + + wantVal := true + + for { + + var tmp_uj__RecoveredFrameworks mesos.FrameworkInfo + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__RecoveredFrameworks type=mesos.FrameworkInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__RecoveredFrameworks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.RecoveredFrameworks = append(uj.RecoveredFrameworks, tmp_uj__RecoveredFrameworks) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetFrameworks_Framework) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetFrameworks_Framework) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"framework_info":`) + + { + + err = mj.FrameworkInfo.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + if mj.Active { + buf.WriteString(`,"active":true`) + } else { + buf.WriteString(`,"active":false`) + } + if mj.Connected { + buf.WriteString(`,"connected":true`) + } else { + buf.WriteString(`,"connected":false`) + } + if mj.Recovered { + buf.WriteString(`,"recovered":true`) + } else { + buf.WriteString(`,"recovered":false`) + } + buf.WriteByte(',') + if mj.RegisteredTime != nil { + if true { + buf.WriteString(`"registered_time":`) + + { + + err = mj.RegisteredTime.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.ReregisteredTime != nil { + if true { + buf.WriteString(`"reregistered_time":`) + + { + + err = mj.ReregisteredTime.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.UnregisteredTime != nil { + if true { + buf.WriteString(`"unregistered_time":`) + + { + + err = mj.UnregisteredTime.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + buf.WriteString(`"offers":`) + if mj.Offers != nil { + buf.WriteString(`[`) + for i, v := range mj.Offers { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"inverse_offers":`) + if mj.InverseOffers != nil { + buf.WriteString(`[`) + for i, v := range mj.InverseOffers { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"allocated_resources":`) + if mj.AllocatedResources != nil { + buf.WriteString(`[`) + for i, v := range mj.AllocatedResources { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"offered_resources":`) + if mj.OfferedResources != nil { + buf.WriteString(`[`) + for i, v := range mj.OfferedResources { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetFrameworks_Frameworkbase = iota + ffj_t_Response_GetFrameworks_Frameworkno_such_key + + ffj_t_Response_GetFrameworks_Framework_FrameworkInfo + + ffj_t_Response_GetFrameworks_Framework_Active + + ffj_t_Response_GetFrameworks_Framework_Connected + + ffj_t_Response_GetFrameworks_Framework_Recovered + + ffj_t_Response_GetFrameworks_Framework_RegisteredTime + + ffj_t_Response_GetFrameworks_Framework_ReregisteredTime + + ffj_t_Response_GetFrameworks_Framework_UnregisteredTime + + ffj_t_Response_GetFrameworks_Framework_Offers + + ffj_t_Response_GetFrameworks_Framework_InverseOffers + + ffj_t_Response_GetFrameworks_Framework_AllocatedResources + + ffj_t_Response_GetFrameworks_Framework_OfferedResources +) + +var ffj_key_Response_GetFrameworks_Framework_FrameworkInfo = []byte("framework_info") + +var ffj_key_Response_GetFrameworks_Framework_Active = []byte("active") + +var ffj_key_Response_GetFrameworks_Framework_Connected = []byte("connected") + +var ffj_key_Response_GetFrameworks_Framework_Recovered = []byte("recovered") + +var ffj_key_Response_GetFrameworks_Framework_RegisteredTime = []byte("registered_time") + +var ffj_key_Response_GetFrameworks_Framework_ReregisteredTime = []byte("reregistered_time") + +var ffj_key_Response_GetFrameworks_Framework_UnregisteredTime = []byte("unregistered_time") + +var ffj_key_Response_GetFrameworks_Framework_Offers = []byte("offers") + +var ffj_key_Response_GetFrameworks_Framework_InverseOffers = []byte("inverse_offers") + +var ffj_key_Response_GetFrameworks_Framework_AllocatedResources = []byte("allocated_resources") + +var ffj_key_Response_GetFrameworks_Framework_OfferedResources = []byte("offered_resources") + +func (uj *Response_GetFrameworks_Framework) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetFrameworks_Framework) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetFrameworks_Frameworkbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetFrameworks_Frameworkno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'a': + + if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_Active, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_Active + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_AllocatedResources, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_AllocatedResources + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'c': + + if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_Connected, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_Connected + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'f': + + if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_FrameworkInfo, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_FrameworkInfo + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'i': + + if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_InverseOffers, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_InverseOffers + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'o': + + if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_Offers, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_Offers + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_OfferedResources, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_OfferedResources + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_Recovered, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_Recovered + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_RegisteredTime, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_RegisteredTime + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_ReregisteredTime, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_ReregisteredTime + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'u': + + if bytes.Equal(ffj_key_Response_GetFrameworks_Framework_UnregisteredTime, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_UnregisteredTime + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_Framework_OfferedResources, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_OfferedResources + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_Framework_AllocatedResources, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_AllocatedResources + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_Framework_InverseOffers, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_InverseOffers + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_Framework_Offers, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_Offers + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_Framework_UnregisteredTime, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_UnregisteredTime + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_Framework_ReregisteredTime, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_ReregisteredTime + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_Framework_RegisteredTime, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_RegisteredTime + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Response_GetFrameworks_Framework_Recovered, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_Recovered + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Response_GetFrameworks_Framework_Connected, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_Connected + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_Response_GetFrameworks_Framework_Active, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_Active + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetFrameworks_Framework_FrameworkInfo, kn) { + currentKey = ffj_t_Response_GetFrameworks_Framework_FrameworkInfo + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetFrameworks_Frameworkno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetFrameworks_Framework_FrameworkInfo: + goto handle_FrameworkInfo + + case ffj_t_Response_GetFrameworks_Framework_Active: + goto handle_Active + + case ffj_t_Response_GetFrameworks_Framework_Connected: + goto handle_Connected + + case ffj_t_Response_GetFrameworks_Framework_Recovered: + goto handle_Recovered + + case ffj_t_Response_GetFrameworks_Framework_RegisteredTime: + goto handle_RegisteredTime + + case ffj_t_Response_GetFrameworks_Framework_ReregisteredTime: + goto handle_ReregisteredTime + + case ffj_t_Response_GetFrameworks_Framework_UnregisteredTime: + goto handle_UnregisteredTime + + case ffj_t_Response_GetFrameworks_Framework_Offers: + goto handle_Offers + + case ffj_t_Response_GetFrameworks_Framework_InverseOffers: + goto handle_InverseOffers + + case ffj_t_Response_GetFrameworks_Framework_AllocatedResources: + goto handle_AllocatedResources + + case ffj_t_Response_GetFrameworks_Framework_OfferedResources: + goto handle_OfferedResources + + case ffj_t_Response_GetFrameworks_Frameworkno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_FrameworkInfo: + + /* handler: uj.FrameworkInfo type=mesos.FrameworkInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.FrameworkInfo.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Active: + + /* handler: uj.Active type=bool kind=bool quoted=false*/ + + { + if tok != fflib.FFTok_bool && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for bool", tok)) + } + } + + { + if tok == fflib.FFTok_null { + + } else { + tmpb := fs.Output.Bytes() + + if bytes.Compare([]byte{'t', 'r', 'u', 'e'}, tmpb) == 0 { + + uj.Active = true + + } else if bytes.Compare([]byte{'f', 'a', 'l', 's', 'e'}, tmpb) == 0 { + + uj.Active = false + + } else { + err = errors.New("unexpected bytes for true/false value") + return fs.WrapErr(err) + } + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Connected: + + /* handler: uj.Connected type=bool kind=bool quoted=false*/ + + { + if tok != fflib.FFTok_bool && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for bool", tok)) + } + } + + { + if tok == fflib.FFTok_null { + + } else { + tmpb := fs.Output.Bytes() + + if bytes.Compare([]byte{'t', 'r', 'u', 'e'}, tmpb) == 0 { + + uj.Connected = true + + } else if bytes.Compare([]byte{'f', 'a', 'l', 's', 'e'}, tmpb) == 0 { + + uj.Connected = false + + } else { + err = errors.New("unexpected bytes for true/false value") + return fs.WrapErr(err) + } + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Recovered: + + /* handler: uj.Recovered type=bool kind=bool quoted=false*/ + + { + if tok != fflib.FFTok_bool && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for bool", tok)) + } + } + + { + if tok == fflib.FFTok_null { + + } else { + tmpb := fs.Output.Bytes() + + if bytes.Compare([]byte{'t', 'r', 'u', 'e'}, tmpb) == 0 { + + uj.Recovered = true + + } else if bytes.Compare([]byte{'f', 'a', 'l', 's', 'e'}, tmpb) == 0 { + + uj.Recovered = false + + } else { + err = errors.New("unexpected bytes for true/false value") + return fs.WrapErr(err) + } + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_RegisteredTime: + + /* handler: uj.RegisteredTime type=mesos.TimeInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.RegisteredTime = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.RegisteredTime == nil { + uj.RegisteredTime = new(mesos.TimeInfo) + } + + err = uj.RegisteredTime.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_ReregisteredTime: + + /* handler: uj.ReregisteredTime type=mesos.TimeInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.ReregisteredTime = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.ReregisteredTime == nil { + uj.ReregisteredTime = new(mesos.TimeInfo) + } + + err = uj.ReregisteredTime.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_UnregisteredTime: + + /* handler: uj.UnregisteredTime type=mesos.TimeInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.UnregisteredTime = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.UnregisteredTime == nil { + uj.UnregisteredTime = new(mesos.TimeInfo) + } + + err = uj.UnregisteredTime.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Offers: + + /* handler: uj.Offers type=[]mesos.Offer kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Offers = nil + } else { + + uj.Offers = []mesos.Offer{} + + wantVal := true + + for { + + var tmp_uj__Offers mesos.Offer + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Offers type=mesos.Offer kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Offers.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Offers = append(uj.Offers, tmp_uj__Offers) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_InverseOffers: + + /* handler: uj.InverseOffers type=[]mesos.InverseOffer kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.InverseOffers = nil + } else { + + uj.InverseOffers = []mesos.InverseOffer{} + + wantVal := true + + for { + + var tmp_uj__InverseOffers mesos.InverseOffer + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__InverseOffers type=mesos.InverseOffer kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__InverseOffers.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.InverseOffers = append(uj.InverseOffers, tmp_uj__InverseOffers) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_AllocatedResources: + + /* handler: uj.AllocatedResources type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.AllocatedResources = nil + } else { + + uj.AllocatedResources = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__AllocatedResources mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__AllocatedResources type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__AllocatedResources.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.AllocatedResources = append(uj.AllocatedResources, tmp_uj__AllocatedResources) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_OfferedResources: + + /* handler: uj.OfferedResources type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.OfferedResources = nil + } else { + + uj.OfferedResources = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__OfferedResources mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__OfferedResources type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__OfferedResources.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.OfferedResources = append(uj.OfferedResources, tmp_uj__OfferedResources) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetHealth) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetHealth) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + if mj.Healthy { + buf.WriteString(`{"healthy":true`) + } else { + buf.WriteString(`{"healthy":false`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetHealthbase = iota + ffj_t_Response_GetHealthno_such_key + + ffj_t_Response_GetHealth_Healthy +) + +var ffj_key_Response_GetHealth_Healthy = []byte("healthy") + +func (uj *Response_GetHealth) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetHealth) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetHealthbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetHealthno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'h': + + if bytes.Equal(ffj_key_Response_GetHealth_Healthy, kn) { + currentKey = ffj_t_Response_GetHealth_Healthy + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Response_GetHealth_Healthy, kn) { + currentKey = ffj_t_Response_GetHealth_Healthy + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetHealthno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetHealth_Healthy: + goto handle_Healthy + + case ffj_t_Response_GetHealthno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Healthy: + + /* handler: uj.Healthy type=bool kind=bool quoted=false*/ + + { + if tok != fflib.FFTok_bool && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for bool", tok)) + } + } + + { + if tok == fflib.FFTok_null { + + } else { + tmpb := fs.Output.Bytes() + + if bytes.Compare([]byte{'t', 'r', 'u', 'e'}, tmpb) == 0 { + + uj.Healthy = true + + } else if bytes.Compare([]byte{'f', 'a', 'l', 's', 'e'}, tmpb) == 0 { + + uj.Healthy = false + + } else { + err = errors.New("unexpected bytes for true/false value") + return fs.WrapErr(err) + } + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetLoggingLevel) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetLoggingLevel) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"level":`) + fflib.FormatBits2(buf, uint64(mj.Level), 10, false) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetLoggingLevelbase = iota + ffj_t_Response_GetLoggingLevelno_such_key + + ffj_t_Response_GetLoggingLevel_Level +) + +var ffj_key_Response_GetLoggingLevel_Level = []byte("level") + +func (uj *Response_GetLoggingLevel) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetLoggingLevel) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetLoggingLevelbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetLoggingLevelno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'l': + + if bytes.Equal(ffj_key_Response_GetLoggingLevel_Level, kn) { + currentKey = ffj_t_Response_GetLoggingLevel_Level + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Response_GetLoggingLevel_Level, kn) { + currentKey = ffj_t_Response_GetLoggingLevel_Level + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetLoggingLevelno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetLoggingLevel_Level: + goto handle_Level + + case ffj_t_Response_GetLoggingLevelno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Level: + + /* handler: uj.Level type=uint32 kind=uint32 quoted=false*/ + + { + if tok != fflib.FFTok_integer && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for uint32", tok)) + } + } + + { + + if tok == fflib.FFTok_null { + + } else { + + tval, err := fflib.ParseUint(fs.Output.Bytes(), 10, 32) + + if err != nil { + return fs.WrapErr(err) + } + + uj.Level = uint32(tval) + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetMaintenanceSchedule) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetMaintenanceSchedule) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"schedule":`) + + { + + err = mj.Schedule.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetMaintenanceSchedulebase = iota + ffj_t_Response_GetMaintenanceScheduleno_such_key + + ffj_t_Response_GetMaintenanceSchedule_Schedule +) + +var ffj_key_Response_GetMaintenanceSchedule_Schedule = []byte("schedule") + +func (uj *Response_GetMaintenanceSchedule) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetMaintenanceSchedule) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetMaintenanceSchedulebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetMaintenanceScheduleno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 's': + + if bytes.Equal(ffj_key_Response_GetMaintenanceSchedule_Schedule, kn) { + currentKey = ffj_t_Response_GetMaintenanceSchedule_Schedule + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetMaintenanceSchedule_Schedule, kn) { + currentKey = ffj_t_Response_GetMaintenanceSchedule_Schedule + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetMaintenanceScheduleno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetMaintenanceSchedule_Schedule: + goto handle_Schedule + + case ffj_t_Response_GetMaintenanceScheduleno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Schedule: + + /* handler: uj.Schedule type=maintenance.Schedule kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Schedule.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetMaintenanceStatus) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetMaintenanceStatus) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"status":`) + + { + + err = mj.Status.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetMaintenanceStatusbase = iota + ffj_t_Response_GetMaintenanceStatusno_such_key + + ffj_t_Response_GetMaintenanceStatus_Status +) + +var ffj_key_Response_GetMaintenanceStatus_Status = []byte("status") + +func (uj *Response_GetMaintenanceStatus) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetMaintenanceStatus) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetMaintenanceStatusbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetMaintenanceStatusno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 's': + + if bytes.Equal(ffj_key_Response_GetMaintenanceStatus_Status, kn) { + currentKey = ffj_t_Response_GetMaintenanceStatus_Status + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetMaintenanceStatus_Status, kn) { + currentKey = ffj_t_Response_GetMaintenanceStatus_Status + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetMaintenanceStatusno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetMaintenanceStatus_Status: + goto handle_Status + + case ffj_t_Response_GetMaintenanceStatusno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Status: + + /* handler: uj.Status type=maintenance.ClusterStatus kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Status.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetMaster) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetMaster) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ `) + if mj.MasterInfo != nil { + if true { + buf.WriteString(`"master_info":`) + + { + + err = mj.MasterInfo.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetMasterbase = iota + ffj_t_Response_GetMasterno_such_key + + ffj_t_Response_GetMaster_MasterInfo +) + +var ffj_key_Response_GetMaster_MasterInfo = []byte("master_info") + +func (uj *Response_GetMaster) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetMaster) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetMasterbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetMasterno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'm': + + if bytes.Equal(ffj_key_Response_GetMaster_MasterInfo, kn) { + currentKey = ffj_t_Response_GetMaster_MasterInfo + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetMaster_MasterInfo, kn) { + currentKey = ffj_t_Response_GetMaster_MasterInfo + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetMasterno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetMaster_MasterInfo: + goto handle_MasterInfo + + case ffj_t_Response_GetMasterno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_MasterInfo: + + /* handler: uj.MasterInfo type=mesos.MasterInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.MasterInfo = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.MasterInfo == nil { + uj.MasterInfo = new(mesos.MasterInfo) + } + + err = uj.MasterInfo.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetMetrics) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetMetrics) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"metrics":`) + if mj.Metrics != nil { + buf.WriteString(`[`) + for i, v := range mj.Metrics { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetMetricsbase = iota + ffj_t_Response_GetMetricsno_such_key + + ffj_t_Response_GetMetrics_Metrics +) + +var ffj_key_Response_GetMetrics_Metrics = []byte("metrics") + +func (uj *Response_GetMetrics) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetMetrics) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetMetricsbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetMetricsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'm': + + if bytes.Equal(ffj_key_Response_GetMetrics_Metrics, kn) { + currentKey = ffj_t_Response_GetMetrics_Metrics + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetMetrics_Metrics, kn) { + currentKey = ffj_t_Response_GetMetrics_Metrics + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetMetricsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetMetrics_Metrics: + goto handle_Metrics + + case ffj_t_Response_GetMetricsno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Metrics: + + /* handler: uj.Metrics type=[]mesos.Metric kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Metrics = nil + } else { + + uj.Metrics = []mesos.Metric{} + + wantVal := true + + for { + + var tmp_uj__Metrics mesos.Metric + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Metrics type=mesos.Metric kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Metrics.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Metrics = append(uj.Metrics, tmp_uj__Metrics) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetQuota) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetQuota) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"status":`) + + { + + err = mj.Status.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetQuotabase = iota + ffj_t_Response_GetQuotano_such_key + + ffj_t_Response_GetQuota_Status +) + +var ffj_key_Response_GetQuota_Status = []byte("status") + +func (uj *Response_GetQuota) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetQuota) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetQuotabase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetQuotano_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 's': + + if bytes.Equal(ffj_key_Response_GetQuota_Status, kn) { + currentKey = ffj_t_Response_GetQuota_Status + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetQuota_Status, kn) { + currentKey = ffj_t_Response_GetQuota_Status + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetQuotano_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetQuota_Status: + goto handle_Status + + case ffj_t_Response_GetQuotano_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Status: + + /* handler: uj.Status type=quota.QuotaStatus kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.Status.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetRoles) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetRoles) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"roles":`) + if mj.Roles != nil { + buf.WriteString(`[`) + for i, v := range mj.Roles { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetRolesbase = iota + ffj_t_Response_GetRolesno_such_key + + ffj_t_Response_GetRoles_Roles +) + +var ffj_key_Response_GetRoles_Roles = []byte("roles") + +func (uj *Response_GetRoles) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetRoles) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetRolesbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetRolesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'r': + + if bytes.Equal(ffj_key_Response_GetRoles_Roles, kn) { + currentKey = ffj_t_Response_GetRoles_Roles + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetRoles_Roles, kn) { + currentKey = ffj_t_Response_GetRoles_Roles + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetRolesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetRoles_Roles: + goto handle_Roles + + case ffj_t_Response_GetRolesno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Roles: + + /* handler: uj.Roles type=[]mesos.Role kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Roles = nil + } else { + + uj.Roles = []mesos.Role{} + + wantVal := true + + for { + + var tmp_uj__Roles mesos.Role + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Roles type=mesos.Role kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Roles.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Roles = append(uj.Roles, tmp_uj__Roles) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetState) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetState) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ `) + if mj.GetTasks != nil { + if true { + buf.WriteString(`"get_tasks":`) + + { + + err = mj.GetTasks.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetExecutors != nil { + if true { + buf.WriteString(`"get_executors":`) + + { + + err = mj.GetExecutors.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetFrameworks != nil { + if true { + buf.WriteString(`"get_frameworks":`) + + { + + err = mj.GetFrameworks.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + if mj.GetAgents != nil { + if true { + buf.WriteString(`"get_agents":`) + + { + + err = mj.GetAgents.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte(',') + } + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetStatebase = iota + ffj_t_Response_GetStateno_such_key + + ffj_t_Response_GetState_GetTasks + + ffj_t_Response_GetState_GetExecutors + + ffj_t_Response_GetState_GetFrameworks + + ffj_t_Response_GetState_GetAgents +) + +var ffj_key_Response_GetState_GetTasks = []byte("get_tasks") + +var ffj_key_Response_GetState_GetExecutors = []byte("get_executors") + +var ffj_key_Response_GetState_GetFrameworks = []byte("get_frameworks") + +var ffj_key_Response_GetState_GetAgents = []byte("get_agents") + +func (uj *Response_GetState) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetState) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetStatebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetStateno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'g': + + if bytes.Equal(ffj_key_Response_GetState_GetTasks, kn) { + currentKey = ffj_t_Response_GetState_GetTasks + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetState_GetExecutors, kn) { + currentKey = ffj_t_Response_GetState_GetExecutors + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetState_GetFrameworks, kn) { + currentKey = ffj_t_Response_GetState_GetFrameworks + state = fflib.FFParse_want_colon + goto mainparse + + } else if bytes.Equal(ffj_key_Response_GetState_GetAgents, kn) { + currentKey = ffj_t_Response_GetState_GetAgents + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetState_GetAgents, kn) { + currentKey = ffj_t_Response_GetState_GetAgents + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetState_GetFrameworks, kn) { + currentKey = ffj_t_Response_GetState_GetFrameworks + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetState_GetExecutors, kn) { + currentKey = ffj_t_Response_GetState_GetExecutors + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetState_GetTasks, kn) { + currentKey = ffj_t_Response_GetState_GetTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetStateno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetState_GetTasks: + goto handle_GetTasks + + case ffj_t_Response_GetState_GetExecutors: + goto handle_GetExecutors + + case ffj_t_Response_GetState_GetFrameworks: + goto handle_GetFrameworks + + case ffj_t_Response_GetState_GetAgents: + goto handle_GetAgents + + case ffj_t_Response_GetStateno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_GetTasks: + + /* handler: uj.GetTasks type=master.Response_GetTasks kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetTasks = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetTasks == nil { + uj.GetTasks = new(Response_GetTasks) + } + + err = uj.GetTasks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetExecutors: + + /* handler: uj.GetExecutors type=master.Response_GetExecutors kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetExecutors = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetExecutors == nil { + uj.GetExecutors = new(Response_GetExecutors) + } + + err = uj.GetExecutors.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetFrameworks: + + /* handler: uj.GetFrameworks type=master.Response_GetFrameworks kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetFrameworks = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetFrameworks == nil { + uj.GetFrameworks = new(Response_GetFrameworks) + } + + err = uj.GetFrameworks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_GetAgents: + + /* handler: uj.GetAgents type=master.Response_GetAgents kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + uj.GetAgents = nil + + state = fflib.FFParse_after_value + goto mainparse + } + + if uj.GetAgents == nil { + uj.GetAgents = new(Response_GetAgents) + } + + err = uj.GetAgents.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetTasks) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetTasks) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"pending_tasks":`) + if mj.PendingTasks != nil { + buf.WriteString(`[`) + for i, v := range mj.PendingTasks { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"tasks":`) + if mj.Tasks != nil { + buf.WriteString(`[`) + for i, v := range mj.Tasks { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"unreachable_tasks":`) + if mj.UnreachableTasks != nil { + buf.WriteString(`[`) + for i, v := range mj.UnreachableTasks { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"completed_tasks":`) + if mj.CompletedTasks != nil { + buf.WriteString(`[`) + for i, v := range mj.CompletedTasks { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteString(`,"orphan_tasks":`) + if mj.OrphanTasks != nil { + buf.WriteString(`[`) + for i, v := range mj.OrphanTasks { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetTasksbase = iota + ffj_t_Response_GetTasksno_such_key + + ffj_t_Response_GetTasks_PendingTasks + + ffj_t_Response_GetTasks_Tasks + + ffj_t_Response_GetTasks_UnreachableTasks + + ffj_t_Response_GetTasks_CompletedTasks + + ffj_t_Response_GetTasks_OrphanTasks +) + +var ffj_key_Response_GetTasks_PendingTasks = []byte("pending_tasks") + +var ffj_key_Response_GetTasks_Tasks = []byte("tasks") + +var ffj_key_Response_GetTasks_UnreachableTasks = []byte("unreachable_tasks") + +var ffj_key_Response_GetTasks_CompletedTasks = []byte("completed_tasks") + +var ffj_key_Response_GetTasks_OrphanTasks = []byte("orphan_tasks") + +func (uj *Response_GetTasks) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetTasks) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetTasksbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetTasksno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'c': + + if bytes.Equal(ffj_key_Response_GetTasks_CompletedTasks, kn) { + currentKey = ffj_t_Response_GetTasks_CompletedTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'o': + + if bytes.Equal(ffj_key_Response_GetTasks_OrphanTasks, kn) { + currentKey = ffj_t_Response_GetTasks_OrphanTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'p': + + if bytes.Equal(ffj_key_Response_GetTasks_PendingTasks, kn) { + currentKey = ffj_t_Response_GetTasks_PendingTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + case 't': + + if bytes.Equal(ffj_key_Response_GetTasks_Tasks, kn) { + currentKey = ffj_t_Response_GetTasks_Tasks + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'u': + + if bytes.Equal(ffj_key_Response_GetTasks_UnreachableTasks, kn) { + currentKey = ffj_t_Response_GetTasks_UnreachableTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetTasks_OrphanTasks, kn) { + currentKey = ffj_t_Response_GetTasks_OrphanTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetTasks_CompletedTasks, kn) { + currentKey = ffj_t_Response_GetTasks_CompletedTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetTasks_UnreachableTasks, kn) { + currentKey = ffj_t_Response_GetTasks_UnreachableTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetTasks_Tasks, kn) { + currentKey = ffj_t_Response_GetTasks_Tasks + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_GetTasks_PendingTasks, kn) { + currentKey = ffj_t_Response_GetTasks_PendingTasks + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetTasksno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetTasks_PendingTasks: + goto handle_PendingTasks + + case ffj_t_Response_GetTasks_Tasks: + goto handle_Tasks + + case ffj_t_Response_GetTasks_UnreachableTasks: + goto handle_UnreachableTasks + + case ffj_t_Response_GetTasks_CompletedTasks: + goto handle_CompletedTasks + + case ffj_t_Response_GetTasks_OrphanTasks: + goto handle_OrphanTasks + + case ffj_t_Response_GetTasksno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_PendingTasks: + + /* handler: uj.PendingTasks type=[]mesos.Task kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.PendingTasks = nil + } else { + + uj.PendingTasks = []mesos.Task{} + + wantVal := true + + for { + + var tmp_uj__PendingTasks mesos.Task + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__PendingTasks type=mesos.Task kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__PendingTasks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.PendingTasks = append(uj.PendingTasks, tmp_uj__PendingTasks) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Tasks: + + /* handler: uj.Tasks type=[]mesos.Task kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Tasks = nil + } else { + + uj.Tasks = []mesos.Task{} + + wantVal := true + + for { + + var tmp_uj__Tasks mesos.Task + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Tasks type=mesos.Task kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Tasks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Tasks = append(uj.Tasks, tmp_uj__Tasks) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_UnreachableTasks: + + /* handler: uj.UnreachableTasks type=[]mesos.Task kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.UnreachableTasks = nil + } else { + + uj.UnreachableTasks = []mesos.Task{} + + wantVal := true + + for { + + var tmp_uj__UnreachableTasks mesos.Task + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__UnreachableTasks type=mesos.Task kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__UnreachableTasks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.UnreachableTasks = append(uj.UnreachableTasks, tmp_uj__UnreachableTasks) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_CompletedTasks: + + /* handler: uj.CompletedTasks type=[]mesos.Task kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.CompletedTasks = nil + } else { + + uj.CompletedTasks = []mesos.Task{} + + wantVal := true + + for { + + var tmp_uj__CompletedTasks mesos.Task + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__CompletedTasks type=mesos.Task kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__CompletedTasks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.CompletedTasks = append(uj.CompletedTasks, tmp_uj__CompletedTasks) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_OrphanTasks: + + /* handler: uj.OrphanTasks type=[]mesos.Task kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.OrphanTasks = nil + } else { + + uj.OrphanTasks = []mesos.Task{} + + wantVal := true + + for { + + var tmp_uj__OrphanTasks mesos.Task + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__OrphanTasks type=mesos.Task kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__OrphanTasks.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.OrphanTasks = append(uj.OrphanTasks, tmp_uj__OrphanTasks) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetVersion) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetVersion) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"version_info":`) + + { + + err = mj.VersionInfo.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetVersionbase = iota + ffj_t_Response_GetVersionno_such_key + + ffj_t_Response_GetVersion_VersionInfo +) + +var ffj_key_Response_GetVersion_VersionInfo = []byte("version_info") + +func (uj *Response_GetVersion) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetVersion) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetVersionbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetVersionno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'v': + + if bytes.Equal(ffj_key_Response_GetVersion_VersionInfo, kn) { + currentKey = ffj_t_Response_GetVersion_VersionInfo + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetVersion_VersionInfo, kn) { + currentKey = ffj_t_Response_GetVersion_VersionInfo + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetVersionno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetVersion_VersionInfo: + goto handle_VersionInfo + + case ffj_t_Response_GetVersionno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_VersionInfo: + + /* handler: uj.VersionInfo type=mesos.VersionInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = uj.VersionInfo.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_GetWeights) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_GetWeights) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"weight_infos":`) + if mj.WeightInfos != nil { + buf.WriteString(`[`) + for i, v := range mj.WeightInfos { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_GetWeightsbase = iota + ffj_t_Response_GetWeightsno_such_key + + ffj_t_Response_GetWeights_WeightInfos +) + +var ffj_key_Response_GetWeights_WeightInfos = []byte("weight_infos") + +func (uj *Response_GetWeights) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_GetWeights) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_GetWeightsbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_GetWeightsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'w': + + if bytes.Equal(ffj_key_Response_GetWeights_WeightInfos, kn) { + currentKey = ffj_t_Response_GetWeights_WeightInfos + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_GetWeights_WeightInfos, kn) { + currentKey = ffj_t_Response_GetWeights_WeightInfos + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_GetWeightsno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_GetWeights_WeightInfos: + goto handle_WeightInfos + + case ffj_t_Response_GetWeightsno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_WeightInfos: + + /* handler: uj.WeightInfos type=[]mesos.WeightInfo kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.WeightInfos = nil + } else { + + uj.WeightInfos = []mesos.WeightInfo{} + + wantVal := true + + for { + + var tmp_uj__WeightInfos mesos.WeightInfo + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__WeightInfos type=mesos.WeightInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__WeightInfos.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.WeightInfos = append(uj.WeightInfos, tmp_uj__WeightInfos) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_ListFiles) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_ListFiles) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"file_infos":`) + if mj.FileInfos != nil { + buf.WriteString(`[`) + for i, v := range mj.FileInfos { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_ListFilesbase = iota + ffj_t_Response_ListFilesno_such_key + + ffj_t_Response_ListFiles_FileInfos +) + +var ffj_key_Response_ListFiles_FileInfos = []byte("file_infos") + +func (uj *Response_ListFiles) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_ListFiles) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_ListFilesbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_ListFilesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'f': + + if bytes.Equal(ffj_key_Response_ListFiles_FileInfos, kn) { + currentKey = ffj_t_Response_ListFiles_FileInfos + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_Response_ListFiles_FileInfos, kn) { + currentKey = ffj_t_Response_ListFiles_FileInfos + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_ListFilesno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_ListFiles_FileInfos: + goto handle_FileInfos + + case ffj_t_Response_ListFilesno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_FileInfos: + + /* handler: uj.FileInfos type=[]mesos.FileInfo kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.FileInfos = nil + } else { + + uj.FileInfos = []mesos.FileInfo{} + + wantVal := true + + for { + + var tmp_uj__FileInfos mesos.FileInfo + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__FileInfos type=mesos.FileInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__FileInfos.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.FileInfos = append(uj.FileInfos, tmp_uj__FileInfos) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *Response_ReadFile) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *Response_ReadFile) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{ "size":`) + fflib.FormatBits2(buf, uint64(mj.Size), 10, false) + buf.WriteByte(',') + if len(mj.Data) != 0 { + buf.WriteString(`"data":`) + if mj.Data != nil { + buf.WriteString(`"`) + { + enc := base64.NewEncoder(base64.StdEncoding, buf) + enc.Write(reflect.Indirect(reflect.ValueOf(mj.Data)).Bytes()) + enc.Close() + } + buf.WriteString(`"`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte(',') + } + buf.Rewind(1) + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_Response_ReadFilebase = iota + ffj_t_Response_ReadFileno_such_key + + ffj_t_Response_ReadFile_Size + + ffj_t_Response_ReadFile_Data +) + +var ffj_key_Response_ReadFile_Size = []byte("size") + +var ffj_key_Response_ReadFile_Data = []byte("data") + +func (uj *Response_ReadFile) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *Response_ReadFile) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_Response_ReadFilebase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_Response_ReadFileno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'd': + + if bytes.Equal(ffj_key_Response_ReadFile_Data, kn) { + currentKey = ffj_t_Response_ReadFile_Data + state = fflib.FFParse_want_colon + goto mainparse + } + + case 's': + + if bytes.Equal(ffj_key_Response_ReadFile_Size, kn) { + currentKey = ffj_t_Response_ReadFile_Size + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_Response_ReadFile_Data, kn) { + currentKey = ffj_t_Response_ReadFile_Data + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.EqualFoldRight(ffj_key_Response_ReadFile_Size, kn) { + currentKey = ffj_t_Response_ReadFile_Size + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_Response_ReadFileno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_Response_ReadFile_Size: + goto handle_Size + + case ffj_t_Response_ReadFile_Data: + goto handle_Data + + case ffj_t_Response_ReadFileno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Size: + + /* handler: uj.Size type=uint64 kind=uint64 quoted=false*/ + + { + if tok != fflib.FFTok_integer && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for uint64", tok)) + } + } + + { + + if tok == fflib.FFTok_null { + + } else { + + tval, err := fflib.ParseUint(fs.Output.Bytes(), 10, 64) + + if err != nil { + return fs.WrapErr(err) + } + + uj.Size = uint64(tval) + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Data: + + /* handler: uj.Data type=[]uint8 kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_string && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Data = nil + } else { + b := make([]byte, base64.StdEncoding.DecodedLen(fs.Output.Len())) + n, err := base64.StdEncoding.Decode(b, fs.Output.Bytes()) + if err != nil { + return fs.WrapErr(err) + } + + v := reflect.ValueOf(&uj.Data).Elem() + v.SetBytes(b[0:n]) + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} diff --git a/api/v1/lib/master/master.proto b/api/v1/lib/master/master.proto new file mode 100644 index 00000000..02439c98 --- /dev/null +++ b/api/v1/lib/master/master.proto @@ -0,0 +1,560 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto2"; + +package mesos.master; + +import "github.com/mesos/mesos-go/api/v1/lib/mesos.proto"; +import "github.com/mesos/mesos-go/api/v1/lib/allocator/allocator.proto"; +import "github.com/mesos/mesos-go/api/v1/lib/maintenance/maintenance.proto"; +import "github.com/mesos/mesos-go/api/v1/lib/quota/quota.proto"; +import "github.com/gogo/protobuf/gogoproto/gogo.proto"; + +option go_package = "master"; +option (gogoproto.benchgen_all) = true; +option (gogoproto.enum_stringer_all) = true; +option (gogoproto.equal_all) = true; +option (gogoproto.goproto_enum_prefix_all) = false; +option (gogoproto.goproto_enum_stringer_all) = false; +option (gogoproto.goproto_stringer_all) = false; +option (gogoproto.goproto_unrecognized_all) = false; +option (gogoproto.gostring_all) = true; +option (gogoproto.marshaler_all) = true; +option (gogoproto.populate_all) = true; +option (gogoproto.protosizer_all) = true; +option (gogoproto.stringer_all) = true; +option (gogoproto.testgen_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.verbose_equal_all) = true; + +/** + * Calls that can be sent to the v1 master API. + * + * A call is described using the standard protocol buffer "union" + * trick, see + * https://developers.google.com/protocol-buffers/docs/techniques#union. + */ +message Call { + enum Type { + // If a call of type `Call::FOO` requires additional parameters they can be + // included in the corresponding `Call::Foo` message. Similarly, if a call + // receives a synchronous response it will be returned as a `Response` + // message of type `Response::FOO`. Currently all calls except + // `Call::SUBSCRIBE` receive synchronous responses; `Call::SUBSCRIBE` returns + // a streaming response of `Event`. + UNKNOWN = 0; + + GET_HEALTH = 1; // Retrieves the master's health status. + GET_FLAGS = 2; // Retrieves the master's flag configuration. + GET_VERSION = 3; // Retrieves the master's version information. + GET_METRICS = 4; // See 'GetMetrics' below. + + GET_LOGGING_LEVEL = 5; // Retrieves the master's logging level. + SET_LOGGING_LEVEL = 6; // See 'SetLoggingLevel' below. + + LIST_FILES = 7; + READ_FILE = 8; // See 'ReadFile' below. + + GET_STATE = 9; + + GET_AGENTS = 10; + GET_FRAMEWORKS = 11; + GET_EXECUTORS = 12; // Retrieves the information about all executors. + GET_TASKS = 13; // Retrieves the information about all known tasks. + GET_ROLES = 14; // Retrieves the information about roles. + + GET_WEIGHTS = 15; // Retrieves the information about role weights. + UPDATE_WEIGHTS = 16; + + GET_MASTER = 17; // Retrieves the master's information. + + SUBSCRIBE = 18; // Subscribes the master to receive events. + + RESERVE_RESOURCES = 19; + UNRESERVE_RESOURCES = 20; + + CREATE_VOLUMES = 21; // See 'CreateVolumes' below. + DESTROY_VOLUMES = 22; // See 'DestroyVolumes' below. + + // Retrieves the cluster's maintenance status. + GET_MAINTENANCE_STATUS = 23; + // Retrieves the cluster's maintenance schedule. + GET_MAINTENANCE_SCHEDULE = 24; + UPDATE_MAINTENANCE_SCHEDULE = 25; // See 'UpdateMaintenanceSchedule' below. + START_MAINTENANCE = 26; // See 'StartMaintenance' below. + STOP_MAINTENANCE = 27; // See 'StopMaintenance' below. + + GET_QUOTA = 28; + SET_QUOTA = 29; // See 'SetQuota' below. + REMOVE_QUOTA = 30; // See 'RemoveQuota' below. + + option (gogoproto.goproto_enum_prefix) = true; + } + + // Provides a snapshot of the current metrics tracked by the master. + message GetMetrics { + // If set, `timeout` would be used to determines the maximum amount of time + // the API will take to respond. If the timeout is exceeded, some metrics + // may not be included in the response. + optional DurationInfo timeout = 1; + } + + // Sets the logging verbosity level for a specified duration. Mesos uses + // [glog](https://github.com/google/glog) for logging. The library only uses + // verbose logging which means nothing will be output unless the verbosity + // level is set (by default it's 0, libprocess uses levels 1, 2, and 3). + message SetLoggingLevel { + // The verbosity level. + required uint32 level = 1 [(gogoproto.nullable) = false]; + // The duration to keep verbosity level toggled. After this duration, the + // verbosity level of log would revert to the original level. + required DurationInfo duration = 2 [(gogoproto.nullable) = false]; + } + + // Provides the file listing for a directory. + message ListFiles { + required string path = 1 [(gogoproto.nullable) = false]; + } + + // Reads data from a file. + message ReadFile { + // The path of file. + required string path = 1 [(gogoproto.nullable) = false]; + + // Initial offset in file to start reading from. + required uint64 offset = 2 [(gogoproto.nullable) = false]; + + // The maximum number of bytes to read. The read length is capped at 16 + // memory pages. + optional uint64 length = 3; + } + + message UpdateWeights { + repeated WeightInfo weight_infos = 1 [(gogoproto.nullable) = false]; + } + + // Reserve resources dynamically on a specific agent. + message ReserveResources { + required AgentID agent_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "AgentID"]; + repeated Resource resources = 2 [(gogoproto.nullable) = false]; + } + + // Unreserve resources dynamically on a specific agent. + message UnreserveResources { + required AgentID agent_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "AgentID"]; + repeated Resource resources = 2 [(gogoproto.nullable) = false]; + } + + // Create persistent volumes on reserved resources. The request is forwarded + // asynchronously to the Mesos agent where the reserved resources are located. + // That asynchronous message may not be delivered or creating the volumes at + // the agent might fail. Volume creation can be verified by sending a + // `GET_VOLUMES` call. + message CreateVolumes { + required AgentID agent_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "AgentID"]; + repeated Resource volumes = 2 [(gogoproto.nullable) = false]; + } + + // Destroy persistent volumes. The request is forwarded asynchronously to the + // Mesos agent where the reserved resources are located. That asynchronous + // message may not be delivered or destroying the volumes at the agent might + // fail. Volume deletion can be verified by sending a `GET_VOLUMES` call. + message DestroyVolumes { + required AgentID agent_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "AgentID"]; + repeated Resource volumes = 2 [(gogoproto.nullable) = false]; + } + + // Updates the cluster's maintenance schedule. + message UpdateMaintenanceSchedule { + required maintenance.Schedule schedule = 1 [(gogoproto.nullable) = false]; + } + + // Starts the maintenance of the cluster, this would bring a set of machines + // down. + message StartMaintenance { + repeated MachineID machines = 1 [(gogoproto.nullable) = false]; + } + + // Stops the maintenance of the cluster, this would bring a set of machines + // back up. + message StopMaintenance { + repeated MachineID machines = 1 [(gogoproto.nullable) = false]; + } + + // Sets the quota for resources to be used by a particular role. + message SetQuota { + required quota.QuotaRequest quota_request = 1 [(gogoproto.nullable) = false]; + } + + message RemoveQuota { + required string role = 1 [(gogoproto.nullable) = false]; + } + + optional Type type = 1 [(gogoproto.nullable) = false]; + + optional GetMetrics get_metrics = 2; + optional SetLoggingLevel set_logging_level = 3; + optional ListFiles list_files = 4; + optional ReadFile read_file = 5; + optional UpdateWeights update_weights = 6; + optional ReserveResources reserve_resources = 7; + optional UnreserveResources unreserve_resources = 8; + optional CreateVolumes create_volumes = 9; + optional DestroyVolumes destroy_volumes = 10; + optional UpdateMaintenanceSchedule update_maintenance_schedule = 11; + optional StartMaintenance start_maintenance = 12; + optional StopMaintenance stop_maintenance = 13; + optional SetQuota set_quota = 14; + optional RemoveQuota remove_quota = 15; +} + + +/** + * Synchronous responses for all calls (except Call::SUBSCRIBE) made to + * the v1 master API. + */ +message Response { + // Each of the responses of type `FOO` corresponds to `Foo` message below. + enum Type { + UNKNOWN = 0; + + GET_HEALTH = 1; // See 'GetHealth' below. + GET_FLAGS = 2; // See 'GetFlags' below. + GET_VERSION = 3; // See 'GetVersion' below. + GET_METRICS = 4; // See 'GetMetrics' below. + + GET_LOGGING_LEVEL = 5; // See 'GetLoggingLevel' below. + + LIST_FILES = 6; + READ_FILE = 7; // See 'ReadFile' below. + + GET_STATE = 8; + + GET_AGENTS = 9; + GET_FRAMEWORKS = 10; + GET_EXECUTORS = 11; // See 'GetExecutors' below. + GET_TASKS = 12; // See 'GetTasks' below. + GET_ROLES = 13; // See 'GetRoles' below. + + GET_WEIGHTS = 14; // See 'GetWeights' below. + + GET_MASTER = 15; // See 'GetMaster' below. + + GET_MAINTENANCE_STATUS = 16; // See 'GetMaintenanceStatus' below. + GET_MAINTENANCE_SCHEDULE = 17; // See 'GetMaintenanceSchedule' below. + + GET_QUOTA = 18; + + option (gogoproto.goproto_enum_prefix) = true; + } + + // `healthy` would be true if the master is healthy. Delayed responses are + // also indicative of the poor health of the master. + message GetHealth { + required bool healthy = 1 [(gogoproto.nullable) = false]; + } + + // Contains the flag configuration of the master. + message GetFlags { + repeated Flag flags = 1 [(gogoproto.nullable) = false]; + } + + // Contains the version information of the master. + message GetVersion { + required VersionInfo version_info = 1 [(gogoproto.nullable) = false]; + } + + // Contains a snapshot of the current metrics. + message GetMetrics { + repeated Metric metrics = 1 [(gogoproto.nullable) = false]; + } + + // Contains the logging level of the master. + message GetLoggingLevel { + required uint32 level = 1 [(gogoproto.nullable) = false]; + } + + // Contains the file listing(similar to `ls -l`) for a directory. + message ListFiles { + repeated FileInfo file_infos = 1 [(gogoproto.nullable) = false]; + } + + // Contains the file data. + message ReadFile { + // The size of file (in bytes). + required uint64 size = 1 [(gogoproto.nullable) = false]; + + required bytes data = 2; + } + + // Contains full state of the master i.e. information about the tasks, + // agents, frameworks and executors running in the cluster. + message GetState { + optional GetTasks get_tasks = 1; + optional GetExecutors get_executors = 2; + optional GetFrameworks get_frameworks = 3; + optional GetAgents get_agents = 4; + } + + message GetAgents { + message Agent { + required AgentInfo agent_info = 1 [(gogoproto.nullable) = false]; + required bool active = 2 [(gogoproto.nullable) = false]; + required string version = 3 [(gogoproto.nullable) = false]; + + optional string pid = 4 [(gogoproto.customname) = "PID"]; + optional TimeInfo registered_time = 5; + optional TimeInfo reregistered_time = 6; + + // Total resources (including oversubscribed resources) the agent + // provides. + repeated Resource total_resources = 7 [(gogoproto.nullable) = false]; + + repeated Resource allocated_resources = 8 [(gogoproto.nullable) = false]; + repeated Resource offered_resources = 9 [(gogoproto.nullable) = false]; + } + + // Registered agents. + repeated Agent agents = 1 [(gogoproto.nullable) = false]; + + // Agents which are recovered from registry but not reregistered yet. + repeated AgentInfo recovered_agents = 2 [(gogoproto.nullable) = false]; + } + + // Information about all the frameworks known to the master at the current + // time. Note that there might be frameworks unknown to the master running + // on partitioned or unsubscribed agents. + message GetFrameworks { + message Framework { + required FrameworkInfo framework_info = 1 [(gogoproto.nullable) = false]; + required bool active = 2 [(gogoproto.nullable) = false]; + required bool connected = 3 [(gogoproto.nullable) = false]; + + // If true, this framework was previously subscribed but hasn't + // yet re-subscribed after a master failover. Recovered frameworks + // are only reported if one or more agents running a task or + // executor for the framework have re-registered after master + // failover. + required bool recovered = 11 [(gogoproto.nullable) = false]; + + optional TimeInfo registered_time = 4; + optional TimeInfo reregistered_time = 5; + optional TimeInfo unregistered_time = 6; + + repeated Offer offers = 7 [(gogoproto.nullable) = false]; + repeated InverseOffer inverse_offers = 8 [(gogoproto.nullable) = false]; + + repeated Resource allocated_resources = 9 [(gogoproto.nullable) = false]; + repeated Resource offered_resources = 10 [(gogoproto.nullable) = false]; + } + + // Frameworks that have subscribed with the master. Note that this includes + // frameworks that are disconnected and in the process of re-subscribing. + repeated Framework frameworks = 1 [(gogoproto.nullable) = false]; + + // Frameworks that have been teared down. + repeated Framework completed_frameworks = 2 [(gogoproto.nullable) = false]; + + // This field previously contained frameworks that previously + // subscribed but haven't yet re-subscribed after a master failover. + // As of Mesos 1.2, this field will always be empty; recovered + // frameworks are now reported in the `frameworks` list with the + // `recovered` field set to true. + // + // TODO(neilc): Remove this field after a deprecation cycle starting + // in Mesos 1.2. + repeated FrameworkInfo recovered_frameworks = 3 [(gogoproto.nullable) = false]; + } + + // Lists information about all the executors known to the master at the + // current time. Note that there might be executors unknown to the master + // running on partitioned or unsubscribed agents. + message GetExecutors { + message Executor { + required ExecutorInfo executor_info = 1 [(gogoproto.nullable) = false]; + required AgentID agent_id = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "AgentID"]; + } + + repeated Executor executors = 1 [(gogoproto.nullable) = false]; + + // As of Mesos 1.2, this field will always be empty. + // + // TODO(neilc): Remove this field after a deprecation cycle starting + // in Mesos 1.2. + repeated Executor orphan_executors = 2 [(gogoproto.nullable) = false]; + } + + // Lists information about all the tasks known to the master at the current + // time. Note that there might be tasks unknown to the master running on + // partitioned or unsubscribed agents. + message GetTasks { + // Tasks that are enqueued on the master waiting (e.g., authorizing) + // to be launched. + repeated Task pending_tasks = 1 [(gogoproto.nullable) = false]; + + // Tasks that have been forwarded to the agent for launch. This + // includes tasks that are staging or running; it also includes + // tasks that have reached a terminal state but the terminal status + // update has not yet been acknowledged by the scheduler. + repeated Task tasks = 2 [(gogoproto.nullable) = false]; + + // Tasks that were running on agents that have become partitioned + // from the master. If/when the agent is no longer partitioned, + // tasks running on that agent will no longer be unreachable (they + // will either be running or completed). Note that the master only + // stores a limited number of unreachable tasks; information about + // unreachable tasks is also not preserved across master failover. + repeated Task unreachable_tasks = 5 [(gogoproto.nullable) = false]; + + // Tasks that have reached terminal state and have all their updates + // acknowledged by the scheduler. + repeated Task completed_tasks = 3 [(gogoproto.nullable) = false]; + + // As of Mesos 1.2, this field will always be empty. + // + // TODO(neilc): Remove this field after a deprecation cycle starting + // in Mesos 1.2. + repeated Task orphan_tasks = 4 [(gogoproto.nullable) = false]; + } + + // Provides information about every role that is on the role whitelist (if + // enabled), has one or more registered frameworks or has a non-default weight + // or quota. + message GetRoles { + repeated Role roles = 1 [(gogoproto.nullable) = false]; + } + + // Provides the weight information about every role. + message GetWeights { + repeated WeightInfo weight_infos = 1 [(gogoproto.nullable) = false]; + } + + // Contains the master's information. + message GetMaster { + optional MasterInfo master_info = 1; + } + + // Contains the cluster's maintenance status. + message GetMaintenanceStatus { + required maintenance.ClusterStatus status = 1 [(gogoproto.nullable) = false]; + } + + // Contains the cluster's maintenance schedule. + message GetMaintenanceSchedule { + required maintenance.Schedule schedule = 1 [(gogoproto.nullable) = false]; + } + + // Contains the cluster's configured quotas. + message GetQuota { + required quota.QuotaStatus status = 1 [(gogoproto.nullable) = false]; + } + + optional Type type = 1 [(gogoproto.nullable) = false]; + + optional GetHealth get_health = 2; + optional GetFlags get_flags = 3; + optional GetVersion get_version = 4; + optional GetMetrics get_metrics = 5; + optional GetLoggingLevel get_logging_level = 6; + optional ListFiles list_files = 7; + optional ReadFile read_file = 8; + optional GetState get_state = 9; + optional GetAgents get_agents = 10; + optional GetFrameworks get_frameworks = 11; + optional GetExecutors get_executors = 12; + optional GetTasks get_tasks = 13; + optional GetRoles get_roles = 14; + optional GetWeights get_weights = 15; + optional GetMaster get_master = 16; + optional GetMaintenanceStatus get_maintenance_status = 17; + optional GetMaintenanceSchedule get_maintenance_schedule = 18; + optional GetQuota get_quota = 19; +} + + +/** + * Streaming response to `Call::SUBSCRIBE` made to the master. + */ +message Event { + enum Type { + UNKNOWN = 0; + SUBSCRIBED = 1; // See `Subscribed` below. + TASK_ADDED = 2; // See `TaskAdded` below. + TASK_UPDATED = 3; // See `TaskUpdated` below. + AGENT_ADDED = 4; // See `AgentAdded` below. + AGENT_REMOVED = 5; // See `AgentRemoved` below. + + // TODO(vinod): Fill in more events. + + option (gogoproto.goproto_enum_prefix) = true; + } + + // First event received when a client subscribes. + message Subscribed { + // Snapshot of the entire cluster state. Further updates to the + // cluster state are sent as separate events on the stream. + optional Response.GetState get_state = 1; + } + + // Forwarded by the master when a task becomes known to it. This can happen + // when a new task is launched by the scheduler or when the task becomes + // known to the master upon an agent (re-)registration after a failover. + message TaskAdded { + required Task task = 1 [(gogoproto.nullable) = false]; + } + + // Forwarded by the master when an existing task transitions to a new state. + message TaskUpdated { + required FrameworkID framework_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "FrameworkID"]; + + // This is the status of the task corresponding to the last + // status update acknowledged by the scheduler. + required TaskStatus status = 2 [(gogoproto.nullable) = false]; + + // This is the latest state of the task according to the agent. + required TaskState state = 3; + } + + // Forwarded by the master when an agent becomes known to it. + // This can happen when an agent registered for the first + // time, or reregistered after a master failover. + message AgentAdded { + required Response.GetAgents.Agent agent = 1 [(gogoproto.nullable) = false]; + } + + // Forwarded by the master when an agent is removed. + // This can happen when an agent does not re-register + // within `--agent_reregister_timeout` upon a master failover, + // or when the agent is scheduled for maintenance. + // + // NOTE: It's possible that an agent might become + // active once it has been removed, i.e. if the master + // has gc'ed its list of known "dead" agents. + // See MESOS-5965 for context. + message AgentRemoved { + required AgentID agent_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "AgentID"]; + } + + optional Type type = 1 [(gogoproto.nullable) = false]; + + optional Subscribed subscribed = 2; + optional TaskAdded task_added = 3; + optional TaskUpdated task_updated = 4; + optional AgentAdded agent_added = 5; + optional AgentRemoved agent_removed = 6; +} diff --git a/api/v1/lib/master/masterpb_test.go b/api/v1/lib/master/masterpb_test.go new file mode 100644 index 00000000..115f1a01 --- /dev/null +++ b/api/v1/lib/master/masterpb_test.go @@ -0,0 +1,9928 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: master/master.proto + +/* +Package master is a generated protocol buffer package. + +It is generated from these files: + master/master.proto + +It has these top-level messages: + Call + Response + Event +*/ +package master + +import testing "testing" +import math_rand "math/rand" +import time "time" +import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" +import github_com_gogo_protobuf_jsonpb "github.com/gogo/protobuf/jsonpb" +import fmt "fmt" +import go_parser "go/parser" +import proto "github.com/gogo/protobuf/proto" +import math "math" +import _ "github.com/mesos/mesos-go/api/v1/lib" +import _ "github.com/mesos/mesos-go/api/v1/lib/allocator" +import _ "github.com/mesos/mesos-go/api/v1/lib/maintenance" +import _ "github.com/mesos/mesos-go/api/v1/lib/quota" +import _ "github.com/gogo/protobuf/gogoproto" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +func TestCallProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCallMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCallProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCallProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_GetMetricsProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_GetMetrics(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_GetMetrics{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_GetMetricsMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_GetMetrics(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_GetMetrics{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_GetMetricsProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_GetMetrics, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_GetMetrics(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_GetMetricsProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_GetMetrics(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_GetMetrics{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_SetLoggingLevelProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetLoggingLevel(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_SetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_SetLoggingLevelMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetLoggingLevel(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_SetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_SetLoggingLevelProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_SetLoggingLevel, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_SetLoggingLevel(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_SetLoggingLevelProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_SetLoggingLevel(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_SetLoggingLevel{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_ListFilesProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ListFiles(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_ListFiles{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_ListFilesMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ListFiles(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_ListFiles{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_ListFilesProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_ListFiles, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_ListFiles(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_ListFilesProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_ListFiles(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_ListFiles{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_ReadFileProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReadFile(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_ReadFile{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_ReadFileMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReadFile(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_ReadFile{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_ReadFileProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_ReadFile, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_ReadFile(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_ReadFileProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_ReadFile(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_ReadFile{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_UpdateWeightsProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateWeights(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_UpdateWeights{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_UpdateWeightsMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateWeights(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_UpdateWeights{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_UpdateWeightsProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_UpdateWeights, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_UpdateWeights(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_UpdateWeightsProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_UpdateWeights(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_UpdateWeights{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_ReserveResourcesProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReserveResources(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_ReserveResources{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_ReserveResourcesMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReserveResources(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_ReserveResources{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_ReserveResourcesProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_ReserveResources, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_ReserveResources(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_ReserveResourcesProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_ReserveResources(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_ReserveResources{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_UnreserveResourcesProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UnreserveResources(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_UnreserveResources{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_UnreserveResourcesMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UnreserveResources(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_UnreserveResources{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_UnreserveResourcesProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_UnreserveResources, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_UnreserveResources(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_UnreserveResourcesProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_UnreserveResources(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_UnreserveResources{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_CreateVolumesProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_CreateVolumes(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_CreateVolumes{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_CreateVolumesMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_CreateVolumes(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_CreateVolumes{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_CreateVolumesProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_CreateVolumes, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_CreateVolumes(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_CreateVolumesProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_CreateVolumes(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_CreateVolumes{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_DestroyVolumesProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_DestroyVolumes(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_DestroyVolumes{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_DestroyVolumesMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_DestroyVolumes(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_DestroyVolumes{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_DestroyVolumesProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_DestroyVolumes, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_DestroyVolumes(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_DestroyVolumesProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_DestroyVolumes(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_DestroyVolumes{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_UpdateMaintenanceScheduleProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateMaintenanceSchedule(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_UpdateMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_UpdateMaintenanceScheduleMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateMaintenanceSchedule(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_UpdateMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_UpdateMaintenanceScheduleProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_UpdateMaintenanceSchedule, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_UpdateMaintenanceSchedule(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_UpdateMaintenanceScheduleProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_UpdateMaintenanceSchedule(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_UpdateMaintenanceSchedule{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_StartMaintenanceProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StartMaintenance(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_StartMaintenance{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_StartMaintenanceMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StartMaintenance(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_StartMaintenance{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_StartMaintenanceProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_StartMaintenance, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_StartMaintenance(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_StartMaintenanceProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_StartMaintenance(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_StartMaintenance{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_StopMaintenanceProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StopMaintenance(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_StopMaintenance{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_StopMaintenanceMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StopMaintenance(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_StopMaintenance{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_StopMaintenanceProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_StopMaintenance, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_StopMaintenance(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_StopMaintenanceProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_StopMaintenance(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_StopMaintenance{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_SetQuotaProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetQuota(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_SetQuota{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_SetQuotaMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetQuota(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_SetQuota{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_SetQuotaProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_SetQuota, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_SetQuota(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_SetQuotaProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_SetQuota(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_SetQuota{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_RemoveQuotaProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_RemoveQuota(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_RemoveQuota{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestCall_RemoveQuotaMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_RemoveQuota(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_RemoveQuota{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkCall_RemoveQuotaProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_RemoveQuota, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedCall_RemoveQuota(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkCall_RemoveQuotaProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedCall_RemoveQuota(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Call_RemoveQuota{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponseProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponseMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponseProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponseProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetHealthProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetHealth(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetHealth{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetHealthMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetHealth(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetHealth{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetHealthProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetHealth, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetHealth(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetHealthProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetHealth(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetHealth{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetFlagsProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFlags(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetFlags{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetFlagsMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFlags(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetFlags{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetFlagsProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetFlags, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetFlags(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetFlagsProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetFlags(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetFlags{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetVersionProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetVersion(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetVersion{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetVersionMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetVersion(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetVersion{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetVersionProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetVersion, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetVersion(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetVersionProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetVersion(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetVersion{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetMetricsProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMetrics(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMetrics{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetMetricsMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMetrics(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMetrics{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetMetricsProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetMetrics, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetMetrics(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetMetricsProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetMetrics(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetMetrics{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetLoggingLevelProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetLoggingLevel(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetLoggingLevelMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetLoggingLevel(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetLoggingLevelProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetLoggingLevel, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetLoggingLevel(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetLoggingLevelProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetLoggingLevel(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetLoggingLevel{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_ListFilesProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ListFiles(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_ListFiles{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_ListFilesMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ListFiles(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_ListFiles{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_ListFilesProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_ListFiles, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_ListFiles(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_ListFilesProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_ListFiles(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_ListFiles{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_ReadFileProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ReadFile(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_ReadFile{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_ReadFileMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ReadFile(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_ReadFile{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_ReadFileProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_ReadFile, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_ReadFile(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_ReadFileProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_ReadFile(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_ReadFile{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetStateProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetState(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetState{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetStateMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetState(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetState{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetStateProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetState, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetState(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetStateProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetState(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetState{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetAgentsProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetAgents{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetAgentsMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetAgents{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetAgentsProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetAgents, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetAgents(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetAgentsProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetAgents(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetAgents{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetAgents_AgentProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents_Agent(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetAgents_Agent{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetAgents_AgentMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents_Agent(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetAgents_Agent{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetAgents_AgentProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetAgents_Agent, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetAgents_Agent(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetAgents_AgentProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetAgents_Agent(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetAgents_Agent{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetFrameworksProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetFrameworks{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetFrameworksMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetFrameworks{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetFrameworksProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetFrameworks, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetFrameworks(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetFrameworksProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetFrameworks(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetFrameworks{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetFrameworks_FrameworkProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks_Framework(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetFrameworks_Framework{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetFrameworks_FrameworkMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks_Framework(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetFrameworks_Framework{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetFrameworks_FrameworkProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetFrameworks_Framework, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetFrameworks_Framework(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetFrameworks_FrameworkProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetFrameworks_Framework(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetFrameworks_Framework{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetExecutorsProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetExecutors{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetExecutorsMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetExecutors{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetExecutorsProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetExecutors, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetExecutors(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetExecutorsProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetExecutors(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetExecutors{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetExecutors_ExecutorProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors_Executor(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetExecutors_Executor{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetExecutors_ExecutorMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors_Executor(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetExecutors_Executor{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetExecutors_ExecutorProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetExecutors_Executor, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetExecutors_Executor(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetExecutors_ExecutorProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetExecutors_Executor(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetExecutors_Executor{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetTasksProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetTasks(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetTasks{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetTasksMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetTasks(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetTasks{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetTasksProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetTasks, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetTasks(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetTasksProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetTasks(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetTasks{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetRolesProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetRoles(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetRoles{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetRolesMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetRoles(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetRoles{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetRolesProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetRoles, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetRoles(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetRolesProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetRoles(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetRoles{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetWeightsProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetWeights(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetWeights{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetWeightsMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetWeights(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetWeights{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetWeightsProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetWeights, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetWeights(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetWeightsProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetWeights(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetWeights{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetMasterProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaster(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMaster{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetMasterMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaster(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMaster{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetMasterProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetMaster, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetMaster(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetMasterProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetMaster(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetMaster{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetMaintenanceStatusProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceStatus(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMaintenanceStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetMaintenanceStatusMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceStatus(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMaintenanceStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetMaintenanceStatusProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetMaintenanceStatus, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetMaintenanceStatus(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetMaintenanceStatusProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetMaintenanceStatus(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetMaintenanceStatus{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetMaintenanceScheduleProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceSchedule(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetMaintenanceScheduleMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceSchedule(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetMaintenanceScheduleProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetMaintenanceSchedule, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetMaintenanceSchedule(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetMaintenanceScheduleProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetMaintenanceSchedule(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetMaintenanceSchedule{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetQuotaProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetQuota(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetQuota{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestResponse_GetQuotaMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetQuota(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetQuota{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkResponse_GetQuotaProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetQuota, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedResponse_GetQuota(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkResponse_GetQuotaProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedResponse_GetQuota(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Response_GetQuota{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestEventProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestEventMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkEventProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedEvent(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkEventProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedEvent(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Event{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_SubscribedProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_Subscribed(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_Subscribed{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestEvent_SubscribedMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_Subscribed(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_Subscribed{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkEvent_SubscribedProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_Subscribed, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedEvent_Subscribed(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkEvent_SubscribedProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedEvent_Subscribed(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Event_Subscribed{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_TaskAddedProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskAdded(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_TaskAdded{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestEvent_TaskAddedMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskAdded(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_TaskAdded{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkEvent_TaskAddedProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_TaskAdded, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedEvent_TaskAdded(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkEvent_TaskAddedProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedEvent_TaskAdded(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Event_TaskAdded{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_TaskUpdatedProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskUpdated(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_TaskUpdated{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestEvent_TaskUpdatedMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskUpdated(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_TaskUpdated{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkEvent_TaskUpdatedProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_TaskUpdated, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedEvent_TaskUpdated(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkEvent_TaskUpdatedProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedEvent_TaskUpdated(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Event_TaskUpdated{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_AgentAddedProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentAdded(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_AgentAdded{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestEvent_AgentAddedMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentAdded(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_AgentAdded{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkEvent_AgentAddedProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_AgentAdded, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedEvent_AgentAdded(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkEvent_AgentAddedProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedEvent_AgentAdded(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Event_AgentAdded{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_AgentRemovedProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentRemoved(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_AgentRemoved{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestEvent_AgentRemovedMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentRemoved(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_AgentRemoved{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkEvent_AgentRemovedProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_AgentRemoved, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedEvent_AgentRemoved(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkEvent_AgentRemovedProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedEvent_AgentRemoved(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &Event_AgentRemoved{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestCallJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_GetMetricsJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_GetMetrics(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_GetMetrics{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_SetLoggingLevelJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetLoggingLevel(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_SetLoggingLevel{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_ListFilesJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ListFiles(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_ListFiles{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_ReadFileJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReadFile(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_ReadFile{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_UpdateWeightsJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateWeights(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_UpdateWeights{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_ReserveResourcesJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReserveResources(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_ReserveResources{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_UnreserveResourcesJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UnreserveResources(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_UnreserveResources{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_CreateVolumesJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_CreateVolumes(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_CreateVolumes{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_DestroyVolumesJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_DestroyVolumes(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_DestroyVolumes{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_UpdateMaintenanceScheduleJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateMaintenanceSchedule(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_UpdateMaintenanceSchedule{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_StartMaintenanceJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StartMaintenance(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_StartMaintenance{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_StopMaintenanceJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StopMaintenance(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_StopMaintenance{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_SetQuotaJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetQuota(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_SetQuota{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCall_RemoveQuotaJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_RemoveQuota(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Call_RemoveQuota{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponseJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetHealthJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetHealth(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetHealth{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetFlagsJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFlags(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetFlags{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetVersionJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetVersion(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetVersion{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetMetricsJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMetrics(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMetrics{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetLoggingLevelJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetLoggingLevel(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetLoggingLevel{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_ListFilesJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ListFiles(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_ListFiles{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_ReadFileJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ReadFile(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_ReadFile{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetStateJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetState(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetState{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetAgentsJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetAgents{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetAgents_AgentJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents_Agent(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetAgents_Agent{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetFrameworksJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetFrameworks{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetFrameworks_FrameworkJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks_Framework(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetFrameworks_Framework{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetExecutorsJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetExecutors{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetExecutors_ExecutorJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors_Executor(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetExecutors_Executor{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetTasksJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetTasks(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetTasks{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetRolesJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetRoles(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetRoles{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetWeightsJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetWeights(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetWeights{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetMasterJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaster(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMaster{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetMaintenanceStatusJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceStatus(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMaintenanceStatus{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetMaintenanceScheduleJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceSchedule(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetMaintenanceSchedule{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestResponse_GetQuotaJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetQuota(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Response_GetQuota{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestEventJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestEvent_SubscribedJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_Subscribed(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_Subscribed{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestEvent_TaskAddedJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskAdded(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_TaskAdded{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestEvent_TaskUpdatedJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskUpdated(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_TaskUpdated{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestEvent_AgentAddedJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentAdded(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_AgentAdded{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestEvent_AgentRemovedJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentRemoved(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &Event_AgentRemoved{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestCallProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCallProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_GetMetricsProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_GetMetrics(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_GetMetrics{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_GetMetricsProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_GetMetrics(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_GetMetrics{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_SetLoggingLevelProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetLoggingLevel(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_SetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_SetLoggingLevelProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetLoggingLevel(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_SetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_ListFilesProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ListFiles(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_ListFiles{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_ListFilesProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ListFiles(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_ListFiles{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_ReadFileProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReadFile(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_ReadFile{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_ReadFileProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReadFile(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_ReadFile{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_UpdateWeightsProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateWeights(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_UpdateWeights{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_UpdateWeightsProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateWeights(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_UpdateWeights{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_ReserveResourcesProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReserveResources(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_ReserveResources{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_ReserveResourcesProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReserveResources(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_ReserveResources{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_UnreserveResourcesProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UnreserveResources(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_UnreserveResources{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_UnreserveResourcesProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UnreserveResources(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_UnreserveResources{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_CreateVolumesProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_CreateVolumes(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_CreateVolumes{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_CreateVolumesProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_CreateVolumes(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_CreateVolumes{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_DestroyVolumesProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_DestroyVolumes(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_DestroyVolumes{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_DestroyVolumesProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_DestroyVolumes(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_DestroyVolumes{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_UpdateMaintenanceScheduleProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateMaintenanceSchedule(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_UpdateMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_UpdateMaintenanceScheduleProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateMaintenanceSchedule(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_UpdateMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_StartMaintenanceProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StartMaintenance(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_StartMaintenance{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_StartMaintenanceProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StartMaintenance(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_StartMaintenance{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_StopMaintenanceProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StopMaintenance(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_StopMaintenance{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_StopMaintenanceProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StopMaintenance(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_StopMaintenance{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_SetQuotaProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetQuota(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_SetQuota{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_SetQuotaProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetQuota(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_SetQuota{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_RemoveQuotaProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_RemoveQuota(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Call_RemoveQuota{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCall_RemoveQuotaProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_RemoveQuota(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Call_RemoveQuota{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponseProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponseProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetHealthProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetHealth(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetHealth{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetHealthProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetHealth(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetHealth{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetFlagsProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFlags(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetFlags{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetFlagsProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFlags(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetFlags{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetVersionProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetVersion(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetVersion{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetVersionProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetVersion(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetVersion{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetMetricsProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMetrics(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetMetrics{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetMetricsProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMetrics(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetMetrics{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetLoggingLevelProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetLoggingLevel(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetLoggingLevelProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetLoggingLevel(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_ListFilesProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ListFiles(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_ListFiles{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_ListFilesProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ListFiles(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_ListFiles{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_ReadFileProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ReadFile(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_ReadFile{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_ReadFileProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ReadFile(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_ReadFile{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetStateProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetState(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetState{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetStateProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetState(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetState{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetAgentsProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetAgents{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetAgentsProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetAgents{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetAgents_AgentProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents_Agent(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetAgents_Agent{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetAgents_AgentProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents_Agent(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetAgents_Agent{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetFrameworksProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetFrameworks{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetFrameworksProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetFrameworks{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetFrameworks_FrameworkProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks_Framework(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetFrameworks_Framework{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetFrameworks_FrameworkProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks_Framework(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetFrameworks_Framework{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetExecutorsProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetExecutors{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetExecutorsProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetExecutors{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetExecutors_ExecutorProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors_Executor(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetExecutors_Executor{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetExecutors_ExecutorProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors_Executor(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetExecutors_Executor{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetTasksProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetTasks(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetTasks{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetTasksProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetTasks(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetTasks{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetRolesProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetRoles(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetRoles{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetRolesProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetRoles(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetRoles{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetWeightsProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetWeights(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetWeights{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetWeightsProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetWeights(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetWeights{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetMasterProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaster(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetMaster{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetMasterProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaster(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetMaster{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetMaintenanceStatusProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceStatus(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetMaintenanceStatus{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetMaintenanceStatusProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceStatus(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetMaintenanceStatus{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetMaintenanceScheduleProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceSchedule(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetMaintenanceScheduleProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceSchedule(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetQuotaProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetQuota(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Response_GetQuota{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestResponse_GetQuotaProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetQuota(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Response_GetQuota{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEventProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Event{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEventProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Event{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_SubscribedProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_Subscribed(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Event_Subscribed{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_SubscribedProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_Subscribed(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Event_Subscribed{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_TaskAddedProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskAdded(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Event_TaskAdded{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_TaskAddedProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskAdded(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Event_TaskAdded{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_TaskUpdatedProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskUpdated(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Event_TaskUpdated{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_TaskUpdatedProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskUpdated(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Event_TaskUpdated{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_AgentAddedProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentAdded(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Event_AgentAdded{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_AgentAddedProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentAdded(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Event_AgentAdded{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_AgentRemovedProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentRemoved(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &Event_AgentRemoved{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestEvent_AgentRemovedProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentRemoved(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &Event_AgentRemoved{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestCallVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_GetMetricsVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_GetMetrics(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_GetMetrics{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_SetLoggingLevelVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_SetLoggingLevel(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_SetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_ListFilesVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_ListFiles(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_ListFiles{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_ReadFileVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_ReadFile(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_ReadFile{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_UpdateWeightsVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_UpdateWeights(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_UpdateWeights{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_ReserveResourcesVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_ReserveResources(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_ReserveResources{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_UnreserveResourcesVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_UnreserveResources(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_UnreserveResources{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_CreateVolumesVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_CreateVolumes(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_CreateVolumes{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_DestroyVolumesVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_DestroyVolumes(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_DestroyVolumes{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_UpdateMaintenanceScheduleVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_UpdateMaintenanceSchedule(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_UpdateMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_StartMaintenanceVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_StartMaintenance(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_StartMaintenance{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_StopMaintenanceVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_StopMaintenance(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_StopMaintenance{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_SetQuotaVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_SetQuota(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_SetQuota{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCall_RemoveQuotaVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_RemoveQuota(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Call_RemoveQuota{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponseVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetHealthVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetHealth(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetHealth{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetFlagsVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetFlags(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetFlags{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetVersionVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetVersion(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetVersion{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetMetricsVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMetrics(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetMetrics{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetLoggingLevelVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetLoggingLevel(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetLoggingLevel{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_ListFilesVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_ListFiles(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_ListFiles{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_ReadFileVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_ReadFile(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_ReadFile{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetStateVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetState(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetState{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetAgentsVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetAgents(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetAgents{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetAgents_AgentVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetAgents_Agent(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetAgents_Agent{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetFrameworksVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetFrameworks(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetFrameworks{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetFrameworks_FrameworkVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetFrameworks_Framework(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetFrameworks_Framework{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetExecutorsVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetExecutors(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetExecutors{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetExecutors_ExecutorVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetExecutors_Executor(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetExecutors_Executor{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetTasksVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetTasks(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetTasks{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetRolesVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetRoles(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetRoles{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetWeightsVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetWeights(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetWeights{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetMasterVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMaster(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetMaster{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetMaintenanceStatusVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMaintenanceStatus(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetMaintenanceStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetMaintenanceScheduleVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMaintenanceSchedule(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetMaintenanceSchedule{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestResponse_GetQuotaVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetQuota(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Response_GetQuota{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestEventVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Event{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestEvent_SubscribedVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_Subscribed(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Event_Subscribed{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestEvent_TaskAddedVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_TaskAdded(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Event_TaskAdded{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestEvent_TaskUpdatedVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_TaskUpdated(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Event_TaskUpdated{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestEvent_AgentAddedVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_AgentAdded(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Event_AgentAdded{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestEvent_AgentRemovedVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_AgentRemoved(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &Event_AgentRemoved{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestCallGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_GetMetricsGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_GetMetrics(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_SetLoggingLevelGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_SetLoggingLevel(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_ListFilesGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_ListFiles(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_ReadFileGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_ReadFile(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_UpdateWeightsGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_UpdateWeights(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_ReserveResourcesGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_ReserveResources(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_UnreserveResourcesGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_UnreserveResources(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_CreateVolumesGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_CreateVolumes(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_DestroyVolumesGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_DestroyVolumes(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_UpdateMaintenanceScheduleGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_UpdateMaintenanceSchedule(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_StartMaintenanceGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_StartMaintenance(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_StopMaintenanceGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_StopMaintenance(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_SetQuotaGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_SetQuota(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCall_RemoveQuotaGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_RemoveQuota(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponseGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetHealthGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetHealth(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetFlagsGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetFlags(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetVersionGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetVersion(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetMetricsGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMetrics(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetLoggingLevelGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetLoggingLevel(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_ListFilesGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_ListFiles(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_ReadFileGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_ReadFile(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetStateGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetState(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetAgentsGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetAgents(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetAgents_AgentGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetAgents_Agent(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetFrameworksGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetFrameworks(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetFrameworks_FrameworkGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetFrameworks_Framework(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetExecutorsGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetExecutors(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetExecutors_ExecutorGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetExecutors_Executor(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetTasksGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetTasks(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetRolesGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetRoles(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetWeightsGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetWeights(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetMasterGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMaster(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetMaintenanceStatusGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMaintenanceStatus(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetMaintenanceScheduleGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMaintenanceSchedule(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestResponse_GetQuotaGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetQuota(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestEventGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestEvent_SubscribedGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_Subscribed(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestEvent_TaskAddedGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_TaskAdded(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestEvent_TaskUpdatedGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_TaskUpdated(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestEvent_AgentAddedGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_AgentAdded(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestEvent_AgentRemovedGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_AgentRemoved(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestCallProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCallProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_GetMetricsProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_GetMetrics(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_GetMetricsProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_GetMetrics, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_GetMetrics(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_SetLoggingLevelProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetLoggingLevel(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_SetLoggingLevelProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_SetLoggingLevel, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_SetLoggingLevel(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_ListFilesProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ListFiles(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_ListFilesProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_ListFiles, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_ListFiles(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_ReadFileProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReadFile(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_ReadFileProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_ReadFile, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_ReadFile(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_UpdateWeightsProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateWeights(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_UpdateWeightsProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_UpdateWeights, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_UpdateWeights(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_ReserveResourcesProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_ReserveResources(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_ReserveResourcesProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_ReserveResources, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_ReserveResources(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_UnreserveResourcesProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UnreserveResources(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_UnreserveResourcesProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_UnreserveResources, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_UnreserveResources(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_CreateVolumesProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_CreateVolumes(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_CreateVolumesProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_CreateVolumes, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_CreateVolumes(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_DestroyVolumesProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_DestroyVolumes(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_DestroyVolumesProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_DestroyVolumes, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_DestroyVolumes(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_UpdateMaintenanceScheduleProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_UpdateMaintenanceSchedule(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_UpdateMaintenanceScheduleProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_UpdateMaintenanceSchedule, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_UpdateMaintenanceSchedule(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_StartMaintenanceProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StartMaintenance(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_StartMaintenanceProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_StartMaintenance, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_StartMaintenance(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_StopMaintenanceProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_StopMaintenance(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_StopMaintenanceProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_StopMaintenance, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_StopMaintenance(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_SetQuotaProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_SetQuota(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_SetQuotaProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_SetQuota, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_SetQuota(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCall_RemoveQuotaProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedCall_RemoveQuota(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkCall_RemoveQuotaProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Call_RemoveQuota, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedCall_RemoveQuota(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponseProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponseProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetHealthProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetHealth(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetHealthProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetHealth, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetHealth(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetFlagsProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFlags(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetFlagsProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetFlags, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetFlags(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetVersionProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetVersion(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetVersionProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetVersion, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetVersion(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetMetricsProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMetrics(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetMetricsProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetMetrics, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetMetrics(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetLoggingLevelProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetLoggingLevel(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetLoggingLevelProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetLoggingLevel, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetLoggingLevel(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_ListFilesProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ListFiles(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_ListFilesProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_ListFiles, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_ListFiles(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_ReadFileProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_ReadFile(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_ReadFileProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_ReadFile, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_ReadFile(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetStateProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetState(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetStateProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetState, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetState(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetAgentsProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetAgentsProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetAgents, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetAgents(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetAgents_AgentProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetAgents_Agent(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetAgents_AgentProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetAgents_Agent, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetAgents_Agent(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetFrameworksProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetFrameworksProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetFrameworks, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetFrameworks(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetFrameworks_FrameworkProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetFrameworks_Framework(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetFrameworks_FrameworkProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetFrameworks_Framework, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetFrameworks_Framework(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetExecutorsProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetExecutorsProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetExecutors, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetExecutors(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetExecutors_ExecutorProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetExecutors_Executor(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetExecutors_ExecutorProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetExecutors_Executor, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetExecutors_Executor(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetTasksProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetTasks(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetTasksProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetTasks, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetTasks(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetRolesProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetRoles(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetRolesProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetRoles, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetRoles(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetWeightsProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetWeights(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetWeightsProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetWeights, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetWeights(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetMasterProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaster(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetMasterProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetMaster, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetMaster(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetMaintenanceStatusProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceStatus(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetMaintenanceStatusProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetMaintenanceStatus, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetMaintenanceStatus(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetMaintenanceScheduleProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetMaintenanceSchedule(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetMaintenanceScheduleProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetMaintenanceSchedule, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetMaintenanceSchedule(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestResponse_GetQuotaProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedResponse_GetQuota(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkResponse_GetQuotaProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Response_GetQuota, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedResponse_GetQuota(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestEventProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkEventProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedEvent(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_SubscribedProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_Subscribed(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkEvent_SubscribedProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_Subscribed, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedEvent_Subscribed(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_TaskAddedProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskAdded(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkEvent_TaskAddedProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_TaskAdded, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedEvent_TaskAdded(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_TaskUpdatedProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_TaskUpdated(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkEvent_TaskUpdatedProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_TaskUpdated, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedEvent_TaskUpdated(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_AgentAddedProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentAdded(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkEvent_AgentAddedProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_AgentAdded, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedEvent_AgentAdded(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestEvent_AgentRemovedProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedEvent_AgentRemoved(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkEvent_AgentRemovedProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*Event_AgentRemoved, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedEvent_AgentRemoved(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestCallStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_GetMetricsStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_GetMetrics(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_SetLoggingLevelStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_SetLoggingLevel(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_ListFilesStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_ListFiles(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_ReadFileStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_ReadFile(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_UpdateWeightsStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_UpdateWeights(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_ReserveResourcesStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_ReserveResources(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_UnreserveResourcesStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_UnreserveResources(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_CreateVolumesStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_CreateVolumes(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_DestroyVolumesStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_DestroyVolumes(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_UpdateMaintenanceScheduleStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_UpdateMaintenanceSchedule(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_StartMaintenanceStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_StartMaintenance(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_StopMaintenanceStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_StopMaintenance(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_SetQuotaStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_SetQuota(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestCall_RemoveQuotaStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedCall_RemoveQuota(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponseStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetHealthStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetHealth(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetFlagsStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetFlags(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetVersionStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetVersion(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetMetricsStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMetrics(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetLoggingLevelStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetLoggingLevel(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_ListFilesStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_ListFiles(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_ReadFileStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_ReadFile(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetStateStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetState(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetAgentsStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetAgents(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetAgents_AgentStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetAgents_Agent(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetFrameworksStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetFrameworks(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetFrameworks_FrameworkStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetFrameworks_Framework(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetExecutorsStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetExecutors(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetExecutors_ExecutorStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetExecutors_Executor(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetTasksStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetTasks(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetRolesStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetRoles(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetWeightsStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetWeights(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetMasterStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMaster(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetMaintenanceStatusStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMaintenanceStatus(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetMaintenanceScheduleStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetMaintenanceSchedule(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestResponse_GetQuotaStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedResponse_GetQuota(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestEventStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestEvent_SubscribedStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_Subscribed(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestEvent_TaskAddedStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_TaskAdded(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestEvent_TaskUpdatedStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_TaskUpdated(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestEvent_AgentAddedStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_AgentAdded(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestEvent_AgentRemovedStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedEvent_AgentRemoved(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} + +//These tests are generated by github.com/gogo/protobuf/plugin/testgen diff --git a/api/v1/lib/mesos.pb.go b/api/v1/lib/mesos.pb.go index 35a4c6b7..551a5fdb 100644 --- a/api/v1/lib/mesos.pb.go +++ b/api/v1/lib/mesos.pb.go @@ -2432,7 +2432,7 @@ type ExecutorInfo struct { Type ExecutorInfo_Type `protobuf:"varint,15,opt,name=type,enum=mesos.ExecutorInfo_Type" json:"type"` ExecutorID ExecutorID `protobuf:"bytes,1,req,name=executor_id,json=executorId" json:"executor_id"` FrameworkID *FrameworkID `protobuf:"bytes,8,opt,name=framework_id,json=frameworkId" json:"framework_id,omitempty"` - Command CommandInfo `protobuf:"bytes,7,req,name=command" json:"command"` + Command *CommandInfo `protobuf:"bytes,7,opt,name=command" json:"command,omitempty"` // Executor provided with a container will launch the container // with the executor's CommandInfo and we expect the container to // act as a Mesos executor. @@ -2498,11 +2498,11 @@ func (m *ExecutorInfo) GetFrameworkID() *FrameworkID { return nil } -func (m *ExecutorInfo) GetCommand() CommandInfo { +func (m *ExecutorInfo) GetCommand() *CommandInfo { if m != nil { return m.Command } - return CommandInfo{} + return nil } func (m *ExecutorInfo) GetContainer() *ContainerInfo { @@ -9975,7 +9975,7 @@ func (this *ExecutorInfo) VerboseEqual(that interface{}) error { if !this.FrameworkID.Equal(that1.FrameworkID) { return fmt.Errorf("FrameworkID this(%v) Not Equal that(%v)", this.FrameworkID, that1.FrameworkID) } - if !this.Command.Equal(&that1.Command) { + if !this.Command.Equal(that1.Command) { return fmt.Errorf("Command this(%v) Not Equal that(%v)", this.Command, that1.Command) } if !this.Container.Equal(that1.Container) { @@ -10055,7 +10055,7 @@ func (this *ExecutorInfo) Equal(that interface{}) bool { if !this.FrameworkID.Equal(that1.FrameworkID) { return false } - if !this.Command.Equal(&that1.Command) { + if !this.Command.Equal(that1.Command) { return false } if !this.Container.Equal(that1.Container) { @@ -21146,7 +21146,7 @@ func (this *MachineInfo) GoString() string { s = append(s, "&mesos.MachineInfo{") s = append(s, "ID: "+strings.Replace(this.ID.GoString(), `&`, ``, 1)+",\n") if this.Mode != nil { - s = append(s, "Mode: "+valueToGoStringMesos(this.Mode, "mesos.MachineInfo_Mode")+",\n") + s = append(s, "Mode: "+valueToGoStringMesos(this.Mode, "MachineInfo_Mode")+",\n") } if this.Unavailability != nil { s = append(s, "Unavailability: "+fmt.Sprintf("%#v", this.Unavailability)+",\n") @@ -21388,7 +21388,9 @@ func (this *ExecutorInfo) GoString() string { if this.FrameworkID != nil { s = append(s, "FrameworkID: "+fmt.Sprintf("%#v", this.FrameworkID)+",\n") } - s = append(s, "Command: "+strings.Replace(this.Command.GoString(), `&`, ``, 1)+",\n") + if this.Command != nil { + s = append(s, "Command: "+fmt.Sprintf("%#v", this.Command)+",\n") + } if this.Container != nil { s = append(s, "Container: "+fmt.Sprintf("%#v", this.Container)+",\n") } @@ -21582,7 +21584,7 @@ func (this *Resource) GoString() string { s = append(s, "&mesos.Resource{") s = append(s, "Name: "+fmt.Sprintf("%#v", this.Name)+",\n") if this.Type != nil { - s = append(s, "Type: "+valueToGoStringMesos(this.Type, "mesos.Value_Type")+",\n") + s = append(s, "Type: "+valueToGoStringMesos(this.Type, "Value_Type")+",\n") } if this.Scalar != nil { s = append(s, "Scalar: "+fmt.Sprintf("%#v", this.Scalar)+",\n") @@ -21679,7 +21681,7 @@ func (this *Resource_DiskInfo_Source) GoString() string { s := make([]string, 0, 7) s = append(s, "&mesos.Resource_DiskInfo_Source{") if this.Type != nil { - s = append(s, "Type: "+valueToGoStringMesos(this.Type, "mesos.Resource_DiskInfo_Source_Type")+",\n") + s = append(s, "Type: "+valueToGoStringMesos(this.Type, "Resource_DiskInfo_Source_Type")+",\n") } if this.Path != nil { s = append(s, "Path: "+fmt.Sprintf("%#v", this.Path)+",\n") @@ -22612,7 +22614,7 @@ func (this *Task) GoString() string { } s = append(s, "AgentID: "+strings.Replace(this.AgentID.GoString(), `&`, ``, 1)+",\n") if this.State != nil { - s = append(s, "State: "+valueToGoStringMesos(this.State, "mesos.TaskState")+",\n") + s = append(s, "State: "+valueToGoStringMesos(this.State, "TaskState")+",\n") } if this.Resources != nil { s = append(s, "Resources: "+fmt.Sprintf("%#v", this.Resources)+",\n") @@ -22621,7 +22623,7 @@ func (this *Task) GoString() string { s = append(s, "Statuses: "+fmt.Sprintf("%#v", this.Statuses)+",\n") } if this.StatusUpdateState != nil { - s = append(s, "StatusUpdateState: "+valueToGoStringMesos(this.StatusUpdateState, "mesos.TaskState")+",\n") + s = append(s, "StatusUpdateState: "+valueToGoStringMesos(this.StatusUpdateState, "TaskState")+",\n") } if this.StatusUpdateUUID != nil { s = append(s, "StatusUpdateUUID: "+valueToGoStringMesos(this.StatusUpdateUUID, "byte")+",\n") @@ -22648,7 +22650,7 @@ func (this *CheckStatusInfo) GoString() string { s := make([]string, 0, 7) s = append(s, "&mesos.CheckStatusInfo{") if this.Type != nil { - s = append(s, "Type: "+valueToGoStringMesos(this.Type, "mesos.CheckInfo_Type")+",\n") + s = append(s, "Type: "+valueToGoStringMesos(this.Type, "CheckInfo_Type")+",\n") } if this.Command != nil { s = append(s, "Command: "+fmt.Sprintf("%#v", this.Command)+",\n") @@ -22691,16 +22693,16 @@ func (this *TaskStatus) GoString() string { s = append(s, "&mesos.TaskStatus{") s = append(s, "TaskID: "+strings.Replace(this.TaskID.GoString(), `&`, ``, 1)+",\n") if this.State != nil { - s = append(s, "State: "+valueToGoStringMesos(this.State, "mesos.TaskState")+",\n") + s = append(s, "State: "+valueToGoStringMesos(this.State, "TaskState")+",\n") } if this.Message != nil { s = append(s, "Message: "+valueToGoStringMesos(this.Message, "string")+",\n") } if this.Source != nil { - s = append(s, "Source: "+valueToGoStringMesos(this.Source, "mesos.TaskStatus_Source")+",\n") + s = append(s, "Source: "+valueToGoStringMesos(this.Source, "TaskStatus_Source")+",\n") } if this.Reason != nil { - s = append(s, "Reason: "+valueToGoStringMesos(this.Reason, "mesos.TaskStatus_Reason")+",\n") + s = append(s, "Reason: "+valueToGoStringMesos(this.Reason, "TaskStatus_Reason")+",\n") } if this.Data != nil { s = append(s, "Data: "+valueToGoStringMesos(this.Data, "byte")+",\n") @@ -22861,7 +22863,7 @@ func (this *Image) GoString() string { s := make([]string, 0, 8) s = append(s, "&mesos.Image{") if this.Type != nil { - s = append(s, "Type: "+valueToGoStringMesos(this.Type, "mesos.Image_Type")+",\n") + s = append(s, "Type: "+valueToGoStringMesos(this.Type, "Image_Type")+",\n") } if this.Appc != nil { s = append(s, "Appc: "+fmt.Sprintf("%#v", this.Appc)+",\n") @@ -22911,7 +22913,7 @@ func (this *Volume) GoString() string { s := make([]string, 0, 9) s = append(s, "&mesos.Volume{") if this.Mode != nil { - s = append(s, "Mode: "+valueToGoStringMesos(this.Mode, "mesos.Volume_Mode")+",\n") + s = append(s, "Mode: "+valueToGoStringMesos(this.Mode, "Volume_Mode")+",\n") } s = append(s, "ContainerPath: "+fmt.Sprintf("%#v", this.ContainerPath)+",\n") if this.HostPath != nil { @@ -23000,7 +23002,7 @@ func (this *NetworkInfo_IPAddress) GoString() string { s := make([]string, 0, 6) s = append(s, "&mesos.NetworkInfo_IPAddress{") if this.Protocol != nil { - s = append(s, "Protocol: "+valueToGoStringMesos(this.Protocol, "mesos.NetworkInfo_Protocol")+",\n") + s = append(s, "Protocol: "+valueToGoStringMesos(this.Protocol, "NetworkInfo_Protocol")+",\n") } if this.IPAddress != nil { s = append(s, "IPAddress: "+valueToGoStringMesos(this.IPAddress, "string")+",\n") @@ -23104,7 +23106,7 @@ func (this *ContainerInfo) GoString() string { s := make([]string, 0, 13) s = append(s, "&mesos.ContainerInfo{") if this.Type != nil { - s = append(s, "Type: "+valueToGoStringMesos(this.Type, "mesos.ContainerInfo_Type")+",\n") + s = append(s, "Type: "+valueToGoStringMesos(this.Type, "ContainerInfo_Type")+",\n") } if this.Volumes != nil { s = append(s, "Volumes: "+fmt.Sprintf("%#v", this.Volumes)+",\n") @@ -23141,7 +23143,7 @@ func (this *ContainerInfo_DockerInfo) GoString() string { s = append(s, "&mesos.ContainerInfo_DockerInfo{") s = append(s, "Image: "+fmt.Sprintf("%#v", this.Image)+",\n") if this.Network != nil { - s = append(s, "Network: "+valueToGoStringMesos(this.Network, "mesos.ContainerInfo_DockerInfo_Network")+",\n") + s = append(s, "Network: "+valueToGoStringMesos(this.Network, "ContainerInfo_DockerInfo_Network")+",\n") } if this.PortMappings != nil { s = append(s, "PortMappings: "+fmt.Sprintf("%#v", this.PortMappings)+",\n") @@ -23271,7 +23273,7 @@ func (this *Port) GoString() string { s = append(s, "Protocol: "+valueToGoStringMesos(this.Protocol, "string")+",\n") } if this.Visibility != nil { - s = append(s, "Visibility: "+valueToGoStringMesos(this.Visibility, "mesos.DiscoveryInfo_Visibility")+",\n") + s = append(s, "Visibility: "+valueToGoStringMesos(this.Visibility, "DiscoveryInfo_Visibility")+",\n") } if this.Labels != nil { s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") @@ -24400,14 +24402,16 @@ func (m *ExecutorInfo) MarshalTo(dAtA []byte) (int, error) { i += n } } - dAtA[i] = 0x3a - i++ - i = encodeVarintMesos(dAtA, i, uint64(m.Command.ProtoSize())) - n18, err := m.Command.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + if m.Command != nil { + dAtA[i] = 0x3a + i++ + i = encodeVarintMesos(dAtA, i, uint64(m.Command.ProtoSize())) + n18, err := m.Command.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n18 } - i += n18 if m.FrameworkID != nil { dAtA[i] = 0x42 i++ @@ -29774,18 +29778,19 @@ func NewPopulatedExecutorInfo(r randyMesos, easy bool) *ExecutorInfo { this.Resources[i] = *v48 } } - v49 := NewPopulatedCommandInfo(r, easy) - this.Command = *v49 + if r.Intn(10) != 0 { + this.Command = NewPopulatedCommandInfo(r, easy) + } if r.Intn(10) != 0 { this.FrameworkID = NewPopulatedFrameworkID(r, easy) } if r.Intn(10) != 0 { - v50 := string(randStringMesos(r)) - this.Name = &v50 + v49 := string(randStringMesos(r)) + this.Name = &v49 } if r.Intn(10) != 0 { - v51 := string(randStringMesos(r)) - this.Source = &v51 + v50 := string(randStringMesos(r)) + this.Source = &v50 } if r.Intn(10) != 0 { this.Container = NewPopulatedContainerInfo(r, easy) @@ -29809,19 +29814,19 @@ func NewPopulatedMasterInfo(r randyMesos, easy bool) *MasterInfo { this := &MasterInfo{} this.ID = string(randStringMesos(r)) this.IP = uint32(r.Uint32()) - v52 := uint32(r.Uint32()) - this.Port = &v52 + v51 := uint32(r.Uint32()) + this.Port = &v51 if r.Intn(10) != 0 { - v53 := string(randStringMesos(r)) - this.PID = &v53 + v52 := string(randStringMesos(r)) + this.PID = &v52 } if r.Intn(10) != 0 { - v54 := string(randStringMesos(r)) - this.Hostname = &v54 + v53 := string(randStringMesos(r)) + this.Hostname = &v53 } if r.Intn(10) != 0 { - v55 := string(randStringMesos(r)) - this.Version = &v55 + v54 := string(randStringMesos(r)) + this.Version = &v54 } if r.Intn(10) != 0 { this.Address = NewPopulatedAddress(r, easy) @@ -29835,30 +29840,30 @@ func NewPopulatedAgentInfo(r randyMesos, easy bool) *AgentInfo { this := &AgentInfo{} this.Hostname = string(randStringMesos(r)) if r.Intn(10) != 0 { - v56 := r.Intn(5) - this.Resources = make([]Resource, v56) - for i := 0; i < v56; i++ { - v57 := NewPopulatedResource(r, easy) - this.Resources[i] = *v57 + v55 := r.Intn(5) + this.Resources = make([]Resource, v55) + for i := 0; i < v55; i++ { + v56 := NewPopulatedResource(r, easy) + this.Resources[i] = *v56 } } if r.Intn(10) != 0 { - v58 := r.Intn(5) - this.Attributes = make([]Attribute, v58) - for i := 0; i < v58; i++ { - v59 := NewPopulatedAttribute(r, easy) - this.Attributes[i] = *v59 + v57 := r.Intn(5) + this.Attributes = make([]Attribute, v57) + for i := 0; i < v57; i++ { + v58 := NewPopulatedAttribute(r, easy) + this.Attributes[i] = *v58 } } if r.Intn(10) != 0 { this.ID = NewPopulatedAgentID(r, easy) } if r.Intn(10) != 0 { - v60 := int32(r.Int31()) + v59 := int32(r.Int31()) if r.Intn(2) == 0 { - v60 *= -1 + v59 *= -1 } - this.Port = &v60 + this.Port = &v59 } if !easy && r.Intn(10) != 0 { } @@ -29916,11 +29921,11 @@ func NewPopulatedValue_Range(r randyMesos, easy bool) *Value_Range { func NewPopulatedValue_Ranges(r randyMesos, easy bool) *Value_Ranges { this := &Value_Ranges{} if r.Intn(10) != 0 { - v61 := r.Intn(5) - this.Range = make([]Value_Range, v61) - for i := 0; i < v61; i++ { - v62 := NewPopulatedValue_Range(r, easy) - this.Range[i] = *v62 + v60 := r.Intn(5) + this.Range = make([]Value_Range, v60) + for i := 0; i < v60; i++ { + v61 := NewPopulatedValue_Range(r, easy) + this.Range[i] = *v61 } } if !easy && r.Intn(10) != 0 { @@ -29931,9 +29936,9 @@ func NewPopulatedValue_Ranges(r randyMesos, easy bool) *Value_Ranges { func NewPopulatedValue_Set(r randyMesos, easy bool) *Value_Set { this := &Value_Set{} if r.Intn(10) != 0 { - v63 := r.Intn(10) - this.Item = make([]string, v63) - for i := 0; i < v63; i++ { + v62 := r.Intn(10) + this.Item = make([]string, v62) + for i := 0; i < v62; i++ { this.Item[i] = string(randStringMesos(r)) } } @@ -29974,8 +29979,8 @@ func NewPopulatedAttribute(r randyMesos, easy bool) *Attribute { func NewPopulatedResource(r randyMesos, easy bool) *Resource { this := &Resource{} this.Name = string(randStringMesos(r)) - v64 := Value_Type([]int32{0, 1, 2, 3}[r.Intn(4)]) - this.Type = &v64 + v63 := Value_Type([]int32{0, 1, 2, 3}[r.Intn(4)]) + this.Type = &v63 if r.Intn(10) != 0 { this.Scalar = NewPopulatedValue_Scalar(r, easy) } @@ -29986,8 +29991,8 @@ func NewPopulatedResource(r randyMesos, easy bool) *Resource { this.Set = NewPopulatedValue_Set(r, easy) } if r.Intn(10) != 0 { - v65 := string(randStringMesos(r)) - this.Role = &v65 + v64 := string(randStringMesos(r)) + this.Role = &v64 } if r.Intn(10) != 0 { this.Disk = NewPopulatedResource_DiskInfo(r, easy) @@ -30012,8 +30017,8 @@ func NewPopulatedResource(r randyMesos, easy bool) *Resource { func NewPopulatedResource_AllocationInfo(r randyMesos, easy bool) *Resource_AllocationInfo { this := &Resource_AllocationInfo{} if r.Intn(10) != 0 { - v66 := string(randStringMesos(r)) - this.Role = &v66 + v65 := string(randStringMesos(r)) + this.Role = &v65 } if !easy && r.Intn(10) != 0 { } @@ -30023,8 +30028,8 @@ func NewPopulatedResource_AllocationInfo(r randyMesos, easy bool) *Resource_Allo func NewPopulatedResource_ReservationInfo(r randyMesos, easy bool) *Resource_ReservationInfo { this := &Resource_ReservationInfo{} if r.Intn(10) != 0 { - v67 := string(randStringMesos(r)) - this.Principal = &v67 + v66 := string(randStringMesos(r)) + this.Principal = &v66 } if r.Intn(10) != 0 { this.Labels = NewPopulatedLabels(r, easy) @@ -30054,8 +30059,8 @@ func NewPopulatedResource_DiskInfo_Persistence(r randyMesos, easy bool) *Resourc this := &Resource_DiskInfo_Persistence{} this.ID = string(randStringMesos(r)) if r.Intn(10) != 0 { - v68 := string(randStringMesos(r)) - this.Principal = &v68 + v67 := string(randStringMesos(r)) + this.Principal = &v67 } if !easy && r.Intn(10) != 0 { } @@ -30064,8 +30069,8 @@ func NewPopulatedResource_DiskInfo_Persistence(r randyMesos, easy bool) *Resourc func NewPopulatedResource_DiskInfo_Source(r randyMesos, easy bool) *Resource_DiskInfo_Source { this := &Resource_DiskInfo_Source{} - v69 := Resource_DiskInfo_Source_Type([]int32{1, 2}[r.Intn(2)]) - this.Type = &v69 + v68 := Resource_DiskInfo_Source_Type([]int32{1, 2}[r.Intn(2)]) + this.Type = &v68 if r.Intn(10) != 0 { this.Path = NewPopulatedResource_DiskInfo_Source_Path(r, easy) } @@ -30110,41 +30115,41 @@ func NewPopulatedResource_SharedInfo(r randyMesos, easy bool) *Resource_SharedIn func NewPopulatedTrafficControlStatistics(r randyMesos, easy bool) *TrafficControlStatistics { this := &TrafficControlStatistics{} this.ID = string(randStringMesos(r)) + if r.Intn(10) != 0 { + v69 := uint64(uint64(r.Uint32())) + this.Backlog = &v69 + } if r.Intn(10) != 0 { v70 := uint64(uint64(r.Uint32())) - this.Backlog = &v70 + this.Bytes = &v70 } if r.Intn(10) != 0 { v71 := uint64(uint64(r.Uint32())) - this.Bytes = &v71 + this.Drops = &v71 } if r.Intn(10) != 0 { v72 := uint64(uint64(r.Uint32())) - this.Drops = &v72 + this.Overlimits = &v72 } if r.Intn(10) != 0 { v73 := uint64(uint64(r.Uint32())) - this.Overlimits = &v73 + this.Packets = &v73 } if r.Intn(10) != 0 { v74 := uint64(uint64(r.Uint32())) - this.Packets = &v74 + this.Qlen = &v74 } if r.Intn(10) != 0 { v75 := uint64(uint64(r.Uint32())) - this.Qlen = &v75 + this.RateBPS = &v75 } if r.Intn(10) != 0 { v76 := uint64(uint64(r.Uint32())) - this.RateBPS = &v76 + this.RatePPS = &v76 } if r.Intn(10) != 0 { v77 := uint64(uint64(r.Uint32())) - this.RatePPS = &v77 - } - if r.Intn(10) != 0 { - v78 := uint64(uint64(r.Uint32())) - this.Requeues = &v78 + this.Requeues = &v77 } if !easy && r.Intn(10) != 0 { } @@ -30153,138 +30158,138 @@ func NewPopulatedTrafficControlStatistics(r randyMesos, easy bool) *TrafficContr func NewPopulatedIpStatistics(r randyMesos, easy bool) *IpStatistics { this := &IpStatistics{} + if r.Intn(10) != 0 { + v78 := int64(r.Int63()) + if r.Intn(2) == 0 { + v78 *= -1 + } + this.Forwarding = &v78 + } if r.Intn(10) != 0 { v79 := int64(r.Int63()) if r.Intn(2) == 0 { v79 *= -1 } - this.Forwarding = &v79 + this.DefaultTTL = &v79 } if r.Intn(10) != 0 { v80 := int64(r.Int63()) if r.Intn(2) == 0 { v80 *= -1 } - this.DefaultTTL = &v80 + this.InReceives = &v80 } if r.Intn(10) != 0 { v81 := int64(r.Int63()) if r.Intn(2) == 0 { v81 *= -1 } - this.InReceives = &v81 + this.InHdrErrors = &v81 } if r.Intn(10) != 0 { v82 := int64(r.Int63()) if r.Intn(2) == 0 { v82 *= -1 } - this.InHdrErrors = &v82 + this.InAddrErrors = &v82 } if r.Intn(10) != 0 { v83 := int64(r.Int63()) if r.Intn(2) == 0 { v83 *= -1 } - this.InAddrErrors = &v83 + this.ForwDatagrams = &v83 } if r.Intn(10) != 0 { v84 := int64(r.Int63()) if r.Intn(2) == 0 { v84 *= -1 } - this.ForwDatagrams = &v84 + this.InUnknownProtos = &v84 } if r.Intn(10) != 0 { v85 := int64(r.Int63()) if r.Intn(2) == 0 { v85 *= -1 } - this.InUnknownProtos = &v85 + this.InDiscards = &v85 } if r.Intn(10) != 0 { v86 := int64(r.Int63()) if r.Intn(2) == 0 { v86 *= -1 } - this.InDiscards = &v86 + this.InDelivers = &v86 } if r.Intn(10) != 0 { v87 := int64(r.Int63()) if r.Intn(2) == 0 { v87 *= -1 } - this.InDelivers = &v87 + this.OutRequests = &v87 } if r.Intn(10) != 0 { v88 := int64(r.Int63()) if r.Intn(2) == 0 { v88 *= -1 } - this.OutRequests = &v88 + this.OutDiscards = &v88 } if r.Intn(10) != 0 { v89 := int64(r.Int63()) if r.Intn(2) == 0 { v89 *= -1 } - this.OutDiscards = &v89 + this.OutNoRoutes = &v89 } if r.Intn(10) != 0 { v90 := int64(r.Int63()) if r.Intn(2) == 0 { v90 *= -1 } - this.OutNoRoutes = &v90 + this.ReasmTimeout = &v90 } if r.Intn(10) != 0 { v91 := int64(r.Int63()) if r.Intn(2) == 0 { v91 *= -1 } - this.ReasmTimeout = &v91 + this.ReasmReqds = &v91 } if r.Intn(10) != 0 { v92 := int64(r.Int63()) if r.Intn(2) == 0 { v92 *= -1 } - this.ReasmReqds = &v92 + this.ReasmOKs = &v92 } if r.Intn(10) != 0 { v93 := int64(r.Int63()) if r.Intn(2) == 0 { v93 *= -1 } - this.ReasmOKs = &v93 + this.ReasmFails = &v93 } if r.Intn(10) != 0 { v94 := int64(r.Int63()) if r.Intn(2) == 0 { v94 *= -1 } - this.ReasmFails = &v94 + this.FragOKs = &v94 } if r.Intn(10) != 0 { v95 := int64(r.Int63()) if r.Intn(2) == 0 { v95 *= -1 } - this.FragOKs = &v95 + this.FragFails = &v95 } if r.Intn(10) != 0 { v96 := int64(r.Int63()) if r.Intn(2) == 0 { v96 *= -1 } - this.FragFails = &v96 - } - if r.Intn(10) != 0 { - v97 := int64(r.Int63()) - if r.Intn(2) == 0 { - v97 *= -1 - } - this.FragCreates = &v97 + this.FragCreates = &v96 } if !easy && r.Intn(10) != 0 { } @@ -30293,194 +30298,194 @@ func NewPopulatedIpStatistics(r randyMesos, easy bool) *IpStatistics { func NewPopulatedIcmpStatistics(r randyMesos, easy bool) *IcmpStatistics { this := &IcmpStatistics{} + if r.Intn(10) != 0 { + v97 := int64(r.Int63()) + if r.Intn(2) == 0 { + v97 *= -1 + } + this.InMsgs = &v97 + } if r.Intn(10) != 0 { v98 := int64(r.Int63()) if r.Intn(2) == 0 { v98 *= -1 } - this.InMsgs = &v98 + this.InErrors = &v98 } if r.Intn(10) != 0 { v99 := int64(r.Int63()) if r.Intn(2) == 0 { v99 *= -1 } - this.InErrors = &v99 + this.InCsumErrors = &v99 } if r.Intn(10) != 0 { v100 := int64(r.Int63()) if r.Intn(2) == 0 { v100 *= -1 } - this.InCsumErrors = &v100 + this.InDestUnreachs = &v100 } if r.Intn(10) != 0 { v101 := int64(r.Int63()) if r.Intn(2) == 0 { v101 *= -1 } - this.InDestUnreachs = &v101 + this.InTimeExcds = &v101 } if r.Intn(10) != 0 { v102 := int64(r.Int63()) if r.Intn(2) == 0 { v102 *= -1 } - this.InTimeExcds = &v102 + this.InParmProbs = &v102 } if r.Intn(10) != 0 { v103 := int64(r.Int63()) if r.Intn(2) == 0 { v103 *= -1 } - this.InParmProbs = &v103 + this.InSrcQuenchs = &v103 } if r.Intn(10) != 0 { v104 := int64(r.Int63()) if r.Intn(2) == 0 { v104 *= -1 } - this.InSrcQuenchs = &v104 + this.InRedirects = &v104 } if r.Intn(10) != 0 { v105 := int64(r.Int63()) if r.Intn(2) == 0 { v105 *= -1 } - this.InRedirects = &v105 + this.InEchos = &v105 } if r.Intn(10) != 0 { v106 := int64(r.Int63()) if r.Intn(2) == 0 { v106 *= -1 } - this.InEchos = &v106 + this.InEchoReps = &v106 } if r.Intn(10) != 0 { v107 := int64(r.Int63()) if r.Intn(2) == 0 { v107 *= -1 } - this.InEchoReps = &v107 + this.InTimestamps = &v107 } if r.Intn(10) != 0 { v108 := int64(r.Int63()) if r.Intn(2) == 0 { v108 *= -1 } - this.InTimestamps = &v108 + this.InTimestampReps = &v108 } if r.Intn(10) != 0 { v109 := int64(r.Int63()) if r.Intn(2) == 0 { v109 *= -1 } - this.InTimestampReps = &v109 + this.InAddrMasks = &v109 } if r.Intn(10) != 0 { v110 := int64(r.Int63()) if r.Intn(2) == 0 { v110 *= -1 } - this.InAddrMasks = &v110 + this.InAddrMaskReps = &v110 } if r.Intn(10) != 0 { v111 := int64(r.Int63()) if r.Intn(2) == 0 { v111 *= -1 } - this.InAddrMaskReps = &v111 + this.OutMsgs = &v111 } if r.Intn(10) != 0 { v112 := int64(r.Int63()) if r.Intn(2) == 0 { v112 *= -1 } - this.OutMsgs = &v112 + this.OutErrors = &v112 } if r.Intn(10) != 0 { v113 := int64(r.Int63()) if r.Intn(2) == 0 { v113 *= -1 } - this.OutErrors = &v113 + this.OutDestUnreachs = &v113 } if r.Intn(10) != 0 { v114 := int64(r.Int63()) if r.Intn(2) == 0 { v114 *= -1 } - this.OutDestUnreachs = &v114 + this.OutTimeExcds = &v114 } if r.Intn(10) != 0 { v115 := int64(r.Int63()) if r.Intn(2) == 0 { v115 *= -1 } - this.OutTimeExcds = &v115 + this.OutParmProbs = &v115 } if r.Intn(10) != 0 { v116 := int64(r.Int63()) if r.Intn(2) == 0 { v116 *= -1 } - this.OutParmProbs = &v116 + this.OutSrcQuenchs = &v116 } if r.Intn(10) != 0 { v117 := int64(r.Int63()) if r.Intn(2) == 0 { v117 *= -1 } - this.OutSrcQuenchs = &v117 + this.OutRedirects = &v117 } if r.Intn(10) != 0 { v118 := int64(r.Int63()) if r.Intn(2) == 0 { v118 *= -1 } - this.OutRedirects = &v118 + this.OutEchos = &v118 } if r.Intn(10) != 0 { v119 := int64(r.Int63()) if r.Intn(2) == 0 { v119 *= -1 } - this.OutEchos = &v119 + this.OutEchoReps = &v119 } if r.Intn(10) != 0 { v120 := int64(r.Int63()) if r.Intn(2) == 0 { v120 *= -1 } - this.OutEchoReps = &v120 + this.OutTimestamps = &v120 } if r.Intn(10) != 0 { v121 := int64(r.Int63()) if r.Intn(2) == 0 { v121 *= -1 } - this.OutTimestamps = &v121 + this.OutTimestampReps = &v121 } if r.Intn(10) != 0 { v122 := int64(r.Int63()) if r.Intn(2) == 0 { v122 *= -1 } - this.OutTimestampReps = &v122 + this.OutAddrMasks = &v122 } if r.Intn(10) != 0 { v123 := int64(r.Int63()) if r.Intn(2) == 0 { v123 *= -1 } - this.OutAddrMasks = &v123 - } - if r.Intn(10) != 0 { - v124 := int64(r.Int63()) - if r.Intn(2) == 0 { - v124 *= -1 - } - this.OutAddrMaskReps = &v124 + this.OutAddrMaskReps = &v123 } if !easy && r.Intn(10) != 0 { } @@ -30489,110 +30494,110 @@ func NewPopulatedIcmpStatistics(r randyMesos, easy bool) *IcmpStatistics { func NewPopulatedTcpStatistics(r randyMesos, easy bool) *TcpStatistics { this := &TcpStatistics{} + if r.Intn(10) != 0 { + v124 := int64(r.Int63()) + if r.Intn(2) == 0 { + v124 *= -1 + } + this.RtoAlgorithm = &v124 + } if r.Intn(10) != 0 { v125 := int64(r.Int63()) if r.Intn(2) == 0 { v125 *= -1 } - this.RtoAlgorithm = &v125 + this.RtoMin = &v125 } if r.Intn(10) != 0 { v126 := int64(r.Int63()) if r.Intn(2) == 0 { v126 *= -1 } - this.RtoMin = &v126 + this.RtoMax = &v126 } if r.Intn(10) != 0 { v127 := int64(r.Int63()) if r.Intn(2) == 0 { v127 *= -1 } - this.RtoMax = &v127 + this.MaxConn = &v127 } if r.Intn(10) != 0 { v128 := int64(r.Int63()) if r.Intn(2) == 0 { v128 *= -1 } - this.MaxConn = &v128 + this.ActiveOpens = &v128 } if r.Intn(10) != 0 { v129 := int64(r.Int63()) if r.Intn(2) == 0 { v129 *= -1 } - this.ActiveOpens = &v129 + this.PassiveOpens = &v129 } if r.Intn(10) != 0 { v130 := int64(r.Int63()) if r.Intn(2) == 0 { v130 *= -1 } - this.PassiveOpens = &v130 + this.AttemptFails = &v130 } if r.Intn(10) != 0 { v131 := int64(r.Int63()) if r.Intn(2) == 0 { v131 *= -1 } - this.AttemptFails = &v131 + this.EstabResets = &v131 } if r.Intn(10) != 0 { v132 := int64(r.Int63()) if r.Intn(2) == 0 { v132 *= -1 } - this.EstabResets = &v132 + this.CurrEstab = &v132 } if r.Intn(10) != 0 { v133 := int64(r.Int63()) if r.Intn(2) == 0 { v133 *= -1 } - this.CurrEstab = &v133 + this.InSegs = &v133 } if r.Intn(10) != 0 { v134 := int64(r.Int63()) if r.Intn(2) == 0 { v134 *= -1 } - this.InSegs = &v134 + this.OutSegs = &v134 } if r.Intn(10) != 0 { v135 := int64(r.Int63()) if r.Intn(2) == 0 { v135 *= -1 } - this.OutSegs = &v135 + this.RetransSegs = &v135 } if r.Intn(10) != 0 { v136 := int64(r.Int63()) if r.Intn(2) == 0 { v136 *= -1 } - this.RetransSegs = &v136 + this.InErrs = &v136 } if r.Intn(10) != 0 { v137 := int64(r.Int63()) if r.Intn(2) == 0 { v137 *= -1 } - this.InErrs = &v137 + this.OutRsts = &v137 } if r.Intn(10) != 0 { v138 := int64(r.Int63()) if r.Intn(2) == 0 { v138 *= -1 } - this.OutRsts = &v138 - } - if r.Intn(10) != 0 { - v139 := int64(r.Int63()) - if r.Intn(2) == 0 { - v139 *= -1 - } - this.InCsumErrors = &v139 + this.InCsumErrors = &v138 } if !easy && r.Intn(10) != 0 { } @@ -30601,61 +30606,61 @@ func NewPopulatedTcpStatistics(r randyMesos, easy bool) *TcpStatistics { func NewPopulatedUdpStatistics(r randyMesos, easy bool) *UdpStatistics { this := &UdpStatistics{} + if r.Intn(10) != 0 { + v139 := int64(r.Int63()) + if r.Intn(2) == 0 { + v139 *= -1 + } + this.InDatagrams = &v139 + } if r.Intn(10) != 0 { v140 := int64(r.Int63()) if r.Intn(2) == 0 { v140 *= -1 } - this.InDatagrams = &v140 + this.NoPorts = &v140 } if r.Intn(10) != 0 { v141 := int64(r.Int63()) if r.Intn(2) == 0 { v141 *= -1 } - this.NoPorts = &v141 + this.InErrors = &v141 } if r.Intn(10) != 0 { v142 := int64(r.Int63()) if r.Intn(2) == 0 { v142 *= -1 } - this.InErrors = &v142 + this.OutDatagrams = &v142 } if r.Intn(10) != 0 { v143 := int64(r.Int63()) if r.Intn(2) == 0 { v143 *= -1 } - this.OutDatagrams = &v143 + this.RcvbufErrors = &v143 } if r.Intn(10) != 0 { v144 := int64(r.Int63()) if r.Intn(2) == 0 { v144 *= -1 } - this.RcvbufErrors = &v144 + this.SndbufErrors = &v144 } if r.Intn(10) != 0 { v145 := int64(r.Int63()) if r.Intn(2) == 0 { v145 *= -1 } - this.SndbufErrors = &v145 + this.InCsumErrors = &v145 } if r.Intn(10) != 0 { v146 := int64(r.Int63()) if r.Intn(2) == 0 { v146 *= -1 } - this.InCsumErrors = &v146 - } - if r.Intn(10) != 0 { - v147 := int64(r.Int63()) - if r.Intn(2) == 0 { - v147 *= -1 - } - this.IgnoredMulti = &v147 + this.IgnoredMulti = &v146 } if !easy && r.Intn(10) != 0 { } @@ -30687,198 +30692,198 @@ func NewPopulatedResourceStatistics(r randyMesos, easy bool) *ResourceStatistics if r.Intn(2) == 0 { this.Timestamp *= -1 } + if r.Intn(10) != 0 { + v147 := float64(r.Float64()) + if r.Intn(2) == 0 { + v147 *= -1 + } + this.CPUsUserTimeSecs = &v147 + } if r.Intn(10) != 0 { v148 := float64(r.Float64()) if r.Intn(2) == 0 { v148 *= -1 } - this.CPUsUserTimeSecs = &v148 + this.CPUsSystemTimeSecs = &v148 } if r.Intn(10) != 0 { v149 := float64(r.Float64()) if r.Intn(2) == 0 { v149 *= -1 } - this.CPUsSystemTimeSecs = &v149 + this.CPUsLimit = &v149 } if r.Intn(10) != 0 { - v150 := float64(r.Float64()) - if r.Intn(2) == 0 { - v150 *= -1 - } - this.CPUsLimit = &v150 + v150 := uint64(uint64(r.Uint32())) + this.MemRSSBytes = &v150 } if r.Intn(10) != 0 { v151 := uint64(uint64(r.Uint32())) - this.MemRSSBytes = &v151 + this.MemLimitBytes = &v151 } if r.Intn(10) != 0 { - v152 := uint64(uint64(r.Uint32())) - this.MemLimitBytes = &v152 + v152 := uint32(r.Uint32()) + this.CPUsNrPeriods = &v152 } if r.Intn(10) != 0 { v153 := uint32(r.Uint32()) - this.CPUsNrPeriods = &v153 - } - if r.Intn(10) != 0 { - v154 := uint32(r.Uint32()) - this.CPUsNrThrottled = &v154 + this.CPUsNrThrottled = &v153 } if r.Intn(10) != 0 { - v155 := float64(r.Float64()) + v154 := float64(r.Float64()) if r.Intn(2) == 0 { - v155 *= -1 + v154 *= -1 } - this.CPUsThrottledTimeSecs = &v155 + this.CPUsThrottledTimeSecs = &v154 + } + if r.Intn(10) != 0 { + v155 := uint64(uint64(r.Uint32())) + this.MemFileBytes = &v155 } if r.Intn(10) != 0 { v156 := uint64(uint64(r.Uint32())) - this.MemFileBytes = &v156 + this.MemAnonBytes = &v156 } if r.Intn(10) != 0 { v157 := uint64(uint64(r.Uint32())) - this.MemAnonBytes = &v157 + this.MemMappedFileBytes = &v157 } if r.Intn(10) != 0 { - v158 := uint64(uint64(r.Uint32())) - this.MemMappedFileBytes = &v158 + this.Perf = NewPopulatedPerfStatistics(r, easy) } if r.Intn(10) != 0 { - this.Perf = NewPopulatedPerfStatistics(r, easy) + v158 := uint64(uint64(r.Uint32())) + this.NetRxPackets = &v158 } if r.Intn(10) != 0 { v159 := uint64(uint64(r.Uint32())) - this.NetRxPackets = &v159 + this.NetRxBytes = &v159 } if r.Intn(10) != 0 { v160 := uint64(uint64(r.Uint32())) - this.NetRxBytes = &v160 + this.NetRxErrors = &v160 } if r.Intn(10) != 0 { v161 := uint64(uint64(r.Uint32())) - this.NetRxErrors = &v161 + this.NetRxDropped = &v161 } if r.Intn(10) != 0 { v162 := uint64(uint64(r.Uint32())) - this.NetRxDropped = &v162 + this.NetTxPackets = &v162 } if r.Intn(10) != 0 { v163 := uint64(uint64(r.Uint32())) - this.NetTxPackets = &v163 + this.NetTxBytes = &v163 } if r.Intn(10) != 0 { v164 := uint64(uint64(r.Uint32())) - this.NetTxBytes = &v164 + this.NetTxErrors = &v164 } if r.Intn(10) != 0 { v165 := uint64(uint64(r.Uint32())) - this.NetTxErrors = &v165 + this.NetTxDropped = &v165 } if r.Intn(10) != 0 { - v166 := uint64(uint64(r.Uint32())) - this.NetTxDropped = &v166 + v166 := float64(r.Float64()) + if r.Intn(2) == 0 { + v166 *= -1 + } + this.NetTCPRttMicrosecsP50 = &v166 } if r.Intn(10) != 0 { v167 := float64(r.Float64()) if r.Intn(2) == 0 { v167 *= -1 } - this.NetTCPRttMicrosecsP50 = &v167 + this.NetTCPRttMicrosecsP90 = &v167 } if r.Intn(10) != 0 { v168 := float64(r.Float64()) if r.Intn(2) == 0 { v168 *= -1 } - this.NetTCPRttMicrosecsP90 = &v168 + this.NetTCPRttMicrosecsP95 = &v168 } if r.Intn(10) != 0 { v169 := float64(r.Float64()) if r.Intn(2) == 0 { v169 *= -1 } - this.NetTCPRttMicrosecsP95 = &v169 + this.NetTCPRttMicrosecsP99 = &v169 } if r.Intn(10) != 0 { - v170 := float64(r.Float64()) - if r.Intn(2) == 0 { - v170 *= -1 - } - this.NetTCPRttMicrosecsP99 = &v170 + v170 := uint64(uint64(r.Uint32())) + this.DiskLimitBytes = &v170 } if r.Intn(10) != 0 { v171 := uint64(uint64(r.Uint32())) - this.DiskLimitBytes = &v171 + this.DiskUsedBytes = &v171 } if r.Intn(10) != 0 { - v172 := uint64(uint64(r.Uint32())) - this.DiskUsedBytes = &v172 + v172 := float64(r.Float64()) + if r.Intn(2) == 0 { + v172 *= -1 + } + this.NetTCPActiveConnections = &v172 } if r.Intn(10) != 0 { v173 := float64(r.Float64()) if r.Intn(2) == 0 { v173 *= -1 } - this.NetTCPActiveConnections = &v173 + this.NetTCPTimeWaitConnections = &v173 } if r.Intn(10) != 0 { - v174 := float64(r.Float64()) - if r.Intn(2) == 0 { - v174 *= -1 - } - this.NetTCPTimeWaitConnections = &v174 + v174 := uint32(r.Uint32()) + this.Processes = &v174 } if r.Intn(10) != 0 { v175 := uint32(r.Uint32()) - this.Processes = &v175 + this.Threads = &v175 } if r.Intn(10) != 0 { - v176 := uint32(r.Uint32()) - this.Threads = &v176 + v176 := uint64(uint64(r.Uint32())) + this.MemLowPressureCounter = &v176 } if r.Intn(10) != 0 { v177 := uint64(uint64(r.Uint32())) - this.MemLowPressureCounter = &v177 + this.MemMediumPressureCounter = &v177 } if r.Intn(10) != 0 { v178 := uint64(uint64(r.Uint32())) - this.MemMediumPressureCounter = &v178 + this.MemCriticalPressureCounter = &v178 } if r.Intn(10) != 0 { - v179 := uint64(uint64(r.Uint32())) - this.MemCriticalPressureCounter = &v179 + v179 := r.Intn(5) + this.NetTrafficControlStatistics = make([]TrafficControlStatistics, v179) + for i := 0; i < v179; i++ { + v180 := NewPopulatedTrafficControlStatistics(r, easy) + this.NetTrafficControlStatistics[i] = *v180 + } } if r.Intn(10) != 0 { - v180 := r.Intn(5) - this.NetTrafficControlStatistics = make([]TrafficControlStatistics, v180) - for i := 0; i < v180; i++ { - v181 := NewPopulatedTrafficControlStatistics(r, easy) - this.NetTrafficControlStatistics[i] = *v181 - } + v181 := uint64(uint64(r.Uint32())) + this.MemTotalBytes = &v181 } if r.Intn(10) != 0 { v182 := uint64(uint64(r.Uint32())) - this.MemTotalBytes = &v182 + this.MemTotalMemswBytes = &v182 } if r.Intn(10) != 0 { v183 := uint64(uint64(r.Uint32())) - this.MemTotalMemswBytes = &v183 + this.MemSoftLimitBytes = &v183 } if r.Intn(10) != 0 { v184 := uint64(uint64(r.Uint32())) - this.MemSoftLimitBytes = &v184 + this.MemCacheBytes = &v184 } if r.Intn(10) != 0 { v185 := uint64(uint64(r.Uint32())) - this.MemCacheBytes = &v185 + this.MemSwapBytes = &v185 } if r.Intn(10) != 0 { v186 := uint64(uint64(r.Uint32())) - this.MemSwapBytes = &v186 - } - if r.Intn(10) != 0 { - v187 := uint64(uint64(r.Uint32())) - this.MemUnevictableBytes = &v187 + this.MemUnevictableBytes = &v186 } if r.Intn(10) != 0 { this.NetSNMPStatistics = NewPopulatedSNMPStatistics(r, easy) @@ -30891,19 +30896,19 @@ func NewPopulatedResourceStatistics(r randyMesos, easy bool) *ResourceStatistics func NewPopulatedResourceUsage(r randyMesos, easy bool) *ResourceUsage { this := &ResourceUsage{} if r.Intn(10) == 0 { - v188 := r.Intn(5) - this.Executors = make([]ResourceUsage_Executor, v188) - for i := 0; i < v188; i++ { - v189 := NewPopulatedResourceUsage_Executor(r, easy) - this.Executors[i] = *v189 + v187 := r.Intn(5) + this.Executors = make([]ResourceUsage_Executor, v187) + for i := 0; i < v187; i++ { + v188 := NewPopulatedResourceUsage_Executor(r, easy) + this.Executors[i] = *v188 } } if r.Intn(10) != 0 { - v190 := r.Intn(5) - this.Total = make([]Resource, v190) - for i := 0; i < v190; i++ { - v191 := NewPopulatedResource(r, easy) - this.Total[i] = *v191 + v189 := r.Intn(5) + this.Total = make([]Resource, v189) + for i := 0; i < v189; i++ { + v190 := NewPopulatedResource(r, easy) + this.Total[i] = *v190 } } if !easy && r.Intn(10) != 0 { @@ -30913,27 +30918,27 @@ func NewPopulatedResourceUsage(r randyMesos, easy bool) *ResourceUsage { func NewPopulatedResourceUsage_Executor(r randyMesos, easy bool) *ResourceUsage_Executor { this := &ResourceUsage_Executor{} - v192 := NewPopulatedExecutorInfo(r, easy) - this.ExecutorInfo = *v192 + v191 := NewPopulatedExecutorInfo(r, easy) + this.ExecutorInfo = *v191 if r.Intn(10) != 0 { - v193 := r.Intn(5) - this.Allocated = make([]Resource, v193) - for i := 0; i < v193; i++ { - v194 := NewPopulatedResource(r, easy) - this.Allocated[i] = *v194 + v192 := r.Intn(5) + this.Allocated = make([]Resource, v192) + for i := 0; i < v192; i++ { + v193 := NewPopulatedResource(r, easy) + this.Allocated[i] = *v193 } } if r.Intn(10) != 0 { this.Statistics = NewPopulatedResourceStatistics(r, easy) } - v195 := NewPopulatedContainerID(r, easy) - this.ContainerID = *v195 + v194 := NewPopulatedContainerID(r, easy) + this.ContainerID = *v194 if r.Intn(10) != 0 { - v196 := r.Intn(5) - this.Tasks = make([]ResourceUsage_Executor_Task, v196) - for i := 0; i < v196; i++ { - v197 := NewPopulatedResourceUsage_Executor_Task(r, easy) - this.Tasks[i] = *v197 + v195 := r.Intn(5) + this.Tasks = make([]ResourceUsage_Executor_Task, v195) + for i := 0; i < v195; i++ { + v196 := NewPopulatedResourceUsage_Executor_Task(r, easy) + this.Tasks[i] = *v196 } } if !easy && r.Intn(10) != 0 { @@ -30944,14 +30949,14 @@ func NewPopulatedResourceUsage_Executor(r randyMesos, easy bool) *ResourceUsage_ func NewPopulatedResourceUsage_Executor_Task(r randyMesos, easy bool) *ResourceUsage_Executor_Task { this := &ResourceUsage_Executor_Task{} this.Name = string(randStringMesos(r)) - v198 := NewPopulatedTaskID(r, easy) - this.ID = *v198 + v197 := NewPopulatedTaskID(r, easy) + this.ID = *v197 if r.Intn(10) != 0 { - v199 := r.Intn(5) - this.Resources = make([]Resource, v199) - for i := 0; i < v199; i++ { - v200 := NewPopulatedResource(r, easy) - this.Resources[i] = *v200 + v198 := r.Intn(5) + this.Resources = make([]Resource, v198) + for i := 0; i < v198; i++ { + v199 := NewPopulatedResource(r, easy) + this.Resources[i] = *v199 } } if r.Intn(10) != 0 { @@ -30972,215 +30977,215 @@ func NewPopulatedPerfStatistics(r randyMesos, easy bool) *PerfStatistics { if r.Intn(2) == 0 { this.Duration *= -1 } + if r.Intn(10) != 0 { + v200 := uint64(uint64(r.Uint32())) + this.Cycles = &v200 + } if r.Intn(10) != 0 { v201 := uint64(uint64(r.Uint32())) - this.Cycles = &v201 + this.StalledCyclesFrontend = &v201 } if r.Intn(10) != 0 { v202 := uint64(uint64(r.Uint32())) - this.StalledCyclesFrontend = &v202 + this.StalledCyclesBackend = &v202 } if r.Intn(10) != 0 { v203 := uint64(uint64(r.Uint32())) - this.StalledCyclesBackend = &v203 + this.Instructions = &v203 } if r.Intn(10) != 0 { v204 := uint64(uint64(r.Uint32())) - this.Instructions = &v204 + this.CacheReferences = &v204 } if r.Intn(10) != 0 { v205 := uint64(uint64(r.Uint32())) - this.CacheReferences = &v205 + this.CacheMisses = &v205 } if r.Intn(10) != 0 { v206 := uint64(uint64(r.Uint32())) - this.CacheMisses = &v206 + this.Branches = &v206 } if r.Intn(10) != 0 { v207 := uint64(uint64(r.Uint32())) - this.Branches = &v207 + this.BranchMisses = &v207 } if r.Intn(10) != 0 { v208 := uint64(uint64(r.Uint32())) - this.BranchMisses = &v208 + this.BusCycles = &v208 } if r.Intn(10) != 0 { v209 := uint64(uint64(r.Uint32())) - this.BusCycles = &v209 + this.RefCycles = &v209 } if r.Intn(10) != 0 { - v210 := uint64(uint64(r.Uint32())) - this.RefCycles = &v210 + v210 := float64(r.Float64()) + if r.Intn(2) == 0 { + v210 *= -1 + } + this.CPUClock = &v210 } if r.Intn(10) != 0 { v211 := float64(r.Float64()) if r.Intn(2) == 0 { v211 *= -1 } - this.CPUClock = &v211 + this.TaskClock = &v211 } if r.Intn(10) != 0 { - v212 := float64(r.Float64()) - if r.Intn(2) == 0 { - v212 *= -1 - } - this.TaskClock = &v212 + v212 := uint64(uint64(r.Uint32())) + this.PageFaults = &v212 } if r.Intn(10) != 0 { v213 := uint64(uint64(r.Uint32())) - this.PageFaults = &v213 + this.MinorFaults = &v213 } if r.Intn(10) != 0 { v214 := uint64(uint64(r.Uint32())) - this.MinorFaults = &v214 + this.MajorFaults = &v214 } if r.Intn(10) != 0 { v215 := uint64(uint64(r.Uint32())) - this.MajorFaults = &v215 + this.ContextSwitches = &v215 } if r.Intn(10) != 0 { v216 := uint64(uint64(r.Uint32())) - this.ContextSwitches = &v216 + this.CPUMigrations = &v216 } if r.Intn(10) != 0 { v217 := uint64(uint64(r.Uint32())) - this.CPUMigrations = &v217 + this.AlignmentFaults = &v217 } if r.Intn(10) != 0 { v218 := uint64(uint64(r.Uint32())) - this.AlignmentFaults = &v218 + this.EmulationFaults = &v218 } if r.Intn(10) != 0 { v219 := uint64(uint64(r.Uint32())) - this.EmulationFaults = &v219 + this.L1DcacheLoads = &v219 } if r.Intn(10) != 0 { v220 := uint64(uint64(r.Uint32())) - this.L1DcacheLoads = &v220 + this.L1DcacheLoadMisses = &v220 } if r.Intn(10) != 0 { v221 := uint64(uint64(r.Uint32())) - this.L1DcacheLoadMisses = &v221 + this.L1DcacheStores = &v221 } if r.Intn(10) != 0 { v222 := uint64(uint64(r.Uint32())) - this.L1DcacheStores = &v222 + this.L1DcacheStoreMisses = &v222 } if r.Intn(10) != 0 { v223 := uint64(uint64(r.Uint32())) - this.L1DcacheStoreMisses = &v223 + this.L1DcachePrefetches = &v223 } if r.Intn(10) != 0 { v224 := uint64(uint64(r.Uint32())) - this.L1DcachePrefetches = &v224 + this.L1DcachePrefetchMisses = &v224 } if r.Intn(10) != 0 { v225 := uint64(uint64(r.Uint32())) - this.L1DcachePrefetchMisses = &v225 + this.L1IcacheLoads = &v225 } if r.Intn(10) != 0 { v226 := uint64(uint64(r.Uint32())) - this.L1IcacheLoads = &v226 + this.L1IcacheLoadMisses = &v226 } if r.Intn(10) != 0 { v227 := uint64(uint64(r.Uint32())) - this.L1IcacheLoadMisses = &v227 + this.L1IcachePrefetches = &v227 } if r.Intn(10) != 0 { v228 := uint64(uint64(r.Uint32())) - this.L1IcachePrefetches = &v228 + this.L1IcachePrefetchMisses = &v228 } if r.Intn(10) != 0 { v229 := uint64(uint64(r.Uint32())) - this.L1IcachePrefetchMisses = &v229 + this.LLCLoads = &v229 } if r.Intn(10) != 0 { v230 := uint64(uint64(r.Uint32())) - this.LLCLoads = &v230 + this.LLCLoadMisses = &v230 } if r.Intn(10) != 0 { v231 := uint64(uint64(r.Uint32())) - this.LLCLoadMisses = &v231 + this.LLCStores = &v231 } if r.Intn(10) != 0 { v232 := uint64(uint64(r.Uint32())) - this.LLCStores = &v232 + this.LLCStoreMisses = &v232 } if r.Intn(10) != 0 { v233 := uint64(uint64(r.Uint32())) - this.LLCStoreMisses = &v233 + this.LLCPrefetches = &v233 } if r.Intn(10) != 0 { v234 := uint64(uint64(r.Uint32())) - this.LLCPrefetches = &v234 + this.LLCPrefetchMisses = &v234 } if r.Intn(10) != 0 { v235 := uint64(uint64(r.Uint32())) - this.LLCPrefetchMisses = &v235 + this.DTLBLoads = &v235 } if r.Intn(10) != 0 { v236 := uint64(uint64(r.Uint32())) - this.DTLBLoads = &v236 + this.DTLBLoadMisses = &v236 } if r.Intn(10) != 0 { v237 := uint64(uint64(r.Uint32())) - this.DTLBLoadMisses = &v237 + this.DTLBStores = &v237 } if r.Intn(10) != 0 { v238 := uint64(uint64(r.Uint32())) - this.DTLBStores = &v238 + this.DTLBStoreMisses = &v238 } if r.Intn(10) != 0 { v239 := uint64(uint64(r.Uint32())) - this.DTLBStoreMisses = &v239 + this.DTLBPrefetches = &v239 } if r.Intn(10) != 0 { v240 := uint64(uint64(r.Uint32())) - this.DTLBPrefetches = &v240 + this.DTLBPrefetchMisses = &v240 } if r.Intn(10) != 0 { v241 := uint64(uint64(r.Uint32())) - this.DTLBPrefetchMisses = &v241 + this.ITLBLoads = &v241 } if r.Intn(10) != 0 { v242 := uint64(uint64(r.Uint32())) - this.ITLBLoads = &v242 + this.ITLBLoadMisses = &v242 } if r.Intn(10) != 0 { v243 := uint64(uint64(r.Uint32())) - this.ITLBLoadMisses = &v243 + this.BranchLoads = &v243 } if r.Intn(10) != 0 { v244 := uint64(uint64(r.Uint32())) - this.BranchLoads = &v244 + this.BranchLoadMisses = &v244 } if r.Intn(10) != 0 { v245 := uint64(uint64(r.Uint32())) - this.BranchLoadMisses = &v245 + this.NodeLoads = &v245 } if r.Intn(10) != 0 { v246 := uint64(uint64(r.Uint32())) - this.NodeLoads = &v246 + this.NodeLoadMisses = &v246 } if r.Intn(10) != 0 { v247 := uint64(uint64(r.Uint32())) - this.NodeLoadMisses = &v247 + this.NodeStores = &v247 } if r.Intn(10) != 0 { v248 := uint64(uint64(r.Uint32())) - this.NodeStores = &v248 + this.NodeStoreMisses = &v248 } if r.Intn(10) != 0 { v249 := uint64(uint64(r.Uint32())) - this.NodeStoreMisses = &v249 + this.NodePrefetches = &v249 } if r.Intn(10) != 0 { v250 := uint64(uint64(r.Uint32())) - this.NodePrefetches = &v250 - } - if r.Intn(10) != 0 { - v251 := uint64(uint64(r.Uint32())) - this.NodePrefetchMisses = &v251 + this.NodePrefetchMisses = &v250 } if !easy && r.Intn(10) != 0 { } @@ -31193,11 +31198,11 @@ func NewPopulatedRequest(r randyMesos, easy bool) *Request { this.AgentID = NewPopulatedAgentID(r, easy) } if r.Intn(10) != 0 { - v252 := r.Intn(5) - this.Resources = make([]Resource, v252) - for i := 0; i < v252; i++ { - v253 := NewPopulatedResource(r, easy) - this.Resources[i] = *v253 + v251 := r.Intn(5) + this.Resources = make([]Resource, v251) + for i := 0; i < v251; i++ { + v252 := NewPopulatedResource(r, easy) + this.Resources[i] = *v252 } } if !easy && r.Intn(10) != 0 { @@ -31207,35 +31212,35 @@ func NewPopulatedRequest(r randyMesos, easy bool) *Request { func NewPopulatedOffer(r randyMesos, easy bool) *Offer { this := &Offer{} - v254 := NewPopulatedOfferID(r, easy) - this.ID = *v254 - v255 := NewPopulatedFrameworkID(r, easy) - this.FrameworkID = *v255 - v256 := NewPopulatedAgentID(r, easy) - this.AgentID = *v256 + v253 := NewPopulatedOfferID(r, easy) + this.ID = *v253 + v254 := NewPopulatedFrameworkID(r, easy) + this.FrameworkID = *v254 + v255 := NewPopulatedAgentID(r, easy) + this.AgentID = *v255 this.Hostname = string(randStringMesos(r)) if r.Intn(10) != 0 { - v257 := r.Intn(5) - this.Resources = make([]Resource, v257) - for i := 0; i < v257; i++ { - v258 := NewPopulatedResource(r, easy) - this.Resources[i] = *v258 + v256 := r.Intn(5) + this.Resources = make([]Resource, v256) + for i := 0; i < v256; i++ { + v257 := NewPopulatedResource(r, easy) + this.Resources[i] = *v257 } } if r.Intn(10) != 0 { - v259 := r.Intn(5) - this.ExecutorIDs = make([]ExecutorID, v259) - for i := 0; i < v259; i++ { - v260 := NewPopulatedExecutorID(r, easy) - this.ExecutorIDs[i] = *v260 + v258 := r.Intn(5) + this.ExecutorIDs = make([]ExecutorID, v258) + for i := 0; i < v258; i++ { + v259 := NewPopulatedExecutorID(r, easy) + this.ExecutorIDs[i] = *v259 } } if r.Intn(10) != 0 { - v261 := r.Intn(5) - this.Attributes = make([]Attribute, v261) - for i := 0; i < v261; i++ { - v262 := NewPopulatedAttribute(r, easy) - this.Attributes[i] = *v262 + v260 := r.Intn(5) + this.Attributes = make([]Attribute, v260) + for i := 0; i < v260; i++ { + v261 := NewPopulatedAttribute(r, easy) + this.Attributes[i] = *v261 } } if r.Intn(10) != 0 { @@ -31281,11 +31286,11 @@ func NewPopulatedOffer_Operation(r randyMesos, easy bool) *Offer_Operation { func NewPopulatedOffer_Operation_Launch(r randyMesos, easy bool) *Offer_Operation_Launch { this := &Offer_Operation_Launch{} if r.Intn(10) != 0 { - v263 := r.Intn(5) - this.TaskInfos = make([]TaskInfo, v263) - for i := 0; i < v263; i++ { - v264 := NewPopulatedTaskInfo(r, easy) - this.TaskInfos[i] = *v264 + v262 := r.Intn(5) + this.TaskInfos = make([]TaskInfo, v262) + for i := 0; i < v262; i++ { + v263 := NewPopulatedTaskInfo(r, easy) + this.TaskInfos[i] = *v263 } } if !easy && r.Intn(10) != 0 { @@ -31295,10 +31300,10 @@ func NewPopulatedOffer_Operation_Launch(r randyMesos, easy bool) *Offer_Operatio func NewPopulatedOffer_Operation_LaunchGroup(r randyMesos, easy bool) *Offer_Operation_LaunchGroup { this := &Offer_Operation_LaunchGroup{} - v265 := NewPopulatedExecutorInfo(r, easy) - this.Executor = *v265 - v266 := NewPopulatedTaskGroupInfo(r, easy) - this.TaskGroup = *v266 + v264 := NewPopulatedExecutorInfo(r, easy) + this.Executor = *v264 + v265 := NewPopulatedTaskGroupInfo(r, easy) + this.TaskGroup = *v265 if !easy && r.Intn(10) != 0 { } return this @@ -31307,11 +31312,11 @@ func NewPopulatedOffer_Operation_LaunchGroup(r randyMesos, easy bool) *Offer_Ope func NewPopulatedOffer_Operation_Reserve(r randyMesos, easy bool) *Offer_Operation_Reserve { this := &Offer_Operation_Reserve{} if r.Intn(10) != 0 { - v267 := r.Intn(5) - this.Resources = make([]Resource, v267) - for i := 0; i < v267; i++ { - v268 := NewPopulatedResource(r, easy) - this.Resources[i] = *v268 + v266 := r.Intn(5) + this.Resources = make([]Resource, v266) + for i := 0; i < v266; i++ { + v267 := NewPopulatedResource(r, easy) + this.Resources[i] = *v267 } } if !easy && r.Intn(10) != 0 { @@ -31322,11 +31327,11 @@ func NewPopulatedOffer_Operation_Reserve(r randyMesos, easy bool) *Offer_Operati func NewPopulatedOffer_Operation_Unreserve(r randyMesos, easy bool) *Offer_Operation_Unreserve { this := &Offer_Operation_Unreserve{} if r.Intn(10) != 0 { - v269 := r.Intn(5) - this.Resources = make([]Resource, v269) - for i := 0; i < v269; i++ { - v270 := NewPopulatedResource(r, easy) - this.Resources[i] = *v270 + v268 := r.Intn(5) + this.Resources = make([]Resource, v268) + for i := 0; i < v268; i++ { + v269 := NewPopulatedResource(r, easy) + this.Resources[i] = *v269 } } if !easy && r.Intn(10) != 0 { @@ -31337,11 +31342,11 @@ func NewPopulatedOffer_Operation_Unreserve(r randyMesos, easy bool) *Offer_Opera func NewPopulatedOffer_Operation_Create(r randyMesos, easy bool) *Offer_Operation_Create { this := &Offer_Operation_Create{} if r.Intn(10) != 0 { - v271 := r.Intn(5) - this.Volumes = make([]Resource, v271) - for i := 0; i < v271; i++ { - v272 := NewPopulatedResource(r, easy) - this.Volumes[i] = *v272 + v270 := r.Intn(5) + this.Volumes = make([]Resource, v270) + for i := 0; i < v270; i++ { + v271 := NewPopulatedResource(r, easy) + this.Volumes[i] = *v271 } } if !easy && r.Intn(10) != 0 { @@ -31352,11 +31357,11 @@ func NewPopulatedOffer_Operation_Create(r randyMesos, easy bool) *Offer_Operatio func NewPopulatedOffer_Operation_Destroy(r randyMesos, easy bool) *Offer_Operation_Destroy { this := &Offer_Operation_Destroy{} if r.Intn(10) != 0 { - v273 := r.Intn(5) - this.Volumes = make([]Resource, v273) - for i := 0; i < v273; i++ { - v274 := NewPopulatedResource(r, easy) - this.Volumes[i] = *v274 + v272 := r.Intn(5) + this.Volumes = make([]Resource, v272) + for i := 0; i < v272; i++ { + v273 := NewPopulatedResource(r, easy) + this.Volumes[i] = *v273 } } if !easy && r.Intn(10) != 0 { @@ -31366,24 +31371,24 @@ func NewPopulatedOffer_Operation_Destroy(r randyMesos, easy bool) *Offer_Operati func NewPopulatedInverseOffer(r randyMesos, easy bool) *InverseOffer { this := &InverseOffer{} - v275 := NewPopulatedOfferID(r, easy) - this.OfferID = *v275 + v274 := NewPopulatedOfferID(r, easy) + this.OfferID = *v274 if r.Intn(10) != 0 { this.URL = NewPopulatedURL(r, easy) } - v276 := NewPopulatedFrameworkID(r, easy) - this.FrameworkID = *v276 + v275 := NewPopulatedFrameworkID(r, easy) + this.FrameworkID = *v275 if r.Intn(10) != 0 { this.AgentID = NewPopulatedAgentID(r, easy) } - v277 := NewPopulatedUnavailability(r, easy) - this.Unavailability = *v277 + v276 := NewPopulatedUnavailability(r, easy) + this.Unavailability = *v276 if r.Intn(10) != 0 { - v278 := r.Intn(5) - this.Resources = make([]Resource, v278) - for i := 0; i < v278; i++ { - v279 := NewPopulatedResource(r, easy) - this.Resources[i] = *v279 + v277 := r.Intn(5) + this.Resources = make([]Resource, v277) + for i := 0; i < v277; i++ { + v278 := NewPopulatedResource(r, easy) + this.Resources[i] = *v278 } } if !easy && r.Intn(10) != 0 { @@ -31394,25 +31399,25 @@ func NewPopulatedInverseOffer(r randyMesos, easy bool) *InverseOffer { func NewPopulatedTaskInfo(r randyMesos, easy bool) *TaskInfo { this := &TaskInfo{} this.Name = string(randStringMesos(r)) - v280 := NewPopulatedTaskID(r, easy) - this.TaskID = *v280 - v281 := NewPopulatedAgentID(r, easy) - this.AgentID = *v281 + v279 := NewPopulatedTaskID(r, easy) + this.TaskID = *v279 + v280 := NewPopulatedAgentID(r, easy) + this.AgentID = *v280 if r.Intn(10) != 0 { - v282 := r.Intn(5) - this.Resources = make([]Resource, v282) - for i := 0; i < v282; i++ { - v283 := NewPopulatedResource(r, easy) - this.Resources[i] = *v283 + v281 := r.Intn(5) + this.Resources = make([]Resource, v281) + for i := 0; i < v281; i++ { + v282 := NewPopulatedResource(r, easy) + this.Resources[i] = *v282 } } if r.Intn(10) != 0 { this.Executor = NewPopulatedExecutorInfo(r, easy) } if r.Intn(10) != 0 { - v284 := r.Intn(100) - this.Data = make([]byte, v284) - for i := 0; i < v284; i++ { + v283 := r.Intn(100) + this.Data = make([]byte, v283) + for i := 0; i < v283; i++ { this.Data[i] = byte(r.Intn(256)) } } @@ -31445,11 +31450,11 @@ func NewPopulatedTaskInfo(r randyMesos, easy bool) *TaskInfo { func NewPopulatedTaskGroupInfo(r randyMesos, easy bool) *TaskGroupInfo { this := &TaskGroupInfo{} if r.Intn(10) != 0 { - v285 := r.Intn(5) - this.Tasks = make([]TaskInfo, v285) - for i := 0; i < v285; i++ { - v286 := NewPopulatedTaskInfo(r, easy) - this.Tasks[i] = *v286 + v284 := r.Intn(5) + this.Tasks = make([]TaskInfo, v284) + for i := 0; i < v284; i++ { + v285 := NewPopulatedTaskInfo(r, easy) + this.Tasks[i] = *v285 } } if !easy && r.Intn(10) != 0 { @@ -31460,41 +31465,41 @@ func NewPopulatedTaskGroupInfo(r randyMesos, easy bool) *TaskGroupInfo { func NewPopulatedTask(r randyMesos, easy bool) *Task { this := &Task{} this.Name = string(randStringMesos(r)) - v287 := NewPopulatedTaskID(r, easy) - this.TaskID = *v287 - v288 := NewPopulatedFrameworkID(r, easy) - this.FrameworkID = *v288 + v286 := NewPopulatedTaskID(r, easy) + this.TaskID = *v286 + v287 := NewPopulatedFrameworkID(r, easy) + this.FrameworkID = *v287 if r.Intn(10) != 0 { this.ExecutorID = NewPopulatedExecutorID(r, easy) } - v289 := NewPopulatedAgentID(r, easy) - this.AgentID = *v289 - v290 := TaskState([]int32{6, 0, 1, 8, 2, 3, 4, 7, 5, 9, 10, 11, 12, 13}[r.Intn(14)]) - this.State = &v290 + v288 := NewPopulatedAgentID(r, easy) + this.AgentID = *v288 + v289 := TaskState([]int32{6, 0, 1, 8, 2, 3, 4, 7, 5, 9, 10, 11, 12, 13}[r.Intn(14)]) + this.State = &v289 if r.Intn(10) != 0 { - v291 := r.Intn(5) - this.Resources = make([]Resource, v291) - for i := 0; i < v291; i++ { - v292 := NewPopulatedResource(r, easy) - this.Resources[i] = *v292 + v290 := r.Intn(5) + this.Resources = make([]Resource, v290) + for i := 0; i < v290; i++ { + v291 := NewPopulatedResource(r, easy) + this.Resources[i] = *v291 } } if r.Intn(10) == 0 { - v293 := r.Intn(5) - this.Statuses = make([]TaskStatus, v293) - for i := 0; i < v293; i++ { - v294 := NewPopulatedTaskStatus(r, easy) - this.Statuses[i] = *v294 + v292 := r.Intn(5) + this.Statuses = make([]TaskStatus, v292) + for i := 0; i < v292; i++ { + v293 := NewPopulatedTaskStatus(r, easy) + this.Statuses[i] = *v293 } } if r.Intn(10) != 0 { - v295 := TaskState([]int32{6, 0, 1, 8, 2, 3, 4, 7, 5, 9, 10, 11, 12, 13}[r.Intn(14)]) - this.StatusUpdateState = &v295 + v294 := TaskState([]int32{6, 0, 1, 8, 2, 3, 4, 7, 5, 9, 10, 11, 12, 13}[r.Intn(14)]) + this.StatusUpdateState = &v294 } if r.Intn(10) != 0 { - v296 := r.Intn(100) - this.StatusUpdateUUID = make([]byte, v296) - for i := 0; i < v296; i++ { + v295 := r.Intn(100) + this.StatusUpdateUUID = make([]byte, v295) + for i := 0; i < v295; i++ { this.StatusUpdateUUID[i] = byte(r.Intn(256)) } } @@ -31508,8 +31513,8 @@ func NewPopulatedTask(r randyMesos, easy bool) *Task { this.Container = NewPopulatedContainerInfo(r, easy) } if r.Intn(10) != 0 { - v297 := string(randStringMesos(r)) - this.User = &v297 + v296 := string(randStringMesos(r)) + this.User = &v296 } if !easy && r.Intn(10) != 0 { } @@ -31519,8 +31524,8 @@ func NewPopulatedTask(r randyMesos, easy bool) *Task { func NewPopulatedCheckStatusInfo(r randyMesos, easy bool) *CheckStatusInfo { this := &CheckStatusInfo{} if r.Intn(10) != 0 { - v298 := CheckInfo_Type([]int32{0, 1, 2}[r.Intn(3)]) - this.Type = &v298 + v297 := CheckInfo_Type([]int32{0, 1, 2}[r.Intn(3)]) + this.Type = &v297 } if r.Intn(10) != 0 { this.Command = NewPopulatedCheckStatusInfo_Command(r, easy) @@ -31536,11 +31541,11 @@ func NewPopulatedCheckStatusInfo(r randyMesos, easy bool) *CheckStatusInfo { func NewPopulatedCheckStatusInfo_Command(r randyMesos, easy bool) *CheckStatusInfo_Command { this := &CheckStatusInfo_Command{} if r.Intn(10) != 0 { - v299 := int32(r.Int31()) + v298 := int32(r.Int31()) if r.Intn(2) == 0 { - v299 *= -1 + v298 *= -1 } - this.ExitCode = &v299 + this.ExitCode = &v298 } if !easy && r.Intn(10) != 0 { } @@ -31550,8 +31555,8 @@ func NewPopulatedCheckStatusInfo_Command(r randyMesos, easy bool) *CheckStatusIn func NewPopulatedCheckStatusInfo_Http(r randyMesos, easy bool) *CheckStatusInfo_Http { this := &CheckStatusInfo_Http{} if r.Intn(10) != 0 { - v300 := uint32(r.Uint32()) - this.StatusCode = &v300 + v299 := uint32(r.Uint32()) + this.StatusCode = &v299 } if !easy && r.Intn(10) != 0 { } @@ -31560,50 +31565,50 @@ func NewPopulatedCheckStatusInfo_Http(r randyMesos, easy bool) *CheckStatusInfo_ func NewPopulatedTaskStatus(r randyMesos, easy bool) *TaskStatus { this := &TaskStatus{} - v301 := NewPopulatedTaskID(r, easy) - this.TaskID = *v301 - v302 := TaskState([]int32{6, 0, 1, 8, 2, 3, 4, 7, 5, 9, 10, 11, 12, 13}[r.Intn(14)]) - this.State = &v302 - if r.Intn(10) != 0 { - v303 := r.Intn(100) - this.Data = make([]byte, v303) - for i := 0; i < v303; i++ { + v300 := NewPopulatedTaskID(r, easy) + this.TaskID = *v300 + v301 := TaskState([]int32{6, 0, 1, 8, 2, 3, 4, 7, 5, 9, 10, 11, 12, 13}[r.Intn(14)]) + this.State = &v301 + if r.Intn(10) != 0 { + v302 := r.Intn(100) + this.Data = make([]byte, v302) + for i := 0; i < v302; i++ { this.Data[i] = byte(r.Intn(256)) } } if r.Intn(10) != 0 { - v304 := string(randStringMesos(r)) - this.Message = &v304 + v303 := string(randStringMesos(r)) + this.Message = &v303 } if r.Intn(10) != 0 { this.AgentID = NewPopulatedAgentID(r, easy) } if r.Intn(10) != 0 { - v305 := float64(r.Float64()) + v304 := float64(r.Float64()) if r.Intn(2) == 0 { - v305 *= -1 + v304 *= -1 } - this.Timestamp = &v305 + this.Timestamp = &v304 } if r.Intn(10) != 0 { this.ExecutorID = NewPopulatedExecutorID(r, easy) } if r.Intn(10) != 0 { - v306 := bool(bool(r.Intn(2) == 0)) - this.Healthy = &v306 + v305 := bool(bool(r.Intn(2) == 0)) + this.Healthy = &v305 } if r.Intn(10) != 0 { - v307 := TaskStatus_Source([]int32{0, 1, 2}[r.Intn(3)]) - this.Source = &v307 + v306 := TaskStatus_Source([]int32{0, 1, 2}[r.Intn(3)]) + this.Source = &v306 } if r.Intn(10) != 0 { - v308 := TaskStatus_Reason([]int32{0, 21, 19, 20, 8, 17, 22, 23, 24, 1, 2, 3, 4, 5, 6, 27, 7, 9, 18, 10, 11, 12, 13, 28, 25, 26, 14, 15, 16}[r.Intn(29)]) - this.Reason = &v308 + v307 := TaskStatus_Reason([]int32{0, 21, 19, 20, 8, 17, 22, 23, 24, 1, 2, 3, 4, 5, 6, 27, 7, 9, 18, 10, 11, 12, 13, 28, 25, 26, 14, 15, 16}[r.Intn(29)]) + this.Reason = &v307 } if r.Intn(10) != 0 { - v309 := r.Intn(100) - this.UUID = make([]byte, v309) - for i := 0; i < v309; i++ { + v308 := r.Intn(100) + this.UUID = make([]byte, v308) + for i := 0; i < v308; i++ { this.UUID[i] = byte(r.Intn(256)) } } @@ -31627,11 +31632,11 @@ func NewPopulatedTaskStatus(r randyMesos, easy bool) *TaskStatus { func NewPopulatedFilters(r randyMesos, easy bool) *Filters { this := &Filters{} if r.Intn(10) != 0 { - v310 := float64(r.Float64()) + v309 := float64(r.Float64()) if r.Intn(2) == 0 { - v310 *= -1 + v309 *= -1 } - this.RefuseSeconds = &v310 + this.RefuseSeconds = &v309 } if !easy && r.Intn(10) != 0 { } @@ -31641,11 +31646,11 @@ func NewPopulatedFilters(r randyMesos, easy bool) *Filters { func NewPopulatedEnvironment(r randyMesos, easy bool) *Environment { this := &Environment{} if r.Intn(10) != 0 { - v311 := r.Intn(5) - this.Variables = make([]Environment_Variable, v311) - for i := 0; i < v311; i++ { - v312 := NewPopulatedEnvironment_Variable(r, easy) - this.Variables[i] = *v312 + v310 := r.Intn(5) + this.Variables = make([]Environment_Variable, v310) + for i := 0; i < v310; i++ { + v311 := NewPopulatedEnvironment_Variable(r, easy) + this.Variables[i] = *v311 } } if !easy && r.Intn(10) != 0 { @@ -31657,8 +31662,8 @@ func NewPopulatedEnvironment_Variable(r randyMesos, easy bool) *Environment_Vari this := &Environment_Variable{} this.Name = string(randStringMesos(r)) if r.Intn(10) != 0 { - v313 := string(randStringMesos(r)) - this.Value = &v313 + v312 := string(randStringMesos(r)) + this.Value = &v312 } if !easy && r.Intn(10) != 0 { } @@ -31677,11 +31682,11 @@ func NewPopulatedParameter(r randyMesos, easy bool) *Parameter { func NewPopulatedParameters(r randyMesos, easy bool) *Parameters { this := &Parameters{} if r.Intn(10) != 0 { - v314 := r.Intn(5) - this.Parameter = make([]Parameter, v314) - for i := 0; i < v314; i++ { - v315 := NewPopulatedParameter(r, easy) - this.Parameter[i] = *v315 + v313 := r.Intn(5) + this.Parameter = make([]Parameter, v313) + for i := 0; i < v313; i++ { + v314 := NewPopulatedParameter(r, easy) + this.Parameter[i] = *v314 } } if !easy && r.Intn(10) != 0 { @@ -31693,8 +31698,8 @@ func NewPopulatedCredential(r randyMesos, easy bool) *Credential { this := &Credential{} this.Principal = string(randStringMesos(r)) if r.Intn(10) != 0 { - v316 := string(randStringMesos(r)) - this.Secret = &v316 + v315 := string(randStringMesos(r)) + this.Secret = &v315 } if !easy && r.Intn(10) != 0 { } @@ -31704,11 +31709,11 @@ func NewPopulatedCredential(r randyMesos, easy bool) *Credential { func NewPopulatedCredentials(r randyMesos, easy bool) *Credentials { this := &Credentials{} if r.Intn(10) != 0 { - v317 := r.Intn(5) - this.Credentials = make([]Credential, v317) - for i := 0; i < v317; i++ { - v318 := NewPopulatedCredential(r, easy) - this.Credentials[i] = *v318 + v316 := r.Intn(5) + this.Credentials = make([]Credential, v316) + for i := 0; i < v316; i++ { + v317 := NewPopulatedCredential(r, easy) + this.Credentials[i] = *v317 } } if !easy && r.Intn(10) != 0 { @@ -31719,16 +31724,16 @@ func NewPopulatedCredentials(r randyMesos, easy bool) *Credentials { func NewPopulatedRateLimit(r randyMesos, easy bool) *RateLimit { this := &RateLimit{} if r.Intn(10) != 0 { - v319 := float64(r.Float64()) + v318 := float64(r.Float64()) if r.Intn(2) == 0 { - v319 *= -1 + v318 *= -1 } - this.QPS = &v319 + this.QPS = &v318 } this.Principal = string(randStringMesos(r)) if r.Intn(10) != 0 { - v320 := uint64(uint64(r.Uint32())) - this.Capacity = &v320 + v319 := uint64(uint64(r.Uint32())) + this.Capacity = &v319 } if !easy && r.Intn(10) != 0 { } @@ -31738,23 +31743,23 @@ func NewPopulatedRateLimit(r randyMesos, easy bool) *RateLimit { func NewPopulatedRateLimits(r randyMesos, easy bool) *RateLimits { this := &RateLimits{} if r.Intn(10) != 0 { - v321 := r.Intn(5) - this.Limits = make([]RateLimit, v321) - for i := 0; i < v321; i++ { - v322 := NewPopulatedRateLimit(r, easy) - this.Limits[i] = *v322 + v320 := r.Intn(5) + this.Limits = make([]RateLimit, v320) + for i := 0; i < v320; i++ { + v321 := NewPopulatedRateLimit(r, easy) + this.Limits[i] = *v321 } } if r.Intn(10) != 0 { - v323 := float64(r.Float64()) + v322 := float64(r.Float64()) if r.Intn(2) == 0 { - v323 *= -1 + v322 *= -1 } - this.AggregateDefaultQPS = &v323 + this.AggregateDefaultQPS = &v322 } if r.Intn(10) != 0 { - v324 := uint64(uint64(r.Uint32())) - this.AggregateDefaultCapacity = &v324 + v323 := uint64(uint64(r.Uint32())) + this.AggregateDefaultCapacity = &v323 } if !easy && r.Intn(10) != 0 { } @@ -31763,8 +31768,8 @@ func NewPopulatedRateLimits(r randyMesos, easy bool) *RateLimits { func NewPopulatedImage(r randyMesos, easy bool) *Image { this := &Image{} - v325 := Image_Type([]int32{1, 2}[r.Intn(2)]) - this.Type = &v325 + v324 := Image_Type([]int32{1, 2}[r.Intn(2)]) + this.Type = &v324 if r.Intn(10) != 0 { this.Appc = NewPopulatedImage_Appc(r, easy) } @@ -31772,8 +31777,8 @@ func NewPopulatedImage(r randyMesos, easy bool) *Image { this.Docker = NewPopulatedImage_Docker(r, easy) } if r.Intn(10) != 0 { - v326 := bool(bool(r.Intn(2) == 0)) - this.Cached = &v326 + v325 := bool(bool(r.Intn(2) == 0)) + this.Cached = &v325 } if !easy && r.Intn(10) != 0 { } @@ -31784,8 +31789,8 @@ func NewPopulatedImage_Appc(r randyMesos, easy bool) *Image_Appc { this := &Image_Appc{} this.Name = string(randStringMesos(r)) if r.Intn(10) != 0 { - v327 := string(randStringMesos(r)) - this.ID = &v327 + v326 := string(randStringMesos(r)) + this.ID = &v326 } if r.Intn(10) != 0 { this.Labels = NewPopulatedLabels(r, easy) @@ -31810,11 +31815,11 @@ func NewPopulatedVolume(r randyMesos, easy bool) *Volume { this := &Volume{} this.ContainerPath = string(randStringMesos(r)) if r.Intn(10) != 0 { - v328 := string(randStringMesos(r)) - this.HostPath = &v328 + v327 := string(randStringMesos(r)) + this.HostPath = &v327 } - v329 := Volume_Mode([]int32{1, 2}[r.Intn(2)]) - this.Mode = &v329 + v328 := Volume_Mode([]int32{1, 2}[r.Intn(2)]) + this.Mode = &v328 if r.Intn(10) != 0 { this.Image = NewPopulatedImage(r, easy) } @@ -31843,8 +31848,8 @@ func NewPopulatedVolume_Source(r randyMesos, easy bool) *Volume_Source { func NewPopulatedVolume_Source_DockerVolume(r randyMesos, easy bool) *Volume_Source_DockerVolume { this := &Volume_Source_DockerVolume{} if r.Intn(10) != 0 { - v330 := string(randStringMesos(r)) - this.Driver = &v330 + v329 := string(randStringMesos(r)) + this.Driver = &v329 } this.Name = string(randStringMesos(r)) if r.Intn(10) != 0 { @@ -31867,9 +31872,9 @@ func NewPopulatedVolume_Source_SandboxPath(r randyMesos, easy bool) *Volume_Sour func NewPopulatedNetworkInfo(r randyMesos, easy bool) *NetworkInfo { this := &NetworkInfo{} if r.Intn(10) != 0 { - v331 := r.Intn(10) - this.Groups = make([]string, v331) - for i := 0; i < v331; i++ { + v330 := r.Intn(10) + this.Groups = make([]string, v330) + for i := 0; i < v330; i++ { this.Groups[i] = string(randStringMesos(r)) } } @@ -31877,23 +31882,23 @@ func NewPopulatedNetworkInfo(r randyMesos, easy bool) *NetworkInfo { this.Labels = NewPopulatedLabels(r, easy) } if r.Intn(10) != 0 { - v332 := r.Intn(5) - this.IPAddresses = make([]NetworkInfo_IPAddress, v332) - for i := 0; i < v332; i++ { - v333 := NewPopulatedNetworkInfo_IPAddress(r, easy) - this.IPAddresses[i] = *v333 + v331 := r.Intn(5) + this.IPAddresses = make([]NetworkInfo_IPAddress, v331) + for i := 0; i < v331; i++ { + v332 := NewPopulatedNetworkInfo_IPAddress(r, easy) + this.IPAddresses[i] = *v332 } } if r.Intn(10) != 0 { - v334 := string(randStringMesos(r)) - this.Name = &v334 + v333 := string(randStringMesos(r)) + this.Name = &v333 } if r.Intn(10) != 0 { - v335 := r.Intn(5) - this.PortMappings = make([]NetworkInfo_PortMapping, v335) - for i := 0; i < v335; i++ { - v336 := NewPopulatedNetworkInfo_PortMapping(r, easy) - this.PortMappings[i] = *v336 + v334 := r.Intn(5) + this.PortMappings = make([]NetworkInfo_PortMapping, v334) + for i := 0; i < v334; i++ { + v335 := NewPopulatedNetworkInfo_PortMapping(r, easy) + this.PortMappings[i] = *v335 } } if !easy && r.Intn(10) != 0 { @@ -31904,12 +31909,12 @@ func NewPopulatedNetworkInfo(r randyMesos, easy bool) *NetworkInfo { func NewPopulatedNetworkInfo_IPAddress(r randyMesos, easy bool) *NetworkInfo_IPAddress { this := &NetworkInfo_IPAddress{} if r.Intn(10) != 0 { - v337 := NetworkInfo_Protocol([]int32{1, 2}[r.Intn(2)]) - this.Protocol = &v337 + v336 := NetworkInfo_Protocol([]int32{1, 2}[r.Intn(2)]) + this.Protocol = &v336 } if r.Intn(10) != 0 { - v338 := string(randStringMesos(r)) - this.IPAddress = &v338 + v337 := string(randStringMesos(r)) + this.IPAddress = &v337 } if !easy && r.Intn(10) != 0 { } @@ -31921,8 +31926,8 @@ func NewPopulatedNetworkInfo_PortMapping(r randyMesos, easy bool) *NetworkInfo_P this.HostPort = uint32(r.Uint32()) this.ContainerPort = uint32(r.Uint32()) if r.Intn(10) != 0 { - v339 := string(randStringMesos(r)) - this.Protocol = &v339 + v338 := string(randStringMesos(r)) + this.Protocol = &v338 } if !easy && r.Intn(10) != 0 { } @@ -31932,9 +31937,9 @@ func NewPopulatedNetworkInfo_PortMapping(r randyMesos, easy bool) *NetworkInfo_P func NewPopulatedCapabilityInfo(r randyMesos, easy bool) *CapabilityInfo { this := &CapabilityInfo{} if r.Intn(10) != 0 { - v340 := r.Intn(10) - this.Capabilities = make([]CapabilityInfo_Capability, v340) - for i := 0; i < v340; i++ { + v339 := r.Intn(10) + this.Capabilities = make([]CapabilityInfo_Capability, v339) + for i := 0; i < v339; i++ { this.Capabilities[i] = CapabilityInfo_Capability([]int32{0, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, 1011, 1012, 1013, 1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, 1023, 1024, 1025, 1026, 1027, 1028, 1029, 1030, 1031, 1032, 1033, 1034, 1035, 1036, 1037}[r.Intn(39)]) } } @@ -31956,11 +31961,11 @@ func NewPopulatedLinuxInfo(r randyMesos, easy bool) *LinuxInfo { func NewPopulatedRLimitInfo(r randyMesos, easy bool) *RLimitInfo { this := &RLimitInfo{} if r.Intn(10) != 0 { - v341 := r.Intn(5) - this.Rlimits = make([]RLimitInfo_RLimit, v341) - for i := 0; i < v341; i++ { - v342 := NewPopulatedRLimitInfo_RLimit(r, easy) - this.Rlimits[i] = *v342 + v340 := r.Intn(5) + this.Rlimits = make([]RLimitInfo_RLimit, v340) + for i := 0; i < v340; i++ { + v341 := NewPopulatedRLimitInfo_RLimit(r, easy) + this.Rlimits[i] = *v341 } } if !easy && r.Intn(10) != 0 { @@ -31972,12 +31977,12 @@ func NewPopulatedRLimitInfo_RLimit(r randyMesos, easy bool) *RLimitInfo_RLimit { this := &RLimitInfo_RLimit{} this.Type = RLimitInfo_RLimit_Type([]int32{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}[r.Intn(17)]) if r.Intn(10) != 0 { - v343 := uint64(uint64(r.Uint32())) - this.Hard = &v343 + v342 := uint64(uint64(r.Uint32())) + this.Hard = &v342 } if r.Intn(10) != 0 { - v344 := uint64(uint64(r.Uint32())) - this.Soft = &v344 + v343 := uint64(uint64(r.Uint32())) + this.Soft = &v343 } if !easy && r.Intn(10) != 0 { } @@ -32005,32 +32010,32 @@ func NewPopulatedTTYInfo_WindowSize(r randyMesos, easy bool) *TTYInfo_WindowSize func NewPopulatedContainerInfo(r randyMesos, easy bool) *ContainerInfo { this := &ContainerInfo{} - v345 := ContainerInfo_Type([]int32{1, 2}[r.Intn(2)]) - this.Type = &v345 + v344 := ContainerInfo_Type([]int32{1, 2}[r.Intn(2)]) + this.Type = &v344 if r.Intn(10) != 0 { - v346 := r.Intn(5) - this.Volumes = make([]Volume, v346) - for i := 0; i < v346; i++ { - v347 := NewPopulatedVolume(r, easy) - this.Volumes[i] = *v347 + v345 := r.Intn(5) + this.Volumes = make([]Volume, v345) + for i := 0; i < v345; i++ { + v346 := NewPopulatedVolume(r, easy) + this.Volumes[i] = *v346 } } if r.Intn(10) != 0 { this.Docker = NewPopulatedContainerInfo_DockerInfo(r, easy) } if r.Intn(10) != 0 { - v348 := string(randStringMesos(r)) - this.Hostname = &v348 + v347 := string(randStringMesos(r)) + this.Hostname = &v347 } if r.Intn(10) != 0 { this.Mesos = NewPopulatedContainerInfo_MesosInfo(r, easy) } if r.Intn(10) != 0 { - v349 := r.Intn(5) - this.NetworkInfos = make([]NetworkInfo, v349) - for i := 0; i < v349; i++ { - v350 := NewPopulatedNetworkInfo(r, easy) - this.NetworkInfos[i] = *v350 + v348 := r.Intn(5) + this.NetworkInfos = make([]NetworkInfo, v348) + for i := 0; i < v348; i++ { + v349 := NewPopulatedNetworkInfo(r, easy) + this.NetworkInfos[i] = *v349 } } if r.Intn(10) != 0 { @@ -32051,36 +32056,36 @@ func NewPopulatedContainerInfo_DockerInfo(r randyMesos, easy bool) *ContainerInf this := &ContainerInfo_DockerInfo{} this.Image = string(randStringMesos(r)) if r.Intn(10) != 0 { - v351 := ContainerInfo_DockerInfo_Network([]int32{1, 2, 3, 4}[r.Intn(4)]) - this.Network = &v351 + v350 := ContainerInfo_DockerInfo_Network([]int32{1, 2, 3, 4}[r.Intn(4)]) + this.Network = &v350 } if r.Intn(10) != 0 { - v352 := r.Intn(5) - this.PortMappings = make([]ContainerInfo_DockerInfo_PortMapping, v352) - for i := 0; i < v352; i++ { - v353 := NewPopulatedContainerInfo_DockerInfo_PortMapping(r, easy) - this.PortMappings[i] = *v353 + v351 := r.Intn(5) + this.PortMappings = make([]ContainerInfo_DockerInfo_PortMapping, v351) + for i := 0; i < v351; i++ { + v352 := NewPopulatedContainerInfo_DockerInfo_PortMapping(r, easy) + this.PortMappings[i] = *v352 } } if r.Intn(10) != 0 { - v354 := bool(bool(r.Intn(2) == 0)) - this.Privileged = &v354 + v353 := bool(bool(r.Intn(2) == 0)) + this.Privileged = &v353 } if r.Intn(10) != 0 { - v355 := r.Intn(5) - this.Parameters = make([]Parameter, v355) - for i := 0; i < v355; i++ { - v356 := NewPopulatedParameter(r, easy) - this.Parameters[i] = *v356 + v354 := r.Intn(5) + this.Parameters = make([]Parameter, v354) + for i := 0; i < v354; i++ { + v355 := NewPopulatedParameter(r, easy) + this.Parameters[i] = *v355 } } if r.Intn(10) != 0 { - v357 := bool(bool(r.Intn(2) == 0)) - this.ForcePullImage = &v357 + v356 := bool(bool(r.Intn(2) == 0)) + this.ForcePullImage = &v356 } if r.Intn(10) != 0 { - v358 := string(randStringMesos(r)) - this.VolumeDriver = &v358 + v357 := string(randStringMesos(r)) + this.VolumeDriver = &v357 } if !easy && r.Intn(10) != 0 { } @@ -32092,8 +32097,8 @@ func NewPopulatedContainerInfo_DockerInfo_PortMapping(r randyMesos, easy bool) * this.HostPort = uint32(r.Uint32()) this.ContainerPort = uint32(r.Uint32()) if r.Intn(10) != 0 { - v359 := string(randStringMesos(r)) - this.Protocol = &v359 + v358 := string(randStringMesos(r)) + this.Protocol = &v358 } if !easy && r.Intn(10) != 0 { } @@ -32113,19 +32118,19 @@ func NewPopulatedContainerInfo_MesosInfo(r randyMesos, easy bool) *ContainerInfo func NewPopulatedContainerStatus(r randyMesos, easy bool) *ContainerStatus { this := &ContainerStatus{} if r.Intn(10) != 0 { - v360 := r.Intn(5) - this.NetworkInfos = make([]NetworkInfo, v360) - for i := 0; i < v360; i++ { - v361 := NewPopulatedNetworkInfo(r, easy) - this.NetworkInfos[i] = *v361 + v359 := r.Intn(5) + this.NetworkInfos = make([]NetworkInfo, v359) + for i := 0; i < v359; i++ { + v360 := NewPopulatedNetworkInfo(r, easy) + this.NetworkInfos[i] = *v360 } } if r.Intn(10) != 0 { this.CgroupInfo = NewPopulatedCgroupInfo(r, easy) } if r.Intn(10) != 0 { - v362 := uint32(r.Uint32()) - this.ExecutorPID = &v362 + v361 := uint32(r.Uint32()) + this.ExecutorPID = &v361 } if r.Intn(10) == 0 { this.ContainerID = NewPopulatedContainerID(r, easy) @@ -32148,8 +32153,8 @@ func NewPopulatedCgroupInfo(r randyMesos, easy bool) *CgroupInfo { func NewPopulatedCgroupInfo_NetCls(r randyMesos, easy bool) *CgroupInfo_NetCls { this := &CgroupInfo_NetCls{} if r.Intn(10) != 0 { - v363 := uint32(r.Uint32()) - this.ClassID = &v363 + v362 := uint32(r.Uint32()) + this.ClassID = &v362 } if !easy && r.Intn(10) != 0 { } @@ -32159,11 +32164,11 @@ func NewPopulatedCgroupInfo_NetCls(r randyMesos, easy bool) *CgroupInfo_NetCls { func NewPopulatedLabels(r randyMesos, easy bool) *Labels { this := &Labels{} if r.Intn(10) != 0 { - v364 := r.Intn(5) - this.Labels = make([]Label, v364) - for i := 0; i < v364; i++ { - v365 := NewPopulatedLabel(r, easy) - this.Labels[i] = *v365 + v363 := r.Intn(5) + this.Labels = make([]Label, v363) + for i := 0; i < v363; i++ { + v364 := NewPopulatedLabel(r, easy) + this.Labels[i] = *v364 } } if !easy && r.Intn(10) != 0 { @@ -32175,8 +32180,8 @@ func NewPopulatedLabel(r randyMesos, easy bool) *Label { this := &Label{} this.Key = string(randStringMesos(r)) if r.Intn(10) != 0 { - v366 := string(randStringMesos(r)) - this.Value = &v366 + v365 := string(randStringMesos(r)) + this.Value = &v365 } if !easy && r.Intn(10) != 0 { } @@ -32187,16 +32192,16 @@ func NewPopulatedPort(r randyMesos, easy bool) *Port { this := &Port{} this.Number = uint32(r.Uint32()) if r.Intn(10) != 0 { - v367 := string(randStringMesos(r)) - this.Name = &v367 + v366 := string(randStringMesos(r)) + this.Name = &v366 } if r.Intn(10) != 0 { - v368 := string(randStringMesos(r)) - this.Protocol = &v368 + v367 := string(randStringMesos(r)) + this.Protocol = &v367 } if r.Intn(10) != 0 { - v369 := DiscoveryInfo_Visibility([]int32{0, 1, 2}[r.Intn(3)]) - this.Visibility = &v369 + v368 := DiscoveryInfo_Visibility([]int32{0, 1, 2}[r.Intn(3)]) + this.Visibility = &v368 } if r.Intn(10) != 0 { this.Labels = NewPopulatedLabels(r, easy) @@ -32209,11 +32214,11 @@ func NewPopulatedPort(r randyMesos, easy bool) *Port { func NewPopulatedPorts(r randyMesos, easy bool) *Ports { this := &Ports{} if r.Intn(10) != 0 { - v370 := r.Intn(5) - this.Ports = make([]Port, v370) - for i := 0; i < v370; i++ { - v371 := NewPopulatedPort(r, easy) - this.Ports[i] = *v371 + v369 := r.Intn(5) + this.Ports = make([]Port, v369) + for i := 0; i < v369; i++ { + v370 := NewPopulatedPort(r, easy) + this.Ports[i] = *v370 } } if !easy && r.Intn(10) != 0 { @@ -32224,21 +32229,21 @@ func NewPopulatedPorts(r randyMesos, easy bool) *Ports { func NewPopulatedDiscoveryInfo(r randyMesos, easy bool) *DiscoveryInfo { this := &DiscoveryInfo{} this.Visibility = DiscoveryInfo_Visibility([]int32{0, 1, 2}[r.Intn(3)]) + if r.Intn(10) != 0 { + v371 := string(randStringMesos(r)) + this.Name = &v371 + } if r.Intn(10) != 0 { v372 := string(randStringMesos(r)) - this.Name = &v372 + this.Environment = &v372 } if r.Intn(10) != 0 { v373 := string(randStringMesos(r)) - this.Environment = &v373 + this.Location = &v373 } if r.Intn(10) != 0 { v374 := string(randStringMesos(r)) - this.Location = &v374 - } - if r.Intn(10) != 0 { - v375 := string(randStringMesos(r)) - this.Version = &v375 + this.Version = &v374 } if r.Intn(10) != 0 { this.Ports = NewPopulatedPorts(r, easy) @@ -32258,8 +32263,8 @@ func NewPopulatedWeightInfo(r randyMesos, easy bool) *WeightInfo { this.Weight *= -1 } if r.Intn(10) != 0 { - v376 := string(randStringMesos(r)) - this.Role = &v376 + v375 := string(randStringMesos(r)) + this.Role = &v375 } if !easy && r.Intn(10) != 0 { } @@ -32270,31 +32275,31 @@ func NewPopulatedVersionInfo(r randyMesos, easy bool) *VersionInfo { this := &VersionInfo{} this.Version = string(randStringMesos(r)) if r.Intn(10) != 0 { - v377 := string(randStringMesos(r)) - this.BuildDate = &v377 + v376 := string(randStringMesos(r)) + this.BuildDate = &v376 } if r.Intn(10) != 0 { - v378 := float64(r.Float64()) + v377 := float64(r.Float64()) if r.Intn(2) == 0 { - v378 *= -1 + v377 *= -1 } - this.BuildTime = &v378 + this.BuildTime = &v377 + } + if r.Intn(10) != 0 { + v378 := string(randStringMesos(r)) + this.BuildUser = &v378 } if r.Intn(10) != 0 { v379 := string(randStringMesos(r)) - this.BuildUser = &v379 + this.GitSHA = &v379 } if r.Intn(10) != 0 { v380 := string(randStringMesos(r)) - this.GitSHA = &v380 + this.GitBranch = &v380 } if r.Intn(10) != 0 { v381 := string(randStringMesos(r)) - this.GitBranch = &v381 - } - if r.Intn(10) != 0 { - v382 := string(randStringMesos(r)) - this.GitTag = &v382 + this.GitTag = &v381 } if !easy && r.Intn(10) != 0 { } @@ -32305,8 +32310,8 @@ func NewPopulatedFlag(r randyMesos, easy bool) *Flag { this := &Flag{} this.Name = string(randStringMesos(r)) if r.Intn(10) != 0 { - v383 := string(randStringMesos(r)) - this.Value = &v383 + v382 := string(randStringMesos(r)) + this.Value = &v382 } if !easy && r.Intn(10) != 0 { } @@ -32321,19 +32326,19 @@ func NewPopulatedRole(r randyMesos, easy bool) *Role { this.Weight *= -1 } if r.Intn(10) != 0 { - v384 := r.Intn(5) - this.Frameworks = make([]FrameworkID, v384) - for i := 0; i < v384; i++ { - v385 := NewPopulatedFrameworkID(r, easy) - this.Frameworks[i] = *v385 + v383 := r.Intn(5) + this.Frameworks = make([]FrameworkID, v383) + for i := 0; i < v383; i++ { + v384 := NewPopulatedFrameworkID(r, easy) + this.Frameworks[i] = *v384 } } if r.Intn(10) != 0 { - v386 := r.Intn(5) - this.Resources = make([]Resource, v386) - for i := 0; i < v386; i++ { - v387 := NewPopulatedResource(r, easy) - this.Resources[i] = *v387 + v385 := r.Intn(5) + this.Resources = make([]Resource, v385) + for i := 0; i < v385; i++ { + v386 := NewPopulatedResource(r, easy) + this.Resources[i] = *v386 } } if !easy && r.Intn(10) != 0 { @@ -32345,11 +32350,11 @@ func NewPopulatedMetric(r randyMesos, easy bool) *Metric { this := &Metric{} this.Name = string(randStringMesos(r)) if r.Intn(10) != 0 { - v388 := float64(r.Float64()) + v387 := float64(r.Float64()) if r.Intn(2) == 0 { - v388 *= -1 + v387 *= -1 } - this.Value = &v388 + this.Value = &v387 } if !easy && r.Intn(10) != 0 { } @@ -32360,30 +32365,30 @@ func NewPopulatedFileInfo(r randyMesos, easy bool) *FileInfo { this := &FileInfo{} this.Path = string(randStringMesos(r)) if r.Intn(10) != 0 { - v389 := int32(r.Int31()) + v388 := int32(r.Int31()) if r.Intn(2) == 0 { - v389 *= -1 + v388 *= -1 } - this.Nlink = &v389 + this.Nlink = &v388 } if r.Intn(10) != 0 { - v390 := uint64(uint64(r.Uint32())) - this.Size = &v390 + v389 := uint64(uint64(r.Uint32())) + this.Size = &v389 } if r.Intn(10) != 0 { this.Mtime = NewPopulatedTimeInfo(r, easy) } if r.Intn(10) != 0 { - v391 := uint32(r.Uint32()) - this.Mode = &v391 + v390 := uint32(r.Uint32()) + this.Mode = &v390 } if r.Intn(10) != 0 { - v392 := string(randStringMesos(r)) - this.UID = &v392 + v391 := string(randStringMesos(r)) + this.UID = &v391 } if r.Intn(10) != 0 { - v393 := string(randStringMesos(r)) - this.GID = &v393 + v392 := string(randStringMesos(r)) + this.GID = &v392 } if !easy && r.Intn(10) != 0 { } @@ -32409,9 +32414,9 @@ func randUTF8RuneMesos(r randyMesos) rune { return rune(ru + 61) } func randStringMesos(r randyMesos) string { - v394 := r.Intn(100) - tmps := make([]rune, v394) - for i := 0; i < v394; i++ { + v393 := r.Intn(100) + tmps := make([]rune, v393) + for i := 0; i < v393; i++ { tmps[i] = randUTF8RuneMesos(r) } return string(tmps) @@ -32433,11 +32438,11 @@ func randFieldMesos(dAtA []byte, r randyMesos, fieldNumber int, wire int) []byte switch wire { case 0: dAtA = encodeVarintPopulateMesos(dAtA, uint64(key)) - v395 := r.Int63() + v394 := r.Int63() if r.Intn(2) == 0 { - v395 *= -1 + v394 *= -1 } - dAtA = encodeVarintPopulateMesos(dAtA, uint64(v395)) + dAtA = encodeVarintPopulateMesos(dAtA, uint64(v394)) case 1: dAtA = encodeVarintPopulateMesos(dAtA, uint64(key)) dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) @@ -32850,8 +32855,10 @@ func (m *ExecutorInfo) ProtoSize() (n int) { n += 1 + l + sovMesos(uint64(l)) } } - l = m.Command.ProtoSize() - n += 1 + l + sovMesos(uint64(l)) + if m.Command != nil { + l = m.Command.ProtoSize() + n += 1 + l + sovMesos(uint64(l)) + } if m.FrameworkID != nil { l = m.FrameworkID.ProtoSize() n += 1 + l + sovMesos(uint64(l)) @@ -35236,7 +35243,7 @@ func (this *ExecutorInfo) String() string { `ExecutorID:` + strings.Replace(strings.Replace(this.ExecutorID.String(), "ExecutorID", "ExecutorID", 1), `&`, ``, 1) + `,`, `Data:` + valueToStringMesos(this.Data) + `,`, `Resources:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Resources), "Resource", "Resource", 1), `&`, ``, 1) + `,`, - `Command:` + strings.Replace(strings.Replace(this.Command.String(), "CommandInfo", "CommandInfo", 1), `&`, ``, 1) + `,`, + `Command:` + strings.Replace(fmt.Sprintf("%v", this.Command), "CommandInfo", "CommandInfo", 1) + `,`, `FrameworkID:` + strings.Replace(fmt.Sprintf("%v", this.FrameworkID), "FrameworkID", "FrameworkID", 1) + `,`, `Name:` + valueToStringMesos(this.Name) + `,`, `Source:` + valueToStringMesos(this.Source) + `,`, @@ -39936,11 +39943,13 @@ func (m *ExecutorInfo) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + if m.Command == nil { + m.Command = &CommandInfo{} + } if err := m.Command.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - hasFields[0] |= uint64(0x00000002) case 8: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field FrameworkID", wireType) @@ -40203,9 +40212,6 @@ func (m *ExecutorInfo) Unmarshal(dAtA []byte) error { if hasFields[0]&uint64(0x00000001) == 0 { return github_com_gogo_protobuf_proto.NewRequiredNotSetError("executor_id") } - if hasFields[0]&uint64(0x00000002) == 0 { - return github_com_gogo_protobuf_proto.NewRequiredNotSetError("command") - } if iNdEx > l { return io.ErrUnexpectedEOF @@ -56579,614 +56585,614 @@ var ( func init() { proto.RegisterFile("mesos.proto", fileDescriptorMesos) } var fileDescriptorMesos = []byte{ - // 9736 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x7d, 0x5b, 0x90, 0x5b, 0x49, - 0x96, 0x50, 0xeb, 0x2d, 0x1d, 0x49, 0x55, 0xd7, 0x59, 0x76, 0xb5, 0x2c, 0xbb, 0xab, 0xdc, 0x1a, - 0xdb, 0xed, 0x76, 0xf7, 0xd8, 0xee, 0xea, 0xb6, 0xa7, 0xed, 0x99, 0xe9, 0x5e, 0x95, 0x74, 0xab, - 0xea, 0xae, 0xf5, 0xda, 0x94, 0x64, 0x77, 0x4f, 0x6c, 0x84, 0xe2, 0x96, 0x74, 0xab, 0xea, 0xae, - 0xa5, 0x7b, 0xd5, 0xf7, 0x5e, 0x95, 0xed, 0x09, 0x22, 0x18, 0x96, 0x05, 0x66, 0xf6, 0x41, 0xec, - 0x07, 0x1f, 0xbb, 0x2c, 0x44, 0x10, 0x40, 0x04, 0x7c, 0xc0, 0xc7, 0x02, 0x1f, 0x7c, 0x10, 0xc0, - 0x07, 0x04, 0xfb, 0xc7, 0x4e, 0xf0, 0x03, 0x3f, 0x05, 0x5d, 0x4b, 0xc4, 0x0e, 0xef, 0x61, 0x79, - 0xec, 0x02, 0xc3, 0x23, 0xf2, 0x79, 0xf3, 0x4a, 0xf5, 0xb0, 0x7b, 0x21, 0xf8, 0x2a, 0xe5, 0x79, - 0xe5, 0xc9, 0x93, 0x27, 0x4f, 0x9e, 0xcc, 0x9b, 0x99, 0x05, 0xf9, 0x89, 0xe5, 0xbb, 0xfe, 0x9d, - 0xa9, 0xe7, 0x06, 0x2e, 0x4a, 0xd1, 0x42, 0xf9, 0xeb, 0xfb, 0x76, 0x70, 0x30, 0xdb, 0xbd, 0x33, - 0x74, 0x27, 0x77, 0xf7, 0xdd, 0x7d, 0xf7, 0x2e, 0xc5, 0xee, 0xce, 0xf6, 0x68, 0x89, 0x16, 0xe8, - 0x2f, 0xc6, 0x55, 0x79, 0x17, 0xf2, 0x5b, 0x9e, 0x39, 0xb1, 0x9e, 0xbb, 0xde, 0x33, 0xa3, 0x8e, - 0xca, 0x90, 0x3a, 0x34, 0xc7, 0x33, 0xab, 0x14, 0xbb, 0x16, 0xbf, 0x95, 0xdb, 0x4c, 0xfe, 0xd6, - 0xd1, 0xfa, 0x1b, 0x98, 0x81, 0x2a, 0x37, 0x20, 0xd3, 0xde, 0xdb, 0xb3, 0xbc, 0xf3, 0xc9, 0xaa, - 0xfb, 0x96, 0x13, 0x9c, 0x43, 0x76, 0x1d, 0xd2, 0x3d, 0xd3, 0x3f, 0xaf, 0xce, 0x5b, 0x00, 0xfa, - 0x0b, 0x6b, 0x38, 0x0b, 0xdc, 0xf3, 0xaa, 0xed, 0x43, 0xbe, 0xe6, 0x3a, 0x81, 0x69, 0x3b, 0xe7, - 0x69, 0x88, 0x6e, 0x43, 0x7a, 0x6a, 0x7a, 0x96, 0x13, 0x94, 0xe2, 0xd7, 0x62, 0xb7, 0xf2, 0x1b, - 0xe8, 0x0e, 0xb3, 0xa3, 0xc2, 0x8f, 0x39, 0x45, 0x65, 0x03, 0xb2, 0x3d, 0x7b, 0x62, 0x19, 0xce, - 0x9e, 0x8b, 0x6e, 0x42, 0xde, 0x31, 0x1d, 0xd7, 0xb7, 0x86, 0xae, 0x33, 0xf2, 0xa9, 0xe4, 0x04, - 0x97, 0xac, 0x22, 0x2a, 0x0f, 0xa0, 0x50, 0x9f, 0x79, 0x66, 0x60, 0xbb, 0xce, 0x6b, 0xf1, 0x3d, - 0x85, 0x4c, 0x75, 0x34, 0xf2, 0x2c, 0xdf, 0x47, 0x65, 0xc8, 0x1e, 0xb8, 0x7e, 0xe0, 0x98, 0x13, - 0xd2, 0x82, 0xd8, 0xad, 0x1c, 0x96, 0x65, 0xb4, 0x0a, 0x71, 0x7b, 0x4a, 0x55, 0xcf, 0x6d, 0xa6, - 0x8f, 0x8f, 0xd6, 0xe3, 0x46, 0x07, 0xc7, 0xed, 0x29, 0x2a, 0x41, 0x72, 0xea, 0x7a, 0x41, 0x29, - 0x71, 0x2d, 0x7e, 0x2b, 0xc5, 0xe5, 0x53, 0x48, 0xe5, 0x37, 0x63, 0x90, 0xe8, 0xe3, 0x06, 0xba, - 0x0a, 0x69, 0x7f, 0x78, 0x60, 0x4d, 0xa2, 0x56, 0xe1, 0x30, 0x74, 0x07, 0x32, 0x26, 0xab, 0xbe, - 0x14, 0xbf, 0x16, 0xbf, 0x95, 0xdf, 0x58, 0xe2, 0x76, 0xe1, 0x4a, 0x71, 0x72, 0x41, 0x84, 0x10, - 0x24, 0xa7, 0x66, 0x70, 0x50, 0x4a, 0x50, 0xfd, 0xe8, 0x6f, 0xf4, 0x3e, 0xa4, 0xbe, 0x98, 0x59, - 0xde, 0xcb, 0x52, 0xf2, 0x5a, 0xe2, 0x56, 0x7e, 0x43, 0xe3, 0x12, 0x3a, 0x26, 0xf1, 0xb1, 0xc0, - 0xf2, 0x44, 0x47, 0x50, 0x22, 0xd2, 0xca, 0x3d, 0xcf, 0xdc, 0x9f, 0x90, 0xae, 0x48, 0xb1, 0x56, - 0x8a, 0x72, 0xc5, 0x81, 0xa5, 0xbe, 0x63, 0x1e, 0x9a, 0xf6, 0xd8, 0xdc, 0xb5, 0xc7, 0x76, 0xf0, - 0x12, 0xbd, 0x07, 0x29, 0x3f, 0x30, 0xbd, 0x80, 0x2a, 0x9f, 0xdf, 0x58, 0xe6, 0xb2, 0x45, 0xf7, - 0x08, 0xd1, 0x94, 0x06, 0xdd, 0x85, 0xec, 0x88, 0xf7, 0x01, 0xef, 0xe5, 0x15, 0x4e, 0xaf, 0x76, - 0x0d, 0x96, 0x44, 0x95, 0x4f, 0x21, 0xd7, 0x34, 0x87, 0x07, 0xb6, 0x63, 0x51, 0xef, 0x79, 0x6d, - 0xf3, 0x57, 0xfe, 0x71, 0x0c, 0xf2, 0x42, 0x02, 0xe9, 0xf5, 0xf7, 0x21, 0x6e, 0x8f, 0xb8, 0xae, - 0xc2, 0x0e, 0xb2, 0x86, 0x4d, 0x20, 0xca, 0x52, 0xee, 0x3a, 0x8e, 0xdb, 0x23, 0xf4, 0x1e, 0x24, - 0x27, 0xee, 0xc8, 0xa2, 0x72, 0x97, 0x36, 0xde, 0x9c, 0xa3, 0x77, 0xf6, 0xdc, 0x3b, 0x4d, 0x77, - 0x64, 0x61, 0x4a, 0x84, 0xbe, 0x0d, 0x4b, 0xb3, 0x88, 0x6d, 0x68, 0x1f, 0xe4, 0x37, 0x2e, 0x71, - 0xb6, 0xa8, 0xe1, 0xf0, 0x1c, 0x71, 0xe5, 0x26, 0x24, 0x89, 0x30, 0x94, 0x86, 0x78, 0xbf, 0xa3, - 0xc5, 0x50, 0x01, 0xb2, 0x75, 0x5c, 0x35, 0x5a, 0x46, 0x6b, 0x5b, 0x8b, 0xa3, 0x2c, 0x24, 0xeb, - 0xed, 0xa7, 0x2d, 0x2d, 0x51, 0xf9, 0x1b, 0x29, 0x28, 0x86, 0xc1, 0x81, 0xb4, 0xa9, 0x04, 0xc9, - 0x99, 0x6f, 0x79, 0x11, 0xf7, 0xa1, 0x10, 0x82, 0xa1, 0xd6, 0x8a, 0xab, 0x18, 0x6a, 0xaf, 0xdb, - 0xd4, 0x0e, 0x89, 0xc8, 0x48, 0x53, 0x42, 0x0e, 0xb7, 0x21, 0xb3, 0xc2, 0xfb, 0xa0, 0xed, 0x99, - 0xf6, 0xd8, 0x3d, 0xb4, 0xbc, 0x41, 0x60, 0x4f, 0x2c, 0x77, 0x16, 0x94, 0x92, 0xd7, 0x62, 0xb7, - 0x62, 0x8f, 0x62, 0xf7, 0xf0, 0xb2, 0x40, 0xf5, 0x18, 0x06, 0xdd, 0x00, 0x18, 0x1e, 0x58, 0xc3, - 0x67, 0x53, 0xd7, 0xe6, 0x0e, 0x94, 0x7d, 0x94, 0xda, 0x33, 0xc7, 0xbe, 0x85, 0x15, 0x04, 0xba, - 0x04, 0x49, 0xcf, 0x1d, 0x5b, 0xa5, 0x34, 0xe9, 0xb2, 0x47, 0xb1, 0xdb, 0x98, 0x16, 0x23, 0x7d, - 0x9c, 0x99, 0xeb, 0xe3, 0xab, 0x90, 0x9b, 0x7a, 0xb6, 0x33, 0xb4, 0xa7, 0xe6, 0xb8, 0x94, 0xa5, - 0xc8, 0x10, 0x80, 0xde, 0x85, 0xdc, 0x73, 0x6b, 0x77, 0x66, 0x0f, 0x66, 0xde, 0xb8, 0x94, 0xa3, - 0x8e, 0x50, 0x38, 0x3e, 0x5a, 0xcf, 0x3e, 0xb5, 0x76, 0xfb, 0x76, 0x1f, 0x37, 0x70, 0x96, 0xa2, - 0xfb, 0xde, 0x18, 0x19, 0x50, 0x18, 0x9a, 0x53, 0x66, 0x78, 0xdb, 0xf2, 0x4b, 0x40, 0x87, 0xc5, - 0xfa, 0x82, 0x19, 0x48, 0x07, 0xd7, 0x04, 0xe1, 0x4b, 0x6e, 0xbf, 0x08, 0x2b, 0xba, 0x01, 0xe9, - 0xb1, 0xb9, 0x6b, 0x8d, 0xfd, 0x52, 0x9e, 0xda, 0xb2, 0xc8, 0x85, 0x34, 0x28, 0x10, 0x73, 0x24, - 0xba, 0x08, 0x29, 0xd2, 0x3c, 0xbf, 0x54, 0xb8, 0x96, 0xb8, 0x95, 0xc3, 0xac, 0x50, 0xfe, 0xdd, - 0x18, 0x40, 0x28, 0x1f, 0x7d, 0x02, 0xc9, 0xe0, 0xe5, 0x94, 0xf9, 0xf6, 0xd2, 0xc6, 0xf5, 0x73, - 0xd4, 0xb9, 0xd3, 0x7b, 0x39, 0xb5, 0x44, 0x9f, 0x12, 0xbe, 0xca, 0xaf, 0xc5, 0x20, 0x49, 0x80, - 0x28, 0x0f, 0x99, 0x7e, 0xeb, 0x71, 0x8b, 0xf8, 0xcb, 0x1b, 0xe8, 0x4d, 0x58, 0xc1, 0xfa, 0x93, - 0x76, 0xad, 0xba, 0xd9, 0xd0, 0x07, 0x58, 0xef, 0xb6, 0xfb, 0xb8, 0xa6, 0x77, 0xb5, 0x18, 0x5a, - 0x05, 0xd4, 0xab, 0x76, 0x1f, 0x0f, 0x1e, 0x1b, 0x8d, 0x86, 0xd1, 0xda, 0x1e, 0x74, 0x7b, 0xd5, - 0x9e, 0xae, 0xc5, 0xd1, 0x05, 0x28, 0x6e, 0x77, 0xfa, 0x0a, 0x69, 0x02, 0x5d, 0x04, 0xad, 0xbb, - 0x53, 0xc5, 0x7a, 0x5d, 0x81, 0x26, 0xd1, 0x0a, 0x2c, 0x77, 0xaa, 0xb8, 0x67, 0xf4, 0x8c, 0x76, - 0x6b, 0x50, 0x7d, 0x5a, 0xc5, 0xba, 0x96, 0x42, 0x4b, 0x00, 0xcd, 0x7e, 0xa3, 0x67, 0x0c, 0x70, - 0xbb, 0xa1, 0x6b, 0xe9, 0x72, 0xf2, 0xfb, 0x7f, 0x69, 0x2d, 0x56, 0xf9, 0x61, 0x02, 0x72, 0x35, - 0xd2, 0xf9, 0xd4, 0x61, 0xef, 0x46, 0x1a, 0x2a, 0xc6, 0x87, 0xc4, 0x2f, 0xb4, 0x0c, 0x6d, 0x40, - 0x66, 0xe8, 0x4e, 0x26, 0xa6, 0x33, 0xe2, 0x61, 0xa3, 0xb4, 0xc0, 0x53, 0x63, 0x78, 0x2c, 0x08, - 0xd1, 0x87, 0x90, 0x3c, 0x08, 0x82, 0xe9, 0xdc, 0x20, 0x0c, 0x19, 0x76, 0x82, 0x60, 0xba, 0x99, - 0x3d, 0x3e, 0x5a, 0x4f, 0xee, 0xf4, 0x7a, 0x1d, 0x4c, 0x89, 0xd1, 0x3b, 0x50, 0x1c, 0x59, 0x63, - 0xf3, 0xe5, 0x40, 0x4c, 0x0b, 0xcc, 0xcf, 0xe3, 0x1f, 0xdc, 0xc7, 0x05, 0x8a, 0xe8, 0x32, 0x38, - 0xfa, 0x3a, 0x68, 0xb6, 0x13, 0x58, 0xde, 0xa1, 0x39, 0x96, 0xb4, 0x29, 0x4e, 0x7b, 0x0f, 0x2f, - 0x0b, 0x9c, 0x20, 0x7f, 0x0f, 0x96, 0xf9, 0xc8, 0x91, 0xd4, 0x69, 0x46, 0xbd, 0x71, 0x0f, 0x2f, - 0x71, 0x14, 0x27, 0x2e, 0x7f, 0x1b, 0x32, 0xbc, 0x35, 0x6a, 0xc3, 0x59, 0xcc, 0x0a, 0x67, 0x45, - 0x0a, 0x55, 0x42, 0xac, 0x20, 0x2c, 0x7f, 0x04, 0x49, 0xd2, 0x36, 0x39, 0xf3, 0x10, 0xc6, 0xa2, - 0x3a, 0xf3, 0xc8, 0x39, 0x22, 0x1e, 0xce, 0x11, 0x95, 0x7b, 0x27, 0xf9, 0x4e, 0x1e, 0x32, 0xb5, - 0x76, 0xb3, 0x59, 0x6d, 0xd5, 0xb5, 0x18, 0x09, 0x41, 0xc4, 0x52, 0x5a, 0x9c, 0xf7, 0xe9, 0xdf, - 0x4c, 0x41, 0x7e, 0xc7, 0x32, 0xc7, 0xc1, 0x01, 0x35, 0x2a, 0x71, 0x5f, 0x6a, 0xf0, 0x18, 0x35, - 0xf8, 0x35, 0xae, 0xa8, 0x42, 0x71, 0x87, 0xb0, 0xcb, 0x0e, 0x38, 0xdf, 0xf6, 0xf1, 0xd7, 0xb0, - 0x7d, 0xe2, 0xb5, 0x6c, 0x9f, 0x3c, 0xcd, 0xf6, 0xe8, 0x23, 0xb8, 0x38, 0x74, 0x1d, 0x9f, 0x24, - 0x37, 0xf6, 0xa1, 0x35, 0x20, 0xc1, 0x6d, 0xe6, 0x59, 0xac, 0x6f, 0x8b, 0x8f, 0x62, 0x1f, 0xe2, - 0x15, 0x05, 0xbd, 0xc5, 0xb1, 0x84, 0x6b, 0xdf, 0x33, 0x87, 0xd6, 0x60, 0x6a, 0x79, 0xb6, 0x3b, - 0x9a, 0xef, 0xe3, 0x0f, 0xee, 0x61, 0x44, 0xf1, 0x1d, 0x8a, 0x16, 0x75, 0xbd, 0x1f, 0x76, 0x6e, - 0x66, 0x2e, 0xe5, 0x91, 0x9d, 0x1b, 0xfa, 0xf3, 0x07, 0x7c, 0xd0, 0x64, 0x23, 0x73, 0x91, 0x6a, - 0xde, 0x85, 0x61, 0xf3, 0x08, 0x12, 0xc1, 0x70, 0x4a, 0x83, 0x61, 0x18, 0xde, 0x22, 0x1c, 0x35, - 0xa5, 0x3f, 0x32, 0xc7, 0x47, 0xeb, 0x89, 0x5e, 0xad, 0x83, 0x09, 0x53, 0xf9, 0x0b, 0x28, 0x46, - 0xba, 0xeb, 0xf5, 0xdc, 0x09, 0xad, 0xca, 0xa4, 0x86, 0x25, 0x22, 0x22, 0x9d, 0x29, 0x43, 0xd6, - 0x0f, 0xcc, 0x60, 0xe6, 0x5b, 0x3e, 0xcd, 0x46, 0x8a, 0x58, 0x96, 0xcb, 0xb7, 0xa0, 0xa0, 0x2a, - 0x74, 0x7a, 0x8d, 0x95, 0x87, 0xaf, 0xee, 0xac, 0x28, 0x03, 0xa4, 0x51, 0x5a, 0x82, 0x7b, 0x6d, - 0x1d, 0xe0, 0xb1, 0x3d, 0x1e, 0x77, 0xdc, 0xb1, 0x3d, 0x7c, 0x89, 0x1e, 0x40, 0x41, 0xed, 0x38, - 0xee, 0xbb, 0x27, 0x26, 0x25, 0x79, 0xa5, 0x07, 0x2b, 0x3f, 0x89, 0x93, 0xc4, 0x56, 0xf6, 0x12, - 0xfa, 0x18, 0x92, 0x33, 0xcf, 0x26, 0x59, 0x24, 0x99, 0x49, 0x56, 0x17, 0xfb, 0xf1, 0x4e, 0x1f, - 0x1b, 0x9b, 0x05, 0x9e, 0x5e, 0x24, 0xfb, 0xd8, 0xf0, 0x31, 0xe5, 0x40, 0x1f, 0x41, 0xde, 0x72, - 0x0e, 0x6d, 0xcf, 0x75, 0x26, 0x8b, 0xb9, 0xaf, 0x1e, 0x62, 0xb0, 0x4a, 0x46, 0xe6, 0x13, 0x96, - 0x48, 0x33, 0xeb, 0xf2, 0x14, 0x1a, 0xf1, 0x44, 0x80, 0x65, 0x6d, 0x2c, 0x05, 0x28, 0x43, 0xca, - 0x3f, 0xb0, 0xc6, 0x63, 0xea, 0x8b, 0xd9, 0x47, 0xc9, 0xc0, 0x9b, 0x59, 0x98, 0x81, 0xc8, 0x84, - 0x6a, 0x7a, 0xfb, 0x33, 0x22, 0xd1, 0x2f, 0x65, 0xe8, 0xcc, 0x14, 0x02, 0xca, 0xbf, 0x46, 0xf3, - 0x53, 0xe3, 0xcc, 0xa4, 0x7d, 0x0d, 0xc0, 0xa2, 0x2b, 0x01, 0x73, 0x77, 0xcc, 0xd2, 0xa4, 0x2c, - 0x56, 0x20, 0x68, 0x0d, 0x32, 0xd6, 0x8b, 0xc0, 0x33, 0x87, 0x01, 0xd5, 0x54, 0xd4, 0x2f, 0x80, - 0xa4, 0x1d, 0x43, 0x73, 0x78, 0x60, 0xd1, 0x11, 0x99, 0xc5, 0xac, 0x80, 0xd6, 0x21, 0xef, 0xce, - 0x82, 0xe9, 0x2c, 0x18, 0xec, 0xd9, 0x63, 0x8b, 0x37, 0x07, 0x18, 0x68, 0xcb, 0x1e, 0x5b, 0x95, - 0x5f, 0x4a, 0x41, 0x41, 0xae, 0x40, 0x88, 0xfd, 0xb7, 0x20, 0x6f, 0xf1, 0xf2, 0x40, 0xe6, 0x77, - 0x17, 0x84, 0x15, 0xe5, 0x5a, 0x65, 0x13, 0xf1, 0x1e, 0x50, 0xd6, 0x2f, 0x42, 0x5f, 0xd7, 0x33, - 0x46, 0xc4, 0x82, 0x23, 0x33, 0x30, 0xa9, 0x3a, 0x05, 0x4c, 0x7f, 0xa3, 0x0f, 0x21, 0xe7, 0x59, - 0xbe, 0x3b, 0xf3, 0x86, 0x34, 0x0e, 0x24, 0x94, 0x2c, 0x17, 0x73, 0x38, 0x37, 0x4a, 0x48, 0xa7, - 0x06, 0xee, 0xcc, 0x2b, 0x06, 0x6e, 0xb4, 0x05, 0x85, 0x3d, 0x31, 0xd9, 0x93, 0x56, 0x64, 0x4f, - 0xcd, 0xce, 0x96, 0x8f, 0x8f, 0xd6, 0xd5, 0x15, 0x22, 0xce, 0x4b, 0x46, 0xd6, 0x08, 0x9a, 0x3f, - 0xe5, 0x98, 0x1b, 0xd0, 0xdc, 0xa9, 0x0c, 0x69, 0xa6, 0x5a, 0x09, 0x68, 0x6a, 0x14, 0x2f, 0xc5, - 0x30, 0x87, 0xa0, 0x0d, 0xc8, 0x0d, 0xc5, 0x22, 0x8b, 0xa7, 0x31, 0x17, 0x17, 0x16, 0x5f, 0x64, - 0x08, 0x84, 0x64, 0x84, 0x67, 0x64, 0xfb, 0x43, 0x92, 0xf8, 0xbd, 0x2c, 0x15, 0x22, 0x3c, 0x75, - 0x01, 0x67, 0x3c, 0x92, 0x0c, 0x6d, 0xc3, 0x25, 0xff, 0x60, 0x16, 0x8c, 0xdc, 0xe7, 0xce, 0x20, - 0x32, 0xea, 0x8a, 0xa7, 0x8f, 0xba, 0x15, 0xc1, 0xb1, 0x1d, 0x8e, 0x3e, 0x25, 0xe9, 0x5a, 0x3a, - 0x2b, 0xe9, 0xda, 0xe0, 0x11, 0x73, 0x99, 0x46, 0xcc, 0xd2, 0xbc, 0x37, 0x9c, 0x94, 0x69, 0x54, - 0x36, 0x4e, 0x89, 0x2c, 0x75, 0x7d, 0xab, 0xda, 0x6f, 0xf4, 0xb4, 0x18, 0x02, 0x48, 0xd7, 0xfa, - 0xdd, 0x5e, 0xbb, 0x29, 0x27, 0xc2, 0x7f, 0x16, 0x03, 0x68, 0x9a, 0x7e, 0xc0, 0xac, 0x84, 0xca, - 0x72, 0x89, 0x91, 0x5b, 0x58, 0x50, 0x94, 0xf9, 0x32, 0x85, 0x04, 0xb4, 0x10, 0xb7, 0xb8, 0x52, - 0x2c, 0x3e, 0x4a, 0xde, 0xbf, 0x77, 0xff, 0x1e, 0x0f, 0xb0, 0x97, 0x21, 0x31, 0xb5, 0x47, 0xd4, - 0x29, 0x73, 0x2c, 0x4c, 0x77, 0x8c, 0x3a, 0x26, 0xb0, 0x48, 0xbe, 0x9c, 0x9a, 0xcb, 0x97, 0x4b, - 0x90, 0x39, 0xb4, 0x3c, 0x9f, 0x2c, 0xb6, 0x68, 0x96, 0x8d, 0x45, 0x11, 0xdd, 0x0a, 0x17, 0x95, - 0x6c, 0xe6, 0x99, 0x5b, 0x54, 0xca, 0xe5, 0x64, 0xe5, 0x87, 0x71, 0xc8, 0xb1, 0x8d, 0x03, 0xd2, - 0xb4, 0x6b, 0x91, 0x15, 0x58, 0x18, 0x0d, 0xc2, 0x3a, 0x23, 0x83, 0x25, 0xf1, 0x8a, 0x83, 0xe5, - 0x01, 0x80, 0x19, 0x04, 0x9e, 0xbd, 0x3b, 0x0b, 0xe4, 0x10, 0x13, 0x8b, 0xb3, 0xaa, 0x40, 0x70, - 0x36, 0x85, 0x12, 0xdd, 0xa4, 0x96, 0x4e, 0x47, 0x5b, 0xc0, 0x76, 0x39, 0x22, 0x0b, 0x18, 0x61, - 0x59, 0x32, 0xa0, 0x52, 0xd4, 0xb2, 0x1f, 0x30, 0xcb, 0x96, 0xfd, 0x48, 0x02, 0xfe, 0x28, 0x92, - 0x97, 0x5e, 0x8b, 0x48, 0x3c, 0x2f, 0xf9, 0x7e, 0xf7, 0x24, 0xc7, 0x89, 0x26, 0xc3, 0x31, 0xee, - 0x2f, 0x3f, 0x4a, 0x40, 0xea, 0x09, 0x0d, 0x9f, 0xef, 0xc9, 0x0a, 0xe3, 0xb7, 0x96, 0x64, 0xbc, - 0xa2, 0xb8, 0xc5, 0xd9, 0xfc, 0x3d, 0x32, 0xa5, 0x9a, 0x63, 0xd3, 0x9b, 0x5b, 0x3a, 0x33, 0xf2, - 0x2e, 0x45, 0x61, 0x4e, 0x42, 0x88, 0x3d, 0xd3, 0xd9, 0xb7, 0x7c, 0x9e, 0xff, 0x46, 0x89, 0x31, - 0x45, 0x61, 0x4e, 0x82, 0x2a, 0x90, 0xf0, 0x2d, 0xb6, 0xa6, 0x0b, 0x0d, 0xcf, 0xc5, 0x5a, 0x01, - 0x26, 0x48, 0x74, 0x03, 0x92, 0x81, 0xf5, 0x82, 0x2d, 0xe8, 0xf2, 0xf3, 0xaa, 0x5a, 0x2f, 0x02, - 0x4c, 0xd1, 0xe5, 0xeb, 0x90, 0x66, 0x9a, 0x44, 0xa7, 0x8d, 0x58, 0x64, 0xda, 0x28, 0x7f, 0x13, - 0x52, 0x54, 0x05, 0x42, 0xb4, 0x6b, 0xed, 0xdb, 0x0e, 0x25, 0x4a, 0x0a, 0x22, 0x0a, 0x42, 0xab, - 0x90, 0xb0, 0x68, 0xc2, 0x1f, 0x62, 0x08, 0xa0, 0xfc, 0x31, 0xa4, 0x99, 0xfe, 0xe8, 0x0e, 0xa4, - 0x68, 0x0b, 0xf8, 0xb4, 0x8b, 0x16, 0xdb, 0x28, 0x24, 0x52, 0xb2, 0xf2, 0x65, 0x48, 0x74, 0x2d, - 0x9a, 0xaf, 0xd8, 0x81, 0x35, 0xa1, 0x5c, 0x39, 0x4c, 0x7f, 0x97, 0x2b, 0x90, 0x24, 0xad, 0x38, - 0x73, 0x33, 0xeb, 0x03, 0xde, 0xc5, 0x00, 0xe9, 0x6e, 0xad, 0xda, 0xa8, 0x62, 0xed, 0x0d, 0xf2, - 0x1b, 0x57, 0x5b, 0xdb, 0x74, 0x41, 0x95, 0x81, 0x44, 0x57, 0xef, 0xb1, 0xc5, 0x7a, 0x4f, 0xff, - 0xac, 0xa7, 0x25, 0x2a, 0xbf, 0x1f, 0x83, 0x9c, 0xf4, 0x60, 0xb9, 0x1c, 0x8f, 0x2d, 0x2c, 0xc7, - 0x85, 0x23, 0xc4, 0x5f, 0xcf, 0x11, 0x12, 0xaf, 0xe3, 0x08, 0xc9, 0xf3, 0x1d, 0xe1, 0xd5, 0x3a, - 0x59, 0xf8, 0x4b, 0xfa, 0x0c, 0x7f, 0xa9, 0xfc, 0x83, 0x1c, 0x64, 0xc5, 0x88, 0x3f, 0xa3, 0xe1, - 0x37, 0xce, 0x69, 0xf8, 0xff, 0xf3, 0x26, 0xf3, 0xb6, 0xa4, 0xce, 0xf2, 0xfd, 0x53, 0xf6, 0x2a, - 0xde, 0x87, 0xe4, 0xc8, 0xf6, 0x9f, 0xf1, 0x10, 0x5a, 0x9a, 0x0b, 0x73, 0x64, 0x1e, 0xa4, 0x79, - 0x2c, 0xa6, 0x54, 0xa8, 0x0a, 0x79, 0xcf, 0xf2, 0xc9, 0xca, 0x84, 0x6e, 0x7f, 0x65, 0x23, 0x49, - 0xb9, 0x64, 0xc2, 0x21, 0x09, 0xcb, 0x3a, 0x15, 0x1e, 0xf4, 0x4d, 0x12, 0x5c, 0x0f, 0xdd, 0x21, - 0x4d, 0xb6, 0x58, 0x56, 0xff, 0xd6, 0xa2, 0x00, 0x4e, 0xc0, 0x66, 0x5f, 0x49, 0x8f, 0x36, 0x20, - 0xed, 0x1f, 0x98, 0x9e, 0x35, 0xa2, 0x19, 0x40, 0x7e, 0xa3, 0x3c, 0xcf, 0xd9, 0xa5, 0x58, 0xca, - 0xc6, 0x29, 0xd1, 0x36, 0x2c, 0x9b, 0xe3, 0xb1, 0x3b, 0xa4, 0xd5, 0x0f, 0x6c, 0x67, 0xcf, 0xe5, - 0xf9, 0xc1, 0xda, 0x3c, 0x73, 0x55, 0x92, 0x51, 0x01, 0x4b, 0x66, 0xa4, 0x5c, 0xbe, 0x0e, 0x4b, - 0x51, 0x0a, 0x32, 0x08, 0xa9, 0x4d, 0xd9, 0x46, 0x1e, 0xfd, 0x5d, 0x7e, 0x02, 0xcb, 0x73, 0xed, - 0x8f, 0xee, 0xf9, 0xc4, 0xe6, 0xf7, 0x7c, 0xc2, 0x44, 0x20, 0x7e, 0x46, 0x22, 0x50, 0xfe, 0xf5, - 0x24, 0x64, 0x45, 0x6f, 0x90, 0x54, 0x71, 0x4a, 0xa6, 0x41, 0x3f, 0xb0, 0x9c, 0xa1, 0xc5, 0x33, - 0xfe, 0xeb, 0xa7, 0x75, 0xde, 0x9d, 0x4e, 0x48, 0x8b, 0x55, 0x46, 0x52, 0xf7, 0xa1, 0x3b, 0x9e, - 0x4d, 0xac, 0xb9, 0xba, 0x9f, 0x50, 0x20, 0xe6, 0x48, 0xf4, 0x0d, 0x99, 0x78, 0x25, 0x4e, 0xee, - 0x71, 0x59, 0x53, 0x97, 0x96, 0x45, 0x56, 0x56, 0xde, 0x86, 0xbc, 0x52, 0xf7, 0x99, 0x59, 0x45, - 0xc4, 0x48, 0xf1, 0x39, 0x23, 0x95, 0x7f, 0x23, 0x0e, 0x69, 0x26, 0x9b, 0x2c, 0x53, 0x94, 0xf9, - 0xe6, 0xfa, 0x39, 0xaa, 0xa8, 0x03, 0xf0, 0x63, 0x65, 0x8d, 0x97, 0x3f, 0x9f, 0xb3, 0x63, 0x06, - 0x07, 0x7c, 0x25, 0xf8, 0x4d, 0x48, 0x4d, 0xdc, 0x99, 0x13, 0xf0, 0xf6, 0xdf, 0x38, 0x8f, 0xb5, - 0x49, 0x88, 0x31, 0xe3, 0x29, 0x5f, 0x83, 0x24, 0x11, 0x45, 0x02, 0x88, 0xe7, 0xba, 0x41, 0x34, - 0x80, 0x10, 0x48, 0xf9, 0x6d, 0x48, 0x51, 0x8e, 0xd3, 0x49, 0x2a, 0x57, 0x78, 0xdc, 0xce, 0x42, - 0xb2, 0x53, 0xed, 0xed, 0x68, 0x31, 0x94, 0x83, 0x54, 0xb3, 0xdd, 0x6f, 0xf5, 0xb4, 0x78, 0x79, - 0x19, 0x8a, 0x91, 0x21, 0x53, 0x2e, 0x00, 0x84, 0x23, 0xa1, 0xf2, 0xb7, 0xe2, 0x50, 0xea, 0x79, - 0xe6, 0xde, 0x9e, 0x3d, 0x24, 0xb9, 0xb0, 0xe7, 0x8e, 0xbb, 0x81, 0x19, 0xd8, 0x7e, 0x60, 0x0f, - 0xfd, 0x33, 0xfb, 0xa4, 0x04, 0x99, 0x5d, 0x73, 0xf8, 0x6c, 0xec, 0xee, 0x53, 0x9b, 0x25, 0xb1, - 0x28, 0x92, 0x35, 0xcf, 0xee, 0xcb, 0x80, 0xcf, 0xcc, 0x49, 0xcc, 0x0a, 0x04, 0x3a, 0xf2, 0xdc, - 0x29, 0x8b, 0x59, 0x49, 0xcc, 0x0a, 0x64, 0x7d, 0x45, 0x72, 0xe7, 0xb1, 0x3d, 0xb1, 0x03, 0xb6, - 0x09, 0x91, 0xc4, 0x0a, 0x84, 0xd4, 0x32, 0x35, 0x87, 0xcf, 0xac, 0x80, 0xed, 0x35, 0x24, 0xb1, - 0x28, 0x92, 0xf1, 0xf5, 0xc5, 0xd8, 0x72, 0x68, 0x70, 0x4a, 0x62, 0xfa, 0x1b, 0xdd, 0x80, 0x8c, - 0x67, 0x06, 0xd6, 0xee, 0xd4, 0xa7, 0xe1, 0x27, 0xb9, 0x99, 0x3f, 0x3e, 0x5a, 0xcf, 0x60, 0x33, - 0xb0, 0x36, 0x3b, 0x5d, 0x2c, 0x70, 0x82, 0x6c, 0x3a, 0xf5, 0x69, 0x90, 0x51, 0xc8, 0x3a, 0x82, - 0x6c, 0x3a, 0xa5, 0x5f, 0x43, 0x3c, 0xeb, 0x8b, 0x99, 0x35, 0xa3, 0x3b, 0xa8, 0xa4, 0x16, 0x59, - 0xae, 0xfc, 0x42, 0x0a, 0x0a, 0xc6, 0x54, 0x31, 0xd5, 0x1a, 0xc0, 0x96, 0xeb, 0x3d, 0x37, 0xbd, - 0x91, 0xed, 0xec, 0xd3, 0x41, 0x97, 0xc0, 0x0a, 0x84, 0xe0, 0xeb, 0xd6, 0x9e, 0x39, 0x1b, 0x07, - 0xbd, 0x5e, 0x83, 0x5a, 0x2c, 0x81, 0x15, 0x08, 0xc1, 0x1b, 0x0e, 0xb6, 0x86, 0x96, 0x7d, 0xc8, - 0x2d, 0x97, 0xc0, 0x0a, 0x04, 0x5d, 0x83, 0xbc, 0xe1, 0xec, 0x8c, 0x3c, 0xdd, 0xf3, 0x5c, 0x8f, - 0x19, 0x31, 0x81, 0x55, 0x10, 0xaa, 0x40, 0xc1, 0x70, 0x48, 0x7e, 0xcb, 0x49, 0x52, 0x94, 0x24, - 0x02, 0x43, 0xd7, 0xa1, 0x48, 0x74, 0xaa, 0x9b, 0x81, 0xb9, 0xef, 0x99, 0x13, 0x66, 0xd4, 0x04, - 0x8e, 0x02, 0xd1, 0x2d, 0x58, 0x36, 0x9c, 0xbe, 0xf3, 0xcc, 0x71, 0x9f, 0x3b, 0x1d, 0xcf, 0x0d, - 0x5c, 0x96, 0x45, 0x27, 0xf0, 0x3c, 0x98, 0x69, 0x4d, 0xd6, 0x43, 0xa6, 0x37, 0x62, 0x36, 0xa7, - 0x5a, 0x0b, 0x08, 0xc7, 0x5b, 0x63, 0x9b, 0x24, 0xe6, 0xd4, 0xd8, 0x0c, 0xcf, 0x21, 0xa4, 0x55, - 0xed, 0x59, 0x80, 0x89, 0x55, 0xfd, 0x80, 0x59, 0x39, 0x81, 0x55, 0x10, 0xa7, 0x90, 0x55, 0xe4, - 0x25, 0x85, 0xac, 0x83, 0x51, 0xb4, 0x5c, 0xec, 0xd2, 0xec, 0xba, 0x20, 0x29, 0x04, 0x88, 0x58, - 0x06, 0x5b, 0xa6, 0x3f, 0xe1, 0x3b, 0xf8, 0x74, 0x39, 0x96, 0xc0, 0x11, 0x18, 0xd1, 0x94, 0x96, - 0xb1, 0xf5, 0xc5, 0x88, 0x2d, 0xbb, 0x12, 0x58, 0x81, 0x10, 0x67, 0xa0, 0xa5, 0xf6, 0x63, 0x9f, - 0xae, 0xb7, 0x12, 0x58, 0x96, 0x25, 0xef, 0x96, 0x69, 0x8f, 0xfd, 0x92, 0xa6, 0xf0, 0x52, 0x08, - 0x71, 0xe2, 0x2d, 0xcf, 0xdc, 0x27, 0xac, 0x17, 0x28, 0x52, 0x14, 0x49, 0x60, 0x23, 0x3f, 0x19, - 0x23, 0xa2, 0xb8, 0x10, 0x40, 0x5a, 0x46, 0x0a, 0x35, 0xcf, 0x32, 0x49, 0xcb, 0x56, 0x58, 0xcb, - 0x14, 0x50, 0xe5, 0xef, 0x66, 0x60, 0xc9, 0x18, 0x4e, 0x54, 0x47, 0x5c, 0x85, 0xb4, 0xe1, 0x34, - 0xfd, 0x7d, 0x9f, 0x3b, 0x21, 0x2f, 0x91, 0x06, 0x18, 0x0e, 0x77, 0x0d, 0xe6, 0x7e, 0xb2, 0xcc, - 0x5c, 0xa7, 0xe6, 0xcf, 0x26, 0x1c, 0x9f, 0x10, 0xae, 0x13, 0xc2, 0xd0, 0x4d, 0x58, 0x22, 0x1d, - 0xe7, 0x07, 0x7d, 0xc7, 0xb3, 0xcc, 0xe1, 0x81, 0xf0, 0xc1, 0x39, 0x28, 0x73, 0x54, 0x62, 0x55, - 0xfd, 0xc5, 0x70, 0x24, 0xbc, 0x50, 0x05, 0x31, 0x8a, 0x8e, 0xe9, 0x4d, 0x3a, 0x9e, 0xbb, 0x2b, - 0x5c, 0x50, 0x05, 0x31, 0x7d, 0xba, 0xde, 0xf0, 0x67, 0x66, 0x96, 0x43, 0x6a, 0xca, 0x08, 0x7d, - 0x42, 0x18, 0x93, 0x82, 0xad, 0x91, 0xed, 0x59, 0xc3, 0x40, 0xf8, 0x9e, 0x0a, 0x22, 0x66, 0x37, - 0x1c, 0x7d, 0x78, 0xe0, 0x0a, 0xcf, 0x13, 0x45, 0xe6, 0x96, 0xe4, 0x27, 0xb6, 0xa6, 0xc2, 0xeb, - 0x14, 0x08, 0xab, 0x9f, 0x28, 0xec, 0x07, 0xe6, 0x64, 0x2a, 0xbc, 0x2e, 0x02, 0x63, 0x83, 0x44, - 0x96, 0xa9, 0xa0, 0x82, 0x18, 0x24, 0x11, 0x30, 0xd3, 0x94, 0x0c, 0xc2, 0xa6, 0xe9, 0x3f, 0xf3, - 0xb9, 0xf7, 0xa9, 0x20, 0x66, 0x5b, 0x51, 0xa4, 0xa2, 0x96, 0x84, 0x6d, 0x55, 0x28, 0x69, 0x51, - 0x7b, 0x16, 0xd0, 0xce, 0x65, 0x3e, 0x28, 0x8a, 0xc4, 0x91, 0xda, 0xb3, 0x80, 0x77, 0x1f, 0xf3, - 0xc0, 0x10, 0x40, 0x74, 0x25, 0x23, 0x46, 0xed, 0x3c, 0xe6, 0x88, 0xf3, 0x60, 0xd2, 0xf2, 0xf6, - 0x2c, 0x08, 0xbb, 0x8f, 0xf9, 0x64, 0x04, 0xc6, 0x69, 0xc2, 0x0e, 0x5c, 0x91, 0x34, 0x61, 0x0f, - 0x5e, 0x87, 0x62, 0x7b, 0x16, 0x28, 0x5d, 0x78, 0x91, 0x05, 0x9a, 0x08, 0x90, 0x4b, 0x0a, 0x3b, - 0xf1, 0x92, 0x94, 0x14, 0xf6, 0x62, 0x19, 0xb2, 0xa4, 0x21, 0xb4, 0x1b, 0x57, 0x99, 0xdf, 0x8a, - 0x32, 0x1f, 0xfa, 0xb2, 0x23, 0xdf, 0x94, 0x43, 0x5f, 0xf6, 0x24, 0xd3, 0x43, 0xe9, 0xca, 0x92, - 0xd4, 0x43, 0xe9, 0xcb, 0xdb, 0xa0, 0xa9, 0x00, 0x2a, 0xec, 0x32, 0x25, 0x5c, 0x80, 0x73, 0x9d, - 0xc3, 0xee, 0x2c, 0x4b, 0x9d, 0xc3, 0xfe, 0x64, 0xf6, 0x8e, 0x74, 0xe8, 0x15, 0x69, 0x6f, 0x15, - 0x5c, 0xf9, 0x61, 0x02, 0x8a, 0xbd, 0xa1, 0x3a, 0x7e, 0x49, 0xb0, 0x0a, 0xdc, 0xea, 0x78, 0xdf, - 0xf5, 0xec, 0xe0, 0x60, 0xc2, 0x47, 0x71, 0x04, 0x46, 0xc6, 0x38, 0x0e, 0xdc, 0xa6, 0xed, 0xf0, - 0x91, 0xcc, 0x4b, 0x02, 0x6e, 0xbe, 0xe0, 0x23, 0x98, 0x97, 0x88, 0xdf, 0x34, 0xcd, 0x17, 0x35, - 0xd7, 0x71, 0xf8, 0xa0, 0x15, 0x45, 0x62, 0xc1, 0xea, 0x30, 0xb0, 0x0f, 0xad, 0xf6, 0xd4, 0x72, - 0xe4, 0x68, 0x55, 0x40, 0x44, 0x9f, 0x8e, 0xe9, 0xfb, 0x92, 0x84, 0x0d, 0xd7, 0x08, 0x8c, 0xd0, - 0x54, 0x83, 0xc0, 0x9a, 0x4c, 0x03, 0x16, 0xc9, 0xf8, 0x78, 0x55, 0x61, 0xa4, 0x26, 0xdd, 0x0f, - 0xcc, 0x5d, 0x92, 0x00, 0x87, 0xe3, 0x55, 0x01, 0x11, 0x1f, 0xae, 0xcd, 0x3c, 0x8f, 0x82, 0xf8, - 0x88, 0x0d, 0x01, 0x2c, 0xae, 0x75, 0xad, 0x7d, 0x31, 0x5e, 0x79, 0x89, 0x8f, 0x09, 0x8a, 0xc8, - 0xcb, 0x31, 0x41, 0x31, 0xd7, 0x20, 0x8f, 0xad, 0xc0, 0x33, 0x1d, 0x9f, 0x62, 0xf9, 0xc4, 0xa0, - 0x80, 0x98, 0x4c, 0xdd, 0xf3, 0xc4, 0xa0, 0xe4, 0x25, 0x2e, 0x13, 0x93, 0x29, 0x69, 0x49, 0xca, - 0x24, 0xc5, 0x85, 0x48, 0xb9, 0xbc, 0x18, 0x29, 0x2b, 0xbf, 0x11, 0x87, 0x62, 0x7f, 0xa4, 0xf6, - 0x29, 0x8d, 0x00, 0xe1, 0xa4, 0x1b, 0x13, 0x11, 0x20, 0x9c, 0x72, 0x4b, 0x90, 0x69, 0xb9, 0x1d, - 0xd7, 0x0b, 0x44, 0x70, 0x16, 0xc5, 0x48, 0xdc, 0x4e, 0x2c, 0xc6, 0x6d, 0x32, 0x80, 0xa5, 0xe0, - 0xa4, 0xf4, 0xc5, 0x50, 0x32, 0xf1, 0xa7, 0xe1, 0xe1, 0xee, 0x6c, 0x2f, 0x9a, 0x16, 0xa8, 0x30, - 0x42, 0xd3, 0x75, 0x46, 0x21, 0x0d, 0xef, 0x63, 0x15, 0xb6, 0xd0, 0xf2, 0xcc, 0x09, 0x73, 0x04, - 0xa1, 0xd9, 0x77, 0x5c, 0xcf, 0x1a, 0x35, 0x67, 0xe3, 0xc0, 0xe6, 0x9d, 0x1c, 0x81, 0x55, 0xfe, - 0x4c, 0x1c, 0x96, 0xba, 0xad, 0x66, 0x47, 0x31, 0xcf, 0x37, 0x21, 0x6b, 0x4f, 0x07, 0x7e, 0x60, - 0x06, 0xfe, 0xdc, 0x07, 0x0a, 0x35, 0xc5, 0x62, 0x59, 0x9a, 0x41, 0x19, 0x7d, 0x9c, 0xb1, 0x29, - 0xca, 0x47, 0x35, 0x00, 0x7b, 0x38, 0x11, 0xec, 0xf1, 0xc8, 0xc7, 0xd0, 0xe8, 0xd4, 0xb8, 0x59, - 0x3c, 0x3e, 0x5a, 0xcf, 0x19, 0xb5, 0x26, 0x17, 0x91, 0xb3, 0x39, 0xda, 0x47, 0x9f, 0x42, 0x2e, - 0x18, 0x0a, 0x19, 0x89, 0xc8, 0x6e, 0x6f, 0x64, 0x74, 0xb2, 0x2f, 0xee, 0xbd, 0x1a, 0x97, 0x90, - 0x0d, 0x86, 0xa1, 0x80, 0xd9, 0x48, 0x08, 0x48, 0x46, 0x04, 0x44, 0x5c, 0x81, 0x09, 0xe8, 0xd7, - 0x85, 0x80, 0x19, 0x43, 0xfa, 0x95, 0x3f, 0x8f, 0x00, 0x89, 0x25, 0x43, 0x24, 0x1a, 0xe4, 0x02, - 0x11, 0x7e, 0x22, 0x1b, 0x4d, 0x21, 0x18, 0xd5, 0x60, 0x65, 0x38, 0x9d, 0xf9, 0x83, 0x99, 0xcf, - 0xcf, 0x2f, 0x0c, 0x7c, 0x6b, 0xc8, 0xbf, 0x2e, 0x6e, 0x5e, 0x3c, 0x3e, 0x5a, 0xd7, 0x6a, 0x9d, - 0xbe, 0xdf, 0xf7, 0xd9, 0x11, 0x86, 0xae, 0x35, 0xf4, 0xb1, 0x46, 0x18, 0x54, 0x08, 0x32, 0xe0, - 0x12, 0x15, 0xe2, 0xbf, 0xf4, 0x03, 0x6b, 0xa2, 0x88, 0xa1, 0x1f, 0x1e, 0x37, 0x57, 0x8f, 0x8f, - 0xd6, 0x11, 0x11, 0xd3, 0xa5, 0x78, 0x29, 0x08, 0x11, 0xa6, 0x28, 0x0c, 0xbd, 0x0f, 0x40, 0x45, - 0xd1, 0x14, 0x9e, 0x7d, 0x8a, 0x64, 0xa6, 0x27, 0xfc, 0x0d, 0x02, 0xc4, 0x39, 0x42, 0x40, 0x7f, - 0xa2, 0x0f, 0xa1, 0x38, 0xb1, 0x26, 0x03, 0xcf, 0xf7, 0x07, 0x6c, 0xd5, 0x40, 0x17, 0x01, 0xec, - 0x0b, 0x40, 0xd3, 0x9a, 0xe0, 0x6e, 0x77, 0x93, 0x80, 0x71, 0x7e, 0x62, 0x4d, 0xb0, 0xef, 0xd3, - 0x02, 0xba, 0x09, 0xcb, 0x84, 0x89, 0xd6, 0xc0, 0xd9, 0xd8, 0xf2, 0x80, 0xc8, 0xa2, 0x72, 0x19, - 0xdd, 0x43, 0x58, 0xa6, 0xaa, 0x38, 0x1e, 0xdf, 0x8a, 0x67, 0x7e, 0x5b, 0xdc, 0xbc, 0x70, 0x7c, - 0xb4, 0x5e, 0x24, 0xfa, 0xb4, 0x3c, 0xb6, 0xe7, 0xee, 0xe3, 0x22, 0xa1, 0x94, 0x45, 0xf4, 0x29, - 0x5c, 0x10, 0xac, 0xc1, 0x81, 0xe7, 0x06, 0xc1, 0xd8, 0x62, 0x5f, 0x2c, 0x8a, 0x9b, 0x2b, 0xc7, - 0x47, 0xeb, 0xcb, 0x8c, 0xb9, 0x27, 0x50, 0x78, 0x99, 0xb1, 0x4b, 0x00, 0xc2, 0x50, 0xa2, 0x02, - 0x24, 0xb7, 0x62, 0xd4, 0x1c, 0x35, 0xca, 0xe5, 0xe3, 0xa3, 0xf5, 0x4b, 0x44, 0x8e, 0x64, 0x92, - 0x76, 0xa5, 0x9d, 0xb1, 0x00, 0x46, 0xd7, 0x61, 0x89, 0xb4, 0x7b, 0xcf, 0x1e, 0x5b, 0xbc, 0xd9, - 0x6c, 0x61, 0x52, 0x98, 0x58, 0x93, 0x2d, 0x7b, 0x6c, 0xb1, 0x56, 0x73, 0x2a, 0xd3, 0x71, 0x1d, - 0x4e, 0x95, 0x97, 0x54, 0x55, 0xc7, 0x75, 0x18, 0xd5, 0x07, 0x70, 0x89, 0x50, 0x4d, 0xcc, 0xe9, - 0xd4, 0x1a, 0xa9, 0x22, 0x0b, 0x94, 0x18, 0x4d, 0xac, 0x49, 0x93, 0xe2, 0x42, 0xc1, 0xef, 0x42, - 0x72, 0x6a, 0x79, 0x7b, 0xfc, 0x7b, 0x86, 0x18, 0x65, 0x1d, 0xcb, 0xdb, 0x0b, 0x5d, 0x16, 0x53, - 0x12, 0xa2, 0x83, 0x63, 0x05, 0x03, 0xef, 0xc5, 0x40, 0xac, 0xdf, 0x96, 0x98, 0x0e, 0x8e, 0x15, - 0xe0, 0x17, 0x1d, 0xbe, 0x88, 0xbb, 0x06, 0x05, 0x4e, 0xc5, 0xaa, 0x5e, 0x66, 0x0b, 0x40, 0x4a, - 0xc3, 0xaa, 0xac, 0x40, 0x91, 0x53, 0x58, 0x61, 0x72, 0x93, 0xc4, 0x79, 0x4a, 0x22, 0x57, 0x35, - 0xa2, 0x2e, 0xb2, 0xa8, 0x9c, 0x5a, 0x23, 0x9a, 0xdd, 0x88, 0xba, 0xea, 0x0c, 0x26, 0xa8, 0x82, - 0x50, 0x23, 0x24, 0xa9, 0x7a, 0xf3, 0x1a, 0x05, 0x42, 0xa3, 0x15, 0xa9, 0x51, 0x2f, 0xaa, 0x51, - 0x20, 0x35, 0xba, 0x28, 0x35, 0xea, 0xcd, 0x69, 0x14, 0x84, 0x1a, 0x5d, 0x52, 0xea, 0x12, 0x1a, - 0x75, 0xe1, 0x32, 0xa5, 0x1a, 0x4e, 0x07, 0x5e, 0x10, 0x0c, 0x26, 0xf6, 0xd0, 0x73, 0x89, 0x7b, - 0x0c, 0xa6, 0xf7, 0xef, 0xd1, 0x5c, 0x87, 0xbb, 0x48, 0xcb, 0x0a, 0x7a, 0xb5, 0x0e, 0x0e, 0x82, - 0xa6, 0xa0, 0xe8, 0xdc, 0xbf, 0x87, 0x2f, 0x11, 0x59, 0xc3, 0xe9, 0x1c, 0xf8, 0x0c, 0xa1, 0x0f, - 0xef, 0xd1, 0x0c, 0xe9, 0x74, 0xa1, 0x0f, 0x4f, 0x16, 0xfa, 0xf0, 0x4c, 0xa1, 0xf7, 0x69, 0x4a, - 0x75, 0x86, 0xd0, 0xfb, 0x27, 0x0b, 0xbd, 0x7f, 0x96, 0xd0, 0x87, 0x34, 0xfd, 0x3a, 0x43, 0xe8, - 0xc3, 0x93, 0x85, 0x3e, 0x44, 0xb7, 0x40, 0x1b, 0xd9, 0xfe, 0xb3, 0x48, 0x68, 0x28, 0x53, 0xdb, - 0x2f, 0x11, 0xb8, 0x12, 0x1b, 0x6e, 0xc2, 0x32, 0xa5, 0x9c, 0xf9, 0xd6, 0x88, 0x13, 0x5e, 0x61, - 0x31, 0x84, 0x80, 0xfb, 0xbe, 0x35, 0x62, 0x74, 0x9f, 0x41, 0x59, 0xa8, 0x69, 0xd2, 0xbc, 0x68, - 0x30, 0x74, 0x1d, 0xc7, 0x1a, 0x06, 0xb6, 0xeb, 0xf8, 0xa5, 0xab, 0x54, 0xcf, 0x2b, 0xc7, 0x47, - 0xeb, 0x6f, 0x32, 0x3d, 0x59, 0xee, 0x54, 0x0b, 0x49, 0xf0, 0x9b, 0x4c, 0xd3, 0x05, 0x04, 0x1a, - 0xc0, 0x5b, 0x42, 0x32, 0x0d, 0x0d, 0xcf, 0x4d, 0x3b, 0x88, 0x08, 0x7f, 0x8b, 0x0a, 0x7f, 0xeb, - 0xf8, 0x68, 0xfd, 0x32, 0x13, 0x4e, 0x02, 0xc1, 0x53, 0xd3, 0x0e, 0x54, 0xf1, 0x97, 0x99, 0xf8, - 0x13, 0x50, 0x6c, 0xdf, 0xcc, 0x1d, 0x5a, 0xbe, 0x6f, 0xf9, 0xa5, 0x35, 0x12, 0xbb, 0x70, 0x08, - 0x20, 0x39, 0x47, 0x70, 0xe0, 0x59, 0xe6, 0xc8, 0x2f, 0xad, 0x53, 0x9c, 0x28, 0xa2, 0x6f, 0x40, - 0x89, 0x86, 0x57, 0xf7, 0xf9, 0x60, 0xea, 0x59, 0xbe, 0x3f, 0xf3, 0x48, 0xa3, 0x67, 0x4e, 0x60, - 0x79, 0xa5, 0x6b, 0xd4, 0x46, 0x24, 0x74, 0x34, 0xdc, 0xe7, 0x1d, 0x8e, 0xad, 0x31, 0x24, 0xfa, - 0x36, 0x5c, 0xa1, 0x31, 0xc5, 0x1a, 0xd9, 0xb3, 0xc9, 0x22, 0xef, 0xdb, 0x94, 0x97, 0xc8, 0x6e, - 0x52, 0x8a, 0x79, 0xf6, 0x2a, 0xbc, 0x45, 0xd8, 0x87, 0x9e, 0x1d, 0xd8, 0x43, 0x73, 0xbc, 0x28, - 0xa0, 0x42, 0x05, 0x94, 0x27, 0xd6, 0xa4, 0xc6, 0x69, 0xe6, 0x45, 0xfc, 0x1c, 0xac, 0x51, 0x9b, - 0xb2, 0x2d, 0x2d, 0x62, 0xcd, 0xc0, 0x73, 0xc7, 0x74, 0x62, 0x66, 0xf1, 0xa9, 0xf4, 0xb5, 0xc8, - 0x61, 0xb8, 0xd3, 0xf6, 0xbe, 0xf8, 0x34, 0x7b, 0x85, 0x18, 0xf7, 0xb4, 0xed, 0x31, 0x3e, 0x0b, - 0x05, 0x6e, 0x60, 0x8e, 0xb9, 0x07, 0x5d, 0x97, 0xb3, 0x50, 0x8f, 0x40, 0x23, 0x91, 0x96, 0xd1, - 0x4d, 0xac, 0x89, 0xff, 0x9c, 0x53, 0xdf, 0x90, 0x91, 0x96, 0x52, 0x37, 0x09, 0x8a, 0xb1, 0xdc, + // 9734 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x7d, 0x4d, 0x90, 0x1b, 0x49, + 0x76, 0xde, 0xe0, 0x1f, 0x78, 0x00, 0xba, 0x8b, 0xd9, 0x64, 0x0f, 0x08, 0x72, 0xba, 0x39, 0x58, + 0x92, 0xc3, 0xe1, 0xcc, 0x92, 0x9c, 0x9e, 0x21, 0x77, 0xc8, 0xdd, 0x9d, 0x11, 0x1a, 0xa8, 0xee, + 0x2e, 0x11, 0x7f, 0x4a, 0x00, 0xe4, 0xcc, 0x86, 0x22, 0x10, 0xd5, 0x40, 0x75, 0x77, 0x89, 0x40, + 0x15, 0xa6, 0xaa, 0xd0, 0x24, 0x37, 0x1c, 0xe1, 0xb5, 0x2c, 0xd9, 0xbb, 0xb2, 0xe4, 0xd0, 0xc1, + 0x07, 0xc9, 0xb2, 0x23, 0x1c, 0xb6, 0x23, 0xec, 0x83, 0x7d, 0x90, 0xed, 0x83, 0x0f, 0x0e, 0xdb, + 0x07, 0x3b, 0xac, 0x9b, 0xb5, 0xe1, 0x8b, 0x7d, 0x69, 0x7b, 0x5a, 0x8e, 0xd0, 0xfa, 0x7f, 0x2d, + 0xff, 0x48, 0xb6, 0xd7, 0x3f, 0x91, 0xbf, 0x95, 0x05, 0xf4, 0x0f, 0x39, 0xb2, 0xc3, 0xa7, 0x46, + 0xbe, 0xfc, 0xde, 0xcb, 0x97, 0x2f, 0x5f, 0x66, 0xbe, 0xcc, 0xca, 0xcc, 0x86, 0xfc, 0xc4, 0xf2, + 0x5d, 0xff, 0xce, 0xd4, 0x73, 0x03, 0x17, 0xa5, 0x68, 0xa2, 0xfc, 0xf5, 0x7d, 0x3b, 0x38, 0x98, + 0xed, 0xde, 0x19, 0xba, 0x93, 0xbb, 0xfb, 0xee, 0xbe, 0x7b, 0x97, 0xe6, 0xee, 0xce, 0xf6, 0x68, + 0x8a, 0x26, 0xe8, 0x2f, 0xc6, 0x55, 0x79, 0x17, 0xf2, 0x5b, 0x9e, 0x39, 0xb1, 0x9e, 0xbb, 0xde, + 0x33, 0xa3, 0x8e, 0xca, 0x90, 0x3a, 0x34, 0xc7, 0x33, 0xab, 0x14, 0xbb, 0x16, 0xbf, 0x95, 0xdb, + 0x4c, 0xfe, 0xd6, 0xd1, 0xfa, 0x1b, 0x98, 0x91, 0x2a, 0x37, 0x20, 0xd3, 0xde, 0xdb, 0xb3, 0xbc, + 0xf3, 0x61, 0xd5, 0x7d, 0xcb, 0x09, 0xce, 0x81, 0x5d, 0x87, 0x74, 0xcf, 0xf4, 0xcf, 0x2b, 0xf3, + 0x16, 0x80, 0xfe, 0xc2, 0x1a, 0xce, 0x02, 0xf7, 0xbc, 0x62, 0xfb, 0x90, 0xaf, 0xb9, 0x4e, 0x60, + 0xda, 0xce, 0x79, 0x1a, 0xa2, 0xdb, 0x90, 0x9e, 0x9a, 0x9e, 0xe5, 0x04, 0xa5, 0xf8, 0xb5, 0xd8, + 0xad, 0xfc, 0x06, 0xba, 0xc3, 0xec, 0xa8, 0xf0, 0x63, 0x8e, 0xa8, 0x6c, 0x40, 0xb6, 0x67, 0x4f, + 0x2c, 0xc3, 0xd9, 0x73, 0xd1, 0x4d, 0xc8, 0x3b, 0xa6, 0xe3, 0xfa, 0xd6, 0xd0, 0x75, 0x46, 0x3e, + 0x95, 0x9c, 0xe0, 0x92, 0xd5, 0x8c, 0xca, 0x03, 0x28, 0xd4, 0x67, 0x9e, 0x19, 0xd8, 0xae, 0xf3, + 0x5a, 0x7c, 0x4f, 0x21, 0x53, 0x1d, 0x8d, 0x3c, 0xcb, 0xf7, 0x51, 0x19, 0xb2, 0x07, 0xae, 0x1f, + 0x38, 0xe6, 0x84, 0xd4, 0x20, 0x76, 0x2b, 0x87, 0x65, 0x1a, 0xad, 0x42, 0xdc, 0x9e, 0x52, 0xd5, + 0x73, 0x9b, 0xe9, 0xe3, 0xa3, 0xf5, 0xb8, 0xd1, 0xc1, 0x71, 0x7b, 0x8a, 0x4a, 0x90, 0x9c, 0xba, + 0x5e, 0x50, 0x4a, 0x5c, 0x8b, 0xdf, 0x4a, 0x71, 0xf9, 0x94, 0x52, 0xf9, 0xcd, 0x18, 0x24, 0xfa, + 0xb8, 0x81, 0xae, 0x42, 0xda, 0x1f, 0x1e, 0x58, 0x93, 0xa8, 0x55, 0x38, 0x0d, 0xdd, 0x81, 0x8c, + 0xc9, 0x8a, 0x2f, 0xc5, 0xaf, 0xc5, 0x6f, 0xe5, 0x37, 0x96, 0xb8, 0x5d, 0xb8, 0x52, 0x1c, 0x2e, + 0x40, 0x08, 0x41, 0x72, 0x6a, 0x06, 0x07, 0xa5, 0x04, 0xd5, 0x8f, 0xfe, 0x46, 0xef, 0x43, 0xea, + 0x8b, 0x99, 0xe5, 0xbd, 0x2c, 0x25, 0xaf, 0x25, 0x6e, 0xe5, 0x37, 0x34, 0x2e, 0xa1, 0x63, 0x12, + 0x1f, 0x0b, 0x2c, 0x4f, 0x34, 0x04, 0x05, 0x91, 0x5a, 0xee, 0x79, 0xe6, 0xfe, 0x84, 0x34, 0x45, + 0x8a, 0xd5, 0x52, 0xa4, 0x2b, 0x0e, 0x2c, 0xf5, 0x1d, 0xf3, 0xd0, 0xb4, 0xc7, 0xe6, 0xae, 0x3d, + 0xb6, 0x83, 0x97, 0xe8, 0x3d, 0x48, 0xf9, 0x81, 0xe9, 0x05, 0x54, 0xf9, 0xfc, 0xc6, 0x32, 0x97, + 0x2d, 0x9a, 0x47, 0x88, 0xa6, 0x18, 0x74, 0x17, 0xb2, 0x23, 0xde, 0x06, 0xbc, 0x95, 0x57, 0x38, + 0x5e, 0x6d, 0x1a, 0x2c, 0x41, 0x95, 0x4f, 0x21, 0xd7, 0x34, 0x87, 0x07, 0xb6, 0x63, 0x51, 0xef, + 0x79, 0x6d, 0xf3, 0x57, 0xfe, 0x71, 0x0c, 0xf2, 0x42, 0x02, 0x69, 0xf5, 0xf7, 0x21, 0x6e, 0x8f, + 0xb8, 0xae, 0xc2, 0x0e, 0xb2, 0x84, 0x4d, 0x20, 0xca, 0x52, 0xee, 0x3a, 0x8e, 0xdb, 0x23, 0xf4, + 0x1e, 0x24, 0x27, 0xee, 0xc8, 0xa2, 0x72, 0x97, 0x36, 0xde, 0x9c, 0xc3, 0x3b, 0x7b, 0xee, 0x9d, + 0xa6, 0x3b, 0xb2, 0x30, 0x05, 0xa1, 0x6f, 0xc3, 0xd2, 0x2c, 0x62, 0x1b, 0xda, 0x06, 0xf9, 0x8d, + 0x4b, 0x9c, 0x2d, 0x6a, 0x38, 0x3c, 0x07, 0xae, 0xdc, 0x84, 0x24, 0x11, 0x86, 0xd2, 0x10, 0xef, + 0x77, 0xb4, 0x18, 0x2a, 0x40, 0xb6, 0x8e, 0xab, 0x46, 0xcb, 0x68, 0x6d, 0x6b, 0x71, 0x94, 0x85, + 0x64, 0xbd, 0xfd, 0xb4, 0xa5, 0x25, 0x2a, 0x7f, 0x23, 0x05, 0xc5, 0x70, 0x70, 0x20, 0x75, 0x2a, + 0x41, 0x72, 0xe6, 0x5b, 0x5e, 0xc4, 0x7d, 0x28, 0x85, 0xe4, 0x50, 0x6b, 0xc5, 0xd5, 0x1c, 0x6a, + 0xaf, 0xdb, 0xd4, 0x0e, 0x89, 0x48, 0x4f, 0x53, 0x86, 0x1c, 0x6e, 0x43, 0x66, 0x85, 0xf7, 0x41, + 0xdb, 0x33, 0xed, 0xb1, 0x7b, 0x68, 0x79, 0x83, 0xc0, 0x9e, 0x58, 0xee, 0x2c, 0x28, 0x25, 0xaf, + 0xc5, 0x6e, 0xc5, 0x1e, 0xc5, 0xee, 0xe1, 0x65, 0x91, 0xd5, 0x63, 0x39, 0xe8, 0x06, 0xc0, 0xf0, + 0xc0, 0x1a, 0x3e, 0x9b, 0xba, 0x36, 0x77, 0xa0, 0xec, 0xa3, 0xd4, 0x9e, 0x39, 0xf6, 0x2d, 0xac, + 0x64, 0xa0, 0x4b, 0x90, 0xf4, 0xdc, 0xb1, 0x55, 0x4a, 0x93, 0x26, 0x7b, 0x14, 0xbb, 0x8d, 0x69, + 0x32, 0xd2, 0xc6, 0x99, 0xb9, 0x36, 0xbe, 0x0a, 0xb9, 0xa9, 0x67, 0x3b, 0x43, 0x7b, 0x6a, 0x8e, + 0x4b, 0x59, 0x9a, 0x19, 0x12, 0xd0, 0xbb, 0x90, 0x7b, 0x6e, 0xed, 0xce, 0xec, 0xc1, 0xcc, 0x1b, + 0x97, 0x72, 0xd4, 0x11, 0x0a, 0xc7, 0x47, 0xeb, 0xd9, 0xa7, 0xd6, 0x6e, 0xdf, 0xee, 0xe3, 0x06, + 0xce, 0xd2, 0xec, 0xbe, 0x37, 0x46, 0x06, 0x14, 0x86, 0xe6, 0x94, 0x19, 0xde, 0xb6, 0xfc, 0x12, + 0xd0, 0x6e, 0xb1, 0xbe, 0x60, 0x06, 0xd2, 0xc0, 0x35, 0x01, 0x7c, 0xc9, 0xed, 0x17, 0x61, 0x45, + 0x37, 0x20, 0x3d, 0x36, 0x77, 0xad, 0xb1, 0x5f, 0xca, 0x53, 0x5b, 0x16, 0xb9, 0x90, 0x06, 0x25, + 0x62, 0x9e, 0x89, 0x2e, 0x42, 0x8a, 0x54, 0xcf, 0x2f, 0x15, 0xae, 0x25, 0x6e, 0xe5, 0x30, 0x4b, + 0x94, 0x7f, 0x37, 0x06, 0x10, 0xca, 0x47, 0x9f, 0x40, 0x32, 0x78, 0x39, 0x65, 0xbe, 0xbd, 0xb4, + 0x71, 0xfd, 0x1c, 0x75, 0xee, 0xf4, 0x5e, 0x4e, 0x2d, 0xd1, 0xa6, 0x84, 0xaf, 0xf2, 0x6b, 0x31, + 0x48, 0x12, 0x22, 0xca, 0x43, 0xa6, 0xdf, 0x7a, 0xdc, 0x22, 0xfe, 0xf2, 0x06, 0x7a, 0x13, 0x56, + 0xb0, 0xfe, 0xa4, 0x5d, 0xab, 0x6e, 0x36, 0xf4, 0x01, 0xd6, 0xbb, 0xed, 0x3e, 0xae, 0xe9, 0x5d, + 0x2d, 0x86, 0x56, 0x01, 0xf5, 0xaa, 0xdd, 0xc7, 0x83, 0xc7, 0x46, 0xa3, 0x61, 0xb4, 0xb6, 0x07, + 0xdd, 0x5e, 0xb5, 0xa7, 0x6b, 0x71, 0x74, 0x01, 0x8a, 0xdb, 0x9d, 0xbe, 0x02, 0x4d, 0xa0, 0x8b, + 0xa0, 0x75, 0x77, 0xaa, 0x58, 0xaf, 0x2b, 0xd4, 0x24, 0x5a, 0x81, 0xe5, 0x4e, 0x15, 0xf7, 0x8c, + 0x9e, 0xd1, 0x6e, 0x0d, 0xaa, 0x4f, 0xab, 0x58, 0xd7, 0x52, 0x68, 0x09, 0xa0, 0xd9, 0x6f, 0xf4, + 0x8c, 0x01, 0x6e, 0x37, 0x74, 0x2d, 0x5d, 0x4e, 0x7e, 0xff, 0x2f, 0xad, 0xc5, 0x2a, 0x3f, 0x4c, + 0x40, 0xae, 0x46, 0x1a, 0x9f, 0x3a, 0xec, 0xdd, 0x48, 0x45, 0x45, 0xff, 0x90, 0xf9, 0x0b, 0x35, + 0x43, 0x1b, 0x90, 0x19, 0xba, 0x93, 0x89, 0xe9, 0x8c, 0xf8, 0xb0, 0x51, 0x5a, 0xe0, 0xa9, 0xb1, + 0x7c, 0x2c, 0x80, 0xe8, 0x43, 0x48, 0x1e, 0x04, 0xc1, 0x74, 0xae, 0x13, 0x86, 0x0c, 0x3b, 0x41, + 0x30, 0xdd, 0xcc, 0x1e, 0x1f, 0xad, 0x27, 0x77, 0x7a, 0xbd, 0x0e, 0xa6, 0x60, 0xf4, 0x0e, 0x14, + 0x47, 0xd6, 0xd8, 0x7c, 0x39, 0x10, 0xd3, 0x02, 0xf3, 0xf3, 0xf8, 0x07, 0xf7, 0x71, 0x81, 0x66, + 0x74, 0x19, 0x1d, 0x7d, 0x1d, 0x34, 0xdb, 0x09, 0x2c, 0xef, 0xd0, 0x1c, 0x4b, 0x6c, 0x8a, 0x63, + 0xef, 0xe1, 0x65, 0x91, 0x27, 0xe0, 0xef, 0xc1, 0x32, 0xef, 0x39, 0x12, 0x9d, 0x66, 0xe8, 0x8d, + 0x7b, 0x78, 0x89, 0x67, 0x71, 0x70, 0xf9, 0xdb, 0x90, 0xe1, 0xb5, 0x51, 0x2b, 0xce, 0xc6, 0xac, + 0x70, 0x56, 0xa4, 0x54, 0x65, 0x88, 0x15, 0xc0, 0xf2, 0x47, 0x90, 0x24, 0x75, 0x93, 0x33, 0x0f, + 0x61, 0x2c, 0xaa, 0x33, 0x8f, 0x9c, 0x23, 0xe2, 0xe1, 0x1c, 0x51, 0xb9, 0x77, 0x92, 0xef, 0xe4, + 0x21, 0x53, 0x6b, 0x37, 0x9b, 0xd5, 0x56, 0x5d, 0x8b, 0x91, 0x21, 0x88, 0x58, 0x4a, 0x8b, 0xf3, + 0x36, 0xfd, 0x9b, 0x29, 0xc8, 0xef, 0x58, 0xe6, 0x38, 0x38, 0xa0, 0x46, 0x25, 0xee, 0x4b, 0x0d, + 0x1e, 0xa3, 0x06, 0xbf, 0xc6, 0x15, 0x55, 0x10, 0x77, 0x08, 0xbb, 0x6c, 0x80, 0xf3, 0x6d, 0x1f, + 0x7f, 0x0d, 0xdb, 0x27, 0x5e, 0xcb, 0xf6, 0xc9, 0xd3, 0x6c, 0x8f, 0x3e, 0x82, 0x8b, 0x43, 0xd7, + 0xf1, 0x49, 0x70, 0x63, 0x1f, 0x5a, 0x03, 0x32, 0xb8, 0xcd, 0x3c, 0x8b, 0xb5, 0x6d, 0xf1, 0x51, + 0xec, 0x43, 0xbc, 0xa2, 0x64, 0x6f, 0xf1, 0x5c, 0xc2, 0xb5, 0xef, 0x99, 0x43, 0x6b, 0x30, 0xb5, + 0x3c, 0xdb, 0x1d, 0xcd, 0xb7, 0xf1, 0x07, 0xf7, 0x30, 0xa2, 0xf9, 0x1d, 0x9a, 0x2d, 0xca, 0x7a, + 0x3f, 0x6c, 0xdc, 0xcc, 0x5c, 0xc8, 0x23, 0x1b, 0x37, 0xf4, 0xe7, 0x0f, 0x78, 0xa7, 0xc9, 0x46, + 0xe6, 0x22, 0xd5, 0xbc, 0x0b, 0xdd, 0xe6, 0x11, 0x24, 0x82, 0xe1, 0x94, 0x0e, 0x86, 0xe1, 0xf0, + 0x16, 0xe1, 0xa8, 0x29, 0xed, 0x91, 0x39, 0x3e, 0x5a, 0x4f, 0xf4, 0x6a, 0x1d, 0x4c, 0x98, 0xca, + 0x5f, 0x40, 0x31, 0xd2, 0x5c, 0xaf, 0xe7, 0x4e, 0x68, 0x55, 0x06, 0x35, 0x2c, 0x10, 0x11, 0xe1, + 0x4c, 0x19, 0xb2, 0x7e, 0x60, 0x06, 0x33, 0xdf, 0xf2, 0x69, 0x34, 0x52, 0xc4, 0x32, 0x5d, 0xbe, + 0x05, 0x05, 0x55, 0xa1, 0xd3, 0x4b, 0xac, 0x3c, 0x7c, 0x75, 0x67, 0x45, 0x19, 0x20, 0x95, 0xd2, + 0x12, 0xdc, 0x6b, 0xeb, 0x00, 0x8f, 0xed, 0xf1, 0xb8, 0xe3, 0x8e, 0xed, 0xe1, 0x4b, 0xf4, 0x00, + 0x0a, 0x6a, 0xc3, 0x71, 0xdf, 0x3d, 0x31, 0x28, 0xc9, 0x2b, 0x2d, 0x58, 0xf9, 0x49, 0x9c, 0x04, + 0xb6, 0xb2, 0x95, 0xd0, 0xc7, 0x90, 0x9c, 0x79, 0x36, 0x89, 0x22, 0xc9, 0x4c, 0xb2, 0xba, 0xd8, + 0x8e, 0x77, 0xfa, 0xd8, 0xd8, 0x2c, 0xf0, 0xf0, 0x22, 0xd9, 0xc7, 0x86, 0x8f, 0x29, 0x07, 0xfa, + 0x08, 0xf2, 0x96, 0x73, 0x68, 0x7b, 0xae, 0x33, 0x59, 0x8c, 0x7d, 0xf5, 0x30, 0x07, 0xab, 0x30, + 0x32, 0x9f, 0xb0, 0x40, 0x9a, 0x59, 0x97, 0x87, 0xd0, 0x88, 0x07, 0x02, 0x2c, 0x6a, 0x63, 0x21, + 0x40, 0x19, 0x52, 0xfe, 0x81, 0x35, 0x1e, 0x53, 0x5f, 0xcc, 0x3e, 0x4a, 0x06, 0xde, 0xcc, 0xc2, + 0x8c, 0x44, 0x26, 0x54, 0xd3, 0xdb, 0x9f, 0x11, 0x89, 0x7e, 0x29, 0x43, 0x67, 0xa6, 0x90, 0x50, + 0xfe, 0x35, 0x1a, 0x9f, 0x1a, 0x67, 0x06, 0xed, 0x6b, 0x00, 0x16, 0x5d, 0x09, 0x98, 0xbb, 0x63, + 0x16, 0x26, 0x65, 0xb1, 0x42, 0x41, 0x6b, 0x90, 0xb1, 0x5e, 0x04, 0x9e, 0x39, 0x0c, 0xa8, 0xa6, + 0xa2, 0x7c, 0x41, 0x24, 0xf5, 0x18, 0x9a, 0xc3, 0x03, 0x8b, 0xf6, 0xc8, 0x2c, 0x66, 0x09, 0xb4, + 0x0e, 0x79, 0x77, 0x16, 0x4c, 0x67, 0xc1, 0x60, 0xcf, 0x1e, 0x5b, 0xbc, 0x3a, 0xc0, 0x48, 0x5b, + 0xf6, 0xd8, 0xaa, 0xfc, 0x62, 0x0a, 0x0a, 0x72, 0x05, 0x42, 0xec, 0xbf, 0x05, 0x79, 0x8b, 0xa7, + 0x07, 0x32, 0xbe, 0xbb, 0x20, 0xac, 0x28, 0xd7, 0x2a, 0x9b, 0x88, 0xb7, 0x80, 0xb2, 0x7e, 0x11, + 0xfa, 0xba, 0x9e, 0x31, 0x22, 0x16, 0x1c, 0x99, 0x81, 0x49, 0xd5, 0x29, 0x60, 0xfa, 0x1b, 0x7d, + 0x08, 0x39, 0xcf, 0xf2, 0xdd, 0x99, 0x37, 0xa4, 0xe3, 0x40, 0x42, 0x89, 0x72, 0x31, 0xa7, 0x73, + 0xa3, 0x84, 0xb8, 0xd7, 0xec, 0xdb, 0x5b, 0x50, 0xd8, 0x13, 0xd3, 0x3c, 0xd1, 0x3f, 0x7b, 0x6a, + 0x5c, 0xb6, 0x7c, 0x7c, 0xb4, 0xae, 0xae, 0x0d, 0x71, 0x5e, 0x32, 0x32, 0xf5, 0x69, 0xe4, 0x94, + 0x63, 0x0e, 0x40, 0xa3, 0xa6, 0x32, 0xa4, 0x99, 0x52, 0x25, 0xa0, 0x41, 0x51, 0xbc, 0x14, 0xc3, + 0x9c, 0x82, 0x36, 0x20, 0x37, 0x14, 0xcb, 0x2b, 0x1e, 0xc0, 0x5c, 0x5c, 0x58, 0x76, 0x11, 0x4d, + 0x43, 0x18, 0xe1, 0x19, 0xd9, 0xfe, 0x90, 0x84, 0x7c, 0x2f, 0x4b, 0x85, 0x08, 0x4f, 0x5d, 0xd0, + 0x19, 0x8f, 0x84, 0xa1, 0x6d, 0xb8, 0xe4, 0x1f, 0xcc, 0x82, 0x91, 0xfb, 0xdc, 0x19, 0x44, 0xfa, + 0x5b, 0xf1, 0xf4, 0xfe, 0xb6, 0x22, 0x38, 0xb6, 0xc3, 0x7e, 0xa7, 0x84, 0x5b, 0x4b, 0x67, 0x85, + 0x5b, 0x1b, 0x7c, 0xac, 0x5c, 0xa6, 0x63, 0x65, 0x69, 0xde, 0x0f, 0x4e, 0x8a, 0x31, 0x2a, 0x1b, + 0xa7, 0x8c, 0x29, 0x75, 0x7d, 0xab, 0xda, 0x6f, 0xf4, 0xb4, 0x18, 0x02, 0x48, 0xd7, 0xfa, 0xdd, + 0x5e, 0xbb, 0x29, 0xa7, 0xc0, 0x7f, 0x16, 0x03, 0x68, 0x9a, 0x7e, 0xc0, 0xac, 0x84, 0xca, 0x72, + 0x71, 0x91, 0x5b, 0x58, 0x4a, 0x94, 0xf9, 0x02, 0x85, 0x0c, 0x65, 0x61, 0xde, 0xe2, 0x1a, 0xb1, + 0xf8, 0x28, 0x79, 0xff, 0xde, 0xfd, 0x7b, 0x7c, 0x68, 0xbd, 0x0c, 0x89, 0xa9, 0x3d, 0xa2, 0xee, + 0x98, 0x63, 0x03, 0x74, 0xc7, 0xa8, 0x63, 0x42, 0x8b, 0x44, 0xca, 0xa9, 0xb9, 0x48, 0xb9, 0x04, + 0x99, 0x43, 0xcb, 0xf3, 0xc9, 0x32, 0x8b, 0xc6, 0xd7, 0x58, 0x24, 0xd1, 0xad, 0x70, 0x39, 0xc9, + 0xfc, 0x72, 0x6e, 0x39, 0x29, 0x17, 0x92, 0x95, 0x1f, 0xc6, 0x21, 0xc7, 0xb6, 0x0c, 0x48, 0xd5, + 0xae, 0x45, 0xd6, 0x5e, 0xe1, 0x38, 0x10, 0x96, 0x19, 0xe9, 0x26, 0x89, 0x57, 0xec, 0x26, 0x0f, + 0x00, 0xcc, 0x20, 0xf0, 0xec, 0xdd, 0x59, 0x20, 0x3b, 0x97, 0x58, 0x96, 0x55, 0x45, 0x06, 0x67, + 0x53, 0x90, 0xe8, 0x26, 0xb5, 0x74, 0x3a, 0x5a, 0x03, 0xb6, 0xbf, 0x11, 0x59, 0xba, 0x08, 0xcb, + 0x92, 0x0e, 0x95, 0xa2, 0x96, 0xfd, 0x80, 0x59, 0xb6, 0xec, 0x47, 0x42, 0xef, 0x47, 0x91, 0x88, + 0xf4, 0x5a, 0x44, 0xe2, 0x79, 0x61, 0xf7, 0xbb, 0x27, 0x39, 0x4e, 0x34, 0x0c, 0x8e, 0x71, 0x7f, + 0xf9, 0x51, 0x02, 0x52, 0x4f, 0xe8, 0xc0, 0xf9, 0x9e, 0x2c, 0x30, 0x7e, 0x6b, 0x49, 0x8e, 0x54, + 0x34, 0x6f, 0x71, 0x1e, 0x7f, 0x8f, 0x4c, 0xa6, 0xe6, 0xd8, 0xf4, 0xe6, 0x16, 0xcd, 0x0c, 0xde, + 0xa5, 0x59, 0x98, 0x43, 0x08, 0xd8, 0x33, 0x9d, 0x7d, 0xcb, 0xe7, 0x91, 0x6f, 0x14, 0x8c, 0x69, + 0x16, 0xe6, 0x10, 0x54, 0x81, 0x84, 0x6f, 0xb1, 0xd5, 0x5c, 0x68, 0x78, 0x2e, 0xd6, 0x0a, 0x30, + 0xc9, 0x44, 0x37, 0x20, 0x19, 0x58, 0x2f, 0xd8, 0x52, 0x2e, 0x3f, 0xaf, 0xaa, 0xf5, 0x22, 0xc0, + 0x34, 0xbb, 0x7c, 0x1d, 0xd2, 0x4c, 0x93, 0xe8, 0x84, 0x11, 0x8b, 0x4c, 0x18, 0xe5, 0x6f, 0x42, + 0x8a, 0xaa, 0x40, 0x40, 0xbb, 0xd6, 0xbe, 0xed, 0x50, 0x50, 0x52, 0x80, 0x28, 0x09, 0xad, 0x42, + 0xc2, 0xa2, 0xa1, 0x7e, 0x98, 0x43, 0x08, 0xe5, 0x8f, 0x21, 0xcd, 0xf4, 0x47, 0x77, 0x20, 0x45, + 0x6b, 0xc0, 0x27, 0x5c, 0xb4, 0x58, 0x47, 0x21, 0x91, 0xc2, 0xca, 0x97, 0x21, 0xd1, 0xb5, 0x68, + 0xa4, 0x62, 0x07, 0xd6, 0x84, 0x72, 0xe5, 0x30, 0xfd, 0x5d, 0xae, 0x40, 0x92, 0xd4, 0xe2, 0xcc, + 0x6d, 0xac, 0x0f, 0x78, 0x13, 0x03, 0xa4, 0xbb, 0xb5, 0x6a, 0xa3, 0x8a, 0xb5, 0x37, 0xc8, 0x6f, + 0x5c, 0x6d, 0x6d, 0xd3, 0xa5, 0x54, 0x06, 0x12, 0x5d, 0xbd, 0xc7, 0x96, 0xe9, 0x3d, 0xfd, 0xb3, + 0x9e, 0x96, 0xa8, 0xfc, 0x7e, 0x0c, 0x72, 0xd2, 0x83, 0xe5, 0x42, 0x3c, 0xb6, 0xb0, 0x10, 0x17, + 0x8e, 0x10, 0x7f, 0x3d, 0x47, 0x48, 0xbc, 0x8e, 0x23, 0x24, 0xcf, 0x77, 0x84, 0x57, 0x6b, 0x64, + 0xe1, 0x2f, 0xe9, 0x33, 0xfc, 0xa5, 0xf2, 0x0f, 0x72, 0x90, 0x15, 0x3d, 0xfe, 0x8c, 0x8a, 0xdf, + 0x38, 0xa7, 0xe2, 0xff, 0xcf, 0xab, 0xcc, 0xeb, 0x92, 0x3a, 0xcb, 0xf7, 0x4f, 0xd9, 0xa5, 0x78, + 0x1f, 0x92, 0x23, 0xdb, 0x7f, 0xc6, 0x87, 0xd0, 0xd2, 0xdc, 0x30, 0x47, 0xe6, 0x41, 0x1a, 0xc1, + 0x62, 0x8a, 0x42, 0x55, 0xc8, 0x7b, 0x96, 0x4f, 0xd6, 0x24, 0x74, 0xe3, 0x2b, 0x1b, 0x09, 0xc7, + 0x25, 0x13, 0x0e, 0x21, 0x2c, 0xde, 0x54, 0x78, 0xd0, 0x37, 0xc9, 0xe0, 0x7a, 0xe8, 0x0e, 0x69, + 0x98, 0xc5, 0xe2, 0xf9, 0xb7, 0x16, 0x05, 0x70, 0x00, 0x9b, 0x7d, 0x25, 0x1e, 0x6d, 0x40, 0xda, + 0x3f, 0x30, 0x3d, 0x6b, 0x44, 0x23, 0x80, 0xfc, 0x46, 0x79, 0x9e, 0xb3, 0x4b, 0x73, 0x29, 0x1b, + 0x47, 0xa2, 0x6d, 0x58, 0x36, 0xc7, 0x63, 0x77, 0x48, 0x8b, 0x1f, 0xd8, 0xce, 0x9e, 0xcb, 0xe3, + 0x83, 0xb5, 0x79, 0xe6, 0xaa, 0x84, 0x51, 0x01, 0x4b, 0x66, 0x24, 0x5d, 0xbe, 0x0e, 0x4b, 0x51, + 0x04, 0xe9, 0x84, 0xd4, 0xa6, 0x6c, 0x0b, 0x8f, 0xfe, 0x2e, 0x3f, 0x81, 0xe5, 0xb9, 0xfa, 0x47, + 0x77, 0x7b, 0x62, 0xf3, 0xbb, 0x3d, 0x61, 0x20, 0x10, 0x3f, 0x23, 0x10, 0x28, 0xff, 0x7a, 0x12, + 0xb2, 0xa2, 0x35, 0x48, 0x90, 0x38, 0x25, 0xd3, 0xa0, 0x1f, 0x58, 0xce, 0xd0, 0xe2, 0xb1, 0xfe, + 0xf5, 0xd3, 0x1a, 0xef, 0x4e, 0x27, 0xc4, 0x62, 0x95, 0x91, 0x94, 0x7d, 0xe8, 0x8e, 0x67, 0x13, + 0x6b, 0xae, 0xec, 0x27, 0x94, 0x88, 0x79, 0x26, 0xfa, 0x86, 0x0c, 0xbc, 0x12, 0x27, 0xb7, 0xb8, + 0x2c, 0xa9, 0x4b, 0xd3, 0x22, 0x2a, 0x2b, 0x6f, 0x43, 0x5e, 0x29, 0xfb, 0xcc, 0xa8, 0x22, 0x62, + 0xa4, 0xf8, 0x9c, 0x91, 0xca, 0xbf, 0x11, 0x87, 0x34, 0x93, 0x4d, 0x16, 0x28, 0xca, 0x7c, 0x73, + 0xfd, 0x1c, 0x55, 0xd4, 0x0e, 0xf8, 0xb1, 0xb2, 0xba, 0xcb, 0x9f, 0xcf, 0xd9, 0x31, 0x83, 0x03, + 0xbe, 0x06, 0xfc, 0x26, 0xa4, 0x26, 0xee, 0xcc, 0x09, 0x78, 0xfd, 0x6f, 0x9c, 0xc7, 0xda, 0x24, + 0x60, 0xcc, 0x78, 0xca, 0xd7, 0x20, 0x49, 0x44, 0x91, 0x01, 0xc4, 0x73, 0xdd, 0x20, 0x3a, 0x80, + 0x10, 0x4a, 0xf9, 0x6d, 0x48, 0x51, 0x8e, 0xd3, 0x21, 0x95, 0x2b, 0x7c, 0xdc, 0xce, 0x42, 0xb2, + 0x53, 0xed, 0xed, 0x68, 0x31, 0x94, 0x83, 0x54, 0xb3, 0xdd, 0x6f, 0xf5, 0xb4, 0x78, 0x79, 0x19, + 0x8a, 0x91, 0x2e, 0x53, 0x2e, 0x00, 0x84, 0x3d, 0xa1, 0xf2, 0xb7, 0xe2, 0x50, 0xea, 0x79, 0xe6, + 0xde, 0x9e, 0x3d, 0x24, 0xb1, 0xb0, 0xe7, 0x8e, 0xbb, 0x81, 0x19, 0xd8, 0x7e, 0x60, 0x0f, 0xfd, + 0x33, 0xdb, 0xa4, 0x04, 0x99, 0x5d, 0x73, 0xf8, 0x6c, 0xec, 0xee, 0x53, 0x9b, 0x25, 0xb1, 0x48, + 0x92, 0xd5, 0xce, 0xee, 0xcb, 0x80, 0xcf, 0xcc, 0x49, 0xcc, 0x12, 0x84, 0x3a, 0xf2, 0xdc, 0x29, + 0x1b, 0xb3, 0x92, 0x98, 0x25, 0xc8, 0xca, 0x8a, 0xc4, 0xce, 0x63, 0x7b, 0x62, 0x07, 0x6c, 0xfb, + 0x21, 0x89, 0x15, 0x0a, 0x29, 0x65, 0x6a, 0x0e, 0x9f, 0x59, 0x01, 0xdb, 0x65, 0x48, 0x62, 0x91, + 0x24, 0xfd, 0xeb, 0x8b, 0xb1, 0xe5, 0xd0, 0xc1, 0x29, 0x89, 0xe9, 0x6f, 0x74, 0x03, 0x32, 0x9e, + 0x19, 0x58, 0xbb, 0x53, 0x9f, 0x0e, 0x3f, 0xc9, 0xcd, 0xfc, 0xf1, 0xd1, 0x7a, 0x06, 0x9b, 0x81, + 0xb5, 0xd9, 0xe9, 0x62, 0x91, 0x27, 0x60, 0xd3, 0xa9, 0x4f, 0x07, 0x19, 0x05, 0xd6, 0x11, 0xb0, + 0xe9, 0x94, 0x7e, 0x07, 0xf1, 0xac, 0x2f, 0x66, 0xd6, 0x8c, 0xee, 0x9d, 0x92, 0x52, 0x64, 0xba, + 0xf2, 0x0b, 0x29, 0x28, 0x18, 0x53, 0xc5, 0x54, 0x6b, 0x00, 0x5b, 0xae, 0xf7, 0xdc, 0xf4, 0x46, + 0xb6, 0xb3, 0x4f, 0x3b, 0x5d, 0x02, 0x2b, 0x14, 0x92, 0x5f, 0xb7, 0xf6, 0xcc, 0xd9, 0x38, 0xe8, + 0xf5, 0x1a, 0xd4, 0x62, 0x09, 0xac, 0x50, 0x48, 0xbe, 0xe1, 0x60, 0x6b, 0x68, 0xd9, 0x87, 0xdc, + 0x72, 0x09, 0xac, 0x50, 0xd0, 0x35, 0xc8, 0x1b, 0xce, 0xce, 0xc8, 0xd3, 0x3d, 0xcf, 0xf5, 0x98, + 0x11, 0x13, 0x58, 0x25, 0xa1, 0x0a, 0x14, 0x0c, 0x87, 0xc4, 0xb7, 0x1c, 0x92, 0xa2, 0x90, 0x08, + 0x0d, 0x5d, 0x87, 0x22, 0xd1, 0xa9, 0x6e, 0x06, 0xe6, 0xbe, 0x67, 0x4e, 0x98, 0x51, 0x13, 0x38, + 0x4a, 0x44, 0xb7, 0x60, 0xd9, 0x70, 0xfa, 0xce, 0x33, 0xc7, 0x7d, 0xee, 0x74, 0x3c, 0x37, 0x70, + 0x59, 0x14, 0x9d, 0xc0, 0xf3, 0x64, 0xa6, 0x35, 0x59, 0x0f, 0x99, 0xde, 0x88, 0xd9, 0x9c, 0x6a, + 0x2d, 0x28, 0x3c, 0xdf, 0x1a, 0xdb, 0x24, 0x30, 0xa7, 0xc6, 0x66, 0xf9, 0x9c, 0x42, 0x6a, 0xd5, + 0x9e, 0x05, 0x98, 0x58, 0xd5, 0x0f, 0x98, 0x95, 0x13, 0x58, 0x25, 0x71, 0x84, 0x2c, 0x22, 0x2f, + 0x11, 0xb2, 0x0c, 0x86, 0x68, 0xb9, 0xd8, 0xa5, 0xd1, 0x75, 0x41, 0x22, 0x04, 0x89, 0x58, 0x06, + 0x5b, 0xa6, 0x3f, 0xe1, 0x7b, 0xf7, 0x74, 0x39, 0x96, 0xc0, 0x11, 0x1a, 0xd1, 0x94, 0xa6, 0xb1, + 0xf5, 0xc5, 0x88, 0x2d, 0xbb, 0x12, 0x58, 0xa1, 0x10, 0x67, 0xa0, 0xa9, 0xf6, 0x63, 0x9f, 0xae, + 0xb7, 0x12, 0x58, 0xa6, 0x25, 0xef, 0x96, 0x69, 0x8f, 0xfd, 0x92, 0xa6, 0xf0, 0x52, 0x0a, 0x71, + 0xe2, 0x2d, 0xcf, 0xdc, 0x27, 0xac, 0x17, 0x68, 0xa6, 0x48, 0x92, 0x81, 0x8d, 0xfc, 0x64, 0x8c, + 0x88, 0xe6, 0x85, 0x04, 0x52, 0x33, 0x92, 0xa8, 0x79, 0x96, 0x49, 0x6a, 0xb6, 0xc2, 0x6a, 0xa6, + 0x90, 0x2a, 0x7f, 0x37, 0x03, 0x4b, 0xc6, 0x70, 0xa2, 0x3a, 0xe2, 0x2a, 0xa4, 0x0d, 0xa7, 0xe9, + 0xef, 0xfb, 0xdc, 0x09, 0x79, 0x8a, 0x54, 0xc0, 0x70, 0xb8, 0x6b, 0x30, 0xf7, 0x93, 0x69, 0xe6, + 0x3a, 0x35, 0x7f, 0x36, 0xe1, 0xf9, 0x09, 0xe1, 0x3a, 0x21, 0x0d, 0xdd, 0x84, 0x25, 0xd2, 0x70, + 0x7e, 0xd0, 0x77, 0x3c, 0xcb, 0x1c, 0x1e, 0x08, 0x1f, 0x9c, 0xa3, 0x32, 0x47, 0x25, 0x56, 0xd5, + 0x5f, 0x0c, 0x47, 0xc2, 0x0b, 0x55, 0x12, 0x43, 0x74, 0x4c, 0x6f, 0xd2, 0xf1, 0xdc, 0x5d, 0xe1, + 0x82, 0x2a, 0x89, 0xe9, 0xd3, 0xf5, 0x86, 0x3f, 0x33, 0xb3, 0x1c, 0x52, 0x52, 0x46, 0xe8, 0x13, + 0xd2, 0x98, 0x14, 0x6c, 0x8d, 0x6c, 0xcf, 0x1a, 0x06, 0xc2, 0xf7, 0x54, 0x12, 0x31, 0xbb, 0xe1, + 0xe8, 0xc3, 0x03, 0x57, 0x78, 0x9e, 0x48, 0x32, 0xb7, 0x24, 0x3f, 0xb1, 0x35, 0x15, 0x5e, 0xa7, + 0x50, 0x58, 0xf9, 0x44, 0x61, 0x3f, 0x30, 0x27, 0x53, 0xe1, 0x75, 0x11, 0x1a, 0xeb, 0x24, 0x32, + 0x4d, 0x05, 0x15, 0x44, 0x27, 0x89, 0x90, 0x99, 0xa6, 0xa4, 0x13, 0x36, 0x4d, 0xff, 0x99, 0xcf, + 0xbd, 0x4f, 0x25, 0x31, 0xdb, 0x8a, 0x24, 0x15, 0xb5, 0x24, 0x6c, 0xab, 0x52, 0x49, 0x8d, 0xda, + 0xb3, 0x80, 0x36, 0x2e, 0xf3, 0x41, 0x91, 0x24, 0x8e, 0xd4, 0x9e, 0x05, 0xbc, 0xf9, 0x98, 0x07, + 0x86, 0x04, 0xa2, 0x2b, 0xe9, 0x31, 0x6a, 0xe3, 0x31, 0x47, 0x9c, 0x27, 0x93, 0x9a, 0xb7, 0x67, + 0x41, 0xd8, 0x7c, 0xcc, 0x27, 0x23, 0x34, 0x8e, 0x09, 0x1b, 0x70, 0x45, 0x62, 0xc2, 0x16, 0xbc, + 0x0e, 0xc5, 0xf6, 0x2c, 0x50, 0x9a, 0xf0, 0x22, 0x1b, 0x68, 0x22, 0x44, 0x2e, 0x29, 0x6c, 0xc4, + 0x4b, 0x52, 0x52, 0xd8, 0x8a, 0x65, 0xc8, 0x92, 0x8a, 0xd0, 0x66, 0x5c, 0x65, 0x7e, 0x2b, 0xd2, + 0xbc, 0xeb, 0xcb, 0x86, 0x7c, 0x53, 0x76, 0x7d, 0xd9, 0x92, 0x4c, 0x0f, 0xa5, 0x29, 0x4b, 0x52, + 0x0f, 0xa5, 0x2d, 0x6f, 0x83, 0xa6, 0x12, 0xa8, 0xb0, 0xcb, 0x14, 0xb8, 0x40, 0xe7, 0x3a, 0x87, + 0xcd, 0x59, 0x96, 0x3a, 0x87, 0xed, 0xc9, 0xec, 0x1d, 0x69, 0xd0, 0x2b, 0xd2, 0xde, 0x2a, 0xb9, + 0xf2, 0xc3, 0x04, 0x14, 0x7b, 0x43, 0xb5, 0xff, 0x92, 0xc1, 0x2a, 0x70, 0xab, 0xe3, 0x7d, 0xd7, + 0xb3, 0x83, 0x83, 0x09, 0xef, 0xc5, 0x11, 0x1a, 0xe9, 0xe3, 0x38, 0x70, 0x9b, 0xb6, 0xc3, 0x7b, + 0x32, 0x4f, 0x09, 0xba, 0xf9, 0x82, 0xf7, 0x60, 0x9e, 0x22, 0x7e, 0xd3, 0x34, 0x5f, 0xd4, 0x5c, + 0xc7, 0xe1, 0x9d, 0x56, 0x24, 0x89, 0x05, 0xab, 0xc3, 0xc0, 0x3e, 0xb4, 0xda, 0x53, 0xcb, 0x91, + 0xbd, 0x55, 0x21, 0x11, 0x7d, 0x3a, 0xa6, 0xef, 0x4b, 0x08, 0xeb, 0xae, 0x11, 0x1a, 0xc1, 0x54, + 0x83, 0xc0, 0x9a, 0x4c, 0x03, 0x36, 0x92, 0xf1, 0xfe, 0xaa, 0xd2, 0x48, 0x49, 0xba, 0x1f, 0x98, + 0xbb, 0x24, 0x00, 0x0e, 0xfb, 0xab, 0x42, 0x22, 0x3e, 0x5c, 0x9b, 0x79, 0x1e, 0x25, 0xf1, 0x1e, + 0x1b, 0x12, 0xd8, 0xb8, 0xd6, 0xb5, 0xf6, 0x45, 0x7f, 0xe5, 0x29, 0xde, 0x27, 0x68, 0x46, 0x5e, + 0xf6, 0x09, 0x9a, 0x73, 0x0d, 0xf2, 0xd8, 0x0a, 0x3c, 0xd3, 0xf1, 0x69, 0x2e, 0x9f, 0x18, 0x14, + 0x12, 0x93, 0xa9, 0x7b, 0x9e, 0xe8, 0x94, 0x3c, 0xc5, 0x65, 0x62, 0x32, 0x25, 0x2d, 0x49, 0x99, + 0x24, 0xb9, 0x30, 0x52, 0x2e, 0x2f, 0x8e, 0x94, 0x95, 0xdf, 0x88, 0x43, 0xb1, 0x3f, 0x52, 0xdb, + 0x94, 0x8e, 0x00, 0xe1, 0xa4, 0x1b, 0x13, 0x23, 0x40, 0x38, 0xe5, 0x96, 0x20, 0xd3, 0x72, 0x3b, + 0xae, 0x17, 0x88, 0xc1, 0x59, 0x24, 0x23, 0xe3, 0x76, 0x62, 0x71, 0xdc, 0x26, 0x1d, 0x58, 0x0a, + 0x4e, 0x4a, 0x5f, 0x0c, 0x25, 0x13, 0x7f, 0x1a, 0x1e, 0xee, 0xce, 0xf6, 0xa2, 0x61, 0x81, 0x4a, + 0x23, 0x98, 0xae, 0x33, 0x0a, 0x31, 0xbc, 0x8d, 0x55, 0xda, 0x42, 0xcd, 0x33, 0x27, 0xcc, 0x11, + 0x04, 0xb3, 0xef, 0xb8, 0x9e, 0x35, 0x6a, 0xce, 0xc6, 0x81, 0xcd, 0x1b, 0x39, 0x42, 0xab, 0xfc, + 0x99, 0x38, 0x2c, 0x75, 0x5b, 0xcd, 0x8e, 0x62, 0x9e, 0x6f, 0x42, 0xd6, 0x9e, 0x0e, 0xfc, 0xc0, + 0x0c, 0xfc, 0xb9, 0x4f, 0x13, 0x6a, 0x88, 0xc5, 0xa2, 0x34, 0x83, 0x32, 0xfa, 0x38, 0x63, 0xd3, + 0x2c, 0x1f, 0xd5, 0x00, 0xec, 0xe1, 0x44, 0xb0, 0xc7, 0x23, 0x9f, 0x41, 0xa3, 0x53, 0xe3, 0x66, + 0xf1, 0xf8, 0x68, 0x3d, 0x67, 0xd4, 0x9a, 0x5c, 0x44, 0xce, 0xe6, 0xd9, 0x3e, 0xfa, 0x14, 0x72, + 0xc1, 0x50, 0xc8, 0x48, 0x44, 0x76, 0x7b, 0x23, 0xbd, 0x93, 0x7d, 0x6b, 0xef, 0xd5, 0xb8, 0x84, + 0x6c, 0x30, 0x0c, 0x05, 0xcc, 0x46, 0x42, 0x40, 0x32, 0x22, 0x20, 0xe2, 0x0a, 0x4c, 0x40, 0xbf, + 0x2e, 0x04, 0xcc, 0x58, 0xa6, 0x5f, 0xf9, 0xf3, 0x08, 0x90, 0x58, 0x32, 0x44, 0x46, 0x83, 0x5c, + 0x20, 0x86, 0x9f, 0xc8, 0x46, 0x53, 0x48, 0x46, 0x35, 0x58, 0x19, 0x4e, 0x67, 0xfe, 0x60, 0xe6, + 0xf3, 0x93, 0x0b, 0x03, 0xdf, 0x1a, 0xf2, 0xef, 0x8a, 0x9b, 0x17, 0x8f, 0x8f, 0xd6, 0xb5, 0x5a, + 0xa7, 0xef, 0xf7, 0x7d, 0x76, 0x78, 0xa1, 0x6b, 0x0d, 0x7d, 0xac, 0x11, 0x06, 0x95, 0x82, 0x0c, + 0xb8, 0x44, 0x85, 0xf8, 0x2f, 0xfd, 0xc0, 0x9a, 0x28, 0x62, 0xe8, 0x27, 0xc7, 0xcd, 0xd5, 0xe3, + 0xa3, 0x75, 0x44, 0xc4, 0x74, 0x69, 0xbe, 0x14, 0x84, 0x08, 0x53, 0x94, 0x86, 0xde, 0x07, 0xa0, + 0xa2, 0x68, 0x08, 0xcf, 0x3e, 0x42, 0x32, 0xd3, 0x13, 0xfe, 0x06, 0x21, 0xe2, 0x1c, 0x01, 0xd0, + 0x9f, 0xe8, 0x43, 0x28, 0x4e, 0xac, 0xc9, 0xc0, 0xf3, 0xfd, 0x01, 0x5b, 0x35, 0xd0, 0x45, 0x00, + 0xfb, 0x02, 0xd0, 0xb4, 0x26, 0xb8, 0xdb, 0xdd, 0x24, 0x64, 0x9c, 0x9f, 0x58, 0x13, 0xec, 0xfb, + 0x34, 0x81, 0x6e, 0xc2, 0x32, 0x61, 0xa2, 0x25, 0x70, 0x36, 0xb6, 0x3c, 0x20, 0xb2, 0xa8, 0x5c, + 0x86, 0x7b, 0x08, 0xcb, 0x54, 0x15, 0xc7, 0xe3, 0x5b, 0xf1, 0xcc, 0x6f, 0x8b, 0x9b, 0x17, 0x8e, + 0x8f, 0xd6, 0x8b, 0x44, 0x9f, 0x96, 0xc7, 0xf6, 0xdc, 0x7d, 0x5c, 0x24, 0x48, 0x99, 0x44, 0x9f, + 0xc2, 0x05, 0xc1, 0x1a, 0x1c, 0x78, 0x6e, 0x10, 0x8c, 0x2d, 0xf6, 0xc5, 0xa2, 0xb8, 0xb9, 0x72, + 0x7c, 0xb4, 0xbe, 0xcc, 0x98, 0x7b, 0x22, 0x0b, 0x2f, 0x33, 0x76, 0x49, 0x40, 0x18, 0x4a, 0x54, + 0x80, 0xe4, 0x56, 0x8c, 0x9a, 0xa3, 0x46, 0xb9, 0x7c, 0x7c, 0xb4, 0x7e, 0x89, 0xc8, 0x91, 0x4c, + 0xd2, 0xae, 0xb4, 0x31, 0x16, 0xc8, 0xe8, 0x3a, 0x2c, 0x91, 0x7a, 0xef, 0xd9, 0x63, 0x8b, 0x57, + 0x9b, 0x2d, 0x4c, 0x0a, 0x13, 0x6b, 0xb2, 0x65, 0x8f, 0x2d, 0x56, 0x6b, 0x8e, 0x32, 0x1d, 0xd7, + 0xe1, 0xa8, 0xbc, 0x44, 0x55, 0x1d, 0xd7, 0x61, 0xa8, 0x0f, 0xe0, 0x12, 0x41, 0x4d, 0xcc, 0xe9, + 0xd4, 0x1a, 0xa9, 0x22, 0x0b, 0x14, 0x8c, 0x26, 0xd6, 0xa4, 0x49, 0xf3, 0x42, 0xc1, 0xef, 0x42, + 0x72, 0x6a, 0x79, 0x7b, 0xfc, 0x7b, 0x86, 0xe8, 0x65, 0x1d, 0xcb, 0xdb, 0x0b, 0x5d, 0x16, 0x53, + 0x08, 0xd1, 0xc1, 0xb1, 0x82, 0x81, 0xf7, 0x62, 0x20, 0xd6, 0x6f, 0x4b, 0x4c, 0x07, 0xc7, 0x0a, + 0xf0, 0x8b, 0x0e, 0x5f, 0xc4, 0x5d, 0x83, 0x02, 0x47, 0xb1, 0xa2, 0x97, 0xd9, 0x02, 0x90, 0x62, + 0x58, 0x91, 0x15, 0x28, 0x72, 0x84, 0x15, 0x06, 0x37, 0x49, 0x9c, 0xa7, 0x10, 0xb9, 0xaa, 0x11, + 0x65, 0x91, 0x45, 0xe5, 0xd4, 0x1a, 0xd1, 0xe8, 0x46, 0x94, 0x55, 0x67, 0x34, 0x81, 0x0a, 0x42, + 0x8d, 0x90, 0x44, 0xf5, 0xe6, 0x35, 0x0a, 0x84, 0x46, 0x2b, 0x52, 0xa3, 0x5e, 0x54, 0xa3, 0x40, + 0x6a, 0x74, 0x51, 0x6a, 0xd4, 0x9b, 0xd3, 0x28, 0x08, 0x35, 0xba, 0xa4, 0x94, 0x25, 0x34, 0xea, + 0xc2, 0x65, 0x8a, 0x1a, 0x4e, 0x07, 0x5e, 0x10, 0x0c, 0x26, 0xf6, 0xd0, 0x73, 0x89, 0x7b, 0x0c, + 0xa6, 0xf7, 0xef, 0xd1, 0x58, 0x87, 0xbb, 0x48, 0xcb, 0x0a, 0x7a, 0xb5, 0x0e, 0x0e, 0x82, 0xa6, + 0x40, 0x74, 0xee, 0xdf, 0xc3, 0x97, 0x88, 0xac, 0xe1, 0x74, 0x8e, 0x7c, 0x86, 0xd0, 0x87, 0xf7, + 0x68, 0x84, 0x74, 0xba, 0xd0, 0x87, 0x27, 0x0b, 0x7d, 0x78, 0xa6, 0xd0, 0xfb, 0x34, 0xa4, 0x3a, + 0x43, 0xe8, 0xfd, 0x93, 0x85, 0xde, 0x3f, 0x4b, 0xe8, 0x43, 0x1a, 0x7e, 0x9d, 0x21, 0xf4, 0xe1, + 0xc9, 0x42, 0x1f, 0xa2, 0x5b, 0xa0, 0x8d, 0x6c, 0xff, 0x59, 0x64, 0x68, 0x28, 0x53, 0xdb, 0x2f, + 0x11, 0xba, 0x32, 0x36, 0xdc, 0x84, 0x65, 0x8a, 0x9c, 0xf9, 0xd6, 0x88, 0x03, 0xaf, 0xb0, 0x31, + 0x84, 0x90, 0xfb, 0xbe, 0x35, 0x62, 0xb8, 0xcf, 0xa0, 0x2c, 0xd4, 0x34, 0x69, 0x5c, 0x34, 0x18, + 0xba, 0x8e, 0x63, 0x0d, 0x03, 0xdb, 0x75, 0xfc, 0xd2, 0x55, 0xaa, 0xe7, 0x95, 0xe3, 0xa3, 0xf5, + 0x37, 0x99, 0x9e, 0x2c, 0x76, 0xaa, 0x85, 0x10, 0xfc, 0x26, 0xd3, 0x74, 0x21, 0x03, 0x0d, 0xe0, + 0x2d, 0x21, 0x99, 0x0e, 0x0d, 0xcf, 0x4d, 0x3b, 0x88, 0x08, 0x7f, 0x8b, 0x0a, 0x7f, 0xeb, 0xf8, + 0x68, 0xfd, 0x32, 0x13, 0x4e, 0x06, 0x82, 0xa7, 0xa6, 0x1d, 0xa8, 0xe2, 0x2f, 0x33, 0xf1, 0x27, + 0x64, 0xb1, 0x7d, 0x33, 0x77, 0x68, 0xf9, 0xbe, 0xe5, 0x97, 0xd6, 0xc8, 0xd8, 0x85, 0x43, 0x02, + 0x89, 0x39, 0x82, 0x03, 0xcf, 0x32, 0x47, 0x7e, 0x69, 0x9d, 0xe6, 0x89, 0x24, 0xfa, 0x06, 0x94, + 0xe8, 0xf0, 0xea, 0x3e, 0x1f, 0x4c, 0x3d, 0xcb, 0xf7, 0x67, 0x1e, 0xa9, 0xf4, 0xcc, 0x09, 0x2c, + 0xaf, 0x74, 0x8d, 0xda, 0x88, 0x0c, 0x1d, 0x0d, 0xf7, 0x79, 0x87, 0xe7, 0xd6, 0x58, 0x26, 0xfa, + 0x36, 0x5c, 0xa1, 0x63, 0x8a, 0x35, 0xb2, 0x67, 0x93, 0x45, 0xde, 0xb7, 0x29, 0x2f, 0x91, 0xdd, + 0xa4, 0x88, 0x79, 0xf6, 0x2a, 0xbc, 0x45, 0xd8, 0x87, 0x9e, 0x1d, 0xd8, 0x43, 0x73, 0xbc, 0x28, + 0xa0, 0x42, 0x05, 0x94, 0x27, 0xd6, 0xa4, 0xc6, 0x31, 0xf3, 0x22, 0x7e, 0x0e, 0xd6, 0xa8, 0x4d, + 0xd9, 0x96, 0x16, 0xb1, 0x66, 0xe0, 0xb9, 0x63, 0x3a, 0x31, 0xb3, 0xf1, 0xa9, 0xf4, 0xb5, 0xc8, + 0x31, 0xb8, 0xd3, 0xf6, 0xbe, 0xf8, 0x34, 0x7b, 0x85, 0x18, 0xf7, 0xb4, 0xed, 0x31, 0x3e, 0x0b, + 0x05, 0x6e, 0x60, 0x8e, 0xb9, 0x07, 0x5d, 0x97, 0xb3, 0x50, 0x8f, 0x50, 0x23, 0x23, 0x2d, 0xc3, + 0x4d, 0xac, 0x89, 0xff, 0x9c, 0xa3, 0x6f, 0xc8, 0x91, 0x96, 0xa2, 0x9b, 0x24, 0x8b, 0xb1, 0xdc, 0x85, 0x8b, 0x84, 0xc5, 0x77, 0xf7, 0x82, 0x88, 0x2b, 0xdf, 0xa4, 0x1c, 0x17, 0x26, 0xd6, 0xa4, - 0xeb, 0xee, 0x05, 0x51, 0x6f, 0xa6, 0xa6, 0x33, 0x87, 0x07, 0x22, 0x8e, 0xbf, 0x23, 0x75, 0xa9, - 0x11, 0x68, 0x64, 0x6e, 0xf0, 0x9f, 0x9b, 0x53, 0x4e, 0x76, 0x4b, 0xce, 0x0d, 0xdd, 0xe7, 0xe6, - 0x94, 0x51, 0x6d, 0x30, 0x8d, 0x67, 0x8e, 0x75, 0x68, 0x0f, 0xe9, 0x49, 0x07, 0x4e, 0xfc, 0x2e, - 0x25, 0x5e, 0x99, 0x58, 0x93, 0x7e, 0x88, 0x63, 0x3c, 0x3f, 0x0b, 0x2b, 0xc4, 0xf2, 0xbe, 0xc3, - 0x93, 0x31, 0x6e, 0xee, 0xdb, 0x91, 0xb9, 0x22, 0x9a, 0xf9, 0x6d, 0x5e, 0x3a, 0x3e, 0x5a, 0xbf, - 0xd0, 0xb2, 0x82, 0x28, 0x18, 0x5f, 0x70, 0xac, 0xa0, 0xeb, 0xa8, 0xb9, 0x5b, 0xe5, 0xf7, 0x92, - 0x50, 0x14, 0xf9, 0x51, 0xdf, 0x37, 0xf7, 0x2d, 0x54, 0x85, 0x9c, 0x38, 0xd8, 0x20, 0xce, 0xa5, - 0xcc, 0x7f, 0x2c, 0xa0, 0x84, 0xf2, 0x83, 0xb8, 0xc8, 0x9c, 0x24, 0x17, 0x7a, 0x0f, 0x52, 0xb4, - 0x53, 0x4a, 0xf1, 0xb3, 0x3e, 0xe4, 0x32, 0x9a, 0xf2, 0x8f, 0x13, 0x90, 0x15, 0xa2, 0xd0, 0x27, - 0x50, 0x0c, 0xcf, 0x63, 0x38, 0x7b, 0x2e, 0x3f, 0x91, 0xb1, 0x72, 0xc2, 0x37, 0x78, 0x71, 0xac, - 0xd2, 0x52, 0xcf, 0x73, 0x7c, 0x08, 0x39, 0xfe, 0x05, 0xc1, 0x1a, 0x9d, 0x5d, 0x7b, 0x48, 0x87, - 0x1e, 0x02, 0x28, 0x86, 0x65, 0x69, 0xea, 0xe5, 0x39, 0x2e, 0xc5, 0x8a, 0x0a, 0x31, 0xfa, 0x69, - 0x28, 0xc8, 0xb3, 0x0d, 0x03, 0xfa, 0xa9, 0x3d, 0x7e, 0xf2, 0x11, 0xf4, 0xcd, 0x15, 0xbe, 0xcf, - 0xab, 0x9e, 0x6b, 0xc7, 0x79, 0xc9, 0x6c, 0x8c, 0xd0, 0x27, 0x90, 0x0a, 0xe8, 0xd2, 0x97, 0x7d, - 0xc8, 0xae, 0x9c, 0x69, 0xf4, 0x3b, 0x3d, 0xd3, 0x7f, 0x26, 0x0d, 0x49, 0xd8, 0xca, 0x7f, 0x25, - 0x06, 0x49, 0x02, 0x3d, 0xe3, 0xab, 0xd9, 0xbb, 0x74, 0xe3, 0x99, 0x9d, 0x07, 0x17, 0xdf, 0x1d, - 0xd8, 0xb9, 0xfd, 0x85, 0x7d, 0xe8, 0xaf, 0xf4, 0x41, 0x3e, 0xfc, 0xae, 0x92, 0x3c, 0xe3, 0xbb, - 0x4a, 0xe5, 0x4f, 0xaf, 0xc0, 0x52, 0x34, 0xbb, 0x79, 0xa5, 0x84, 0xfc, 0x5a, 0xe4, 0x14, 0x78, - 0x48, 0x22, 0xa1, 0x64, 0xe1, 0x39, 0x7c, 0x39, 0x1c, 0xcb, 0x3d, 0x72, 0x5e, 0x42, 0x0f, 0xe0, - 0x4d, 0x3f, 0x30, 0xc7, 0x24, 0x59, 0x64, 0x90, 0xc1, 0x9e, 0xe7, 0x3a, 0x81, 0xe5, 0x8c, 0xf8, - 0xb6, 0xf9, 0x25, 0x8e, 0xae, 0x51, 0xec, 0x16, 0x47, 0xa2, 0x8f, 0x60, 0x75, 0x8e, 0x6f, 0x97, - 0xe4, 0x33, 0xce, 0x88, 0x6f, 0xa9, 0x5f, 0x8c, 0xb0, 0x6d, 0x32, 0x1c, 0x59, 0xae, 0xd9, 0x8e, - 0x1f, 0x78, 0x33, 0x3e, 0xdd, 0xb0, 0x14, 0x3a, 0x02, 0x43, 0xef, 0x82, 0xc6, 0x62, 0x8a, 0x67, - 0xed, 0x59, 0x9e, 0xe5, 0x10, 0x2b, 0xb3, 0x2d, 0xf7, 0x65, 0x0a, 0xc7, 0x12, 0x8c, 0xde, 0x86, - 0x02, 0x23, 0x9d, 0xd8, 0x74, 0xc2, 0xc9, 0xb2, 0xbc, 0x88, 0xc2, 0x9a, 0x14, 0x44, 0x16, 0xb3, - 0xbb, 0x9e, 0xe9, 0x0c, 0x0f, 0x2c, 0xbe, 0xf5, 0x8e, 0x65, 0x19, 0x7d, 0x0d, 0x8a, 0xec, 0xb7, - 0xe0, 0xe7, 0xa9, 0x2d, 0x03, 0x72, 0x01, 0x6f, 0x01, 0xec, 0xce, 0x7c, 0xde, 0x48, 0x9e, 0xd6, - 0xe6, 0x76, 0x67, 0x3e, 0x6b, 0x18, 0x41, 0x7b, 0xd6, 0x9e, 0x40, 0xb3, 0x44, 0x36, 0xe7, 0x59, - 0x7b, 0x1c, 0xfd, 0x2e, 0x90, 0x85, 0xc7, 0x60, 0x38, 0x76, 0x87, 0xcf, 0x68, 0x12, 0x1b, 0x63, - 0xeb, 0xb1, 0x5a, 0xa7, 0x5f, 0x23, 0x30, 0x9c, 0x1d, 0x4e, 0x67, 0xf4, 0x17, 0x91, 0x44, 0xbc, - 0x95, 0xd3, 0x92, 0xdc, 0x35, 0x86, 0x73, 0x04, 0xc2, 0xd0, 0xeb, 0x90, 0x9f, 0x9a, 0xfb, 0xd6, - 0x80, 0xee, 0xdf, 0xcb, 0xbc, 0x95, 0x80, 0xb6, 0x28, 0x84, 0x18, 0x63, 0x62, 0x3b, 0xae, 0x27, - 0x28, 0x78, 0xda, 0x4a, 0x61, 0x0a, 0x89, 0xf9, 0x73, 0x21, 0xc9, 0x05, 0x4e, 0x42, 0x60, 0x9c, - 0x84, 0x58, 0x9f, 0x74, 0xf1, 0x8b, 0x60, 0xe0, 0x3f, 0xb7, 0x03, 0x6a, 0x37, 0xc4, 0xad, 0xcf, - 0xe0, 0x5d, 0x0e, 0x46, 0x1f, 0xc3, 0x12, 0x69, 0xdb, 0xc4, 0xde, 0x67, 0x3e, 0xc6, 0x53, 0x57, - 0xb9, 0xd2, 0x69, 0x4a, 0x04, 0x5d, 0xe9, 0x84, 0x45, 0x52, 0x89, 0x39, 0xb6, 0xf7, 0xe9, 0xc1, - 0x3b, 0xa1, 0x0b, 0xcb, 0x69, 0x97, 0x25, 0x3c, 0xd4, 0xc7, 0x9a, 0xcc, 0xc6, 0xec, 0x73, 0x29, - 0x27, 0x65, 0x99, 0xed, 0xb2, 0x84, 0x73, 0xd2, 0x9b, 0xb0, 0x3c, 0xfe, 0x60, 0x30, 0x62, 0x1e, - 0x31, 0x76, 0x49, 0x96, 0xb1, 0xca, 0x26, 0xa4, 0xf1, 0x07, 0x75, 0x0a, 0x6d, 0x10, 0x20, 0x99, - 0x1c, 0xa3, 0x74, 0xa2, 0xfb, 0xdf, 0x64, 0x93, 0xa3, 0x4a, 0xcd, 0x9d, 0xe0, 0x16, 0x68, 0x21, - 0x8b, 0x1f, 0xb8, 0x9e, 0xc5, 0xf6, 0xf5, 0x92, 0x78, 0x49, 0x50, 0x77, 0x29, 0x14, 0x7d, 0x08, - 0xab, 0x73, 0x94, 0x42, 0xfa, 0x65, 0x36, 0x91, 0x45, 0xe8, 0xb9, 0xf8, 0x7b, 0x70, 0x31, 0x64, - 0x9a, 0x12, 0xbf, 0x67, 0x86, 0x2f, 0x47, 0x15, 0xea, 0x48, 0x0c, 0x7a, 0x08, 0x97, 0x17, 0x39, - 0x44, 0x4d, 0x2c, 0xa9, 0x5c, 0x9d, 0x67, 0xe3, 0x95, 0x31, 0x33, 0xd9, 0xaa, 0x99, 0xae, 0x0a, - 0x33, 0x19, 0x0b, 0x66, 0xb2, 0x17, 0xcd, 0xf4, 0x96, 0xd0, 0xca, 0x98, 0x37, 0x13, 0x6b, 0x87, - 0xbd, 0xd0, 0x8e, 0xb5, 0x28, 0xc7, 0x42, 0x3b, 0xec, 0x93, 0xdb, 0xb1, 0x2e, 0xda, 0x61, 0x9c, - 0xd4, 0x8e, 0x77, 0x21, 0x37, 0x1e, 0x0f, 0x79, 0x0b, 0x68, 0x8e, 0xc8, 0x86, 0x56, 0xa3, 0x51, - 0xa3, 0x0d, 0xc0, 0xd9, 0xf1, 0x78, 0xc8, 0x9a, 0xf2, 0x10, 0x96, 0x05, 0xa9, 0x90, 0xfd, 0x76, - 0xe8, 0xaa, 0x9c, 0x81, 0x89, 0xc5, 0x45, 0xce, 0xc5, 0x6b, 0x79, 0x1f, 0x80, 0xb0, 0xf2, 0x3e, - 0xa7, 0xd9, 0x20, 0xdb, 0x5a, 0x68, 0x34, 0x6a, 0xac, 0xcb, 0x31, 0x51, 0x83, 0xf7, 0xfe, 0xb7, - 0x40, 0x93, 0xd4, 0xa2, 0xa6, 0xaf, 0x51, 0x1e, 0x74, 0x7c, 0xb4, 0xbe, 0x24, 0x78, 0x78, 0x55, - 0x4b, 0x82, 0x91, 0xd7, 0xf5, 0x31, 0x10, 0x88, 0x6a, 0xb8, 0xeb, 0x11, 0x2d, 0x43, 0xbb, 0x51, - 0x2d, 0x15, 0x33, 0xea, 0xb0, 0xa2, 0x72, 0x8a, 0xaa, 0x69, 0xb6, 0xc7, 0x52, 0x1e, 0x85, 0x9d, - 0xd7, 0x7e, 0x41, 0x11, 0x11, 0x36, 0x76, 0x14, 0x8c, 0x77, 0xb9, 0x4d, 0x6f, 0x86, 0x8d, 0xad, - 0xf7, 0x1a, 0x9b, 0xcc, 0xa8, 0x39, 0x42, 0xc0, 0xac, 0xfa, 0x2d, 0xd0, 0x24, 0xb5, 0xa8, 0xf1, - 0x9d, 0xb0, 0xb1, 0x82, 0x47, 0x34, 0x56, 0x30, 0xf2, 0xba, 0xee, 0x42, 0x9e, 0x72, 0x73, 0xcb, - 0xd2, 0x9c, 0x70, 0x73, 0xe9, 0xf8, 0x68, 0x1d, 0x08, 0x23, 0x37, 0x2d, 0x55, 0x87, 0xdb, 0xf6, - 0x53, 0xb8, 0x10, 0x32, 0x88, 0xfa, 0x68, 0x76, 0xc8, 0xb6, 0x47, 0x24, 0x1b, 0xaf, 0x70, 0x59, - 0xf2, 0xf2, 0x1a, 0xbf, 0x09, 0x14, 0xa4, 0xda, 0xf7, 0x76, 0x54, 0x5d, 0xc5, 0xc0, 0x54, 0x5d, - 0xc5, 0xc2, 0x3b, 0x70, 0x31, 0xc2, 0x2c, 0x14, 0x78, 0x8f, 0x4a, 0xa0, 0x9b, 0x55, 0xaa, 0x04, - 0xae, 0x03, 0x52, 0xa5, 0x84, 0x46, 0xb6, 0x43, 0x23, 0xbf, 0x1f, 0x1a, 0xd9, 0x08, 0x8d, 0x6c, - 0xab, 0x46, 0xb6, 0xe7, 0x8d, 0xfc, 0xf5, 0x50, 0x6b, 0x63, 0xce, 0xc8, 0x76, 0xd4, 0xc8, 0x6f, - 0x03, 0x9f, 0xcc, 0x78, 0x6d, 0x77, 0x58, 0xc0, 0x67, 0x30, 0x56, 0xc1, 0xfb, 0x80, 0x14, 0x12, - 0x51, 0xc5, 0x5d, 0x4a, 0xa8, 0x85, 0x84, 0xe1, 0x6c, 0xe8, 0xb8, 0x23, 0x11, 0x37, 0xee, 0xb1, - 0xe9, 0x8e, 0x40, 0x98, 0xb0, 0x5b, 0xa0, 0x49, 0xb4, 0x10, 0xf5, 0x01, 0x8b, 0x93, 0x82, 0x88, - 0x0b, 0x5a, 0x87, 0x3c, 0xa5, 0xe4, 0xdd, 0xbf, 0xc1, 0x37, 0x3d, 0xdc, 0x91, 0x08, 0xa4, 0xb7, - 0xe1, 0x42, 0x48, 0x20, 0x64, 0x7d, 0xc8, 0x22, 0xbf, 0x24, 0xe3, 0xc2, 0xde, 0x01, 0x0a, 0x52, - 0x7b, 0xf6, 0xa3, 0xb0, 0x56, 0xa5, 0x17, 0xef, 0xc1, 0xc5, 0x08, 0xa1, 0x90, 0x7b, 0x9f, 0x05, - 0x28, 0x95, 0x9a, 0x89, 0xae, 0x1c, 0x42, 0x86, 0x7f, 0x19, 0x46, 0x0f, 0x20, 0x6b, 0xee, 0x93, - 0x19, 0xcb, 0x16, 0xa7, 0xda, 0xe7, 0x4f, 0x48, 0xd2, 0xfd, 0x62, 0x5e, 0xc0, 0x19, 0x4a, 0x6c, - 0xcc, 0xe5, 0x8b, 0xf1, 0x57, 0xcb, 0x17, 0x2b, 0x7f, 0xb6, 0x00, 0x29, 0x7a, 0x0b, 0x95, 0xdf, - 0x2b, 0x8b, 0x45, 0x6e, 0x2a, 0xf2, 0xfb, 0xa9, 0x0b, 0xa9, 0xe9, 0x4f, 0xcf, 0x9d, 0x77, 0x8e, - 0x47, 0x92, 0x6e, 0xf5, 0xbc, 0xb3, 0x4c, 0xba, 0x4f, 0x3d, 0xf3, 0xfc, 0x48, 0x69, 0x6e, 0x22, - 0x7a, 0x4f, 0x92, 0x37, 0x77, 0x99, 0xcb, 0x58, 0x6c, 0xb2, 0x7a, 0xaa, 0x35, 0x79, 0xfe, 0xa9, - 0xd6, 0x57, 0x3d, 0x02, 0x6e, 0x40, 0x41, 0x39, 0x93, 0x4e, 0xd2, 0xc7, 0xc4, 0xc9, 0x87, 0xd2, - 0x65, 0xeb, 0x42, 0x98, 0x8f, 0xf3, 0xe1, 0xa9, 0xf4, 0xf9, 0x03, 0xb2, 0x99, 0x57, 0x3e, 0x20, - 0x7b, 0x03, 0x12, 0x33, 0x6f, 0xcc, 0xcf, 0x9a, 0x81, 0xd8, 0x70, 0xc7, 0x0d, 0x76, 0x88, 0xb8, - 0x8f, 0x1b, 0x98, 0xe0, 0x4f, 0xb8, 0xb9, 0x98, 0x7b, 0x8d, 0x9b, 0x8b, 0x27, 0x9d, 0x12, 0x83, - 0xaf, 0x74, 0x4a, 0xec, 0x9f, 0x64, 0x20, 0xd7, 0x9e, 0x5a, 0x7c, 0x11, 0x70, 0x3f, 0x72, 0x1a, - 0xf7, 0x8a, 0xea, 0x4c, 0x77, 0x24, 0xd5, 0xe2, 0xe9, 0xc8, 0xfb, 0x64, 0xed, 0x32, 0x73, 0x86, - 0xe2, 0xac, 0xd2, 0x5b, 0xa7, 0x30, 0x36, 0x28, 0x11, 0xe6, 0xc4, 0xe8, 0x63, 0xc8, 0xb0, 0xa3, - 0x76, 0xe2, 0xa0, 0xd6, 0xda, 0x29, 0x7c, 0xec, 0x84, 0x9a, 0x85, 0x05, 0x39, 0xfa, 0x04, 0x72, - 0x33, 0x47, 0xf0, 0x26, 0x23, 0x97, 0x9f, 0xe6, 0x79, 0xfb, 0x82, 0x0e, 0x87, 0x2c, 0x44, 0xe1, - 0x21, 0x3d, 0xaf, 0xc0, 0x0f, 0x21, 0x9e, 0xa6, 0x30, 0x3b, 0xd4, 0x80, 0x39, 0x31, 0x51, 0x78, - 0x64, 0xf9, 0x81, 0xe7, 0xbe, 0xe4, 0x07, 0x31, 0x4f, 0x53, 0xb8, 0xce, 0xa8, 0xb0, 0x20, 0x47, - 0x3a, 0x14, 0x58, 0xa3, 0x07, 0xfb, 0x9e, 0x3b, 0x9b, 0xf2, 0xf3, 0x8b, 0x95, 0x33, 0xed, 0xb4, - 0x4d, 0x28, 0x71, 0x7e, 0x1c, 0x16, 0xca, 0x9f, 0x40, 0x9a, 0xe1, 0xd0, 0x47, 0x7c, 0x31, 0x40, - 0xba, 0x5e, 0xec, 0x35, 0x2c, 0xab, 0xcb, 0xd2, 0x70, 0x99, 0x4f, 0xd7, 0x08, 0xa4, 0xec, 0x97, - 0xff, 0x28, 0xe4, 0x15, 0xd9, 0xe8, 0x3e, 0x64, 0x85, 0xcb, 0x9f, 0xbf, 0x5b, 0x20, 0x49, 0xc9, - 0xa2, 0x9f, 0xd6, 0xcd, 0x9a, 0xc2, 0x42, 0xc8, 0x45, 0xa5, 0x6e, 0x2a, 0x7c, 0x5e, 0x01, 0xd1, - 0x80, 0x0c, 0xef, 0xcc, 0xe8, 0x00, 0x8f, 0xbd, 0xda, 0x00, 0x2f, 0xff, 0x14, 0xe4, 0x64, 0x87, - 0x7e, 0x35, 0x09, 0x0f, 0x21, 0xcd, 0x7a, 0x15, 0xdd, 0x85, 0x0c, 0x3b, 0x30, 0x78, 0x0e, 0xb3, - 0xa0, 0x2a, 0x3f, 0x82, 0x0c, 0xef, 0xd8, 0xd7, 0xe6, 0xad, 0x8c, 0x4f, 0x3a, 0xab, 0x0e, 0x90, - 0x6e, 0x54, 0xfb, 0xad, 0xda, 0x8e, 0x16, 0x43, 0x1a, 0x14, 0xd8, 0xef, 0xc1, 0x36, 0x6e, 0xf7, - 0x3b, 0x5a, 0x9a, 0x90, 0x62, 0xbd, 0xab, 0xe3, 0x27, 0xba, 0x16, 0x47, 0x45, 0xc8, 0xf5, 0x5b, - 0xa2, 0x98, 0xa0, 0x37, 0x22, 0xb0, 0x5e, 0xed, 0xe9, 0x5a, 0x92, 0x5d, 0x95, 0xe8, 0xf6, 0x70, - 0xfb, 0x73, 0x2d, 0xc5, 0x8f, 0xbb, 0xff, 0xcb, 0x38, 0x14, 0x0c, 0xe7, 0xd0, 0xf2, 0x7c, 0x8b, - 0xcd, 0x11, 0x77, 0xcf, 0x98, 0x23, 0x64, 0x94, 0xe6, 0x00, 0x3a, 0x51, 0xf0, 0x30, 0x16, 0x3f, - 0x27, 0x8c, 0xcd, 0xcf, 0x27, 0x89, 0x3f, 0xc4, 0x7c, 0xa2, 0x4e, 0x9f, 0xc9, 0xd7, 0x98, 0x3e, - 0x6b, 0x0b, 0xa1, 0x34, 0x45, 0xb5, 0x38, 0x39, 0x94, 0xf2, 0x8e, 0x99, 0x0f, 0xa8, 0x11, 0x5f, - 0x4a, 0xbf, 0xe2, 0x1c, 0x7c, 0x9c, 0x84, 0xac, 0x18, 0x6c, 0x67, 0x6c, 0x1d, 0x3d, 0x80, 0x0c, - 0x1b, 0xab, 0xa7, 0xec, 0x1f, 0x2d, 0x71, 0xd3, 0xf0, 0x77, 0x20, 0x70, 0x9a, 0x0e, 0xd8, 0x3f, - 0xdc, 0x04, 0x1b, 0x69, 0x4f, 0xf2, 0x15, 0xa7, 0xcf, 0xbb, 0x4a, 0x3c, 0x48, 0x45, 0xbe, 0x7a, - 0xab, 0xf1, 0x40, 0x89, 0x04, 0xe2, 0xee, 0x56, 0x5a, 0xb9, 0xbb, 0xf5, 0x7a, 0x57, 0x2c, 0xef, - 0x43, 0xe1, 0x80, 0xde, 0x8e, 0x1c, 0xd0, 0x8b, 0xea, 0x73, 0x17, 0xb0, 0x94, 0x8b, 0x93, 0x38, - 0x7f, 0xa0, 0x5c, 0x7c, 0x8d, 0xdc, 0x9f, 0xca, 0xbd, 0xda, 0xfd, 0xa9, 0x70, 0x87, 0x0d, 0xce, - 0xbe, 0xc2, 0xa4, 0x5c, 0xb3, 0xca, 0xbf, 0xda, 0x35, 0xab, 0x0d, 0xc8, 0x3f, 0xb3, 0xc7, 0xe3, - 0xc1, 0x94, 0x5e, 0x71, 0xe4, 0x97, 0xb3, 0x44, 0xda, 0x11, 0xde, 0x7d, 0xc4, 0xf0, 0x2c, 0xbc, - 0x07, 0x79, 0x13, 0x52, 0xac, 0xc9, 0xc5, 0xc8, 0x39, 0x78, 0x79, 0x17, 0x13, 0x33, 0x74, 0xe5, - 0x5b, 0x50, 0x8c, 0x04, 0x55, 0xba, 0x45, 0x4c, 0x37, 0x3b, 0xcf, 0x8c, 0xfa, 0x8c, 0xa6, 0xf2, - 0xcf, 0x53, 0xe7, 0xee, 0x6c, 0x7e, 0x55, 0xf7, 0xfc, 0xbf, 0x39, 0xf6, 0x37, 0xa3, 0x97, 0x09, - 0x93, 0x11, 0x03, 0x2a, 0x79, 0xdb, 0xd2, 0x19, 0x17, 0x09, 0xd5, 0xe1, 0x92, 0x7a, 0xcd, 0xe1, - 0x72, 0x93, 0x3e, 0xa9, 0x11, 0x58, 0xa5, 0x34, 0x3d, 0xa6, 0xad, 0x29, 0x16, 0xe8, 0x12, 0x38, - 0x66, 0xe8, 0xe8, 0xb0, 0xca, 0xbc, 0xe2, 0xb0, 0xfa, 0x50, 0xb9, 0x80, 0x9b, 0x8d, 0x64, 0xa4, - 0x42, 0xfe, 0x4c, 0x7c, 0xdc, 0x91, 0x84, 0xe8, 0xa7, 0x60, 0x85, 0xfd, 0x1e, 0xcc, 0xa6, 0x23, - 0x33, 0xb0, 0x06, 0x4c, 0xbf, 0x1c, 0xcd, 0xcc, 0x16, 0xf5, 0xbb, 0xc0, 0x88, 0xfb, 0x94, 0x96, - 0x82, 0xd0, 0x26, 0xa0, 0xa8, 0x84, 0xd9, 0xcc, 0x66, 0x37, 0x11, 0x0a, 0xec, 0x0c, 0x46, 0x57, - 0x61, 0xe9, 0xf7, 0x8d, 0x3a, 0xd6, 0x54, 0x21, 0xfd, 0x99, 0x3d, 0x7a, 0xd5, 0xb7, 0x16, 0xbe, - 0xca, 0xd5, 0xc4, 0xc8, 0x10, 0x2e, 0xbe, 0xda, 0x10, 0x16, 0xb7, 0x6d, 0x97, 0xc2, 0xdb, 0xb6, - 0x95, 0x9f, 0x8f, 0xc3, 0x32, 0x1d, 0x34, 0xac, 0x39, 0x74, 0x88, 0xbc, 0xfb, 0x0a, 0xaf, 0x1d, - 0xc8, 0x53, 0xf6, 0x73, 0xef, 0x1c, 0xac, 0xa9, 0xd4, 0xa1, 0xcc, 0xc5, 0xd7, 0x0e, 0x1e, 0x46, - 0x5e, 0x3b, 0xb8, 0x72, 0x0a, 0xdb, 0x49, 0x6f, 0x1e, 0x94, 0x6f, 0x86, 0xcf, 0x0d, 0x5c, 0x81, - 0x9c, 0xf5, 0x82, 0x7e, 0x64, 0x1d, 0x31, 0x7d, 0x53, 0x24, 0xbe, 0xda, 0x41, 0xcd, 0x1d, 0x59, - 0xe5, 0x77, 0xf8, 0xbb, 0x02, 0xeb, 0x90, 0xe7, 0x5d, 0x29, 0xc9, 0x8a, 0xec, 0x63, 0xca, 0xcc, - 0x27, 0x84, 0x95, 0xbf, 0x5c, 0x04, 0x08, 0x9d, 0x49, 0x1d, 0xd2, 0xb1, 0xd7, 0x19, 0xd2, 0x72, - 0x18, 0xc4, 0xcf, 0x1e, 0x06, 0x22, 0xee, 0x27, 0x94, 0xb8, 0x5f, 0x82, 0xcc, 0xc4, 0xf2, 0x7d, - 0x73, 0x9f, 0x65, 0xe4, 0x39, 0x2c, 0x8a, 0x91, 0x89, 0x3d, 0xf5, 0x1a, 0x13, 0xfb, 0x55, 0xf5, - 0xc3, 0x46, 0x9a, 0xef, 0x77, 0xcb, 0x4f, 0x1a, 0x73, 0x21, 0x23, 0xf3, 0x55, 0x42, 0x46, 0x09, - 0x32, 0x6c, 0x56, 0x79, 0x49, 0x27, 0x9e, 0x2c, 0x16, 0x45, 0x74, 0x4f, 0xde, 0x21, 0xc9, 0x45, - 0xae, 0xb2, 0x86, 0x46, 0x9e, 0xbb, 0x3c, 0x42, 0x38, 0x3c, 0xcb, 0xf4, 0x5d, 0x87, 0x0e, 0xb1, - 0x13, 0x39, 0x30, 0xc5, 0x63, 0x4e, 0x87, 0xae, 0x42, 0x92, 0x0e, 0xc9, 0x3c, 0x1d, 0x92, 0xd4, - 0x47, 0xe8, 0x30, 0xa4, 0x50, 0x65, 0xe8, 0x15, 0xce, 0x1a, 0x7a, 0x55, 0xb6, 0x1f, 0xcf, 0x3e, - 0xa3, 0x31, 0x8f, 0xe0, 0xa3, 0x69, 0x75, 0x7e, 0x34, 0x31, 0x2d, 0xd8, 0x3e, 0xbd, 0x02, 0x40, - 0x8f, 0x40, 0x9b, 0xb1, 0xd3, 0xb6, 0xf4, 0xb3, 0x2a, 0x31, 0x31, 0xbf, 0xe5, 0x3b, 0xff, 0xb4, - 0x10, 0x5e, 0x56, 0x08, 0x09, 0x10, 0x3d, 0x84, 0x02, 0x9d, 0xa6, 0x44, 0xd5, 0xcb, 0xd1, 0xaa, - 0xa3, 0x83, 0x01, 0xe7, 0x87, 0x21, 0xa0, 0xb2, 0x29, 0xef, 0xc8, 0x5c, 0x80, 0x22, 0x7b, 0xe2, - 0x64, 0xd0, 0xac, 0x76, 0x7b, 0x3a, 0xd6, 0xde, 0x20, 0xe9, 0x30, 0x07, 0x55, 0xb7, 0xf5, 0x56, - 0x4f, 0x8b, 0xa1, 0x15, 0x58, 0xe6, 0x10, 0xfd, 0x33, 0xbd, 0xd6, 0xef, 0xb5, 0xb1, 0x16, 0xaf, - 0xfc, 0xb5, 0x0c, 0xa4, 0x99, 0x55, 0x51, 0x05, 0xd6, 0xb0, 0x5e, 0xed, 0xb6, 0x5b, 0x03, 0xfe, - 0x24, 0x81, 0xa4, 0x1b, 0x6c, 0x55, 0x8d, 0x86, 0x5e, 0xd7, 0xde, 0x88, 0xd0, 0xb4, 0x7a, 0x55, - 0xa3, 0xa5, 0xe3, 0x01, 0xcf, 0xba, 0x39, 0xcd, 0x25, 0xb4, 0x0e, 0x57, 0x16, 0x69, 0x8c, 0xa6, - 0xd1, 0xab, 0xf6, 0x8c, 0x76, 0x4b, 0x5b, 0x41, 0xd7, 0xe1, 0xda, 0x19, 0x04, 0x83, 0xba, 0xd1, - 0x7d, 0xac, 0x5d, 0x44, 0x37, 0xa1, 0x72, 0x16, 0x55, 0x53, 0x6f, 0xb6, 0xf1, 0xe7, 0x5a, 0x16, - 0xad, 0x41, 0x79, 0x81, 0xae, 0x83, 0x75, 0xbd, 0xd9, 0xe9, 0xe9, 0x75, 0xed, 0xc2, 0x89, 0x2a, - 0xf7, 0x3b, 0xf5, 0x6a, 0x4f, 0x17, 0x2a, 0xaf, 0xa2, 0x5b, 0x70, 0x9d, 0xd3, 0xc8, 0x26, 0x63, - 0x7d, 0xdb, 0xe8, 0xf6, 0x30, 0xab, 0xac, 0x67, 0x34, 0xf5, 0x76, 0xbf, 0xa7, 0xbd, 0x89, 0x6e, - 0xc3, 0xcd, 0x45, 0xca, 0x13, 0x69, 0x4b, 0x8a, 0x66, 0x92, 0xb6, 0xa7, 0xe3, 0xa6, 0xd1, 0xaa, - 0x12, 0xcd, 0x62, 0xe8, 0x1a, 0x5c, 0x9d, 0xc7, 0x93, 0x25, 0x0a, 0x91, 0xa5, 0x63, 0xbd, 0xae, - 0xc5, 0xd1, 0x55, 0x28, 0x71, 0x8a, 0x2d, 0x5c, 0x6d, 0xea, 0x4f, 0xdb, 0xf8, 0xf1, 0x00, 0xeb, - 0xcd, 0xf6, 0x13, 0xbd, 0xae, 0x25, 0x48, 0x87, 0x72, 0xec, 0x76, 0x6d, 0xa0, 0x63, 0xdc, 0xc6, - 0x5a, 0x52, 0xa9, 0xd4, 0x68, 0x3d, 0xa9, 0x36, 0x8c, 0x7a, 0xc8, 0x6a, 0xd4, 0xb5, 0x14, 0xba, - 0x0c, 0x97, 0xe6, 0xf0, 0xed, 0xad, 0x2d, 0x1d, 0x77, 0xb5, 0xb4, 0xa2, 0x8f, 0xd1, 0x1e, 0x74, - 0x9f, 0x1a, 0xbd, 0xda, 0xce, 0x66, 0xbb, 0x8a, 0x89, 0x1f, 0x18, 0x44, 0xe3, 0x2b, 0x8a, 0x70, - 0xe6, 0x67, 0xa4, 0xaf, 0x6a, 0xed, 0x56, 0x4b, 0xaf, 0x11, 0x7c, 0x46, 0x11, 0x8e, 0xf5, 0x5a, - 0xbb, 0x55, 0x33, 0x1a, 0x06, 0xeb, 0xf4, 0x9c, 0xd2, 0x14, 0xf9, 0x1c, 0xcf, 0x40, 0x2c, 0xe4, - 0x10, 0x7a, 0x0b, 0x2e, 0x73, 0x2c, 0xf5, 0xd6, 0xa8, 0x5c, 0x40, 0x25, 0xb8, 0x18, 0x41, 0x0b, - 0x1b, 0xe4, 0x51, 0x19, 0x56, 0xe7, 0x30, 0xdd, 0x5e, 0x15, 0x13, 0xae, 0xc2, 0x02, 0x97, 0xa8, - 0xae, 0xa8, 0x78, 0x20, 0x7d, 0x4d, 0xa8, 0xb6, 0xa3, 0xd7, 0x1e, 0xd3, 0xb7, 0x84, 0xfa, 0x5d, - 0xee, 0x1b, 0x75, 0xed, 0xaa, 0xa2, 0x14, 0xa5, 0xa2, 0xcb, 0x4a, 0x61, 0x35, 0xed, 0xb2, 0xe2, - 0x58, 0x0a, 0xba, 0xdf, 0xaa, 0xf6, 0x7b, 0x3b, 0x6d, 0x6c, 0x7c, 0x47, 0xaf, 0x6b, 0x65, 0xf6, - 0x90, 0x51, 0x48, 0x23, 0x98, 0x97, 0x14, 0x73, 0x50, 0x44, 0x84, 0x6d, 0x79, 0x9e, 0x4d, 0x28, - 0xae, 0x55, 0x3e, 0x84, 0xcc, 0x96, 0x3d, 0x0e, 0x2c, 0x7a, 0x01, 0x60, 0xc9, 0xb3, 0xf6, 0x66, - 0xbe, 0x35, 0x08, 0x9f, 0x83, 0xa3, 0xef, 0x5b, 0xdd, 0xc7, 0x45, 0x86, 0xe0, 0x6f, 0xb6, 0x54, - 0x7e, 0x31, 0x06, 0x79, 0xe5, 0x55, 0x0e, 0xf4, 0x29, 0xe4, 0x0e, 0x4d, 0xcf, 0x26, 0x21, 0x48, - 0xe4, 0xc0, 0x57, 0x16, 0x1f, 0xef, 0xb8, 0xf3, 0x84, 0xd3, 0x88, 0x7c, 0x4c, 0xf2, 0x94, 0x1f, - 0x41, 0x56, 0x20, 0xcf, 0x48, 0x8b, 0xe5, 0x7b, 0x1f, 0x71, 0xe5, 0xbd, 0x8f, 0xca, 0xa7, 0x90, - 0x93, 0x6f, 0xb8, 0xa1, 0x55, 0x48, 0x3c, 0xb3, 0x5e, 0x46, 0x78, 0x09, 0x20, 0xbc, 0xd1, 0x1c, - 0x5f, 0xbc, 0xd1, 0xbc, 0x09, 0x20, 0x05, 0xf8, 0xe8, 0x23, 0xc8, 0x4d, 0x45, 0x89, 0xb7, 0xe5, - 0xb4, 0xa7, 0xe2, 0x42, 0xc2, 0xca, 0x0e, 0x40, 0xcd, 0xb3, 0x46, 0x96, 0x13, 0xd8, 0xe6, 0x18, - 0x55, 0xa2, 0xf7, 0x35, 0xc3, 0x1a, 0x95, 0x5b, 0x9b, 0xab, 0x90, 0xf6, 0xad, 0xa1, 0x67, 0x05, - 0xbc, 0x35, 0xbc, 0x54, 0xd9, 0x81, 0x7c, 0x28, 0xc9, 0x47, 0x0f, 0x21, 0x3f, 0x0c, 0x8b, 0x5c, - 0x21, 0x31, 0xa7, 0x86, 0x84, 0xe2, 0xcd, 0x3e, 0x85, 0xb6, 0xb2, 0x07, 0x39, 0x6c, 0x06, 0x16, - 0x3b, 0x41, 0x7b, 0x19, 0x12, 0x5f, 0x4c, 0x79, 0x8f, 0xb2, 0x1d, 0x83, 0x9f, 0xe9, 0x74, 0x31, - 0x81, 0x45, 0xb5, 0x8d, 0x9f, 0xac, 0x6d, 0x19, 0xb2, 0x43, 0x73, 0x6a, 0x0e, 0xc5, 0x83, 0x6e, - 0x49, 0x2c, 0xcb, 0x95, 0x7f, 0x14, 0x03, 0x90, 0x15, 0xf9, 0xe8, 0x0e, 0xa4, 0xf9, 0x4d, 0xbd, - 0xa8, 0xf5, 0x24, 0x89, 0x78, 0xdb, 0x8f, 0xdf, 0xde, 0x7b, 0x0c, 0x97, 0xcc, 0xfd, 0x7d, 0xcf, - 0xda, 0x27, 0x09, 0xf1, 0x88, 0x5d, 0x76, 0x1b, 0x10, 0x5d, 0xd9, 0xd9, 0xe4, 0x37, 0x8f, 0x8f, - 0xd6, 0x57, 0xaa, 0x82, 0x80, 0x5f, 0x86, 0x23, 0xba, 0xaf, 0x98, 0xf3, 0xc0, 0xa9, 0x8f, 0xbe, - 0x05, 0xe5, 0x45, 0x61, 0x73, 0x9a, 0x97, 0xe6, 0x19, 0x6b, 0xa2, 0x25, 0x3f, 0x89, 0x43, 0xca, - 0x98, 0x90, 0x04, 0xe9, 0xc6, 0x89, 0x6f, 0x12, 0x50, 0x9c, 0x9a, 0xaa, 0xde, 0x80, 0xa4, 0x39, - 0x9d, 0x0e, 0x79, 0x9e, 0x1a, 0x25, 0xab, 0x4e, 0xa7, 0x43, 0x4c, 0xd1, 0xe8, 0x3d, 0x48, 0x8f, - 0xdc, 0xe1, 0x33, 0x6b, 0xfe, 0xe2, 0x36, 0x23, 0xac, 0x53, 0x14, 0xe6, 0x24, 0xe8, 0x2a, 0xa4, - 0xe9, 0xb7, 0x53, 0xb6, 0xe6, 0x12, 0x8f, 0xc5, 0x70, 0x58, 0x79, 0x00, 0x49, 0x22, 0xf8, 0x8c, - 0x51, 0xb2, 0xca, 0x8f, 0xc5, 0x84, 0x8f, 0x00, 0x8a, 0xfd, 0x23, 0x91, 0xbd, 0x24, 0xce, 0xba, - 0x26, 0xdc, 0x87, 0x34, 0x53, 0xe8, 0x8c, 0x2a, 0x3e, 0x00, 0x08, 0x1d, 0x6d, 0xae, 0xf1, 0xa1, - 0x4f, 0x62, 0x85, 0xa8, 0x52, 0x09, 0xaf, 0x9f, 0x56, 0x3b, 0x9d, 0x1a, 0x7b, 0x42, 0xa4, 0xde, - 0xae, 0x3d, 0xd6, 0xb1, 0x7c, 0x42, 0xe4, 0x07, 0x69, 0x48, 0xb3, 0x8b, 0xc3, 0xe8, 0x3d, 0x58, - 0x0a, 0x73, 0x28, 0x7a, 0xe7, 0x56, 0xd5, 0xa2, 0x28, 0x71, 0xf4, 0x5e, 0xec, 0x15, 0xc8, 0x1d, - 0xb8, 0x7e, 0x30, 0x50, 0xde, 0x5f, 0xa2, 0x5f, 0x2d, 0x28, 0xf2, 0x26, 0x7f, 0xbd, 0x30, 0x41, - 0x7b, 0x12, 0x45, 0xee, 0x27, 0xab, 0x0f, 0x17, 0x56, 0x20, 0x65, 0x4f, 0x44, 0xaa, 0x9c, 0xdf, - 0x28, 0xa8, 0x5d, 0x84, 0x19, 0x0a, 0xbd, 0x2f, 0x53, 0xd0, 0x54, 0x64, 0x75, 0xc4, 0xa5, 0xcd, - 0xdd, 0x5d, 0xfe, 0x8b, 0x49, 0x99, 0x4e, 0x7d, 0x18, 0x59, 0xfd, 0x5c, 0x3e, 0x89, 0x6d, 0x71, - 0x0f, 0x7f, 0x0b, 0x8a, 0xcc, 0x25, 0x06, 0x91, 0x2b, 0xd6, 0x6f, 0x9f, 0xc8, 0xcd, 0xfa, 0x8c, - 0x5f, 0xbb, 0x2e, 0x8c, 0x94, 0x12, 0xaa, 0x41, 0xc1, 0x37, 0x9d, 0xd1, 0xae, 0xfb, 0x62, 0x20, - 0x1f, 0xc5, 0x0c, 0x77, 0xe7, 0xa3, 0x62, 0xba, 0x8c, 0x90, 0xde, 0x5c, 0xce, 0xfb, 0x61, 0xa1, - 0xfc, 0x5d, 0x28, 0xa8, 0x55, 0x90, 0xf0, 0x35, 0xf2, 0xec, 0x43, 0x1a, 0x23, 0x69, 0xf8, 0x62, - 0xa5, 0x33, 0x1e, 0x5b, 0xfc, 0x18, 0x96, 0x18, 0xcd, 0xc0, 0x9d, 0xb2, 0xb3, 0x27, 0x89, 0x88, - 0xe3, 0x84, 0x31, 0x18, 0x17, 0x19, 0x61, 0x9b, 0xd1, 0x95, 0x7f, 0x3d, 0x06, 0x79, 0x45, 0x31, - 0xf4, 0x69, 0xc4, 0x9a, 0x37, 0xce, 0x6b, 0xc8, 0xa2, 0x65, 0x4b, 0xf2, 0x1e, 0xb7, 0xa2, 0x24, - 0x7d, 0x00, 0xee, 0xee, 0x49, 0x9b, 0xc2, 0x59, 0x48, 0x76, 0xf5, 0xc6, 0x16, 0xf3, 0xd9, 0x4e, - 0x15, 0xeb, 0xf4, 0xce, 0x34, 0xf3, 0xd9, 0x4f, 0x4e, 0x62, 0xb8, 0x00, 0x45, 0xe6, 0xda, 0x83, - 0x27, 0xed, 0x46, 0xbf, 0xa9, 0xb3, 0xcd, 0xe4, 0x6e, 0xb5, 0x55, 0xdf, 0x6c, 0x7f, 0x36, 0xa0, - 0xd7, 0xaf, 0x05, 0xff, 0x6a, 0xf8, 0xe0, 0x25, 0x7e, 0xaa, 0xc5, 0xe8, 0xdf, 0xb6, 0x16, 0xaf, - 0xfc, 0x6a, 0x12, 0xf2, 0x2d, 0x2b, 0x90, 0xcf, 0x5b, 0xae, 0x42, 0x9a, 0x6e, 0xee, 0xb3, 0xe3, - 0x6b, 0x39, 0xcc, 0x4b, 0xaf, 0x78, 0x48, 0x0d, 0xf5, 0xa0, 0x60, 0x4f, 0x07, 0xfc, 0x3d, 0x1b, - 0xf9, 0xf9, 0xee, 0x2a, 0x27, 0x56, 0x2a, 0xba, 0x63, 0x74, 0xc4, 0x9b, 0xaa, 0x72, 0x8f, 0x48, - 0x82, 0x2c, 0x1f, 0xe7, 0xed, 0xa9, 0x2c, 0xc8, 0x37, 0x96, 0xd2, 0xca, 0x1b, 0x4b, 0x06, 0x14, - 0xa7, 0xae, 0x17, 0xd0, 0x2b, 0x03, 0xb6, 0xb3, 0x2f, 0xf6, 0x64, 0xd6, 0x4e, 0xa8, 0xaa, 0xe3, - 0x7a, 0x41, 0x93, 0x91, 0x89, 0xf3, 0x8f, 0xd3, 0x10, 0xe4, 0x97, 0x3d, 0xc8, 0xc9, 0xaa, 0xd1, - 0x37, 0x20, 0x4b, 0x9f, 0x05, 0x1e, 0xba, 0xe3, 0xb9, 0x8f, 0x61, 0x11, 0x91, 0x9c, 0x04, 0x4b, - 0x62, 0xfa, 0xf1, 0x5e, 0x36, 0x9d, 0xc7, 0x45, 0xf6, 0xf1, 0x5e, 0xc8, 0xc6, 0x39, 0xd9, 0xa8, - 0xf2, 0x4b, 0xc8, 0x2b, 0x6a, 0xa1, 0xb7, 0x45, 0x68, 0x99, 0x7f, 0x82, 0x8d, 0x05, 0x18, 0xd7, - 0x0b, 0xe6, 0x42, 0x15, 0xa1, 0x8b, 0x2b, 0x74, 0x4a, 0xa8, 0x22, 0xc4, 0x65, 0xa5, 0x15, 0xec, - 0xd5, 0x32, 0x59, 0xae, 0xac, 0x41, 0x56, 0xa8, 0x4f, 0x5c, 0xce, 0xe8, 0x1c, 0x7e, 0xc4, 0xde, - 0x70, 0x33, 0x3a, 0x87, 0x0f, 0xb4, 0x78, 0xe5, 0x38, 0x05, 0x4b, 0xe1, 0xe3, 0x3b, 0xd4, 0x2b, - 0xea, 0x73, 0x6f, 0x78, 0x92, 0x19, 0x37, 0x7c, 0xb3, 0x27, 0x4a, 0xac, 0x14, 0xa3, 0xcf, 0x77, - 0x56, 0x7e, 0x90, 0x8a, 0x3c, 0x00, 0x34, 0xf7, 0x41, 0x24, 0x55, 0xdb, 0x21, 0x3f, 0x7f, 0x94, - 0x41, 0x17, 0xa0, 0x50, 0xaf, 0xd6, 0x06, 0xed, 0x27, 0x3a, 0xc6, 0x46, 0x5d, 0xd7, 0xfe, 0x55, - 0x06, 0x5d, 0x84, 0x65, 0x02, 0xc2, 0x7a, 0xb5, 0x3e, 0xe8, 0xea, 0x55, 0x5c, 0xdb, 0xd1, 0xfe, - 0x75, 0x06, 0xe5, 0x21, 0xbd, 0xd5, 0x7e, 0xda, 0xd2, 0xb1, 0xf6, 0x6f, 0x58, 0xa1, 0xab, 0xf7, - 0x8c, 0xba, 0xf6, 0x6f, 0x33, 0x28, 0x07, 0xc9, 0xc7, 0x46, 0xa3, 0xa1, 0xfd, 0x3b, 0x0a, 0xef, - 0xea, 0xbd, 0x6d, 0xa3, 0xae, 0xfd, 0x7b, 0x51, 0xe8, 0x1b, 0x75, 0xed, 0x3f, 0x64, 0x50, 0x01, - 0x32, 0x5d, 0xbd, 0xd7, 0xa9, 0x55, 0x3b, 0xda, 0x8f, 0x69, 0x15, 0x0d, 0xa3, 0xd5, 0xff, 0x6c, - 0x60, 0x34, 0x9b, 0xfd, 0x5e, 0x75, 0xb3, 0xa1, 0x6b, 0xff, 0x31, 0x83, 0x2e, 0x81, 0xd6, 0xd2, - 0x7b, 0x83, 0x4d, 0xa3, 0x45, 0x2a, 0xc6, 0x4f, 0x8c, 0x9a, 0xae, 0xfd, 0x5e, 0x06, 0x21, 0x28, - 0x52, 0x30, 0x6e, 0x57, 0xeb, 0xb5, 0x6a, 0xb7, 0xa7, 0xfd, 0xa7, 0x0c, 0x5a, 0x82, 0x1c, 0x81, - 0x55, 0xeb, 0x4d, 0xa3, 0xa5, 0xfd, 0x67, 0x2a, 0x9e, 0x94, 0x71, 0xf5, 0xa9, 0xf6, 0x5f, 0x32, - 0xa8, 0x08, 0x59, 0xa3, 0x53, 0x1b, 0x34, 0xda, 0xb5, 0xc7, 0xda, 0x7f, 0xa5, 0xc4, 0xa4, 0xc8, - 0xb4, 0xff, 0xfd, 0x0c, 0x5a, 0x06, 0xe8, 0x7e, 0xde, 0x1d, 0x34, 0xdb, 0xf5, 0x7e, 0x43, 0xd7, - 0xfe, 0x80, 0x12, 0x10, 0x00, 0xae, 0x3e, 0x35, 0xda, 0xda, 0x7f, 0x93, 0x04, 0xb5, 0x1d, 0xdc, - 0x6e, 0xf7, 0xb4, 0xff, 0x2e, 0x01, 0x9d, 0x1e, 0xae, 0xd6, 0x74, 0xed, 0x7f, 0x48, 0x8e, 0x4e, - 0xb5, 0x56, 0xeb, 0x69, 0x3f, 0x91, 0x65, 0xa6, 0xcf, 0xff, 0xa4, 0x1a, 0x90, 0xf2, 0x26, 0xe1, - 0xff, 0x5f, 0xb2, 0xd8, 0x22, 0x2d, 0xfa, 0xdf, 0xd4, 0xe8, 0xb4, 0x3e, 0xbe, 0xc6, 0xd1, 0xbe, - 0x97, 0x15, 0x14, 0x64, 0x5d, 0xa8, 0xfd, 0xb1, 0x2c, 0x5a, 0x81, 0x25, 0x5a, 0xec, 0x7d, 0x4e, - 0x16, 0xa4, 0x5b, 0xc6, 0xb6, 0xf6, 0xf3, 0x59, 0xd2, 0x6f, 0xcd, 0xc7, 0xad, 0x76, 0x5d, 0xfb, - 0xe3, 0xf4, 0x77, 0x43, 0xaf, 0x76, 0x75, 0xed, 0x17, 0xb2, 0x48, 0x83, 0x7c, 0xb5, 0x5f, 0x37, - 0x7a, 0x83, 0xa7, 0xd8, 0xe8, 0xe9, 0xda, 0x9f, 0xc8, 0x12, 0x93, 0x31, 0x08, 0x59, 0xcd, 0xe2, - 0x76, 0x43, 0xfb, 0x93, 0x59, 0xde, 0x03, 0x5b, 0xa4, 0x07, 0xfe, 0x54, 0x96, 0xa8, 0xd0, 0x54, - 0xfb, 0xfd, 0xfb, 0x59, 0xd2, 0x06, 0x02, 0x62, 0x6d, 0xf8, 0x41, 0x96, 0xf6, 0xdf, 0xe7, 0xdd, - 0x46, 0x7b, 0x5b, 0xfb, 0xc5, 0x2c, 0xb1, 0xc0, 0xd3, 0xea, 0x63, 0x7d, 0x50, 0x6d, 0x54, 0x71, - 0x53, 0xfb, 0x25, 0x5a, 0xc5, 0x26, 0x31, 0xf0, 0xa0, 0xdb, 0xef, 0x76, 0xf4, 0x56, 0x5d, 0xfb, - 0x65, 0x4a, 0xc4, 0xaa, 0x25, 0xbe, 0xa3, 0xfd, 0x4a, 0x96, 0xc7, 0xc3, 0xc7, 0x90, 0x6b, 0xd8, - 0xce, 0xec, 0x05, 0x75, 0xef, 0x4f, 0x60, 0x59, 0x3a, 0xea, 0x4b, 0x71, 0x84, 0x3a, 0xf2, 0x90, - 0x69, 0xc4, 0xc3, 0xf1, 0xd2, 0x30, 0x52, 0xae, 0xfc, 0x30, 0x01, 0x80, 0x69, 0xce, 0xc9, 0xdf, - 0x27, 0xcc, 0x78, 0x91, 0xd4, 0x54, 0xbe, 0x56, 0x23, 0x69, 0xf8, 0x4f, 0xf1, 0xad, 0x90, 0x93, - 0x97, 0xbf, 0x9f, 0x80, 0x34, 0xc3, 0xa0, 0x6f, 0x44, 0x26, 0x9f, 0xb7, 0x4e, 0x93, 0xb0, 0x38, - 0xe9, 0x20, 0x48, 0x1e, 0x98, 0xde, 0x88, 0x3f, 0x84, 0x41, 0x7f, 0x13, 0x98, 0xef, 0xee, 0x05, - 0x3c, 0x31, 0xa5, 0xbf, 0x2b, 0x7f, 0x2e, 0x7e, 0xca, 0xeb, 0x6b, 0xb8, 0xd1, 0xec, 0x0d, 0xaa, - 0x5d, 0x2d, 0x86, 0x8a, 0x90, 0xa3, 0x85, 0x5a, 0x1b, 0xeb, 0x5a, 0x1c, 0x15, 0x20, 0xcb, 0x8a, - 0x9d, 0xbe, 0x96, 0x90, 0xc8, 0x7a, 0xb5, 0x57, 0xd5, 0x92, 0x68, 0x89, 0x98, 0xa0, 0xd9, 0x1b, - 0x6c, 0x75, 0x8d, 0xef, 0xf0, 0xa7, 0x69, 0x69, 0x99, 0xf4, 0x07, 0x59, 0xa3, 0x6b, 0x50, 0xa0, - 0xe5, 0xa6, 0xde, 0xa4, 0x63, 0x80, 0x78, 0x5c, 0x91, 0x41, 0xba, 0xdb, 0x3f, 0xd3, 0xd7, 0xfb, - 0xba, 0x96, 0x95, 0x32, 0xa9, 0x53, 0xe6, 0xd0, 0x32, 0xe4, 0x59, 0xb1, 0xbd, 0x65, 0x34, 0x74, - 0x0d, 0xa4, 0xd0, 0x56, 0x07, 0xb7, 0x6b, 0x5a, 0x5e, 0x6a, 0x84, 0xbb, 0x5d, 0xad, 0x20, 0xc9, - 0x71, 0xaf, 0x83, 0x8d, 0xb6, 0x56, 0x54, 0x00, 0xd4, 0x87, 0x97, 0xe8, 0xc6, 0x03, 0x01, 0x74, - 0x8d, 0x6d, 0xe2, 0x1f, 0x46, 0x6b, 0x5b, 0x5b, 0x96, 0x42, 0xbb, 0xbd, 0x6a, 0xed, 0xb1, 0xa6, - 0x71, 0x07, 0xf9, 0x95, 0x18, 0x64, 0x7a, 0xbd, 0xcf, 0x69, 0x87, 0x3e, 0x82, 0xfc, 0x73, 0xdb, - 0x19, 0xb9, 0xcf, 0x07, 0xbe, 0xfd, 0x5d, 0xf1, 0x8a, 0x8d, 0xc8, 0xae, 0x38, 0xd1, 0x9d, 0xa7, - 0x94, 0xa2, 0x6b, 0x7f, 0xd7, 0xc2, 0xf0, 0x5c, 0xfe, 0x2e, 0x6f, 0x01, 0x84, 0x18, 0xf6, 0x6e, - 0xca, 0x73, 0x3f, 0xfa, 0xca, 0x26, 0x81, 0xa0, 0x35, 0xc8, 0x0c, 0x49, 0x66, 0xe1, 0xf8, 0x91, - 0xb8, 0x2e, 0x80, 0x95, 0xa3, 0x2c, 0x14, 0x23, 0xbb, 0xe3, 0xe8, 0xeb, 0x91, 0xb5, 0xc3, 0xe5, - 0x93, 0x76, 0xd0, 0xd5, 0x35, 0xc4, 0xd7, 0xc3, 0x0f, 0xd7, 0xec, 0xa4, 0x51, 0xf4, 0x0d, 0x9d, - 0xb9, 0xcf, 0xd6, 0xe8, 0x1b, 0x73, 0x6b, 0x89, 0xf5, 0x13, 0xe5, 0xb3, 0x5c, 0x8d, 0x3d, 0x63, - 0xc4, 0xd7, 0x15, 0xe5, 0xc8, 0x01, 0x9f, 0xe8, 0x23, 0x79, 0x1f, 0x01, 0x7b, 0xa2, 0x9f, 0xe7, - 0xb5, 0x6b, 0x27, 0xca, 0x6c, 0x12, 0x18, 0xfb, 0x0e, 0x46, 0xd1, 0xe8, 0xdb, 0xf4, 0x92, 0x1b, - 0xfb, 0xd8, 0x44, 0x0f, 0x3d, 0x64, 0x22, 0x2f, 0x90, 0x29, 0xf3, 0xb2, 0x98, 0xde, 0x9d, 0x10, - 0xe4, 0xa3, 0xbb, 0x00, 0x63, 0x32, 0xd4, 0xd9, 0xc0, 0xce, 0x46, 0xbe, 0xb9, 0xc9, 0x18, 0x80, - 0x73, 0x63, 0x19, 0x0e, 0x36, 0x20, 0xcf, 0x06, 0x24, 0xe3, 0xc8, 0x45, 0xd2, 0xc7, 0x70, 0x04, - 0x62, 0x60, 0x54, 0x94, 0xe7, 0x01, 0x64, 0x03, 0x11, 0x3b, 0x20, 0xb2, 0xd3, 0xcd, 0xfd, 0x83, - 0xed, 0x74, 0xf3, 0x02, 0xce, 0x04, 0x2c, 0x74, 0x94, 0x7f, 0x33, 0x09, 0x10, 0x1a, 0x11, 0x95, - 0xc5, 0xea, 0x20, 0xf2, 0x22, 0x1a, 0x5b, 0x15, 0x6c, 0x43, 0x86, 0xb7, 0x8b, 0x3f, 0x91, 0xfe, - 0xce, 0x39, 0x5d, 0x22, 0x2c, 0xf3, 0x28, 0xb9, 0xd3, 0xee, 0xf6, 0xb0, 0xe0, 0x46, 0x4f, 0xe6, - 0x53, 0x27, 0x76, 0x53, 0xe1, 0xbd, 0xf3, 0xc4, 0x9d, 0x93, 0x47, 0xa1, 0x1b, 0x00, 0x53, 0xcf, - 0x3e, 0xb4, 0xc7, 0xd6, 0xbe, 0x5c, 0x55, 0x8a, 0xc7, 0xc8, 0x43, 0x04, 0x7a, 0x00, 0x20, 0x37, - 0x34, 0xe6, 0x1f, 0x20, 0x9c, 0xdf, 0xfa, 0x50, 0x28, 0xd1, 0x2d, 0xd0, 0xf6, 0x5c, 0x6f, 0x68, - 0x0d, 0xa6, 0xb3, 0xf1, 0x78, 0xc0, 0xcc, 0x44, 0xdf, 0x59, 0xc5, 0x4b, 0x14, 0xde, 0x99, 0x8d, - 0xc7, 0x6c, 0x55, 0xfd, 0x0e, 0x14, 0x99, 0x1b, 0x0f, 0xf8, 0xda, 0x21, 0x23, 0x9f, 0xe1, 0x2c, - 0x30, 0x44, 0x9d, 0xc2, 0xff, 0x7f, 0x66, 0x61, 0x0f, 0x21, 0xc3, 0xbb, 0x87, 0x3e, 0x9f, 0xdb, - 0xee, 0xf2, 0xe7, 0x2e, 0x37, 0xb1, 0x51, 0xdf, 0xd6, 0xd9, 0xbb, 0x76, 0xad, 0x76, 0x4b, 0xd7, - 0x12, 0xe4, 0x57, 0xbf, 0xab, 0x63, 0x2d, 0xc9, 0x42, 0x53, 0xf9, 0x2e, 0xe4, 0xe4, 0x18, 0x09, - 0xd7, 0x93, 0xb1, 0x53, 0xd7, 0x93, 0x95, 0xaf, 0x85, 0x6f, 0xe9, 0xf1, 0xa5, 0x30, 0x7b, 0x95, - 0x49, 0xef, 0xb6, 0xbb, 0x72, 0x85, 0xf0, 0xbd, 0x38, 0x2c, 0xcf, 0x7d, 0x30, 0x58, 0x1c, 0x79, - 0xb1, 0xd7, 0x1a, 0x79, 0x1b, 0x90, 0x1f, 0xd2, 0xf5, 0x03, 0x1b, 0x17, 0x73, 0x0b, 0xf8, 0x7d, - 0xf1, 0x5d, 0x1b, 0xc3, 0x50, 0xfe, 0x46, 0x1b, 0xca, 0x41, 0xbe, 0x29, 0x7f, 0x35, 0xbf, 0xc8, - 0x6e, 0x60, 0x8b, 0x4f, 0x39, 0x1d, 0xa3, 0x1e, 0x9e, 0xd8, 0xeb, 0xd8, 0xf4, 0x2d, 0xd7, 0xb9, - 0x0b, 0x45, 0xa7, 0xfc, 0x4f, 0x0b, 0x26, 0xe7, 0xb4, 0xcb, 0x44, 0x95, 0x3f, 0x02, 0x10, 0x6a, - 0x85, 0xbe, 0x4d, 0xc7, 0xdb, 0x60, 0x38, 0x16, 0x0f, 0x01, 0x94, 0x16, 0x34, 0x27, 0x16, 0xa8, - 0x8d, 0xfd, 0x4d, 0x38, 0x3e, 0x5a, 0x4f, 0x93, 0xdf, 0x8d, 0x2e, 0x4e, 0x3b, 0x14, 0x56, 0xbe, - 0x0b, 0x69, 0x86, 0x45, 0x37, 0x20, 0x33, 0x1c, 0x9b, 0xbe, 0xcf, 0x0f, 0x87, 0x16, 0x59, 0x28, - 0xa8, 0x11, 0x90, 0x51, 0xc7, 0x02, 0x57, 0xf9, 0x08, 0xd2, 0x6c, 0x35, 0x85, 0x6e, 0xcb, 0xc5, - 0x16, 0xb3, 0x77, 0x41, 0x5d, 0x6c, 0xc9, 0x6d, 0x2d, 0x76, 0x2d, 0xe8, 0x3e, 0xa4, 0x28, 0xf8, - 0xd4, 0x2d, 0xc9, 0x93, 0x77, 0x33, 0xff, 0x4e, 0x0c, 0x92, 0xd4, 0x47, 0xaf, 0x42, 0xda, 0x99, - 0x4d, 0x76, 0xf9, 0x7f, 0x34, 0x10, 0x8e, 0xcc, 0x61, 0x72, 0xe5, 0x15, 0x8f, 0xbc, 0x6e, 0x7b, - 0xaa, 0x57, 0xa3, 0x4f, 0x01, 0x0e, 0x6d, 0xdf, 0xe6, 0xa7, 0x71, 0x92, 0x34, 0x4c, 0xad, 0x9f, - 0xf4, 0x3d, 0xf8, 0xce, 0x13, 0x49, 0x86, 0x15, 0x16, 0x65, 0x9d, 0x99, 0x3a, 0xeb, 0x32, 0xd4, - 0x3d, 0x48, 0xb1, 0x17, 0x29, 0xde, 0x81, 0xd4, 0x94, 0xbe, 0x54, 0xc1, 0x2c, 0x95, 0x17, 0x71, - 0xc4, 0xf5, 0x44, 0x72, 0xc5, 0xf0, 0x95, 0x7f, 0x18, 0x87, 0x62, 0x44, 0x03, 0xa4, 0x47, 0x74, - 0x65, 0xb3, 0xe8, 0x79, 0xba, 0x8a, 0xb0, 0xa4, 0x68, 0x7c, 0x92, 0x89, 0xae, 0x45, 0xdf, 0x99, - 0x66, 0x56, 0x8a, 0xbc, 0x29, 0x5d, 0x86, 0xac, 0x38, 0x8d, 0x29, 0x66, 0x49, 0x51, 0x56, 0x9f, - 0x92, 0x4d, 0x45, 0x9f, 0x92, 0xad, 0x88, 0xd6, 0xa6, 0x23, 0x83, 0x9d, 0x9a, 0x82, 0x37, 0x54, - 0xb1, 0x60, 0xe6, 0x2c, 0x0b, 0x3e, 0x00, 0x08, 0x9b, 0x45, 0x12, 0x2f, 0xf9, 0xb5, 0x85, 0xbf, - 0xee, 0xdd, 0xe8, 0xd3, 0x0f, 0x72, 0xf4, 0x7f, 0x63, 0xe8, 0x9f, 0xf5, 0x74, 0xdc, 0xaa, 0x36, - 0xb4, 0x78, 0xe5, 0x13, 0x80, 0xa7, 0x96, 0xbd, 0x7f, 0x10, 0xf0, 0x17, 0x03, 0xd3, 0xcf, 0x69, - 0x29, 0x72, 0xfd, 0x8c, 0xc3, 0xe4, 0xb3, 0x83, 0xf1, 0xf0, 0xd9, 0xc1, 0xca, 0xef, 0xc6, 0x20, - 0xff, 0x84, 0x35, 0x87, 0x4a, 0x58, 0x0b, 0x1b, 0xab, 0xba, 0xae, 0x6c, 0x32, 0xbd, 0x64, 0x65, - 0x8f, 0x47, 0x83, 0x11, 0xfb, 0x3a, 0x4d, 0xdf, 0xdb, 0xa3, 0x90, 0xba, 0x19, 0x58, 0x21, 0x9a, - 0x7e, 0xbb, 0x4c, 0xb0, 0x4f, 0xc5, 0x14, 0x42, 0x3f, 0x52, 0x4a, 0x34, 0x3d, 0x3c, 0x90, 0x54, - 0xb8, 0xfb, 0xbe, 0xe5, 0xa1, 0xaf, 0x41, 0x66, 0xdf, 0x0e, 0x06, 0xfe, 0x81, 0xc9, 0x2c, 0xcd, - 0x06, 0xf2, 0xb6, 0x1d, 0x74, 0x77, 0xaa, 0x38, 0xbd, 0x6f, 0x07, 0xdd, 0x03, 0x93, 0xc8, 0x20, - 0x44, 0xec, 0xc0, 0x3b, 0xdf, 0x83, 0xc8, 0xed, 0xdb, 0xc1, 0x26, 0x05, 0xa0, 0x37, 0x99, 0x8c, - 0xc0, 0xdc, 0xe7, 0xff, 0x43, 0x83, 0xf0, 0xf5, 0xcc, 0xfd, 0xca, 0x03, 0x48, 0x6e, 0x8d, 0xcd, - 0xfd, 0xd7, 0xfe, 0xd0, 0xf0, 0xd7, 0x63, 0x90, 0xc4, 0xee, 0x99, 0x5f, 0x28, 0x42, 0xb3, 0xc7, - 0x4f, 0x30, 0xfb, 0xc7, 0x00, 0xf2, 0x84, 0x8d, 0x98, 0xdc, 0x4f, 0x3a, 0x9c, 0xc3, 0x7d, 0x39, - 0xa4, 0xfd, 0x4a, 0x67, 0xc7, 0x2a, 0x1f, 0x43, 0xba, 0x69, 0x05, 0x9e, 0x3d, 0x7c, 0xd5, 0x96, - 0xc6, 0x44, 0x4b, 0xff, 0x5e, 0x0c, 0xb2, 0x5b, 0x36, 0x7b, 0x0a, 0x50, 0x6e, 0x96, 0xc5, 0xe6, - 0x37, 0xcb, 0x08, 0xb3, 0x33, 0xb6, 0x1d, 0x96, 0xf6, 0xa4, 0x30, 0x2b, 0xd0, 0x35, 0x0d, 0xc9, - 0xc6, 0xc5, 0x9a, 0x86, 0xa4, 0xd7, 0x37, 0x20, 0x35, 0xa1, 0x8e, 0x90, 0x3c, 0xf9, 0x23, 0x36, - 0xc3, 0x12, 0x56, 0xba, 0x57, 0x4b, 0xff, 0xcf, 0x00, 0xdf, 0x97, 0xbd, 0x0c, 0x89, 0x19, 0x7f, - 0xdf, 0x98, 0x3f, 0xfb, 0xdc, 0x37, 0xea, 0x98, 0xc0, 0x08, 0x6a, 0x9f, 0x9f, 0x33, 0xe0, 0xa8, - 0x6d, 0x82, 0xda, 0xb7, 0x47, 0xb7, 0x7f, 0x16, 0xd2, 0x7c, 0xaa, 0x5c, 0x05, 0x54, 0xc7, 0xc6, - 0x13, 0x1d, 0x0f, 0x5a, 0x6d, 0xba, 0x96, 0xc0, 0xec, 0x5b, 0x29, 0x82, 0x25, 0x0e, 0xc7, 0xfd, - 0x16, 0xff, 0xff, 0x32, 0x21, 0xac, 0xba, 0xd9, 0xa6, 0x74, 0x09, 0x05, 0xd6, 0xed, 0xb5, 0x3b, - 0x1d, 0xbd, 0xae, 0x25, 0x6f, 0xff, 0x72, 0x1c, 0x72, 0xf2, 0x04, 0x06, 0x59, 0x41, 0xd1, 0x8f, - 0x6a, 0xdd, 0x5e, 0x75, 0x9b, 0xc8, 0x49, 0x93, 0x15, 0x94, 0x80, 0xe0, 0x1e, 0x01, 0xbd, 0x21, - 0x89, 0x44, 0x65, 0x31, 0x09, 0xe1, 0xff, 0x79, 0x44, 0xcb, 0x4a, 0xb6, 0x2d, 0xa3, 0x65, 0x74, - 0x77, 0xe8, 0xf7, 0xda, 0x65, 0xc8, 0x33, 0x10, 0xfb, 0xb0, 0x9c, 0x90, 0x00, 0xc2, 0x45, 0x74, - 0x21, 0xab, 0x24, 0x0a, 0x60, 0x9f, 0x6b, 0x33, 0x24, 0x62, 0xd0, 0x72, 0x83, 0x64, 0x2e, 0x29, - 0x59, 0x4b, 0x1d, 0x33, 0xe5, 0x73, 0xe8, 0x22, 0x68, 0xfc, 0x1b, 0x20, 0xd6, 0xab, 0xb5, 0x1d, - 0xba, 0xa1, 0x02, 0x92, 0x6d, 0x9b, 0xa4, 0x36, 0x79, 0x74, 0x19, 0x2e, 0xc9, 0xe2, 0x60, 0xf3, - 0xf3, 0x41, 0xbb, 0xa3, 0xe3, 0x6a, 0xaf, 0x8d, 0xb5, 0x82, 0x94, 0x28, 0x3f, 0x7e, 0x6e, 0xd6, - 0x7e, 0xfb, 0xcb, 0xb5, 0x37, 0xfe, 0xe9, 0x97, 0x6b, 0x6f, 0xfc, 0x8b, 0x2f, 0xd7, 0x62, 0x3f, - 0xfe, 0x72, 0x2d, 0xf6, 0x07, 0x5f, 0xae, 0xc5, 0xbe, 0x77, 0xbc, 0x16, 0xfb, 0xab, 0xc7, 0x6b, - 0xb1, 0xbf, 0x7d, 0xbc, 0x16, 0xfb, 0xfb, 0xc7, 0x6b, 0xb1, 0xdf, 0x3a, 0x5e, 0x8b, 0xfd, 0xf6, - 0xf1, 0x5a, 0xec, 0x47, 0xc7, 0x6b, 0x6f, 0xfc, 0xf8, 0x78, 0x2d, 0xf6, 0xab, 0xbf, 0xb3, 0xf6, - 0xc6, 0x5f, 0xf8, 0x9d, 0xb5, 0xd8, 0x77, 0xd8, 0x62, 0xe2, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, - 0x51, 0x75, 0xe9, 0x08, 0x31, 0x6c, 0x00, 0x00, + 0xeb, 0xee, 0x05, 0x51, 0x6f, 0xa6, 0xa6, 0x33, 0x87, 0x07, 0x62, 0x1c, 0x7f, 0x47, 0xea, 0x52, + 0x23, 0xd4, 0xc8, 0xdc, 0xe0, 0x3f, 0x37, 0xa7, 0x1c, 0x76, 0x4b, 0xce, 0x0d, 0xdd, 0xe7, 0xe6, + 0x94, 0xa1, 0x36, 0x98, 0xc6, 0x33, 0xc7, 0x3a, 0xb4, 0x87, 0xf4, 0x8c, 0x03, 0x07, 0xbf, 0x4b, + 0xc1, 0x2b, 0x13, 0x6b, 0xd2, 0x0f, 0xf3, 0x18, 0xcf, 0xcf, 0xc2, 0x0a, 0xb1, 0xbc, 0xef, 0xf0, + 0x60, 0x8c, 0x9b, 0xfb, 0x76, 0x64, 0xae, 0x88, 0x46, 0x7e, 0x9b, 0x97, 0x8e, 0x8f, 0xd6, 0x2f, + 0xb4, 0xac, 0x20, 0x4a, 0xc6, 0x17, 0x1c, 0x2b, 0xe8, 0x3a, 0x6a, 0xec, 0x56, 0xf9, 0xbd, 0x24, + 0x14, 0x45, 0x7c, 0xd4, 0xf7, 0xcd, 0x7d, 0x0b, 0x55, 0x21, 0x27, 0x8e, 0x34, 0x88, 0x13, 0x29, + 0xf3, 0x1f, 0x0b, 0x28, 0x50, 0x7e, 0x10, 0x17, 0x91, 0x93, 0xe4, 0x42, 0xef, 0x41, 0x8a, 0x36, + 0x4a, 0x29, 0x7e, 0xd6, 0x87, 0x5c, 0x86, 0x29, 0xff, 0x38, 0x01, 0x59, 0x21, 0x0a, 0x7d, 0x02, + 0xc5, 0xf0, 0x24, 0x86, 0xb3, 0xe7, 0xf2, 0xb3, 0x18, 0x2b, 0x27, 0x7c, 0x83, 0x17, 0x07, 0x2a, + 0x2d, 0xf5, 0x24, 0xc7, 0x87, 0x90, 0xe3, 0x5f, 0x10, 0xac, 0xd1, 0xd9, 0xa5, 0x87, 0x38, 0xf4, + 0x10, 0x40, 0x31, 0x2c, 0x0b, 0x53, 0x2f, 0xcf, 0x71, 0x29, 0x56, 0x54, 0xc0, 0xe8, 0xa7, 0xa1, + 0x20, 0xcf, 0x36, 0x0c, 0xe8, 0xa7, 0xf6, 0xf8, 0xc9, 0x87, 0xcf, 0x37, 0x57, 0xf8, 0x3e, 0xaf, + 0x7a, 0xa2, 0x1d, 0xe7, 0x25, 0xb3, 0x31, 0x42, 0x9f, 0x40, 0x2a, 0xa0, 0x4b, 0x5f, 0xf6, 0x21, + 0xbb, 0x72, 0xa6, 0xd1, 0xef, 0xf4, 0x4c, 0xff, 0x99, 0x34, 0x24, 0x61, 0x2b, 0xff, 0x95, 0x18, + 0x24, 0x09, 0xf5, 0x8c, 0xaf, 0x66, 0xef, 0xd2, 0x8d, 0x67, 0x76, 0x12, 0x5c, 0x7c, 0x77, 0x60, + 0x27, 0xf6, 0x17, 0xf6, 0xa1, 0xbf, 0xd2, 0x07, 0xf9, 0xf0, 0xbb, 0x4a, 0xf2, 0x8c, 0xef, 0x2a, + 0x95, 0x3f, 0xbd, 0x02, 0x4b, 0xd1, 0xe8, 0xe6, 0x95, 0x02, 0xf2, 0x6b, 0x91, 0xf3, 0xdf, 0x21, + 0x44, 0x52, 0xc9, 0xc2, 0x73, 0xf8, 0x72, 0x38, 0x96, 0x7b, 0xe4, 0x3c, 0x85, 0x1e, 0xc0, 0x9b, + 0x7e, 0x60, 0x8e, 0x49, 0xb0, 0xc8, 0x28, 0x83, 0x3d, 0xcf, 0x75, 0x02, 0xcb, 0x19, 0xf1, 0x6d, + 0xf3, 0x4b, 0x3c, 0xbb, 0x46, 0x73, 0xb7, 0x78, 0x26, 0xfa, 0x08, 0x56, 0xe7, 0xf8, 0x76, 0x49, + 0x3c, 0xe3, 0x8c, 0xf8, 0x96, 0xfa, 0xc5, 0x08, 0xdb, 0x26, 0xcb, 0x23, 0xcb, 0x35, 0xdb, 0xf1, + 0x03, 0x6f, 0xc6, 0xa7, 0x1b, 0x16, 0x42, 0x47, 0x68, 0xe8, 0x5d, 0xd0, 0xd8, 0x98, 0xe2, 0x59, + 0x7b, 0x96, 0x67, 0x39, 0xc4, 0xca, 0x6c, 0xcb, 0x7d, 0x99, 0xd2, 0xb1, 0x24, 0xa3, 0xb7, 0xa1, + 0xc0, 0xa0, 0x13, 0x9b, 0x4e, 0x38, 0x59, 0x16, 0x17, 0x51, 0x5a, 0x93, 0x92, 0xc8, 0x62, 0x76, + 0xd7, 0x33, 0x9d, 0xe1, 0x81, 0xc5, 0xb7, 0xde, 0xb1, 0x4c, 0xa3, 0xaf, 0x41, 0x91, 0xfd, 0x16, + 0xfc, 0x3c, 0xb4, 0x65, 0x44, 0x2e, 0xe0, 0x2d, 0x80, 0xdd, 0x99, 0xcf, 0x2b, 0xc9, 0xc3, 0xda, + 0xdc, 0xee, 0xcc, 0x67, 0x15, 0x23, 0xd9, 0x9e, 0xb5, 0x27, 0xb2, 0x59, 0x20, 0x9b, 0xf3, 0xac, + 0x3d, 0x9e, 0xfd, 0x2e, 0x90, 0x85, 0xc7, 0x60, 0x38, 0x76, 0x87, 0xcf, 0x68, 0x10, 0x1b, 0x63, + 0xeb, 0xb1, 0x5a, 0xa7, 0x5f, 0x23, 0x34, 0x9c, 0x1d, 0x4e, 0x67, 0xf4, 0x17, 0x91, 0x44, 0xbc, + 0x95, 0x63, 0x49, 0xec, 0x1a, 0xc3, 0x39, 0x42, 0x61, 0xd9, 0xeb, 0x90, 0x9f, 0x9a, 0xfb, 0xd6, + 0x80, 0xee, 0xdf, 0xcb, 0xb8, 0x95, 0x90, 0xb6, 0x28, 0x85, 0x18, 0x63, 0x62, 0x3b, 0xae, 0x27, + 0x10, 0x3c, 0x6c, 0xa5, 0x34, 0x05, 0x62, 0xfe, 0x5c, 0x08, 0xb9, 0xc0, 0x21, 0x84, 0xc6, 0x21, + 0xc4, 0xfa, 0xa4, 0x89, 0x5f, 0x04, 0x03, 0xff, 0xb9, 0x1d, 0x50, 0xbb, 0x21, 0x6e, 0x7d, 0x46, + 0xef, 0x72, 0x32, 0xfa, 0x18, 0x96, 0x48, 0xdd, 0x26, 0xf6, 0x3e, 0xf3, 0x31, 0x1e, 0xba, 0xca, + 0x95, 0x4e, 0x53, 0x66, 0xd0, 0x95, 0x4e, 0x98, 0x24, 0x85, 0x98, 0x63, 0x7b, 0x9f, 0x1e, 0xb9, + 0x13, 0xba, 0xb0, 0x98, 0x76, 0x59, 0xd2, 0x43, 0x7d, 0xac, 0xc9, 0x6c, 0xcc, 0x3e, 0x97, 0x72, + 0x28, 0x8b, 0x6c, 0x97, 0x25, 0x9d, 0x43, 0x6f, 0xc2, 0xf2, 0xf8, 0x83, 0xc1, 0x88, 0x79, 0xc4, + 0xd8, 0x25, 0x51, 0xc6, 0x2a, 0x9b, 0x90, 0xc6, 0x1f, 0xd4, 0x29, 0xb5, 0x41, 0x88, 0x64, 0x72, + 0x8c, 0xe2, 0x44, 0xf3, 0xbf, 0xc9, 0x26, 0x47, 0x15, 0xcd, 0x9d, 0xe0, 0x16, 0x68, 0x21, 0x8b, + 0x1f, 0xb8, 0x9e, 0xc5, 0xf6, 0xf5, 0x92, 0x78, 0x49, 0xa0, 0xbb, 0x94, 0x8a, 0x3e, 0x84, 0xd5, + 0x39, 0xa4, 0x90, 0x7e, 0x99, 0x4d, 0x64, 0x11, 0x3c, 0x17, 0x7f, 0x0f, 0x2e, 0x86, 0x4c, 0x53, + 0xe2, 0xf7, 0xcc, 0xf0, 0xe5, 0xa8, 0x42, 0x1d, 0x99, 0x83, 0x1e, 0xc2, 0xe5, 0x45, 0x0e, 0x51, + 0x12, 0x0b, 0x2a, 0x57, 0xe7, 0xd9, 0x78, 0x61, 0xcc, 0x4c, 0xb6, 0x6a, 0xa6, 0xab, 0xc2, 0x4c, + 0xc6, 0x82, 0x99, 0xec, 0x45, 0x33, 0xbd, 0x25, 0xb4, 0x32, 0xe6, 0xcd, 0xc4, 0xea, 0x61, 0x2f, + 0xd4, 0x63, 0x2d, 0xca, 0xb1, 0x50, 0x0f, 0xfb, 0xe4, 0x7a, 0xac, 0x8b, 0x7a, 0x18, 0x27, 0xd5, + 0xe3, 0x5d, 0xc8, 0x8d, 0xc7, 0x43, 0x5e, 0x03, 0x1a, 0x23, 0xb2, 0xae, 0xd5, 0x68, 0xd4, 0x68, + 0x05, 0x70, 0x76, 0x3c, 0x1e, 0xb2, 0xaa, 0x3c, 0x84, 0x65, 0x01, 0x15, 0xb2, 0xdf, 0x0e, 0x5d, + 0x95, 0x33, 0x30, 0xb1, 0xb8, 0xc8, 0xb9, 0x78, 0x29, 0xef, 0x03, 0x10, 0x56, 0xde, 0xe6, 0x34, + 0x1a, 0x64, 0x5b, 0x0b, 0x8d, 0x46, 0x8d, 0x35, 0x39, 0x26, 0x6a, 0xf0, 0xd6, 0xff, 0x16, 0x68, + 0x12, 0x2d, 0x4a, 0xfa, 0x1a, 0xe5, 0x41, 0xc7, 0x47, 0xeb, 0x4b, 0x82, 0x87, 0x17, 0xb5, 0x24, + 0x18, 0x79, 0x59, 0x1f, 0x03, 0xa1, 0xa8, 0x86, 0xbb, 0x1e, 0xd1, 0x32, 0xb4, 0x1b, 0xd5, 0x52, + 0x31, 0xa3, 0x0e, 0x2b, 0x2a, 0xa7, 0x28, 0x9a, 0x46, 0x7b, 0x2c, 0xe4, 0x51, 0xd8, 0x79, 0xe9, + 0x17, 0x14, 0x11, 0x61, 0x65, 0x47, 0xc1, 0x78, 0x97, 0xdb, 0xf4, 0x66, 0x58, 0xd9, 0x7a, 0xaf, + 0xb1, 0xc9, 0x8c, 0x9a, 0x23, 0x00, 0x66, 0xd5, 0x6f, 0x81, 0x26, 0xd1, 0xa2, 0xc4, 0x77, 0xc2, + 0xca, 0x0a, 0x1e, 0x51, 0x59, 0xc1, 0xc8, 0xcb, 0xba, 0x0b, 0x79, 0xca, 0xcd, 0x2d, 0x4b, 0x63, + 0xc2, 0xcd, 0xa5, 0xe3, 0xa3, 0x75, 0x20, 0x8c, 0xdc, 0xb4, 0x54, 0x1d, 0x6e, 0xdb, 0x4f, 0xe1, + 0x42, 0xc8, 0x20, 0xca, 0xa3, 0xd1, 0x21, 0xdb, 0x1e, 0x91, 0x6c, 0xbc, 0xc0, 0x65, 0xc9, 0xcb, + 0x4b, 0xfc, 0x26, 0x50, 0x92, 0x6a, 0xdf, 0xdb, 0x51, 0x75, 0x15, 0x03, 0x53, 0x75, 0x15, 0x0b, + 0xef, 0xc0, 0xc5, 0x08, 0xb3, 0x50, 0xe0, 0x3d, 0x2a, 0x81, 0x6e, 0x56, 0xa9, 0x12, 0xb8, 0x0e, + 0x48, 0x95, 0x12, 0x1a, 0xd9, 0x0e, 0x8d, 0xfc, 0x7e, 0x68, 0x64, 0x23, 0x34, 0xb2, 0xad, 0x1a, + 0xd9, 0x9e, 0x37, 0xf2, 0xd7, 0x43, 0xad, 0x8d, 0x39, 0x23, 0xdb, 0x51, 0x23, 0xbf, 0x0d, 0x7c, + 0x32, 0xe3, 0xa5, 0xdd, 0x61, 0x03, 0x3e, 0xa3, 0xb1, 0x02, 0xde, 0x07, 0xa4, 0x40, 0x44, 0x11, + 0x77, 0x29, 0x50, 0x0b, 0x81, 0xe1, 0x6c, 0xe8, 0xb8, 0x23, 0x31, 0x6e, 0xdc, 0x63, 0xd3, 0x1d, + 0xa1, 0x30, 0x61, 0xb7, 0x40, 0x93, 0xd9, 0x42, 0xd4, 0x07, 0x6c, 0x9c, 0x14, 0x20, 0x2e, 0x68, + 0x1d, 0xf2, 0x14, 0xc9, 0x9b, 0x7f, 0x83, 0x6f, 0x7a, 0xb8, 0x23, 0x31, 0x90, 0xde, 0x86, 0x0b, + 0x21, 0x40, 0xc8, 0xfa, 0x90, 0x8d, 0xfc, 0x12, 0xc6, 0x85, 0xbd, 0x03, 0x94, 0xa4, 0xb6, 0xec, + 0x47, 0x61, 0xa9, 0x4a, 0x2b, 0xde, 0x83, 0x8b, 0x11, 0xa0, 0x90, 0x7b, 0x9f, 0x0d, 0x50, 0x2a, + 0x9a, 0x89, 0xae, 0x1c, 0x42, 0x86, 0x7f, 0x19, 0x46, 0x0f, 0x20, 0x6b, 0xee, 0x93, 0x19, 0xcb, + 0x16, 0xe7, 0xd9, 0xe7, 0x4f, 0x48, 0xd2, 0xfd, 0x62, 0x9e, 0xc0, 0x19, 0x0a, 0x36, 0xe6, 0xe2, + 0xc5, 0xf8, 0xab, 0xc5, 0x8b, 0x95, 0x3f, 0x5b, 0x80, 0x14, 0xbd, 0x7f, 0xca, 0x6f, 0x94, 0xc5, + 0x22, 0x77, 0x14, 0xf9, 0xcd, 0xd4, 0x85, 0xd0, 0xf4, 0xa7, 0xe7, 0xce, 0x3b, 0xc7, 0x23, 0x41, + 0xb7, 0x7a, 0xde, 0x59, 0x06, 0xdd, 0xa7, 0x9e, 0x79, 0x7e, 0xa4, 0x54, 0x37, 0x11, 0xbd, 0x21, + 0xc9, 0xab, 0xbb, 0xcc, 0x65, 0x2c, 0x56, 0x59, 0x3d, 0xd5, 0x9a, 0x3c, 0xff, 0x54, 0xeb, 0xab, + 0x1e, 0xfe, 0x36, 0xa0, 0xa0, 0x9c, 0x46, 0x27, 0xe1, 0x63, 0xe2, 0xe4, 0xe3, 0xe8, 0xb2, 0x76, + 0x21, 0xcd, 0xc7, 0xf9, 0xf0, 0x3c, 0xfa, 0xfc, 0x01, 0xd9, 0xcc, 0x2b, 0x1f, 0x90, 0xbd, 0x01, + 0x89, 0x99, 0x37, 0xe6, 0x67, 0xcd, 0x40, 0x6c, 0xb8, 0xe3, 0x06, 0x3b, 0x44, 0xdc, 0xc7, 0x0d, + 0x4c, 0xf2, 0x4f, 0xb8, 0xb3, 0x98, 0x7b, 0x8d, 0x3b, 0x8b, 0x27, 0x9d, 0x12, 0x83, 0xaf, 0x74, + 0x4a, 0xec, 0x9f, 0x64, 0x20, 0xd7, 0x9e, 0x5a, 0x7c, 0x11, 0x70, 0x3f, 0x72, 0x1a, 0xf7, 0x8a, + 0xea, 0x4c, 0x77, 0x24, 0x6a, 0xf1, 0x74, 0xe4, 0x7d, 0xb2, 0x76, 0x99, 0x39, 0x43, 0x71, 0x56, + 0xe9, 0xad, 0x53, 0x18, 0x1b, 0x14, 0x84, 0x39, 0x18, 0x7d, 0x0c, 0x19, 0x76, 0xd4, 0x4e, 0x1c, + 0xd4, 0x5a, 0x3b, 0x85, 0x8f, 0x9d, 0x50, 0xb3, 0xb0, 0x80, 0xa3, 0x4f, 0x20, 0x37, 0x73, 0x04, + 0x6f, 0x32, 0x72, 0xed, 0x69, 0x9e, 0xb7, 0x2f, 0x70, 0x38, 0x64, 0x21, 0x0a, 0x0f, 0xe9, 0x79, + 0x05, 0x7e, 0x08, 0xf1, 0x34, 0x85, 0xd9, 0xa1, 0x06, 0xcc, 0xc1, 0x44, 0xe1, 0x91, 0xe5, 0x07, + 0x9e, 0xfb, 0x92, 0x1f, 0xc4, 0x3c, 0x4d, 0xe1, 0x3a, 0x43, 0x61, 0x01, 0x47, 0x3a, 0x14, 0x58, + 0xa5, 0x07, 0xfb, 0x9e, 0x3b, 0x9b, 0xf2, 0xf3, 0x8b, 0x95, 0x33, 0xed, 0xb4, 0x4d, 0x90, 0x38, + 0x3f, 0x0e, 0x13, 0xe5, 0x4f, 0x20, 0xcd, 0xf2, 0xd0, 0x47, 0x7c, 0x31, 0x40, 0x9a, 0x5e, 0xec, + 0x35, 0x2c, 0xab, 0xcb, 0xd2, 0x70, 0x99, 0x4f, 0xd7, 0x08, 0x24, 0xed, 0x97, 0xff, 0x28, 0xe4, + 0x15, 0xd9, 0xe8, 0x3e, 0x64, 0x85, 0xcb, 0x9f, 0xbf, 0x5b, 0x20, 0xa1, 0x64, 0xd1, 0x4f, 0xcb, + 0x66, 0x55, 0x61, 0x43, 0xc8, 0x45, 0xa5, 0x6c, 0x2a, 0x7c, 0x5e, 0x01, 0x51, 0x81, 0x0c, 0x6f, + 0xcc, 0x68, 0x07, 0x8f, 0xbd, 0x5a, 0x07, 0x2f, 0xff, 0x14, 0xe4, 0x64, 0x83, 0x7e, 0x35, 0x09, + 0x0f, 0x21, 0xcd, 0x5a, 0x15, 0xdd, 0x85, 0x0c, 0x3b, 0x30, 0x78, 0x0e, 0xb3, 0x40, 0x95, 0x1f, + 0x41, 0x86, 0x37, 0xec, 0x6b, 0xf3, 0x56, 0xc6, 0x27, 0x9d, 0x55, 0x07, 0x48, 0x37, 0xaa, 0xfd, + 0x56, 0x6d, 0x47, 0x8b, 0x21, 0x0d, 0x0a, 0xec, 0xf7, 0x60, 0x1b, 0xb7, 0xfb, 0x1d, 0x2d, 0x4d, + 0xa0, 0x58, 0xef, 0xea, 0xf8, 0x89, 0xae, 0xc5, 0x51, 0x11, 0x72, 0xfd, 0x96, 0x48, 0x26, 0xe8, + 0x8d, 0x08, 0xac, 0x57, 0x7b, 0xba, 0x96, 0x64, 0x57, 0x25, 0xba, 0x3d, 0xdc, 0xfe, 0x5c, 0x4b, + 0xf1, 0xe3, 0xee, 0xff, 0x32, 0x0e, 0x05, 0xc3, 0x39, 0xb4, 0x3c, 0xdf, 0x62, 0x73, 0xc4, 0xdd, + 0x33, 0xe6, 0x08, 0x39, 0x4a, 0x73, 0x02, 0x9d, 0x28, 0xf8, 0x30, 0x16, 0x3f, 0x67, 0x18, 0x9b, + 0x9f, 0x4f, 0x12, 0x7f, 0x88, 0xf9, 0x44, 0x9d, 0x3e, 0x93, 0xaf, 0x31, 0x7d, 0xd6, 0x16, 0x86, + 0xd2, 0x14, 0xd5, 0xe2, 0xe4, 0xa1, 0x94, 0x37, 0xcc, 0xfc, 0x80, 0x1a, 0xf1, 0xa5, 0xf4, 0x2b, + 0xce, 0xc1, 0xc7, 0x49, 0xc8, 0x8a, 0xce, 0x76, 0xc6, 0xd6, 0xd1, 0x03, 0xc8, 0xb0, 0xbe, 0x7a, + 0xca, 0xfe, 0xd1, 0x12, 0x37, 0x0d, 0x7f, 0x01, 0x02, 0xa7, 0x69, 0x87, 0xfd, 0xc3, 0x4d, 0xb0, + 0x91, 0xfa, 0x24, 0x5f, 0x71, 0xfa, 0xbc, 0xab, 0x8c, 0x07, 0xa9, 0xc8, 0x57, 0x6f, 0x75, 0x3c, + 0x50, 0x46, 0x02, 0x71, 0x6b, 0x2b, 0xad, 0xdc, 0xda, 0x7a, 0xbd, 0x0b, 0x58, 0xf7, 0xa1, 0x70, + 0x40, 0xef, 0x45, 0x0e, 0xe8, 0x15, 0xf5, 0xb9, 0x0b, 0x58, 0xca, 0x95, 0x49, 0x9c, 0x3f, 0x50, + 0xae, 0xbc, 0x46, 0xee, 0x4f, 0xe5, 0x5e, 0xed, 0xfe, 0x54, 0xb8, 0xc3, 0x06, 0x67, 0x5f, 0x61, + 0x52, 0xae, 0x59, 0xe5, 0x5f, 0xed, 0x9a, 0xd5, 0x06, 0xe4, 0x9f, 0xd9, 0xe3, 0xf1, 0x60, 0x4a, + 0x2f, 0x37, 0xf2, 0xcb, 0x59, 0x22, 0xec, 0x08, 0x6f, 0x3d, 0x62, 0x78, 0x16, 0xde, 0x80, 0xbc, + 0x09, 0x29, 0x56, 0xe5, 0x62, 0xe4, 0x1c, 0xbc, 0xbc, 0x85, 0x89, 0x59, 0x76, 0xe5, 0x5b, 0x50, + 0x8c, 0x0c, 0xaa, 0x74, 0x8b, 0x98, 0x6e, 0x76, 0x9e, 0x39, 0xea, 0x33, 0x4c, 0xe5, 0x9f, 0xa7, + 0xce, 0xdd, 0xd9, 0xfc, 0xaa, 0xee, 0xf9, 0x7f, 0xb3, 0xef, 0x6f, 0x46, 0xaf, 0x11, 0x26, 0x23, + 0x06, 0x54, 0xe2, 0xb6, 0xa5, 0x33, 0xae, 0x10, 0xaa, 0xdd, 0x25, 0xf5, 0x9a, 0xdd, 0xe5, 0x26, + 0x7d, 0x4c, 0x23, 0xb0, 0x4a, 0x69, 0x7a, 0x4c, 0x5b, 0x53, 0x2c, 0xd0, 0x25, 0x74, 0xcc, 0xb2, + 0xa3, 0xdd, 0x2a, 0xf3, 0x8a, 0xdd, 0xea, 0x43, 0xe5, 0xea, 0x6d, 0x36, 0x12, 0x91, 0x0a, 0xf9, + 0x33, 0xf1, 0x71, 0x47, 0x02, 0xd1, 0x4f, 0xc1, 0x0a, 0xfb, 0x3d, 0x98, 0x4d, 0x47, 0x66, 0x60, + 0x0d, 0x98, 0x7e, 0x39, 0x1a, 0x99, 0x2d, 0xea, 0x77, 0x81, 0x81, 0xfb, 0x14, 0x4b, 0x49, 0x68, + 0x13, 0x50, 0x54, 0xc2, 0x6c, 0x66, 0xb3, 0x9b, 0x08, 0x05, 0x76, 0x06, 0xa3, 0xab, 0xb0, 0xf4, + 0xfb, 0x46, 0x1d, 0x6b, 0xaa, 0x90, 0xfe, 0xcc, 0x1e, 0xbd, 0xea, 0x2b, 0x0b, 0x5f, 0xe5, 0x6a, + 0x62, 0xa4, 0x0b, 0x17, 0x5f, 0xad, 0x0b, 0x8b, 0x7b, 0xb6, 0x4b, 0xe1, 0x3d, 0xdb, 0xca, 0xcf, + 0xc7, 0x61, 0x99, 0x76, 0x1a, 0x56, 0x1d, 0xda, 0x45, 0xde, 0x7d, 0x85, 0x77, 0x0e, 0xe4, 0x29, + 0xfb, 0xb9, 0x17, 0x0e, 0xd6, 0x54, 0x74, 0x28, 0x73, 0xf1, 0x9d, 0x83, 0x87, 0x91, 0x77, 0x0e, + 0xae, 0x9c, 0xc2, 0x76, 0xd2, 0x6b, 0x07, 0xe5, 0x9b, 0xe1, 0x43, 0x03, 0x57, 0x20, 0x67, 0xbd, + 0xa0, 0x1f, 0x59, 0x47, 0x4c, 0xdf, 0x14, 0x19, 0x5f, 0xed, 0xa0, 0xe6, 0x8e, 0xac, 0xf2, 0x3b, + 0xfc, 0x45, 0x81, 0x75, 0xc8, 0xf3, 0xa6, 0x94, 0xb0, 0x22, 0xfb, 0x98, 0x32, 0xf3, 0x09, 0xb0, + 0xf2, 0x97, 0x8b, 0x00, 0xa1, 0x33, 0xa9, 0x5d, 0x3a, 0xf6, 0x3a, 0x5d, 0x5a, 0x76, 0x83, 0xf8, + 0xd9, 0xdd, 0x40, 0x8c, 0xfb, 0x09, 0x65, 0xdc, 0x2f, 0x41, 0x66, 0x62, 0xf9, 0xbe, 0xb9, 0xcf, + 0x22, 0xf2, 0x1c, 0x16, 0xc9, 0xc8, 0xc4, 0x9e, 0x7a, 0x8d, 0x89, 0xfd, 0xaa, 0xfa, 0x61, 0x23, + 0xcd, 0xf7, 0xbb, 0xe5, 0x27, 0x8d, 0xb9, 0x21, 0x23, 0xf3, 0x55, 0x86, 0x8c, 0x12, 0x64, 0xd8, + 0xac, 0xf2, 0x92, 0x4e, 0x3c, 0x59, 0x2c, 0x92, 0xe8, 0x9e, 0xbc, 0x43, 0x92, 0x8b, 0x5c, 0x65, + 0x0d, 0x8d, 0x3c, 0x77, 0x79, 0x84, 0x70, 0x78, 0x96, 0xe9, 0xbb, 0x0e, 0xed, 0x62, 0x27, 0x72, + 0x60, 0x9a, 0x8f, 0x39, 0x0e, 0x5d, 0x85, 0x24, 0xed, 0x92, 0x79, 0xda, 0x25, 0xa9, 0x8f, 0xd0, + 0x6e, 0x48, 0xa9, 0x4a, 0xd7, 0x2b, 0x9c, 0xd5, 0xf5, 0xaa, 0x6c, 0x3f, 0x9e, 0x7d, 0x46, 0x63, + 0x1e, 0xc1, 0x7b, 0xd3, 0xea, 0x7c, 0x6f, 0x62, 0x5a, 0xb0, 0x7d, 0x7a, 0x85, 0x80, 0x1e, 0x81, + 0x36, 0x63, 0xa7, 0x6d, 0xe9, 0x67, 0x55, 0x62, 0x62, 0x7e, 0xcb, 0x77, 0xfe, 0x51, 0x21, 0xbc, + 0xac, 0x00, 0x09, 0x11, 0x3d, 0x84, 0x02, 0x9d, 0xa6, 0x44, 0xd1, 0xcb, 0xd1, 0xa2, 0xa3, 0x9d, + 0x01, 0xe7, 0x87, 0x21, 0xa1, 0xb2, 0x29, 0xef, 0xc8, 0x5c, 0x80, 0x22, 0x7b, 0xdc, 0x64, 0xd0, + 0xac, 0x76, 0x7b, 0x3a, 0xd6, 0xde, 0x20, 0xe1, 0x30, 0x27, 0x55, 0xb7, 0xf5, 0x56, 0x4f, 0x8b, + 0xa1, 0x15, 0x58, 0xe6, 0x14, 0xfd, 0x33, 0xbd, 0xd6, 0xef, 0xb5, 0xb1, 0x16, 0xaf, 0xfc, 0xb5, + 0x0c, 0xa4, 0x99, 0x55, 0x51, 0x05, 0xd6, 0xb0, 0x5e, 0xed, 0xb6, 0x5b, 0x03, 0xfe, 0x18, 0x81, + 0xc4, 0x0d, 0xb6, 0xaa, 0x46, 0x43, 0xaf, 0x6b, 0x6f, 0x44, 0x30, 0xad, 0x5e, 0xd5, 0x68, 0xe9, + 0x78, 0xc0, 0xa3, 0x6e, 0x8e, 0xb9, 0x84, 0xd6, 0xe1, 0xca, 0x22, 0xc6, 0x68, 0x1a, 0xbd, 0x6a, + 0xcf, 0x68, 0xb7, 0xb4, 0x15, 0x74, 0x1d, 0xae, 0x9d, 0x01, 0x18, 0xd4, 0x8d, 0xee, 0x63, 0xed, + 0x22, 0xba, 0x09, 0x95, 0xb3, 0x50, 0x4d, 0xbd, 0xd9, 0xc6, 0x9f, 0x6b, 0x59, 0xb4, 0x06, 0xe5, + 0x05, 0x5c, 0x07, 0xeb, 0x7a, 0xb3, 0xd3, 0xd3, 0xeb, 0xda, 0x85, 0x13, 0x55, 0xee, 0x77, 0xea, + 0xd5, 0x9e, 0x2e, 0x54, 0x5e, 0x45, 0xb7, 0xe0, 0x3a, 0xc7, 0xc8, 0x2a, 0x63, 0x7d, 0xdb, 0xe8, + 0xf6, 0x30, 0x2b, 0xac, 0x67, 0x34, 0xf5, 0x76, 0xbf, 0xa7, 0xbd, 0x89, 0x6e, 0xc3, 0xcd, 0x45, + 0xe4, 0x89, 0xd8, 0x92, 0xa2, 0x99, 0xc4, 0xf6, 0x74, 0xdc, 0x34, 0x5a, 0x55, 0xa2, 0x59, 0x0c, + 0x5d, 0x83, 0xab, 0xf3, 0xf9, 0x64, 0x89, 0x42, 0x64, 0xe9, 0x58, 0xaf, 0x6b, 0x71, 0x74, 0x15, + 0x4a, 0x1c, 0xb1, 0x85, 0xab, 0x4d, 0xfd, 0x69, 0x1b, 0x3f, 0x1e, 0x60, 0xbd, 0xd9, 0x7e, 0xa2, + 0xd7, 0xb5, 0x04, 0x69, 0x50, 0x9e, 0xbb, 0x5d, 0x1b, 0xe8, 0x18, 0xb7, 0xb1, 0x96, 0x54, 0x0a, + 0x35, 0x5a, 0x4f, 0xaa, 0x0d, 0xa3, 0x1e, 0xb2, 0x1a, 0x75, 0x2d, 0x85, 0x2e, 0xc3, 0xa5, 0xb9, + 0xfc, 0xf6, 0xd6, 0x96, 0x8e, 0xbb, 0x5a, 0x5a, 0xd1, 0xc7, 0x68, 0x0f, 0xba, 0x4f, 0x8d, 0x5e, + 0x6d, 0x67, 0xb3, 0x5d, 0xc5, 0xc4, 0x0f, 0x0c, 0xa2, 0xf1, 0x15, 0x45, 0x38, 0xf3, 0x33, 0xd2, + 0x56, 0xb5, 0x76, 0xab, 0xa5, 0xd7, 0x48, 0x7e, 0x46, 0x11, 0x8e, 0xf5, 0x5a, 0xbb, 0x55, 0x33, + 0x1a, 0x06, 0x6b, 0xf4, 0x9c, 0x52, 0x15, 0xf9, 0x10, 0xcf, 0x40, 0x2c, 0xe4, 0x10, 0x7a, 0x0b, + 0x2e, 0xf3, 0x5c, 0xea, 0xad, 0x51, 0xb9, 0x80, 0x4a, 0x70, 0x31, 0x92, 0x2d, 0x6c, 0x90, 0x47, + 0x65, 0x58, 0x9d, 0xcb, 0xe9, 0xf6, 0xaa, 0x98, 0x70, 0x15, 0x16, 0xb8, 0x44, 0x71, 0x45, 0xc5, + 0x03, 0xe9, 0x3b, 0x42, 0xb5, 0x1d, 0xbd, 0xf6, 0x98, 0xbe, 0x22, 0xd4, 0xef, 0x72, 0xdf, 0xa8, + 0x6b, 0x57, 0x15, 0xa5, 0x28, 0x8a, 0x2e, 0x2b, 0x85, 0xd5, 0xb4, 0xcb, 0x8a, 0x63, 0x29, 0xd9, + 0xfd, 0x56, 0xb5, 0xdf, 0xdb, 0x69, 0x63, 0xe3, 0x3b, 0x7a, 0x5d, 0x2b, 0xb3, 0x27, 0x8c, 0x42, + 0x8c, 0x60, 0x5e, 0x52, 0xcc, 0x41, 0x33, 0x22, 0x6c, 0xcb, 0xf3, 0x6c, 0x42, 0x71, 0xad, 0xf2, + 0x21, 0x64, 0xb6, 0xec, 0x71, 0x60, 0xd1, 0x0b, 0x00, 0x4b, 0x9e, 0xb5, 0x37, 0xf3, 0xad, 0x41, + 0xf8, 0x10, 0x1c, 0x7d, 0xd9, 0xea, 0x3e, 0x2e, 0xb2, 0x0c, 0xfe, 0x5a, 0x4b, 0xe5, 0x97, 0x62, + 0x90, 0x57, 0xde, 0xe3, 0x40, 0x9f, 0x42, 0xee, 0xd0, 0xf4, 0x6c, 0x32, 0x04, 0x89, 0x18, 0xf8, + 0xca, 0xe2, 0xb3, 0x1d, 0x77, 0x9e, 0x70, 0x8c, 0x88, 0xc7, 0x24, 0x4f, 0xf9, 0x11, 0x64, 0x45, + 0xe6, 0x19, 0x61, 0xb1, 0x7c, 0xe9, 0x23, 0xae, 0xbc, 0xf4, 0x51, 0xf9, 0x14, 0x72, 0xf2, 0xf5, + 0x36, 0xb4, 0x0a, 0x89, 0x67, 0xd6, 0xcb, 0x08, 0x2f, 0x21, 0x84, 0x37, 0x9a, 0xe3, 0x8b, 0x37, + 0x9a, 0x37, 0x01, 0xa4, 0x00, 0x1f, 0x7d, 0x04, 0xb9, 0xa9, 0x48, 0xf1, 0xba, 0x9c, 0xf6, 0x48, + 0x5c, 0x08, 0xac, 0xec, 0x00, 0xd4, 0x3c, 0x6b, 0x64, 0x39, 0x81, 0x6d, 0x8e, 0x51, 0x25, 0x7a, + 0x5f, 0x33, 0x2c, 0x51, 0xb9, 0xb5, 0xb9, 0x0a, 0x69, 0xdf, 0x1a, 0x7a, 0x56, 0xc0, 0x6b, 0xc3, + 0x53, 0x95, 0x1d, 0xc8, 0x87, 0x92, 0x7c, 0xf4, 0x10, 0xf2, 0xc3, 0x30, 0xc9, 0x15, 0x12, 0x73, + 0x6a, 0x08, 0x14, 0xaf, 0xf5, 0x29, 0xd8, 0xca, 0x1e, 0xe4, 0xb0, 0x19, 0x58, 0xec, 0x04, 0xed, + 0x65, 0x48, 0x7c, 0x31, 0xe5, 0x2d, 0xca, 0x76, 0x0c, 0x7e, 0xa6, 0xd3, 0xc5, 0x84, 0x16, 0xd5, + 0x36, 0x7e, 0xb2, 0xb6, 0x65, 0xc8, 0x0e, 0xcd, 0xa9, 0x39, 0x14, 0x4f, 0xb9, 0x25, 0xb1, 0x4c, + 0x57, 0xfe, 0x51, 0x0c, 0x40, 0x16, 0xe4, 0xa3, 0x3b, 0x90, 0xe6, 0x37, 0xf5, 0xa2, 0xd6, 0x93, + 0x10, 0xf1, 0xaa, 0x1f, 0xbf, 0xbd, 0xf7, 0x18, 0x2e, 0x99, 0xfb, 0xfb, 0x9e, 0xb5, 0x4f, 0x02, + 0xe2, 0x11, 0xbb, 0xec, 0x36, 0x20, 0xba, 0xb2, 0xb3, 0xc9, 0x6f, 0x1e, 0x1f, 0xad, 0xaf, 0x54, + 0x05, 0x80, 0x5f, 0x86, 0x23, 0xba, 0xaf, 0x98, 0xf3, 0xc4, 0xa9, 0x8f, 0xbe, 0x05, 0xe5, 0x45, + 0x61, 0x73, 0x9a, 0x97, 0xe6, 0x19, 0x6b, 0xa2, 0x26, 0x3f, 0x89, 0x43, 0xca, 0x98, 0x90, 0x00, + 0xe9, 0xc6, 0x89, 0x6f, 0x12, 0xd0, 0x3c, 0x35, 0x54, 0xbd, 0x01, 0x49, 0x73, 0x3a, 0x1d, 0xf2, + 0x38, 0x35, 0x0a, 0xab, 0x4e, 0xa7, 0x43, 0x4c, 0xb3, 0xd1, 0x7b, 0x90, 0x1e, 0xb9, 0xc3, 0x67, + 0xd6, 0xfc, 0xc5, 0x6d, 0x06, 0xac, 0xd3, 0x2c, 0xcc, 0x21, 0xe8, 0x2a, 0xa4, 0xe9, 0xb7, 0x53, + 0xb6, 0xe6, 0x12, 0xcf, 0xc4, 0x70, 0x5a, 0x79, 0x00, 0x49, 0x22, 0xf8, 0x8c, 0x5e, 0xb2, 0xca, + 0x8f, 0xc5, 0x84, 0xcf, 0xff, 0x89, 0xfd, 0x23, 0x11, 0xbd, 0x24, 0xce, 0xba, 0x26, 0xdc, 0x87, + 0x34, 0x53, 0xe8, 0x8c, 0x22, 0x3e, 0x00, 0x08, 0x1d, 0x6d, 0xae, 0xf2, 0xa1, 0x4f, 0x62, 0x05, + 0x54, 0xa9, 0x84, 0xd7, 0x4f, 0xab, 0x9d, 0x4e, 0x8d, 0x3d, 0x21, 0x52, 0x6f, 0xd7, 0x1e, 0xeb, + 0x58, 0x3e, 0x21, 0xf2, 0x83, 0x34, 0xa4, 0xd9, 0xc5, 0x61, 0xf4, 0x1e, 0x2c, 0x85, 0x31, 0x14, + 0xbd, 0x73, 0xab, 0x6a, 0x51, 0x94, 0x79, 0xf4, 0x5e, 0xec, 0x15, 0xc8, 0x1d, 0xb8, 0x7e, 0x30, + 0x50, 0x5e, 0x5e, 0xa2, 0x5f, 0x2d, 0x68, 0xe6, 0x4d, 0xfe, 0x6e, 0x61, 0x82, 0xb6, 0x24, 0x8a, + 0xdc, 0x4f, 0x56, 0x9f, 0x2c, 0xac, 0x40, 0xca, 0x9e, 0x88, 0x50, 0x39, 0xbf, 0x51, 0x50, 0x9b, + 0x08, 0xb3, 0x2c, 0xf4, 0xbe, 0x0c, 0x41, 0x53, 0x91, 0xd5, 0x11, 0x97, 0x36, 0x77, 0x77, 0xf9, + 0x2f, 0x26, 0x65, 0x38, 0xf5, 0x61, 0x64, 0xf5, 0x73, 0xf9, 0x24, 0xb6, 0xc5, 0x3d, 0xfc, 0x2d, + 0x28, 0x32, 0x97, 0x18, 0x44, 0xae, 0x58, 0xbf, 0x7d, 0x22, 0x37, 0x6b, 0x33, 0x7e, 0xed, 0xba, + 0x30, 0x52, 0x52, 0xa8, 0x06, 0x05, 0xdf, 0x74, 0x46, 0xbb, 0xee, 0x8b, 0x81, 0x7c, 0x0e, 0x33, + 0xdc, 0x9d, 0x8f, 0x8a, 0xe9, 0x32, 0x20, 0xbd, 0xb9, 0x9c, 0xf7, 0xc3, 0x44, 0xf9, 0xbb, 0x50, + 0x50, 0x8b, 0x20, 0xc3, 0xd7, 0xc8, 0xb3, 0x0f, 0xe9, 0x18, 0x49, 0x87, 0x2f, 0x96, 0x3a, 0xe3, + 0x99, 0xc5, 0x8f, 0x61, 0x89, 0x61, 0x06, 0xee, 0x94, 0x9d, 0x3d, 0x49, 0x44, 0x1c, 0x27, 0x1c, + 0x83, 0x71, 0x91, 0x01, 0xdb, 0x0c, 0x57, 0xfe, 0xf5, 0x18, 0xe4, 0x15, 0xc5, 0xd0, 0xa7, 0x11, + 0x6b, 0xde, 0x38, 0xaf, 0x22, 0x8b, 0x96, 0x2d, 0xc9, 0x7b, 0xdc, 0x8a, 0x92, 0xf4, 0xe9, 0xb7, + 0xbb, 0x27, 0x6d, 0x0a, 0x67, 0x21, 0xd9, 0xd5, 0x1b, 0x5b, 0xcc, 0x67, 0x3b, 0x55, 0xac, 0xd3, + 0x3b, 0xd3, 0xcc, 0x67, 0x3f, 0x39, 0x89, 0xe1, 0x02, 0x14, 0x99, 0x6b, 0x0f, 0x9e, 0xb4, 0x1b, + 0xfd, 0xa6, 0xce, 0x36, 0x93, 0xbb, 0xd5, 0x56, 0x7d, 0xb3, 0xfd, 0xd9, 0x80, 0x5e, 0xbf, 0x16, + 0xfc, 0xab, 0xe1, 0x53, 0x97, 0xf8, 0xa9, 0x16, 0xa3, 0x7f, 0xdb, 0x5a, 0xbc, 0xf2, 0xab, 0x49, + 0xc8, 0xb7, 0xac, 0x40, 0x3e, 0x6c, 0xb9, 0x0a, 0x69, 0xba, 0xb9, 0xcf, 0x8e, 0xaf, 0xe5, 0x30, + 0x4f, 0xbd, 0xe2, 0x21, 0x35, 0xd4, 0x83, 0x82, 0x3d, 0x1d, 0xf0, 0xf7, 0x6c, 0xe4, 0xe7, 0xbb, + 0xab, 0x1c, 0xac, 0x14, 0x74, 0xc7, 0xe8, 0x88, 0xd7, 0x54, 0xe5, 0x1e, 0x91, 0x24, 0x59, 0x3e, + 0xce, 0xdb, 0x53, 0x99, 0x90, 0x6f, 0x2c, 0xa5, 0x95, 0x37, 0x96, 0x0c, 0x28, 0x4e, 0x5d, 0x2f, + 0xa0, 0x57, 0x06, 0x6c, 0x67, 0x5f, 0xec, 0xc9, 0xac, 0x9d, 0x50, 0x54, 0xc7, 0xf5, 0x82, 0x26, + 0x83, 0x89, 0xf3, 0x8f, 0xd3, 0x90, 0xe4, 0x97, 0x3d, 0xc8, 0xc9, 0xa2, 0xd1, 0x37, 0x20, 0x4b, + 0x1f, 0x04, 0x1e, 0xba, 0xe3, 0xb9, 0x8f, 0x61, 0x11, 0x91, 0x1c, 0x82, 0x25, 0x98, 0x7e, 0xbc, + 0x97, 0x55, 0xe7, 0xe3, 0x22, 0xfb, 0x78, 0x2f, 0x64, 0xe3, 0x9c, 0xac, 0x54, 0xf9, 0x25, 0xe4, + 0x15, 0xb5, 0xd0, 0xdb, 0x62, 0x68, 0x99, 0x7f, 0x7c, 0x8d, 0x0d, 0x30, 0xae, 0x17, 0xcc, 0x0d, + 0x55, 0x04, 0x17, 0x57, 0x70, 0xca, 0x50, 0x45, 0xc0, 0x65, 0xa5, 0x16, 0xec, 0xbd, 0x32, 0x99, + 0xae, 0xac, 0x41, 0x56, 0xa8, 0x4f, 0x5c, 0xce, 0xe8, 0x1c, 0x7e, 0xc4, 0x5e, 0x6f, 0x33, 0x3a, + 0x87, 0x0f, 0xb4, 0x78, 0xe5, 0x38, 0x05, 0x4b, 0xe1, 0xe3, 0x3b, 0xd4, 0x2b, 0xea, 0x73, 0xaf, + 0x77, 0x92, 0x19, 0x37, 0x7c, 0xb3, 0x27, 0x0a, 0x56, 0x92, 0xd1, 0x87, 0x3b, 0x2b, 0x3f, 0x48, + 0x45, 0x1e, 0x00, 0x9a, 0xfb, 0x20, 0x92, 0xaa, 0xed, 0x90, 0x9f, 0x3f, 0xca, 0xa0, 0x0b, 0x50, + 0xa8, 0x57, 0x6b, 0x83, 0xf6, 0x13, 0x1d, 0x63, 0xa3, 0xae, 0x6b, 0xff, 0x2a, 0x83, 0x2e, 0xc2, + 0x32, 0x21, 0x61, 0xbd, 0x5a, 0x1f, 0x74, 0xf5, 0x2a, 0xae, 0xed, 0x68, 0xff, 0x3a, 0x83, 0xf2, + 0x90, 0xde, 0x6a, 0x3f, 0x6d, 0xe9, 0x58, 0xfb, 0x37, 0x2c, 0xd1, 0xd5, 0x7b, 0x46, 0x5d, 0xfb, + 0xb7, 0x19, 0x94, 0x83, 0xe4, 0x63, 0xa3, 0xd1, 0xd0, 0xfe, 0x1d, 0xa5, 0x77, 0xf5, 0xde, 0xb6, + 0x51, 0xd7, 0xfe, 0xbd, 0x48, 0xf4, 0x8d, 0xba, 0xf6, 0x1f, 0x32, 0xa8, 0x00, 0x99, 0xae, 0xde, + 0xeb, 0xd4, 0xaa, 0x1d, 0xed, 0xc7, 0xb4, 0x88, 0x86, 0xd1, 0xea, 0x7f, 0x36, 0x30, 0x9a, 0xcd, + 0x7e, 0xaf, 0xba, 0xd9, 0xd0, 0xb5, 0xff, 0x98, 0x41, 0x97, 0x40, 0x6b, 0xe9, 0xbd, 0xc1, 0xa6, + 0xd1, 0x22, 0x05, 0xe3, 0x27, 0x46, 0x4d, 0xd7, 0x7e, 0x2f, 0x83, 0x10, 0x14, 0x29, 0x19, 0xb7, + 0xab, 0xf5, 0x5a, 0xb5, 0xdb, 0xd3, 0xfe, 0x53, 0x06, 0x2d, 0x41, 0x8e, 0xd0, 0xaa, 0xf5, 0xa6, + 0xd1, 0xd2, 0xfe, 0x33, 0x15, 0x4f, 0xd2, 0xb8, 0xfa, 0x54, 0xfb, 0x2f, 0x19, 0x54, 0x84, 0xac, + 0xd1, 0xa9, 0x0d, 0x1a, 0xed, 0xda, 0x63, 0xed, 0xbf, 0x52, 0x30, 0x49, 0x32, 0xed, 0x7f, 0x3f, + 0x83, 0x96, 0x01, 0xba, 0x9f, 0x77, 0x07, 0xcd, 0x76, 0xbd, 0xdf, 0xd0, 0xb5, 0x3f, 0xa0, 0x00, + 0x42, 0xc0, 0xd5, 0xa7, 0x46, 0x5b, 0xfb, 0x6f, 0x12, 0x50, 0xdb, 0xc1, 0xed, 0x76, 0x4f, 0xfb, + 0xef, 0x92, 0xd0, 0xe9, 0xe1, 0x6a, 0x4d, 0xd7, 0xfe, 0x87, 0xe4, 0xe8, 0x54, 0x6b, 0xb5, 0x9e, + 0xf6, 0x13, 0x99, 0x66, 0xfa, 0xfc, 0x4f, 0xaa, 0x01, 0x49, 0x6f, 0x12, 0xfe, 0xff, 0x25, 0x93, + 0x2d, 0x52, 0xa3, 0xff, 0x4d, 0x8d, 0x4e, 0xcb, 0xe3, 0x6b, 0x1c, 0xed, 0x7b, 0x59, 0x81, 0x20, + 0xeb, 0x42, 0xed, 0x8f, 0x65, 0xd1, 0x0a, 0x2c, 0xd1, 0x64, 0xef, 0x73, 0xb2, 0x20, 0xdd, 0x32, + 0xb6, 0xb5, 0x9f, 0xcf, 0x92, 0x76, 0x6b, 0x3e, 0x6e, 0xb5, 0xeb, 0xda, 0x1f, 0xa7, 0xbf, 0x1b, + 0x7a, 0xb5, 0xab, 0x6b, 0xbf, 0x90, 0x45, 0x1a, 0xe4, 0xab, 0xfd, 0xba, 0xd1, 0x1b, 0x3c, 0xc5, + 0x46, 0x4f, 0xd7, 0x7e, 0x31, 0x4b, 0x4c, 0xc6, 0x28, 0x64, 0x35, 0x8b, 0xdb, 0x0d, 0xed, 0x4f, + 0x64, 0x79, 0x0b, 0x6c, 0x91, 0x16, 0xf8, 0x93, 0x59, 0xa2, 0x42, 0x53, 0x6d, 0xf7, 0xef, 0x67, + 0x49, 0x1d, 0x08, 0x89, 0xd5, 0xe1, 0x07, 0x59, 0xda, 0x7e, 0x9f, 0x77, 0x1b, 0xed, 0x6d, 0xed, + 0x97, 0xb2, 0xc4, 0x02, 0x4f, 0xab, 0x8f, 0xf5, 0x41, 0xb5, 0x51, 0xc5, 0x4d, 0xed, 0x4f, 0xd1, + 0x22, 0x36, 0x89, 0x81, 0x07, 0xdd, 0x7e, 0xb7, 0xa3, 0xb7, 0xea, 0xda, 0x2f, 0x53, 0x10, 0x2b, + 0x96, 0xf8, 0x8e, 0xf6, 0x2b, 0x59, 0x3e, 0x1e, 0x3e, 0x86, 0x5c, 0xc3, 0x76, 0x66, 0x2f, 0xa8, + 0x7b, 0x7f, 0x02, 0xcb, 0xd2, 0x51, 0x5f, 0x8a, 0x23, 0xd4, 0x91, 0x27, 0x4c, 0x23, 0x1e, 0x8e, + 0x97, 0x86, 0x91, 0x74, 0xe5, 0x87, 0x09, 0x00, 0x4c, 0x63, 0x4e, 0xfe, 0x32, 0x61, 0xc6, 0x8b, + 0x84, 0xa6, 0xf2, 0xb5, 0x1a, 0x89, 0xe1, 0x3f, 0xc5, 0xb7, 0x42, 0x0e, 0x2f, 0x7f, 0x3f, 0x01, + 0x69, 0x96, 0x83, 0xbe, 0x11, 0x99, 0x7c, 0xde, 0x3a, 0x4d, 0xc2, 0xe2, 0xa4, 0x83, 0x20, 0x79, + 0x60, 0x7a, 0x23, 0xfe, 0x10, 0x06, 0xfd, 0x4d, 0x68, 0xbe, 0xbb, 0x17, 0xf0, 0xc0, 0x94, 0xfe, + 0xae, 0xfc, 0xb9, 0xf8, 0x29, 0xaf, 0xaf, 0xe1, 0x46, 0xb3, 0x37, 0xa8, 0x76, 0xb5, 0x18, 0x2a, + 0x42, 0x8e, 0x26, 0x6a, 0x6d, 0xac, 0x6b, 0x71, 0x54, 0x80, 0x2c, 0x4b, 0x76, 0xfa, 0x5a, 0x42, + 0x66, 0xd6, 0xab, 0xbd, 0xaa, 0x96, 0x44, 0x4b, 0xc4, 0x04, 0xcd, 0xde, 0x60, 0xab, 0x6b, 0x7c, + 0x87, 0x3f, 0x4a, 0x4b, 0xd3, 0xa4, 0x3d, 0xc8, 0x1a, 0x5d, 0x83, 0x02, 0x4d, 0x37, 0xf5, 0x26, + 0xed, 0x03, 0xc4, 0xe3, 0x8a, 0x8c, 0xd2, 0xdd, 0xfe, 0x99, 0xbe, 0xde, 0xd7, 0xb5, 0xac, 0x94, + 0x49, 0x9d, 0x32, 0x87, 0x96, 0x21, 0xcf, 0x92, 0xed, 0x2d, 0xa3, 0xa1, 0x6b, 0x20, 0x85, 0xb6, + 0x3a, 0xb8, 0x5d, 0xd3, 0xf2, 0x52, 0x23, 0xdc, 0xed, 0x6a, 0x05, 0x09, 0xc7, 0xbd, 0x0e, 0x36, + 0xda, 0x5a, 0x51, 0x21, 0x50, 0x1f, 0x5e, 0xa2, 0x1b, 0x0f, 0x84, 0xd0, 0x35, 0xb6, 0x89, 0x7f, + 0x18, 0xad, 0x6d, 0x6d, 0x59, 0x0a, 0xed, 0xf6, 0xaa, 0xb5, 0xc7, 0x9a, 0xc6, 0x1d, 0xe4, 0x57, + 0x62, 0x90, 0xe9, 0xf5, 0x3e, 0xa7, 0x0d, 0xfa, 0x08, 0xf2, 0xcf, 0x6d, 0x67, 0xe4, 0x3e, 0x1f, + 0xf8, 0xf6, 0x77, 0xc5, 0x2b, 0x36, 0x22, 0xba, 0xe2, 0xa0, 0x3b, 0x4f, 0x29, 0xa2, 0x6b, 0x7f, + 0xd7, 0xc2, 0xf0, 0x5c, 0xfe, 0x2e, 0x6f, 0x01, 0x84, 0x39, 0xec, 0xdd, 0x94, 0xe7, 0x7e, 0xf4, + 0x7d, 0x4d, 0x42, 0x41, 0x6b, 0x90, 0x19, 0x92, 0xc8, 0xc2, 0xf1, 0x23, 0xe3, 0xba, 0x20, 0x56, + 0x8e, 0xb2, 0x50, 0x8c, 0xec, 0x8e, 0xa3, 0xaf, 0x47, 0xd6, 0x0e, 0x97, 0x4f, 0xda, 0x41, 0x57, + 0xd7, 0x10, 0x5f, 0x0f, 0x3f, 0x5c, 0xb3, 0x93, 0x46, 0xd1, 0x37, 0x74, 0xe6, 0x3e, 0x5b, 0xa3, + 0x6f, 0xcc, 0xad, 0x25, 0xd6, 0x4f, 0x94, 0xcf, 0x62, 0x35, 0xf6, 0x8c, 0x11, 0x5f, 0x57, 0x94, + 0x23, 0x07, 0x7c, 0xa2, 0x8f, 0xe4, 0x7d, 0x04, 0xec, 0x71, 0x7e, 0x1e, 0xd7, 0xae, 0x9d, 0x28, + 0xb3, 0x49, 0x68, 0xec, 0x3b, 0x18, 0xcd, 0x46, 0xdf, 0xa6, 0x97, 0xdc, 0xd8, 0xc7, 0x26, 0x7a, + 0xe8, 0x21, 0x13, 0x79, 0x81, 0x4c, 0x99, 0x97, 0xc5, 0xf4, 0xee, 0x84, 0x24, 0x1f, 0xdd, 0x05, + 0x18, 0x93, 0xae, 0xce, 0x3a, 0x76, 0x36, 0xf2, 0xcd, 0x4d, 0x8e, 0x01, 0x38, 0x37, 0x96, 0xc3, + 0xc1, 0x06, 0xe4, 0x59, 0x87, 0x64, 0x1c, 0xb9, 0x48, 0xf8, 0x18, 0xf6, 0x40, 0x0c, 0x0c, 0x45, + 0x79, 0x1e, 0x40, 0x36, 0x10, 0x63, 0x07, 0x44, 0x76, 0xba, 0xb9, 0x7f, 0xb0, 0x9d, 0x6e, 0x9e, + 0xc0, 0x99, 0x80, 0x0d, 0x1d, 0xe5, 0xdf, 0x4c, 0x02, 0x84, 0x46, 0x44, 0x65, 0xb1, 0x3a, 0x88, + 0xbc, 0x88, 0xc6, 0x56, 0x05, 0xdb, 0x90, 0xe1, 0xf5, 0xe2, 0x8f, 0xa3, 0xbf, 0x73, 0x4e, 0x93, + 0x08, 0xcb, 0x3c, 0x4a, 0xee, 0xb4, 0xbb, 0x3d, 0x2c, 0xb8, 0xd1, 0x93, 0xf9, 0xd0, 0x89, 0xdd, + 0x54, 0x78, 0xef, 0x3c, 0x71, 0xe7, 0xc4, 0x51, 0xe8, 0x06, 0xc0, 0xd4, 0xb3, 0x0f, 0xed, 0xb1, + 0xb5, 0x2f, 0x57, 0x95, 0xe2, 0x19, 0xf2, 0x30, 0x03, 0x3d, 0x00, 0x90, 0x1b, 0x1a, 0xf3, 0x0f, + 0x10, 0xce, 0x6f, 0x7d, 0x28, 0x48, 0x74, 0x0b, 0xb4, 0x3d, 0xd7, 0x1b, 0x5a, 0x83, 0xe9, 0x6c, + 0x3c, 0x1e, 0x30, 0x33, 0xd1, 0x17, 0x56, 0xf1, 0x12, 0xa5, 0x77, 0x66, 0xe3, 0x31, 0x5b, 0x55, + 0xbf, 0x03, 0x45, 0xe6, 0xc6, 0x03, 0xbe, 0x76, 0xc8, 0xc8, 0x67, 0x38, 0x0b, 0x2c, 0xa3, 0x4e, + 0xe9, 0xff, 0x3f, 0xa3, 0xb0, 0x87, 0x90, 0xe1, 0xcd, 0x43, 0x1f, 0xce, 0x6d, 0x77, 0xf9, 0x73, + 0x97, 0x9b, 0xd8, 0xa8, 0x6f, 0xeb, 0xec, 0x5d, 0xbb, 0x56, 0xbb, 0xa5, 0x6b, 0x09, 0xf2, 0xab, + 0xdf, 0xd5, 0xb1, 0x96, 0x64, 0x43, 0x53, 0xf9, 0x2e, 0xe4, 0x64, 0x1f, 0x09, 0xd7, 0x93, 0xb1, + 0x53, 0xd7, 0x93, 0x95, 0xaf, 0x85, 0x6f, 0xe9, 0xf1, 0xa5, 0x30, 0x7b, 0x95, 0x49, 0xef, 0xb6, + 0xbb, 0x72, 0x85, 0xf0, 0xbd, 0x38, 0x2c, 0xcf, 0x7d, 0x30, 0x58, 0xec, 0x79, 0xb1, 0xd7, 0xea, + 0x79, 0x1b, 0x90, 0x1f, 0xd2, 0xf5, 0x03, 0xeb, 0x17, 0x73, 0x0b, 0xf8, 0x7d, 0xf1, 0x5d, 0x1b, + 0xc3, 0x50, 0xfe, 0x46, 0x1b, 0xca, 0x41, 0xbe, 0x29, 0x7f, 0x2f, 0xbf, 0xc8, 0x6e, 0x60, 0x8b, + 0x4f, 0x39, 0x1d, 0xa3, 0x1e, 0x9e, 0xd8, 0xeb, 0xd8, 0xf4, 0x2d, 0xd7, 0xb9, 0x0b, 0x45, 0xa7, + 0xfc, 0x37, 0x0b, 0x26, 0xe7, 0xb4, 0xcb, 0x44, 0x95, 0x3f, 0x02, 0x10, 0x6a, 0x85, 0xbe, 0x4d, + 0xfb, 0xdb, 0x60, 0x38, 0x16, 0x0f, 0x01, 0x94, 0x16, 0x34, 0x27, 0x16, 0xa8, 0x8d, 0xfd, 0x4d, + 0x38, 0x3e, 0x5a, 0x4f, 0x93, 0xdf, 0x8d, 0x2e, 0x4e, 0x3b, 0x94, 0x56, 0xbe, 0x0b, 0x69, 0x96, + 0x8b, 0x6e, 0x40, 0x66, 0x38, 0x36, 0x7d, 0x9f, 0x1f, 0x0e, 0x2d, 0xb2, 0xa1, 0xa0, 0x46, 0x48, + 0x46, 0x1d, 0x8b, 0xbc, 0xca, 0x47, 0x90, 0x66, 0xab, 0x29, 0x74, 0x5b, 0x2e, 0xb6, 0x98, 0xbd, + 0x0b, 0xea, 0x62, 0x4b, 0x6e, 0x6b, 0xb1, 0x6b, 0x41, 0xf7, 0x21, 0x45, 0xc9, 0xa7, 0x6e, 0x49, + 0x9e, 0xbc, 0x9b, 0xf9, 0x77, 0x62, 0x90, 0xa4, 0x3e, 0x7a, 0x15, 0xd2, 0xce, 0x6c, 0xb2, 0xcb, + 0xff, 0x97, 0x81, 0x70, 0x64, 0x4e, 0x93, 0x2b, 0xaf, 0x78, 0xe4, 0x75, 0xdb, 0x53, 0xbd, 0x1a, + 0x7d, 0x0a, 0x70, 0x68, 0xfb, 0x36, 0x3f, 0x8d, 0x93, 0xa4, 0xc3, 0xd4, 0xfa, 0x49, 0xdf, 0x83, + 0xef, 0x3c, 0x91, 0x30, 0xac, 0xb0, 0x28, 0xeb, 0xcc, 0xd4, 0x59, 0x97, 0xa1, 0xee, 0x41, 0x8a, + 0xbd, 0x48, 0xf1, 0x0e, 0xa4, 0xa6, 0xf4, 0xa5, 0x0a, 0x66, 0xa9, 0xbc, 0x18, 0x47, 0x5c, 0x4f, + 0x04, 0x57, 0x2c, 0xbf, 0xf2, 0x0f, 0xe3, 0x50, 0x8c, 0x68, 0x80, 0xf4, 0x88, 0xae, 0x6c, 0x16, + 0x3d, 0x4f, 0x57, 0x31, 0x2c, 0x29, 0x1a, 0x9f, 0x64, 0xa2, 0x6b, 0xd1, 0x17, 0xa6, 0x99, 0x95, + 0x22, 0xaf, 0x49, 0x97, 0x21, 0x2b, 0x4e, 0x63, 0x8a, 0x59, 0x52, 0xa4, 0xd5, 0xa7, 0x64, 0x53, + 0xd1, 0xa7, 0x64, 0x2b, 0xa2, 0xb6, 0xe9, 0x48, 0x67, 0xa7, 0xa6, 0xe0, 0x15, 0x55, 0x2c, 0x98, + 0x39, 0xcb, 0x82, 0x0f, 0x00, 0xc2, 0x6a, 0x91, 0xc0, 0x4b, 0x7e, 0x6d, 0xe1, 0xef, 0x7a, 0x37, + 0xfa, 0xf4, 0x83, 0x1c, 0xfd, 0xaf, 0x18, 0xfa, 0x67, 0x3d, 0x1d, 0xb7, 0xaa, 0x0d, 0x2d, 0x5e, + 0xf9, 0x04, 0xe0, 0xa9, 0x65, 0xef, 0x1f, 0x04, 0xfc, 0xc5, 0xc0, 0xf4, 0x73, 0x9a, 0x8a, 0x5c, + 0x3f, 0xe3, 0x34, 0xf9, 0xec, 0x60, 0x3c, 0x7c, 0x76, 0xb0, 0xf2, 0xbb, 0x31, 0xc8, 0x3f, 0x61, + 0xd5, 0xa1, 0x12, 0xd6, 0xc2, 0xca, 0xaa, 0xae, 0x2b, 0xab, 0x4c, 0x2f, 0x59, 0xd9, 0xe3, 0xd1, + 0x60, 0xc4, 0xbe, 0x4e, 0xd3, 0xf7, 0xf6, 0x28, 0xa5, 0x6e, 0x06, 0x56, 0x98, 0x4d, 0xbf, 0x5d, + 0x26, 0xd8, 0xa7, 0x62, 0x4a, 0xa1, 0x1f, 0x29, 0x65, 0x36, 0x3d, 0x3c, 0x90, 0x54, 0xb8, 0xfb, + 0xbe, 0xe5, 0xa1, 0xaf, 0x41, 0x66, 0xdf, 0x0e, 0x06, 0xfe, 0x81, 0xc9, 0x2c, 0xcd, 0x3a, 0xf2, + 0xb6, 0x1d, 0x74, 0x77, 0xaa, 0x38, 0xbd, 0x6f, 0x07, 0xdd, 0x03, 0x93, 0xc8, 0x20, 0x20, 0x76, + 0xe0, 0x9d, 0xef, 0x41, 0xe4, 0xf6, 0xed, 0x60, 0x93, 0x12, 0xd0, 0x9b, 0x4c, 0x46, 0x60, 0xee, + 0xf3, 0xff, 0x9e, 0x41, 0xf8, 0x7a, 0xe6, 0x7e, 0xe5, 0x01, 0x24, 0xb7, 0xc6, 0xe6, 0xfe, 0x6b, + 0x7f, 0x68, 0xf8, 0xeb, 0x31, 0x48, 0x62, 0xf7, 0xcc, 0x2f, 0x14, 0xa1, 0xd9, 0xe3, 0x27, 0x98, + 0xfd, 0x63, 0x00, 0x79, 0xc2, 0x46, 0x4c, 0xee, 0x27, 0x1d, 0xce, 0xe1, 0xbe, 0x1c, 0x62, 0xbf, + 0xd2, 0xd9, 0xb1, 0xca, 0xc7, 0x90, 0x6e, 0x5a, 0x81, 0x67, 0x0f, 0x5f, 0xb5, 0xa6, 0x31, 0x51, + 0xd3, 0xbf, 0x17, 0x83, 0xec, 0x96, 0xcd, 0x9e, 0x02, 0x94, 0x9b, 0x65, 0xb1, 0xf9, 0xcd, 0x32, + 0xc2, 0xec, 0x8c, 0x6d, 0x87, 0x85, 0x3d, 0x29, 0xcc, 0x12, 0x74, 0x4d, 0x43, 0xa2, 0x71, 0xb1, + 0xa6, 0x21, 0xe1, 0xf5, 0x0d, 0x48, 0x4d, 0xa8, 0x23, 0x24, 0x4f, 0xfe, 0x88, 0xcd, 0x72, 0x09, + 0x2b, 0xdd, 0xab, 0xa5, 0xff, 0x61, 0x80, 0xef, 0xcb, 0x5e, 0x86, 0xc4, 0x8c, 0xbf, 0x6f, 0xcc, + 0x9f, 0x7d, 0xee, 0x1b, 0x75, 0x4c, 0x68, 0x24, 0x6b, 0x9f, 0x9f, 0x33, 0xe0, 0x59, 0xdb, 0x24, + 0x6b, 0xdf, 0x1e, 0xdd, 0xfe, 0x59, 0x48, 0xf3, 0xa9, 0x72, 0x15, 0x50, 0x1d, 0x1b, 0x4f, 0x74, + 0x3c, 0x68, 0xb5, 0xe9, 0x5a, 0x02, 0xb3, 0x6f, 0xa5, 0x08, 0x96, 0x38, 0x1d, 0xf7, 0x5b, 0xfc, + 0x3f, 0xcb, 0x84, 0xb4, 0xea, 0x66, 0x9b, 0xe2, 0x12, 0x0a, 0xad, 0xdb, 0x6b, 0x77, 0x3a, 0x7a, + 0x5d, 0x4b, 0xde, 0xfe, 0xe5, 0x38, 0xe4, 0xe4, 0x09, 0x0c, 0xb2, 0x82, 0xa2, 0x1f, 0xd5, 0xba, + 0xbd, 0xea, 0x36, 0x91, 0x93, 0x26, 0x2b, 0x28, 0x41, 0xc1, 0x3d, 0x42, 0x7a, 0x43, 0x82, 0x44, + 0x61, 0x31, 0x49, 0xe1, 0xff, 0x73, 0x44, 0xcb, 0x4a, 0xb6, 0x2d, 0xa3, 0x65, 0x74, 0x77, 0xe8, + 0xf7, 0xda, 0x65, 0xc8, 0x33, 0x12, 0xfb, 0xb0, 0x9c, 0x90, 0x04, 0xc2, 0x45, 0x74, 0x21, 0xab, + 0x24, 0x4a, 0x60, 0x9f, 0x6b, 0x33, 0x64, 0xc4, 0xa0, 0xe9, 0x06, 0x89, 0x5c, 0x52, 0xb2, 0x94, + 0x3a, 0x66, 0xca, 0xe7, 0xd0, 0x45, 0xd0, 0xf8, 0x37, 0x40, 0xac, 0x57, 0x6b, 0x3b, 0x74, 0x43, + 0x05, 0x24, 0xdb, 0x36, 0x09, 0x6d, 0xf2, 0xe8, 0x32, 0x5c, 0x92, 0xc9, 0xc1, 0xe6, 0xe7, 0x83, + 0x76, 0x47, 0xc7, 0xd5, 0x5e, 0x1b, 0x6b, 0x05, 0x29, 0x51, 0x7e, 0xfc, 0xdc, 0xac, 0xfd, 0xf6, + 0x97, 0x6b, 0x6f, 0xfc, 0xd3, 0x2f, 0xd7, 0xde, 0xf8, 0x17, 0x5f, 0xae, 0xc5, 0x7e, 0xfc, 0xe5, + 0x5a, 0xec, 0x0f, 0xbe, 0x5c, 0x8b, 0x7d, 0xef, 0x78, 0x2d, 0xf6, 0x57, 0x8f, 0xd7, 0x62, 0x7f, + 0xfb, 0x78, 0x2d, 0xf6, 0xf7, 0x8f, 0xd7, 0x62, 0xbf, 0x75, 0xbc, 0x16, 0xfb, 0xed, 0xe3, 0xb5, + 0xd8, 0x8f, 0x8e, 0xd7, 0xde, 0xf8, 0xf1, 0xf1, 0x5a, 0xec, 0x57, 0x7f, 0x67, 0xed, 0x8d, 0xbf, + 0xf0, 0x3b, 0x6b, 0xb1, 0xef, 0xb0, 0xc5, 0xc4, 0xff, 0x09, 0x00, 0x00, 0xff, 0xff, 0x13, 0xff, + 0x56, 0x17, 0x2b, 0x6c, 0x00, 0x00, } diff --git a/api/v1/lib/mesos.pb_ffjson.go b/api/v1/lib/mesos.pb_ffjson.go index 6155cb94..1ae1a754 100644 --- a/api/v1/lib/mesos.pb_ffjson.go +++ b/api/v1/lib/mesos.pb_ffjson.go @@ -9859,17 +9859,21 @@ func (mj *ExecutorInfo) MarshalJSONBuf(buf fflib.EncodingBuffer) error { buf.WriteByte(',') } } - buf.WriteString(`"command":`) + if mj.Command != nil { + if true { + buf.WriteString(`"command":`) - { + { - err = mj.Command.MarshalJSONBuf(buf) - if err != nil { - return err - } + err = mj.Command.MarshalJSONBuf(buf) + if err != nil { + return err + } + } + buf.WriteByte(',') + } } - buf.WriteByte(',') if mj.Container != nil { if true { buf.WriteString(`"container":`) @@ -10407,10 +10411,16 @@ handle_Command: { if tok == fflib.FFTok_null { + uj.Command = nil + state = fflib.FFParse_after_value goto mainparse } + if uj.Command == nil { + uj.Command = new(CommandInfo) + } + err = uj.Command.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) if err != nil { return err diff --git a/api/v1/lib/mesos.proto b/api/v1/lib/mesos.proto index 0bf14503..f99fb958 100644 --- a/api/v1/lib/mesos.proto +++ b/api/v1/lib/mesos.proto @@ -707,7 +707,7 @@ message ExecutorInfo { required ExecutorID executor_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "ExecutorID"]; optional FrameworkID framework_id = 8 [(gogoproto.customname) = "FrameworkID"]; // TODO(benh): Make this required. - required CommandInfo command = 7 [(gogoproto.nullable) = false]; + optional CommandInfo command = 7; // Executor provided with a container will launch the container // with the executor's CommandInfo and we expect the container to diff --git a/api/v1/lib/quota/quota.pb.go b/api/v1/lib/quota/quota.pb.go new file mode 100644 index 00000000..e713679c --- /dev/null +++ b/api/v1/lib/quota/quota.pb.go @@ -0,0 +1,1347 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: quota/quota.proto + +/* + Package quota is a generated protocol buffer package. + + It is generated from these files: + quota/quota.proto + + It has these top-level messages: + QuotaInfo + QuotaRequest + QuotaStatus +*/ +package quota + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" +import mesos "github.com/mesos/mesos-go/api/v1/lib" +import _ "github.com/gogo/protobuf/gogoproto" + +import strings "strings" +import reflect "reflect" + +import io "io" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +// TODO(joerg84): Add limits, i.e. upper bounds of resources that a +// role is allowed to use. +type QuotaInfo struct { + // Quota is granted per role and not per framework, similar to + // dynamic reservations. + Role *string `protobuf:"bytes,1,opt,name=role" json:"role,omitempty"` + // Principal which set the quota. Currently only operators can set quotas. + Principal *string `protobuf:"bytes,2,opt,name=principal" json:"principal,omitempty"` + // The guarantee that these resources are allocatable for the above role. + // NOTE: `guarantee.role` should not specify any role except '*', + // because quota does not reserve specific resources. + Guarantee []mesos.Resource `protobuf:"bytes,3,rep,name=guarantee" json:"guarantee"` +} + +func (m *QuotaInfo) Reset() { *m = QuotaInfo{} } +func (*QuotaInfo) ProtoMessage() {} +func (*QuotaInfo) Descriptor() ([]byte, []int) { return fileDescriptorQuota, []int{0} } + +func (m *QuotaInfo) GetRole() string { + if m != nil && m.Role != nil { + return *m.Role + } + return "" +} + +func (m *QuotaInfo) GetPrincipal() string { + if m != nil && m.Principal != nil { + return *m.Principal + } + return "" +} + +func (m *QuotaInfo) GetGuarantee() []mesos.Resource { + if m != nil { + return m.Guarantee + } + return nil +} + +// * +// `QuotaRequest` provides a schema for set quota JSON requests. +type QuotaRequest struct { + // Disables the capacity heuristic check if set to `true`. + Force *bool `protobuf:"varint,1,opt,name=force,def=0" json:"force,omitempty"` + // The role for which to set quota. + Role *string `protobuf:"bytes,2,opt,name=role" json:"role,omitempty"` + // The requested guarantee that these resources will be allocatable for + // the above role. + Guarantee []mesos.Resource `protobuf:"bytes,3,rep,name=guarantee" json:"guarantee"` +} + +func (m *QuotaRequest) Reset() { *m = QuotaRequest{} } +func (*QuotaRequest) ProtoMessage() {} +func (*QuotaRequest) Descriptor() ([]byte, []int) { return fileDescriptorQuota, []int{1} } + +const Default_QuotaRequest_Force bool = false + +func (m *QuotaRequest) GetForce() bool { + if m != nil && m.Force != nil { + return *m.Force + } + return Default_QuotaRequest_Force +} + +func (m *QuotaRequest) GetRole() string { + if m != nil && m.Role != nil { + return *m.Role + } + return "" +} + +func (m *QuotaRequest) GetGuarantee() []mesos.Resource { + if m != nil { + return m.Guarantee + } + return nil +} + +// * +// `QuotaStatus` describes the internal representation for the /quota/status +// response. +type QuotaStatus struct { + // Quotas which are currently set, i.e. known to the master. + Infos []QuotaInfo `protobuf:"bytes,1,rep,name=infos" json:"infos"` +} + +func (m *QuotaStatus) Reset() { *m = QuotaStatus{} } +func (*QuotaStatus) ProtoMessage() {} +func (*QuotaStatus) Descriptor() ([]byte, []int) { return fileDescriptorQuota, []int{2} } + +func (m *QuotaStatus) GetInfos() []QuotaInfo { + if m != nil { + return m.Infos + } + return nil +} + +func init() { + proto.RegisterType((*QuotaInfo)(nil), "mesos.quota.QuotaInfo") + proto.RegisterType((*QuotaRequest)(nil), "mesos.quota.QuotaRequest") + proto.RegisterType((*QuotaStatus)(nil), "mesos.quota.QuotaStatus") +} +func (this *QuotaInfo) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*QuotaInfo) + if !ok { + that2, ok := that.(QuotaInfo) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *QuotaInfo") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *QuotaInfo but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *QuotaInfo but is not nil && this == nil") + } + if this.Role != nil && that1.Role != nil { + if *this.Role != *that1.Role { + return fmt.Errorf("Role this(%v) Not Equal that(%v)", *this.Role, *that1.Role) + } + } else if this.Role != nil { + return fmt.Errorf("this.Role == nil && that.Role != nil") + } else if that1.Role != nil { + return fmt.Errorf("Role this(%v) Not Equal that(%v)", this.Role, that1.Role) + } + if this.Principal != nil && that1.Principal != nil { + if *this.Principal != *that1.Principal { + return fmt.Errorf("Principal this(%v) Not Equal that(%v)", *this.Principal, *that1.Principal) + } + } else if this.Principal != nil { + return fmt.Errorf("this.Principal == nil && that.Principal != nil") + } else if that1.Principal != nil { + return fmt.Errorf("Principal this(%v) Not Equal that(%v)", this.Principal, that1.Principal) + } + if len(this.Guarantee) != len(that1.Guarantee) { + return fmt.Errorf("Guarantee this(%v) Not Equal that(%v)", len(this.Guarantee), len(that1.Guarantee)) + } + for i := range this.Guarantee { + if !this.Guarantee[i].Equal(&that1.Guarantee[i]) { + return fmt.Errorf("Guarantee this[%v](%v) Not Equal that[%v](%v)", i, this.Guarantee[i], i, that1.Guarantee[i]) + } + } + return nil +} +func (this *QuotaInfo) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*QuotaInfo) + if !ok { + that2, ok := that.(QuotaInfo) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Role != nil && that1.Role != nil { + if *this.Role != *that1.Role { + return false + } + } else if this.Role != nil { + return false + } else if that1.Role != nil { + return false + } + if this.Principal != nil && that1.Principal != nil { + if *this.Principal != *that1.Principal { + return false + } + } else if this.Principal != nil { + return false + } else if that1.Principal != nil { + return false + } + if len(this.Guarantee) != len(that1.Guarantee) { + return false + } + for i := range this.Guarantee { + if !this.Guarantee[i].Equal(&that1.Guarantee[i]) { + return false + } + } + return true +} +func (this *QuotaRequest) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*QuotaRequest) + if !ok { + that2, ok := that.(QuotaRequest) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *QuotaRequest") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *QuotaRequest but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *QuotaRequest but is not nil && this == nil") + } + if this.Force != nil && that1.Force != nil { + if *this.Force != *that1.Force { + return fmt.Errorf("Force this(%v) Not Equal that(%v)", *this.Force, *that1.Force) + } + } else if this.Force != nil { + return fmt.Errorf("this.Force == nil && that.Force != nil") + } else if that1.Force != nil { + return fmt.Errorf("Force this(%v) Not Equal that(%v)", this.Force, that1.Force) + } + if this.Role != nil && that1.Role != nil { + if *this.Role != *that1.Role { + return fmt.Errorf("Role this(%v) Not Equal that(%v)", *this.Role, *that1.Role) + } + } else if this.Role != nil { + return fmt.Errorf("this.Role == nil && that.Role != nil") + } else if that1.Role != nil { + return fmt.Errorf("Role this(%v) Not Equal that(%v)", this.Role, that1.Role) + } + if len(this.Guarantee) != len(that1.Guarantee) { + return fmt.Errorf("Guarantee this(%v) Not Equal that(%v)", len(this.Guarantee), len(that1.Guarantee)) + } + for i := range this.Guarantee { + if !this.Guarantee[i].Equal(&that1.Guarantee[i]) { + return fmt.Errorf("Guarantee this[%v](%v) Not Equal that[%v](%v)", i, this.Guarantee[i], i, that1.Guarantee[i]) + } + } + return nil +} +func (this *QuotaRequest) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*QuotaRequest) + if !ok { + that2, ok := that.(QuotaRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if this.Force != nil && that1.Force != nil { + if *this.Force != *that1.Force { + return false + } + } else if this.Force != nil { + return false + } else if that1.Force != nil { + return false + } + if this.Role != nil && that1.Role != nil { + if *this.Role != *that1.Role { + return false + } + } else if this.Role != nil { + return false + } else if that1.Role != nil { + return false + } + if len(this.Guarantee) != len(that1.Guarantee) { + return false + } + for i := range this.Guarantee { + if !this.Guarantee[i].Equal(&that1.Guarantee[i]) { + return false + } + } + return true +} +func (this *QuotaStatus) VerboseEqual(that interface{}) error { + if that == nil { + if this == nil { + return nil + } + return fmt.Errorf("that == nil && this != nil") + } + + that1, ok := that.(*QuotaStatus) + if !ok { + that2, ok := that.(QuotaStatus) + if ok { + that1 = &that2 + } else { + return fmt.Errorf("that is not of type *QuotaStatus") + } + } + if that1 == nil { + if this == nil { + return nil + } + return fmt.Errorf("that is type *QuotaStatus but is nil && this != nil") + } else if this == nil { + return fmt.Errorf("that is type *QuotaStatus but is not nil && this == nil") + } + if len(this.Infos) != len(that1.Infos) { + return fmt.Errorf("Infos this(%v) Not Equal that(%v)", len(this.Infos), len(that1.Infos)) + } + for i := range this.Infos { + if !this.Infos[i].Equal(&that1.Infos[i]) { + return fmt.Errorf("Infos this[%v](%v) Not Equal that[%v](%v)", i, this.Infos[i], i, that1.Infos[i]) + } + } + return nil +} +func (this *QuotaStatus) Equal(that interface{}) bool { + if that == nil { + if this == nil { + return true + } + return false + } + + that1, ok := that.(*QuotaStatus) + if !ok { + that2, ok := that.(QuotaStatus) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + if this == nil { + return true + } + return false + } else if this == nil { + return false + } + if len(this.Infos) != len(that1.Infos) { + return false + } + for i := range this.Infos { + if !this.Infos[i].Equal(&that1.Infos[i]) { + return false + } + } + return true +} +func (this *QuotaInfo) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, ""a.QuotaInfo{") + if this.Role != nil { + s = append(s, "Role: "+valueToGoStringQuota(this.Role, "string")+",\n") + } + if this.Principal != nil { + s = append(s, "Principal: "+valueToGoStringQuota(this.Principal, "string")+",\n") + } + if this.Guarantee != nil { + s = append(s, "Guarantee: "+fmt.Sprintf("%#v", this.Guarantee)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *QuotaRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, ""a.QuotaRequest{") + if this.Force != nil { + s = append(s, "Force: "+valueToGoStringQuota(this.Force, "bool")+",\n") + } + if this.Role != nil { + s = append(s, "Role: "+valueToGoStringQuota(this.Role, "string")+",\n") + } + if this.Guarantee != nil { + s = append(s, "Guarantee: "+fmt.Sprintf("%#v", this.Guarantee)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *QuotaStatus) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, ""a.QuotaStatus{") + if this.Infos != nil { + s = append(s, "Infos: "+fmt.Sprintf("%#v", this.Infos)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringQuota(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} +func (m *QuotaInfo) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *QuotaInfo) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Role != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintQuota(dAtA, i, uint64(len(*m.Role))) + i += copy(dAtA[i:], *m.Role) + } + if m.Principal != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintQuota(dAtA, i, uint64(len(*m.Principal))) + i += copy(dAtA[i:], *m.Principal) + } + if len(m.Guarantee) > 0 { + for _, msg := range m.Guarantee { + dAtA[i] = 0x1a + i++ + i = encodeVarintQuota(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *QuotaRequest) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *QuotaRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Force != nil { + dAtA[i] = 0x8 + i++ + if *m.Force { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.Role != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintQuota(dAtA, i, uint64(len(*m.Role))) + i += copy(dAtA[i:], *m.Role) + } + if len(m.Guarantee) > 0 { + for _, msg := range m.Guarantee { + dAtA[i] = 0x1a + i++ + i = encodeVarintQuota(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *QuotaStatus) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *QuotaStatus) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Infos) > 0 { + for _, msg := range m.Infos { + dAtA[i] = 0xa + i++ + i = encodeVarintQuota(dAtA, i, uint64(msg.ProtoSize())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func encodeFixed64Quota(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 encodeFixed32Quota(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 encodeVarintQuota(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func NewPopulatedQuotaInfo(r randyQuota, easy bool) *QuotaInfo { + this := &QuotaInfo{} + if r.Intn(10) != 0 { + v1 := string(randStringQuota(r)) + this.Role = &v1 + } + if r.Intn(10) != 0 { + v2 := string(randStringQuota(r)) + this.Principal = &v2 + } + if r.Intn(10) != 0 { + v3 := r.Intn(5) + this.Guarantee = make([]mesos.Resource, v3) + for i := 0; i < v3; i++ { + v4 := mesos.NewPopulatedResource(r, easy) + this.Guarantee[i] = *v4 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedQuotaRequest(r randyQuota, easy bool) *QuotaRequest { + this := &QuotaRequest{} + if r.Intn(10) != 0 { + v5 := bool(bool(r.Intn(2) == 0)) + this.Force = &v5 + } + if r.Intn(10) != 0 { + v6 := string(randStringQuota(r)) + this.Role = &v6 + } + if r.Intn(10) != 0 { + v7 := r.Intn(5) + this.Guarantee = make([]mesos.Resource, v7) + for i := 0; i < v7; i++ { + v8 := mesos.NewPopulatedResource(r, easy) + this.Guarantee[i] = *v8 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedQuotaStatus(r randyQuota, easy bool) *QuotaStatus { + this := &QuotaStatus{} + if r.Intn(10) != 0 { + v9 := r.Intn(5) + this.Infos = make([]QuotaInfo, v9) + for i := 0; i < v9; i++ { + v10 := NewPopulatedQuotaInfo(r, easy) + this.Infos[i] = *v10 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +type randyQuota interface { + Float32() float32 + Float64() float64 + Int63() int64 + Int31() int32 + Uint32() uint32 + Intn(n int) int +} + +func randUTF8RuneQuota(r randyQuota) rune { + ru := r.Intn(62) + if ru < 10 { + return rune(ru + 48) + } else if ru < 36 { + return rune(ru + 55) + } + return rune(ru + 61) +} +func randStringQuota(r randyQuota) string { + v11 := r.Intn(100) + tmps := make([]rune, v11) + for i := 0; i < v11; i++ { + tmps[i] = randUTF8RuneQuota(r) + } + return string(tmps) +} +func randUnrecognizedQuota(r randyQuota, maxFieldNumber int) (dAtA []byte) { + l := r.Intn(5) + for i := 0; i < l; i++ { + wire := r.Intn(4) + if wire == 3 { + wire = 5 + } + fieldNumber := maxFieldNumber + r.Intn(100) + dAtA = randFieldQuota(dAtA, r, fieldNumber, wire) + } + return dAtA +} +func randFieldQuota(dAtA []byte, r randyQuota, fieldNumber int, wire int) []byte { + key := uint32(fieldNumber)<<3 | uint32(wire) + switch wire { + case 0: + dAtA = encodeVarintPopulateQuota(dAtA, uint64(key)) + v12 := r.Int63() + if r.Intn(2) == 0 { + v12 *= -1 + } + dAtA = encodeVarintPopulateQuota(dAtA, uint64(v12)) + case 1: + dAtA = encodeVarintPopulateQuota(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + case 2: + dAtA = encodeVarintPopulateQuota(dAtA, uint64(key)) + ll := r.Intn(100) + dAtA = encodeVarintPopulateQuota(dAtA, uint64(ll)) + for j := 0; j < ll; j++ { + dAtA = append(dAtA, byte(r.Intn(256))) + } + default: + dAtA = encodeVarintPopulateQuota(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + } + return dAtA +} +func encodeVarintPopulateQuota(dAtA []byte, v uint64) []byte { + for v >= 1<<7 { + dAtA = append(dAtA, uint8(uint64(v)&0x7f|0x80)) + v >>= 7 + } + dAtA = append(dAtA, uint8(v)) + return dAtA +} +func (m *QuotaInfo) ProtoSize() (n int) { + var l int + _ = l + if m.Role != nil { + l = len(*m.Role) + n += 1 + l + sovQuota(uint64(l)) + } + if m.Principal != nil { + l = len(*m.Principal) + n += 1 + l + sovQuota(uint64(l)) + } + if len(m.Guarantee) > 0 { + for _, e := range m.Guarantee { + l = e.ProtoSize() + n += 1 + l + sovQuota(uint64(l)) + } + } + return n +} + +func (m *QuotaRequest) ProtoSize() (n int) { + var l int + _ = l + if m.Force != nil { + n += 2 + } + if m.Role != nil { + l = len(*m.Role) + n += 1 + l + sovQuota(uint64(l)) + } + if len(m.Guarantee) > 0 { + for _, e := range m.Guarantee { + l = e.ProtoSize() + n += 1 + l + sovQuota(uint64(l)) + } + } + return n +} + +func (m *QuotaStatus) ProtoSize() (n int) { + var l int + _ = l + if len(m.Infos) > 0 { + for _, e := range m.Infos { + l = e.ProtoSize() + n += 1 + l + sovQuota(uint64(l)) + } + } + return n +} + +func sovQuota(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozQuota(x uint64) (n int) { + return sovQuota(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *QuotaInfo) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QuotaInfo{`, + `Role:` + valueToStringQuota(this.Role) + `,`, + `Principal:` + valueToStringQuota(this.Principal) + `,`, + `Guarantee:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Guarantee), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *QuotaRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QuotaRequest{`, + `Force:` + valueToStringQuota(this.Force) + `,`, + `Role:` + valueToStringQuota(this.Role) + `,`, + `Guarantee:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Guarantee), "Resource", "mesos.Resource", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *QuotaStatus) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QuotaStatus{`, + `Infos:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Infos), "QuotaInfo", "QuotaInfo", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func valueToStringQuota(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *QuotaInfo) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: QuotaInfo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: QuotaInfo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + 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 ErrInvalidLengthQuota + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + s := string(dAtA[iNdEx:postIndex]) + m.Role = &s + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Principal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + 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 ErrInvalidLengthQuota + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + s := string(dAtA[iNdEx:postIndex]) + m.Principal = &s + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Guarantee", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuota + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Guarantee = append(m.Guarantee, mesos.Resource{}) + if err := m.Guarantee[len(m.Guarantee)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuota(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuota + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *QuotaRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: QuotaRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: QuotaRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Force", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.Force = &b + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + 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 ErrInvalidLengthQuota + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + s := string(dAtA[iNdEx:postIndex]) + m.Role = &s + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Guarantee", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuota + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Guarantee = append(m.Guarantee, mesos.Resource{}) + if err := m.Guarantee[len(m.Guarantee)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuota(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuota + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *QuotaStatus) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: QuotaStatus: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: QuotaStatus: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Infos", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuota + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuota + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Infos = append(m.Infos, QuotaInfo{}) + if err := m.Infos[len(m.Infos)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuota(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuota + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipQuota(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowQuota + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowQuota + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowQuota + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthQuota + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowQuota + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipQuota(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthQuota = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowQuota = fmt.Errorf("proto: integer overflow") +) + +func init() { proto.RegisterFile("quota/quota.proto", fileDescriptorQuota) } + +var fileDescriptorQuota = []byte{ + // 331 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x90, 0xbd, 0x4e, 0xeb, 0x40, + 0x10, 0x85, 0x3d, 0x49, 0x2c, 0x5d, 0x6f, 0xae, 0x84, 0x70, 0x81, 0xac, 0x80, 0x86, 0x28, 0x55, + 0x9a, 0xd8, 0x10, 0x3a, 0x3a, 0x42, 0x45, 0x89, 0xe9, 0xe8, 0x1c, 0x6b, 0x6d, 0x2c, 0x39, 0x5e, + 0x67, 0x7f, 0xa8, 0x79, 0x04, 0x1e, 0x81, 0x92, 0x47, 0xa0, 0xa4, 0x4c, 0x99, 0x92, 0x0a, 0xe1, + 0x4d, 0x43, 0x99, 0x92, 0x12, 0xb1, 0x8b, 0x92, 0xd4, 0x34, 0xab, 0x99, 0x6f, 0xe7, 0xcc, 0x39, + 0x1a, 0xb2, 0x3f, 0x57, 0x4c, 0x26, 0x91, 0x79, 0xc3, 0x9a, 0x33, 0xc9, 0xfc, 0xee, 0x8c, 0x0a, + 0x26, 0x42, 0x83, 0x7a, 0x27, 0x79, 0x21, 0xef, 0xd4, 0x34, 0x4c, 0xd9, 0x2c, 0x32, 0xdc, 0xbe, + 0xa3, 0x9c, 0x45, 0x49, 0x5d, 0x44, 0xf7, 0xa7, 0x51, 0x59, 0x4c, 0x2d, 0xb3, 0xf2, 0xde, 0x68, + 0x47, 0x91, 0xb3, 0x9c, 0x45, 0x06, 0x4f, 0x55, 0x66, 0x3a, 0xd3, 0x98, 0xca, 0x8e, 0x0f, 0x38, + 0xf1, 0xae, 0x7f, 0x9c, 0xae, 0xaa, 0x8c, 0xf9, 0x3e, 0xe9, 0x70, 0x56, 0xd2, 0x00, 0xfa, 0x30, + 0xf4, 0x62, 0x53, 0xfb, 0x47, 0xc4, 0xab, 0x79, 0x51, 0xa5, 0x45, 0x9d, 0x94, 0x41, 0xcb, 0x7c, + 0x6c, 0x81, 0x7f, 0x46, 0xbc, 0x5c, 0x25, 0x3c, 0xa9, 0x24, 0xa5, 0x41, 0xbb, 0xdf, 0x1e, 0x76, + 0xc7, 0x7b, 0xa1, 0x8d, 0x13, 0x53, 0xc1, 0x14, 0x4f, 0xe9, 0xa4, 0xb3, 0x78, 0x3f, 0x76, 0xe2, + 0xed, 0xdc, 0x40, 0x92, 0xff, 0xc6, 0x33, 0xa6, 0x73, 0x45, 0x85, 0xf4, 0x0f, 0x89, 0x9b, 0x31, + 0x9e, 0x5a, 0xdf, 0x7f, 0xe7, 0x6e, 0x96, 0x94, 0x82, 0xc6, 0x96, 0x6d, 0x32, 0xb5, 0x76, 0x32, + 0xfd, 0xc9, 0xf5, 0x82, 0x74, 0x8d, 0xeb, 0x8d, 0x4c, 0xa4, 0x12, 0xfe, 0x98, 0xb8, 0x45, 0x95, + 0x31, 0x11, 0x80, 0xd1, 0x1f, 0x84, 0x3b, 0x67, 0x0f, 0x37, 0x27, 0xf9, 0x5d, 0x63, 0x47, 0x27, + 0x97, 0xcb, 0x06, 0x9d, 0xb7, 0x06, 0x9d, 0x8f, 0x06, 0x61, 0xdd, 0x20, 0x7c, 0x35, 0x08, 0x0f, + 0x1a, 0xe1, 0x59, 0x23, 0xbc, 0x68, 0x84, 0x57, 0x8d, 0xb0, 0xd0, 0x08, 0x4b, 0x8d, 0xf0, 0xa9, + 0xd1, 0x59, 0x6b, 0x84, 0xc7, 0x15, 0x3a, 0x4f, 0x2b, 0x84, 0x5b, 0xd7, 0xec, 0xfe, 0x0e, 0x00, + 0x00, 0xff, 0xff, 0xf0, 0x55, 0x30, 0x8e, 0xf3, 0x01, 0x00, 0x00, +} diff --git a/api/v1/lib/quota/quota.pb_ffjson.go b/api/v1/lib/quota/quota.pb_ffjson.go new file mode 100644 index 00000000..62e8265a --- /dev/null +++ b/api/v1/lib/quota/quota.pb_ffjson.go @@ -0,0 +1,1015 @@ +// DO NOT EDIT! +// Code generated by ffjson +// source: quota/quota.pb.go +// DO NOT EDIT! + +package quota + +import ( + "bytes" + "errors" + "fmt" + "github.com/mesos/mesos-go/api/v1/lib" + fflib "github.com/pquerna/ffjson/fflib/v1" +) + +func (mj *QuotaInfo) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *QuotaInfo) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteByte('{') + if mj.Role != nil { + if true { + buf.WriteString(`"role":`) + fflib.WriteJsonString(buf, string(*mj.Role)) + buf.WriteByte(',') + } + } + if mj.Principal != nil { + if true { + buf.WriteString(`"principal":`) + fflib.WriteJsonString(buf, string(*mj.Principal)) + buf.WriteByte(',') + } + } + buf.WriteString(`"guarantee":`) + if mj.Guarantee != nil { + buf.WriteString(`[`) + for i, v := range mj.Guarantee { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_QuotaInfobase = iota + ffj_t_QuotaInfono_such_key + + ffj_t_QuotaInfo_Role + + ffj_t_QuotaInfo_Principal + + ffj_t_QuotaInfo_Guarantee +) + +var ffj_key_QuotaInfo_Role = []byte("role") + +var ffj_key_QuotaInfo_Principal = []byte("principal") + +var ffj_key_QuotaInfo_Guarantee = []byte("guarantee") + +func (uj *QuotaInfo) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *QuotaInfo) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_QuotaInfobase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_QuotaInfono_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'g': + + if bytes.Equal(ffj_key_QuotaInfo_Guarantee, kn) { + currentKey = ffj_t_QuotaInfo_Guarantee + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'p': + + if bytes.Equal(ffj_key_QuotaInfo_Principal, kn) { + currentKey = ffj_t_QuotaInfo_Principal + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_QuotaInfo_Role, kn) { + currentKey = ffj_t_QuotaInfo_Role + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_QuotaInfo_Guarantee, kn) { + currentKey = ffj_t_QuotaInfo_Guarantee + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_QuotaInfo_Principal, kn) { + currentKey = ffj_t_QuotaInfo_Principal + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_QuotaInfo_Role, kn) { + currentKey = ffj_t_QuotaInfo_Role + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_QuotaInfono_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_QuotaInfo_Role: + goto handle_Role + + case ffj_t_QuotaInfo_Principal: + goto handle_Principal + + case ffj_t_QuotaInfo_Guarantee: + goto handle_Guarantee + + case ffj_t_QuotaInfono_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Role: + + /* handler: uj.Role type=string kind=string quoted=false*/ + + { + + { + if tok != fflib.FFTok_string && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for string", tok)) + } + } + + if tok == fflib.FFTok_null { + + uj.Role = nil + + } else { + + var tval string + outBuf := fs.Output.Bytes() + + tval = string(string(outBuf)) + uj.Role = &tval + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Principal: + + /* handler: uj.Principal type=string kind=string quoted=false*/ + + { + + { + if tok != fflib.FFTok_string && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for string", tok)) + } + } + + if tok == fflib.FFTok_null { + + uj.Principal = nil + + } else { + + var tval string + outBuf := fs.Output.Bytes() + + tval = string(string(outBuf)) + uj.Principal = &tval + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Guarantee: + + /* handler: uj.Guarantee type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Guarantee = nil + } else { + + uj.Guarantee = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__Guarantee mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Guarantee type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Guarantee.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Guarantee = append(uj.Guarantee, tmp_uj__Guarantee) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *QuotaRequest) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *QuotaRequest) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteByte('{') + if mj.Force != nil { + if true { + if *mj.Force { + buf.WriteString(`"force":true`) + } else { + buf.WriteString(`"force":false`) + } + buf.WriteByte(',') + } + } + if mj.Role != nil { + if true { + buf.WriteString(`"role":`) + fflib.WriteJsonString(buf, string(*mj.Role)) + buf.WriteByte(',') + } + } + buf.WriteString(`"guarantee":`) + if mj.Guarantee != nil { + buf.WriteString(`[`) + for i, v := range mj.Guarantee { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_QuotaRequestbase = iota + ffj_t_QuotaRequestno_such_key + + ffj_t_QuotaRequest_Force + + ffj_t_QuotaRequest_Role + + ffj_t_QuotaRequest_Guarantee +) + +var ffj_key_QuotaRequest_Force = []byte("force") + +var ffj_key_QuotaRequest_Role = []byte("role") + +var ffj_key_QuotaRequest_Guarantee = []byte("guarantee") + +func (uj *QuotaRequest) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *QuotaRequest) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_QuotaRequestbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_QuotaRequestno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'f': + + if bytes.Equal(ffj_key_QuotaRequest_Force, kn) { + currentKey = ffj_t_QuotaRequest_Force + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'g': + + if bytes.Equal(ffj_key_QuotaRequest_Guarantee, kn) { + currentKey = ffj_t_QuotaRequest_Guarantee + state = fflib.FFParse_want_colon + goto mainparse + } + + case 'r': + + if bytes.Equal(ffj_key_QuotaRequest_Role, kn) { + currentKey = ffj_t_QuotaRequest_Role + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.SimpleLetterEqualFold(ffj_key_QuotaRequest_Guarantee, kn) { + currentKey = ffj_t_QuotaRequest_Guarantee + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_QuotaRequest_Role, kn) { + currentKey = ffj_t_QuotaRequest_Role + state = fflib.FFParse_want_colon + goto mainparse + } + + if fflib.SimpleLetterEqualFold(ffj_key_QuotaRequest_Force, kn) { + currentKey = ffj_t_QuotaRequest_Force + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_QuotaRequestno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_QuotaRequest_Force: + goto handle_Force + + case ffj_t_QuotaRequest_Role: + goto handle_Role + + case ffj_t_QuotaRequest_Guarantee: + goto handle_Guarantee + + case ffj_t_QuotaRequestno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Force: + + /* handler: uj.Force type=bool kind=bool quoted=false*/ + + { + if tok != fflib.FFTok_bool && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for bool", tok)) + } + } + + { + if tok == fflib.FFTok_null { + + uj.Force = nil + + } else { + tmpb := fs.Output.Bytes() + + var tval bool + + if bytes.Compare([]byte{'t', 'r', 'u', 'e'}, tmpb) == 0 { + + tval = true + + } else if bytes.Compare([]byte{'f', 'a', 'l', 's', 'e'}, tmpb) == 0 { + + tval = false + + } else { + err = errors.New("unexpected bytes for true/false value") + return fs.WrapErr(err) + } + + uj.Force = &tval + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Role: + + /* handler: uj.Role type=string kind=string quoted=false*/ + + { + + { + if tok != fflib.FFTok_string && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for string", tok)) + } + } + + if tok == fflib.FFTok_null { + + uj.Role = nil + + } else { + + var tval string + outBuf := fs.Output.Bytes() + + tval = string(string(outBuf)) + uj.Role = &tval + + } + } + + state = fflib.FFParse_after_value + goto mainparse + +handle_Guarantee: + + /* handler: uj.Guarantee type=[]mesos.Resource kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Guarantee = nil + } else { + + uj.Guarantee = []mesos.Resource{} + + wantVal := true + + for { + + var tmp_uj__Guarantee mesos.Resource + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Guarantee type=mesos.Resource kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Guarantee.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Guarantee = append(uj.Guarantee, tmp_uj__Guarantee) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} + +func (mj *QuotaStatus) MarshalJSON() ([]byte, error) { + var buf fflib.Buffer + if mj == nil { + buf.WriteString("null") + return buf.Bytes(), nil + } + err := mj.MarshalJSONBuf(&buf) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} +func (mj *QuotaStatus) MarshalJSONBuf(buf fflib.EncodingBuffer) error { + if mj == nil { + buf.WriteString("null") + return nil + } + var err error + var obj []byte + _ = obj + _ = err + buf.WriteString(`{"infos":`) + if mj.Infos != nil { + buf.WriteString(`[`) + for i, v := range mj.Infos { + if i != 0 { + buf.WriteString(`,`) + } + + { + + err = v.MarshalJSONBuf(buf) + if err != nil { + return err + } + + } + } + buf.WriteString(`]`) + } else { + buf.WriteString(`null`) + } + buf.WriteByte('}') + return nil +} + +const ( + ffj_t_QuotaStatusbase = iota + ffj_t_QuotaStatusno_such_key + + ffj_t_QuotaStatus_Infos +) + +var ffj_key_QuotaStatus_Infos = []byte("infos") + +func (uj *QuotaStatus) UnmarshalJSON(input []byte) error { + fs := fflib.NewFFLexer(input) + return uj.UnmarshalJSONFFLexer(fs, fflib.FFParse_map_start) +} + +func (uj *QuotaStatus) UnmarshalJSONFFLexer(fs *fflib.FFLexer, state fflib.FFParseState) error { + var err error = nil + currentKey := ffj_t_QuotaStatusbase + _ = currentKey + tok := fflib.FFTok_init + wantedTok := fflib.FFTok_init + +mainparse: + for { + tok = fs.Scan() + // println(fmt.Sprintf("debug: tok: %v state: %v", tok, state)) + if tok == fflib.FFTok_error { + goto tokerror + } + + switch state { + + case fflib.FFParse_map_start: + if tok != fflib.FFTok_left_bracket { + wantedTok = fflib.FFTok_left_bracket + goto wrongtokenerror + } + state = fflib.FFParse_want_key + continue + + case fflib.FFParse_after_value: + if tok == fflib.FFTok_comma { + state = fflib.FFParse_want_key + } else if tok == fflib.FFTok_right_bracket { + goto done + } else { + wantedTok = fflib.FFTok_comma + goto wrongtokenerror + } + + case fflib.FFParse_want_key: + // json {} ended. goto exit. woo. + if tok == fflib.FFTok_right_bracket { + goto done + } + if tok != fflib.FFTok_string { + wantedTok = fflib.FFTok_string + goto wrongtokenerror + } + + kn := fs.Output.Bytes() + if len(kn) <= 0 { + // "" case. hrm. + currentKey = ffj_t_QuotaStatusno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } else { + switch kn[0] { + + case 'i': + + if bytes.Equal(ffj_key_QuotaStatus_Infos, kn) { + currentKey = ffj_t_QuotaStatus_Infos + state = fflib.FFParse_want_colon + goto mainparse + } + + } + + if fflib.EqualFoldRight(ffj_key_QuotaStatus_Infos, kn) { + currentKey = ffj_t_QuotaStatus_Infos + state = fflib.FFParse_want_colon + goto mainparse + } + + currentKey = ffj_t_QuotaStatusno_such_key + state = fflib.FFParse_want_colon + goto mainparse + } + + case fflib.FFParse_want_colon: + if tok != fflib.FFTok_colon { + wantedTok = fflib.FFTok_colon + goto wrongtokenerror + } + state = fflib.FFParse_want_value + continue + case fflib.FFParse_want_value: + + if tok == fflib.FFTok_left_brace || tok == fflib.FFTok_left_bracket || tok == fflib.FFTok_integer || tok == fflib.FFTok_double || tok == fflib.FFTok_string || tok == fflib.FFTok_bool || tok == fflib.FFTok_null { + switch currentKey { + + case ffj_t_QuotaStatus_Infos: + goto handle_Infos + + case ffj_t_QuotaStatusno_such_key: + err = fs.SkipField(tok) + if err != nil { + return fs.WrapErr(err) + } + state = fflib.FFParse_after_value + goto mainparse + } + } else { + goto wantedvalue + } + } + } + +handle_Infos: + + /* handler: uj.Infos type=[]quota.QuotaInfo kind=slice quoted=false*/ + + { + + { + if tok != fflib.FFTok_left_brace && tok != fflib.FFTok_null { + return fs.WrapErr(fmt.Errorf("cannot unmarshal %s into Go value for ", tok)) + } + } + + if tok == fflib.FFTok_null { + uj.Infos = nil + } else { + + uj.Infos = []QuotaInfo{} + + wantVal := true + + for { + + var tmp_uj__Infos QuotaInfo + + tok = fs.Scan() + if tok == fflib.FFTok_error { + goto tokerror + } + if tok == fflib.FFTok_right_brace { + break + } + + if tok == fflib.FFTok_comma { + if wantVal == true { + // TODO(pquerna): this isn't an ideal error message, this handles + // things like [,,,] as an array value. + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) + } + continue + } else { + wantVal = true + } + + /* handler: tmp_uj__Infos type=quota.QuotaInfo kind=struct quoted=false*/ + + { + if tok == fflib.FFTok_null { + + state = fflib.FFParse_after_value + goto mainparse + } + + err = tmp_uj__Infos.UnmarshalJSONFFLexer(fs, fflib.FFParse_want_key) + if err != nil { + return err + } + state = fflib.FFParse_after_value + } + + uj.Infos = append(uj.Infos, tmp_uj__Infos) + + wantVal = false + } + } + } + + state = fflib.FFParse_after_value + goto mainparse + +wantedvalue: + return fs.WrapErr(fmt.Errorf("wanted value token, but got token: %v", tok)) +wrongtokenerror: + return fs.WrapErr(fmt.Errorf("ffjson: wanted token: %v, but got token: %v output=%s", wantedTok, tok, fs.Output.String())) +tokerror: + if fs.BigError != nil { + return fs.WrapErr(fs.BigError) + } + err = fs.Error.ToError() + if err != nil { + return fs.WrapErr(err) + } + panic("ffjson-generated: unreachable, please report bug.") +done: + + return nil +} diff --git a/api/v1/lib/quota/quota.proto b/api/v1/lib/quota/quota.proto new file mode 100644 index 00000000..5c3755b8 --- /dev/null +++ b/api/v1/lib/quota/quota.proto @@ -0,0 +1,92 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto2"; + +package mesos.quota; + +import "github.com/mesos/mesos-go/api/v1/lib/mesos.proto"; +import "github.com/gogo/protobuf/gogoproto/gogo.proto"; + +option go_package = "quota"; +option (gogoproto.benchgen_all) = true; +option (gogoproto.enum_stringer_all) = true; +option (gogoproto.equal_all) = true; +option (gogoproto.goproto_enum_prefix_all) = false; +option (gogoproto.goproto_enum_stringer_all) = false; +option (gogoproto.goproto_stringer_all) = false; +option (gogoproto.goproto_unrecognized_all) = false; +option (gogoproto.gostring_all) = true; +option (gogoproto.marshaler_all) = true; +option (gogoproto.populate_all) = true; +option (gogoproto.protosizer_all) = true; +option (gogoproto.stringer_all) = true; +option (gogoproto.testgen_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.verbose_equal_all) = true; + + +/** + * `QuotaInfo` describes the guaranteed resource allocation that a role + * may rely on (i.e. minimum share a role is entitled to receive). + * + * As for now, `QuotaInfo` is an internal message used by the master and + * therefore does not require versioning. However, in the future we may + * want to expose it in the Framework API, which will render it being + * external facing. + */ +// TODO(joerg84): Add limits, i.e. upper bounds of resources that a +// role is allowed to use. +message QuotaInfo { + // Quota is granted per role and not per framework, similar to + // dynamic reservations. + optional string role = 1; + + // Principal which set the quota. Currently only operators can set quotas. + optional string principal = 2; + + // The guarantee that these resources are allocatable for the above role. + // NOTE: `guarantee.role` should not specify any role except '*', + // because quota does not reserve specific resources. + repeated Resource guarantee = 3 [(gogoproto.nullable) = false]; +} + + +/** + * `QuotaRequest` provides a schema for set quota JSON requests. + */ +message QuotaRequest { + // Disables the capacity heuristic check if set to `true`. + optional bool force = 1 [default = false]; + + // The role for which to set quota. + optional string role = 2; + + // The requested guarantee that these resources will be allocatable for + // the above role. + repeated Resource guarantee = 3 [(gogoproto.nullable) = false]; +} + + +/** + * `QuotaStatus` describes the internal representation for the /quota/status + * response. + */ +message QuotaStatus { + // Quotas which are currently set, i.e. known to the master. + repeated QuotaInfo infos = 1 [(gogoproto.nullable) = false]; +} diff --git a/api/v1/lib/quota/quotapb_test.go b/api/v1/lib/quota/quotapb_test.go new file mode 100644 index 00000000..bcf90679 --- /dev/null +++ b/api/v1/lib/quota/quotapb_test.go @@ -0,0 +1,725 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: quota/quota.proto + +/* +Package quota is a generated protocol buffer package. + +It is generated from these files: + quota/quota.proto + +It has these top-level messages: + QuotaInfo + QuotaRequest + QuotaStatus +*/ +package quota + +import testing "testing" +import math_rand "math/rand" +import time "time" +import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" +import github_com_gogo_protobuf_jsonpb "github.com/gogo/protobuf/jsonpb" +import fmt "fmt" +import go_parser "go/parser" +import proto "github.com/gogo/protobuf/proto" +import math "math" +import _ "github.com/mesos/mesos-go/api/v1/lib" +import _ "github.com/gogo/protobuf/gogoproto" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +func TestQuotaInfoProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaInfo(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &QuotaInfo{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestQuotaInfoMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaInfo(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &QuotaInfo{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkQuotaInfoProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*QuotaInfo, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedQuotaInfo(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkQuotaInfoProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedQuotaInfo(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &QuotaInfo{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestQuotaRequestProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaRequest(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &QuotaRequest{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestQuotaRequestMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaRequest(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &QuotaRequest{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkQuotaRequestProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*QuotaRequest, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedQuotaRequest(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkQuotaRequestProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedQuotaRequest(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &QuotaRequest{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestQuotaStatusProto(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaStatus(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &QuotaStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + littlefuzz := make([]byte, len(dAtA)) + copy(littlefuzz, dAtA) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } + if len(littlefuzz) > 0 { + fuzzamount := 100 + for i := 0; i < fuzzamount; i++ { + littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256)) + littlefuzz = append(littlefuzz, byte(popr.Intn(256))) + } + // shouldn't panic + _ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg) + } +} + +func TestQuotaStatusMarshalTo(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaStatus(popr, false) + size := p.ProtoSize() + dAtA := make([]byte, size) + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + _, err := p.MarshalTo(dAtA) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &QuotaStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + for i := range dAtA { + dAtA[i] = byte(popr.Intn(256)) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func BenchmarkQuotaStatusProtoMarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*QuotaStatus, 10000) + for i := 0; i < 10000; i++ { + pops[i] = NewPopulatedQuotaStatus(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(pops[i%10000]) + if err != nil { + panic(err) + } + total += len(dAtA) + } + b.SetBytes(int64(total / b.N)) +} + +func BenchmarkQuotaStatusProtoUnmarshal(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + datas := make([][]byte, 10000) + for i := 0; i < 10000; i++ { + dAtA, err := github_com_gogo_protobuf_proto.Marshal(NewPopulatedQuotaStatus(popr, false)) + if err != nil { + panic(err) + } + datas[i] = dAtA + } + msg := &QuotaStatus{} + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += len(datas[i%10000]) + if err := github_com_gogo_protobuf_proto.Unmarshal(datas[i%10000], msg); err != nil { + panic(err) + } + } + b.SetBytes(int64(total / b.N)) +} + +func TestQuotaInfoJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaInfo(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &QuotaInfo{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestQuotaRequestJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaRequest(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &QuotaRequest{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestQuotaStatusJSON(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaStatus(popr, true) + marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{} + jsondata, err := marshaler.MarshalToString(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + msg := &QuotaStatus{} + err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p) + } +} +func TestQuotaInfoProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaInfo(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &QuotaInfo{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestQuotaInfoProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaInfo(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &QuotaInfo{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestQuotaRequestProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaRequest(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &QuotaRequest{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestQuotaRequestProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaRequest(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &QuotaRequest{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestQuotaStatusProtoText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaStatus(popr, true) + dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p) + msg := &QuotaStatus{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestQuotaStatusProtoCompactText(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaStatus(popr, true) + dAtA := github_com_gogo_protobuf_proto.CompactTextString(p) + msg := &QuotaStatus{} + if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("seed = %d, %#v !VerboseProto %#v, since %v", seed, msg, p, err) + } + if !p.Equal(msg) { + t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p) + } +} + +func TestQuotaInfoVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedQuotaInfo(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &QuotaInfo{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestQuotaRequestVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedQuotaRequest(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &QuotaRequest{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestQuotaStatusVerboseEqual(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedQuotaStatus(popr, false) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + panic(err) + } + msg := &QuotaStatus{} + if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil { + panic(err) + } + if err := p.VerboseEqual(msg); err != nil { + t.Fatalf("%#v !VerboseEqual %#v, since %v", msg, p, err) + } +} +func TestQuotaInfoGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedQuotaInfo(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestQuotaRequestGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedQuotaRequest(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestQuotaStatusGoString(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedQuotaStatus(popr, false) + s1 := p.GoString() + s2 := fmt.Sprintf("%#v", p) + if s1 != s2 { + t.Fatalf("GoString want %v got %v", s1, s2) + } + _, err := go_parser.ParseExpr(s1) + if err != nil { + panic(err) + } +} +func TestQuotaInfoProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaInfo(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkQuotaInfoProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*QuotaInfo, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedQuotaInfo(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestQuotaRequestProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaRequest(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkQuotaRequestProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*QuotaRequest, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedQuotaRequest(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestQuotaStatusProtoSize(t *testing.T) { + seed := time.Now().UnixNano() + popr := math_rand.New(math_rand.NewSource(seed)) + p := NewPopulatedQuotaStatus(popr, true) + size2 := github_com_gogo_protobuf_proto.Size(p) + dAtA, err := github_com_gogo_protobuf_proto.Marshal(p) + if err != nil { + t.Fatalf("seed = %d, err = %v", seed, err) + } + size := p.ProtoSize() + if len(dAtA) != size { + t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA)) + } + if size2 != size { + t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2) + } + size3 := github_com_gogo_protobuf_proto.Size(p) + if size3 != size { + t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3) + } +} + +func BenchmarkQuotaStatusProtoSize(b *testing.B) { + popr := math_rand.New(math_rand.NewSource(616)) + total := 0 + pops := make([]*QuotaStatus, 1000) + for i := 0; i < 1000; i++ { + pops[i] = NewPopulatedQuotaStatus(popr, false) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + total += pops[i%1000].ProtoSize() + } + b.SetBytes(int64(total / b.N)) +} + +func TestQuotaInfoStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedQuotaInfo(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestQuotaRequestStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedQuotaRequest(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} +func TestQuotaStatusStringer(t *testing.T) { + popr := math_rand.New(math_rand.NewSource(time.Now().UnixNano())) + p := NewPopulatedQuotaStatus(popr, false) + s1 := p.String() + s2 := fmt.Sprintf("%v", p) + if s1 != s2 { + t.Fatalf("String want %v got %v", s1, s2) + } +} + +//These tests are generated by github.com/gogo/protobuf/plugin/testgen diff --git a/api/v1/lib/recordio/writer.go b/api/v1/lib/recordio/writer.go new file mode 100644 index 00000000..085f967d --- /dev/null +++ b/api/v1/lib/recordio/writer.go @@ -0,0 +1,34 @@ +package recordio + +import ( + "io" + "strconv" +) + +var lf = []byte{'\n'} + +type Writer struct { + out io.Writer +} + +func NewWriter(out io.Writer) *Writer { + return &Writer{out} +} + +func (w *Writer) writeBuffer(b []byte, err error) error { + if err != nil { + return err + } + n, err := w.out.Write(b) + if err == nil && n != len(b) { + return io.ErrShortWrite + } + return err +} + +func (w *Writer) WriteFrame(b []byte) (err error) { + err = w.writeBuffer(([]byte)(strconv.Itoa(len(b))), err) + err = w.writeBuffer(lf, err) + err = w.writeBuffer(b, err) + return +} diff --git a/api/v1/vendor/vendor.json b/api/v1/vendor/vendor.json index a1d22824..589daacb 100644 --- a/api/v1/vendor/vendor.json +++ b/api/v1/vendor/vendor.json @@ -3,10 +3,10 @@ "ignore": "test", "package": [ { - "checksumSHA1": "PEwAbbGe3XH+ts1Pr5iZGEH6gHU=", + "checksumSHA1": "zZwnVAoDZ1Y+ml/54spzz9KtApU=", "path": "github.com/gogo/protobuf", - "revision": "7b6c6391c4ff245962047fc1e2c6e08b1cdfa0e8", - "revisionTime": "2017-07-20T14:48:05Z", + "revision": "db30a6cdca2260d5b850dc381bbac7d5de4bfcb4", + "revisionTime": "2017-07-30T06:52:33Z", "tree": true }, {