diff options
Diffstat (limited to 'pkg/sentry/socket/rpcinet')
-rw-r--r-- | pkg/sentry/socket/rpcinet/conn/conn.go | 187 | ||||
-rwxr-xr-x | pkg/sentry/socket/rpcinet/conn/conn_state_autogen.go | 4 | ||||
-rw-r--r-- | pkg/sentry/socket/rpcinet/device.go | 19 | ||||
-rw-r--r-- | pkg/sentry/socket/rpcinet/notifier/notifier.go | 230 | ||||
-rwxr-xr-x | pkg/sentry/socket/rpcinet/notifier/notifier_state_autogen.go | 4 | ||||
-rw-r--r-- | pkg/sentry/socket/rpcinet/rpcinet.go | 16 | ||||
-rwxr-xr-x | pkg/sentry/socket/rpcinet/rpcinet_state_autogen.go | 4 | ||||
-rw-r--r-- | pkg/sentry/socket/rpcinet/socket.go | 887 | ||||
-rw-r--r-- | pkg/sentry/socket/rpcinet/stack.go | 135 | ||||
-rw-r--r-- | pkg/sentry/socket/rpcinet/stack_unsafe.go | 193 | ||||
-rwxr-xr-x | pkg/sentry/socket/rpcinet/syscall_rpc_go_proto/syscall_rpc.pb.go | 3938 |
11 files changed, 5617 insertions, 0 deletions
diff --git a/pkg/sentry/socket/rpcinet/conn/conn.go b/pkg/sentry/socket/rpcinet/conn/conn.go new file mode 100644 index 000000000..f537c7f63 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/conn/conn.go @@ -0,0 +1,187 @@ +// Copyright 2018 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package conn is an RPC connection to a syscall RPC server. +package conn + +import ( + "fmt" + "sync" + "sync/atomic" + "syscall" + + "github.com/golang/protobuf/proto" + "gvisor.googlesource.com/gvisor/pkg/binary" + "gvisor.googlesource.com/gvisor/pkg/syserr" + "gvisor.googlesource.com/gvisor/pkg/unet" + + pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" +) + +type request struct { + response []byte + ready chan struct{} + ignoreResult bool +} + +// RPCConnection represents a single RPC connection to a syscall gofer. +type RPCConnection struct { + // reqID is the ID of the last request and must be accessed atomically. + reqID uint64 + + sendMu sync.Mutex + socket *unet.Socket + + reqMu sync.Mutex + requests map[uint64]request +} + +// NewRPCConnection initializes a RPC connection to a socket gofer. +func NewRPCConnection(s *unet.Socket) *RPCConnection { + conn := &RPCConnection{socket: s, requests: map[uint64]request{}} + go func() { // S/R-FIXME(b/77962828) + var nums [16]byte + for { + for n := 0; n < len(nums); { + nn, err := conn.socket.Read(nums[n:]) + if err != nil { + panic(fmt.Sprint("error reading length from socket rpc gofer: ", err)) + } + n += nn + } + + b := make([]byte, binary.LittleEndian.Uint64(nums[:8])) + id := binary.LittleEndian.Uint64(nums[8:]) + + for n := 0; n < len(b); { + nn, err := conn.socket.Read(b[n:]) + if err != nil { + panic(fmt.Sprint("error reading request from socket rpc gofer: ", err)) + } + n += nn + } + + conn.reqMu.Lock() + r := conn.requests[id] + if r.ignoreResult { + delete(conn.requests, id) + } else { + r.response = b + conn.requests[id] = r + } + conn.reqMu.Unlock() + close(r.ready) + } + }() + return conn +} + +// NewRequest makes a request to the RPC gofer and returns the request ID and a +// channel which will be closed once the request completes. +func (c *RPCConnection) NewRequest(req pb.SyscallRequest, ignoreResult bool) (uint64, chan struct{}) { + b, err := proto.Marshal(&req) + if err != nil { + panic(fmt.Sprint("invalid proto: ", err)) + } + + id := atomic.AddUint64(&c.reqID, 1) + ch := make(chan struct{}) + + c.reqMu.Lock() + c.requests[id] = request{ready: ch, ignoreResult: ignoreResult} + c.reqMu.Unlock() + + c.sendMu.Lock() + defer c.sendMu.Unlock() + + var nums [16]byte + binary.LittleEndian.PutUint64(nums[:8], uint64(len(b))) + binary.LittleEndian.PutUint64(nums[8:], id) + for n := 0; n < len(nums); { + nn, err := c.socket.Write(nums[n:]) + if err != nil { + panic(fmt.Sprint("error writing length and ID to socket gofer: ", err)) + } + n += nn + } + + for n := 0; n < len(b); { + nn, err := c.socket.Write(b[n:]) + if err != nil { + panic(fmt.Sprint("error writing request to socket gofer: ", err)) + } + n += nn + } + + return id, ch +} + +// RPCReadFile will execute the ReadFile helper RPC method which avoids the +// common pattern of open(2), read(2), close(2) by doing all three operations +// as a single RPC. It will read the entire file or return EFBIG if the file +// was too large. +func (c *RPCConnection) RPCReadFile(path string) ([]byte, *syserr.Error) { + req := &pb.SyscallRequest_ReadFile{&pb.ReadFileRequest{ + Path: path, + }} + + id, ch := c.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-ch + + res := c.Request(id).Result.(*pb.SyscallResponse_ReadFile).ReadFile.Result + if e, ok := res.(*pb.ReadFileResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.ReadFileResponse_Data).Data, nil +} + +// RPCWriteFile will execute the WriteFile helper RPC method which avoids the +// common pattern of open(2), write(2), write(2), close(2) by doing all +// operations as a single RPC. +func (c *RPCConnection) RPCWriteFile(path string, data []byte) (int64, *syserr.Error) { + req := &pb.SyscallRequest_WriteFile{&pb.WriteFileRequest{ + Path: path, + Content: data, + }} + + id, ch := c.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-ch + + res := c.Request(id).Result.(*pb.SyscallResponse_WriteFile).WriteFile + if e := res.ErrorNumber; e != 0 { + return int64(res.Written), syserr.FromHost(syscall.Errno(e)) + } + + return int64(res.Written), nil +} + +// Request retrieves the request corresponding to the given request ID. +// +// The channel returned by NewRequest must have been closed before Request can +// be called. This will happen automatically, do not manually close the +// channel. +func (c *RPCConnection) Request(id uint64) pb.SyscallResponse { + c.reqMu.Lock() + r := c.requests[id] + delete(c.requests, id) + c.reqMu.Unlock() + + var resp pb.SyscallResponse + if err := proto.Unmarshal(r.response, &resp); err != nil { + panic(fmt.Sprint("invalid proto: ", err)) + } + + return resp +} diff --git a/pkg/sentry/socket/rpcinet/conn/conn_state_autogen.go b/pkg/sentry/socket/rpcinet/conn/conn_state_autogen.go new file mode 100755 index 000000000..f6c927a60 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/conn/conn_state_autogen.go @@ -0,0 +1,4 @@ +// automatically generated by stateify. + +package conn + diff --git a/pkg/sentry/socket/rpcinet/device.go b/pkg/sentry/socket/rpcinet/device.go new file mode 100644 index 000000000..44c0a39b7 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/device.go @@ -0,0 +1,19 @@ +// Copyright 2018 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rpcinet + +import "gvisor.googlesource.com/gvisor/pkg/sentry/device" + +var socketDevice = device.NewAnonDevice() diff --git a/pkg/sentry/socket/rpcinet/notifier/notifier.go b/pkg/sentry/socket/rpcinet/notifier/notifier.go new file mode 100644 index 000000000..601e05994 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/notifier/notifier.go @@ -0,0 +1,230 @@ +// Copyright 2018 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package notifier implements an FD notifier implementation over RPC. +package notifier + +import ( + "fmt" + "sync" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/conn" + pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +type fdInfo struct { + queue *waiter.Queue + waiting bool +} + +// Notifier holds all the state necessary to issue notifications when IO events +// occur in the observed FDs. +type Notifier struct { + // rpcConn is the connection that is used for sending RPCs. + rpcConn *conn.RPCConnection + + // epFD is the epoll file descriptor used to register for io + // notifications. + epFD uint32 + + // mu protects fdMap. + mu sync.Mutex + + // fdMap maps file descriptors to their notification queues and waiting + // status. + fdMap map[uint32]*fdInfo +} + +// NewRPCNotifier creates a new notifier object. +func NewRPCNotifier(cn *conn.RPCConnection) (*Notifier, error) { + id, c := cn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollCreate1{&pb.EpollCreate1Request{}}}, false /* ignoreResult */) + <-c + + res := cn.Request(id).Result.(*pb.SyscallResponse_EpollCreate1).EpollCreate1.Result + if e, ok := res.(*pb.EpollCreate1Response_ErrorNumber); ok { + return nil, syscall.Errno(e.ErrorNumber) + } + + w := &Notifier{ + rpcConn: cn, + epFD: res.(*pb.EpollCreate1Response_Fd).Fd, + fdMap: make(map[uint32]*fdInfo), + } + + go w.waitAndNotify() // S/R-FIXME(b/77962828) + + return w, nil +} + +// waitFD waits on mask for fd. The fdMap mutex must be hold. +func (n *Notifier) waitFD(fd uint32, fi *fdInfo, mask waiter.EventMask) error { + if !fi.waiting && mask == 0 { + return nil + } + + e := pb.EpollEvent{ + Events: mask.ToLinux() | -syscall.EPOLLET, + Fd: fd, + } + + switch { + case !fi.waiting && mask != 0: + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollCtl{&pb.EpollCtlRequest{Epfd: n.epFD, Op: syscall.EPOLL_CTL_ADD, Fd: fd, Event: &e}}}, false /* ignoreResult */) + <-c + + e := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_EpollCtl).EpollCtl.ErrorNumber + if e != 0 { + return syscall.Errno(e) + } + + fi.waiting = true + case fi.waiting && mask == 0: + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollCtl{&pb.EpollCtlRequest{Epfd: n.epFD, Op: syscall.EPOLL_CTL_DEL, Fd: fd}}}, false /* ignoreResult */) + <-c + n.rpcConn.Request(id) + + fi.waiting = false + case fi.waiting && mask != 0: + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollCtl{&pb.EpollCtlRequest{Epfd: n.epFD, Op: syscall.EPOLL_CTL_MOD, Fd: fd, Event: &e}}}, false /* ignoreResult */) + <-c + + e := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_EpollCtl).EpollCtl.ErrorNumber + if e != 0 { + return syscall.Errno(e) + } + } + + return nil +} + +// addFD adds an FD to the list of FDs observed by n. +func (n *Notifier) addFD(fd uint32, queue *waiter.Queue) { + n.mu.Lock() + defer n.mu.Unlock() + + // Panic if we're already notifying on this FD. + if _, ok := n.fdMap[fd]; ok { + panic(fmt.Sprintf("File descriptor %d added twice", fd)) + } + + // We have nothing to wait for at the moment. Just add it to the map. + n.fdMap[fd] = &fdInfo{queue: queue} +} + +// updateFD updates the set of events the FD needs to be notified on. +func (n *Notifier) updateFD(fd uint32) error { + n.mu.Lock() + defer n.mu.Unlock() + + if fi, ok := n.fdMap[fd]; ok { + return n.waitFD(fd, fi, fi.queue.Events()) + } + + return nil +} + +// RemoveFD removes an FD from the list of FDs observed by n. +func (n *Notifier) removeFD(fd uint32) { + n.mu.Lock() + defer n.mu.Unlock() + + // Remove from map, then from epoll object. + n.waitFD(fd, n.fdMap[fd], 0) + delete(n.fdMap, fd) +} + +// hasFD returns true if the FD is in the list of observed FDs. +func (n *Notifier) hasFD(fd uint32) bool { + n.mu.Lock() + defer n.mu.Unlock() + + _, ok := n.fdMap[fd] + return ok +} + +// waitAndNotify loops waiting for io event notifications from the epoll +// object. Once notifications arrive, they are dispatched to the +// registered queue. +func (n *Notifier) waitAndNotify() error { + for { + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollWait{&pb.EpollWaitRequest{Fd: n.epFD, NumEvents: 100, Msec: -1}}}, false /* ignoreResult */) + <-c + + res := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_EpollWait).EpollWait.Result + if e, ok := res.(*pb.EpollWaitResponse_ErrorNumber); ok { + err := syscall.Errno(e.ErrorNumber) + // NOTE(magi): I don't think epoll_wait can return EAGAIN but I'm being + // conseratively careful here since exiting the notification thread + // would be really bad. + if err == syscall.EINTR || err == syscall.EAGAIN { + continue + } + return err + } + + n.mu.Lock() + for _, e := range res.(*pb.EpollWaitResponse_Events).Events.Events { + if fi, ok := n.fdMap[e.Fd]; ok { + fi.queue.Notify(waiter.EventMaskFromLinux(e.Events)) + } + } + n.mu.Unlock() + } +} + +// AddFD adds an FD to the list of observed FDs. +func (n *Notifier) AddFD(fd uint32, queue *waiter.Queue) error { + n.addFD(fd, queue) + return nil +} + +// UpdateFD updates the set of events the FD needs to be notified on. +func (n *Notifier) UpdateFD(fd uint32) error { + return n.updateFD(fd) +} + +// RemoveFD removes an FD from the list of observed FDs. +func (n *Notifier) RemoveFD(fd uint32) { + n.removeFD(fd) +} + +// HasFD returns true if the FD is in the list of observed FDs. +// +// This should only be used by tests to assert that FDs are correctly +// registered. +func (n *Notifier) HasFD(fd uint32) bool { + return n.hasFD(fd) +} + +// NonBlockingPoll polls the given fd in non-blocking fashion. It is used just +// to query the FD's current state; this method will block on the RPC response +// although the syscall is non-blocking. +func (n *Notifier) NonBlockingPoll(fd uint32, mask waiter.EventMask) waiter.EventMask { + for { + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Poll{&pb.PollRequest{Fd: fd, Events: mask.ToLinux()}}}, false /* ignoreResult */) + <-c + + res := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_Poll).Poll.Result + if e, ok := res.(*pb.PollResponse_ErrorNumber); ok { + if syscall.Errno(e.ErrorNumber) == syscall.EINTR { + continue + } + return mask + } + + return waiter.EventMaskFromLinux(res.(*pb.PollResponse_Events).Events) + } +} diff --git a/pkg/sentry/socket/rpcinet/notifier/notifier_state_autogen.go b/pkg/sentry/socket/rpcinet/notifier/notifier_state_autogen.go new file mode 100755 index 000000000..f108d91c1 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/notifier/notifier_state_autogen.go @@ -0,0 +1,4 @@ +// automatically generated by stateify. + +package notifier + diff --git a/pkg/sentry/socket/rpcinet/rpcinet.go b/pkg/sentry/socket/rpcinet/rpcinet.go new file mode 100644 index 000000000..5d4fd4dac --- /dev/null +++ b/pkg/sentry/socket/rpcinet/rpcinet.go @@ -0,0 +1,16 @@ +// Copyright 2018 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package rpcinet implements sockets using an RPC for each syscall. +package rpcinet diff --git a/pkg/sentry/socket/rpcinet/rpcinet_state_autogen.go b/pkg/sentry/socket/rpcinet/rpcinet_state_autogen.go new file mode 100755 index 000000000..d3076c7e3 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/rpcinet_state_autogen.go @@ -0,0 +1,4 @@ +// automatically generated by stateify. + +package rpcinet + diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go new file mode 100644 index 000000000..55e0b6665 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -0,0 +1,887 @@ +// Copyright 2018 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rpcinet + +import ( + "sync/atomic" + "syscall" + "time" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/binary" + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs" + ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/conn" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/notifier" + pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" + "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserr" + "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/tcpip" + "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +// socketOperations implements fs.FileOperations and socket.Socket for a socket +// implemented using a host socket. +type socketOperations struct { + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` + socket.SendReceiveTimeout + + family int // Read-only. + fd uint32 // must be O_NONBLOCK + wq *waiter.Queue + rpcConn *conn.RPCConnection + notifier *notifier.Notifier + + // shState is the state of the connection with respect to shutdown. Because + // we're mixing non-blocking semantics on the other side we have to adapt for + // some strange differences between blocking and non-blocking sockets. + shState int32 +} + +// Verify that we actually implement socket.Socket. +var _ = socket.Socket(&socketOperations{}) + +// New creates a new RPC socket. +func newSocketFile(ctx context.Context, stack *Stack, family int, skType int, protocol int) (*fs.File, *syserr.Error) { + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Socket{&pb.SocketRequest{Family: int64(family), Type: int64(skType | syscall.SOCK_NONBLOCK), Protocol: int64(protocol)}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Socket).Socket.Result + if e, ok := res.(*pb.SocketResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + fd := res.(*pb.SocketResponse_Fd).Fd + + var wq waiter.Queue + stack.notifier.AddFD(fd, &wq) + + dirent := socket.NewDirent(ctx, socketDevice) + defer dirent.DecRef() + return fs.NewFile(ctx, dirent, fs.FileFlags{Read: true, Write: true}, &socketOperations{ + family: family, + wq: &wq, + fd: fd, + rpcConn: stack.rpcConn, + notifier: stack.notifier, + }), nil +} + +func isBlockingErrno(err error) bool { + return err == syscall.EAGAIN || err == syscall.EWOULDBLOCK +} + +func translateIOSyscallError(err error) error { + if isBlockingErrno(err) { + return syserror.ErrWouldBlock + } + return err +} + +// setShutdownFlags will set the shutdown flag so we can handle blocking reads +// after a read shutdown. +func (s *socketOperations) setShutdownFlags(how int) { + var f tcpip.ShutdownFlags + switch how { + case linux.SHUT_RD: + f = tcpip.ShutdownRead + case linux.SHUT_WR: + f = tcpip.ShutdownWrite + case linux.SHUT_RDWR: + f = tcpip.ShutdownWrite | tcpip.ShutdownRead + } + + // Atomically update the flags. + for { + old := atomic.LoadInt32(&s.shState) + if atomic.CompareAndSwapInt32(&s.shState, old, old|int32(f)) { + break + } + } +} + +func (s *socketOperations) resetShutdownFlags() { + atomic.StoreInt32(&s.shState, 0) +} + +func (s *socketOperations) isShutRdSet() bool { + return atomic.LoadInt32(&s.shState)&int32(tcpip.ShutdownRead) != 0 +} + +func (s *socketOperations) isShutWrSet() bool { + return atomic.LoadInt32(&s.shState)&int32(tcpip.ShutdownWrite) != 0 +} + +// Release implements fs.FileOperations.Release. +func (s *socketOperations) Release() { + s.notifier.RemoveFD(s.fd) + + // We always need to close the FD. + _, _ = s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Close{&pb.CloseRequest{Fd: s.fd}}}, true /* ignoreResult */) +} + +// Readiness implements waiter.Waitable.Readiness. +func (s *socketOperations) Readiness(mask waiter.EventMask) waiter.EventMask { + return s.notifier.NonBlockingPoll(s.fd, mask) +} + +// EventRegister implements waiter.Waitable.EventRegister. +func (s *socketOperations) EventRegister(e *waiter.Entry, mask waiter.EventMask) { + s.wq.EventRegister(e, mask) + s.notifier.UpdateFD(s.fd) +} + +// EventUnregister implements waiter.Waitable.EventUnregister. +func (s *socketOperations) EventUnregister(e *waiter.Entry) { + s.wq.EventUnregister(e) + s.notifier.UpdateFD(s.fd) +} + +func rpcRead(t *kernel.Task, req *pb.SyscallRequest_Read) (*pb.ReadResponse_Data, *syserr.Error) { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Read).Read.Result + if e, ok := res.(*pb.ReadResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.ReadResponse_Data), nil +} + +// Read implements fs.FileOperations.Read. +func (s *socketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, _ int64) (int64, error) { + req := &pb.SyscallRequest_Read{&pb.ReadRequest{ + Fd: s.fd, + Length: uint32(dst.NumBytes()), + }} + + res, se := rpcRead(ctx.(*kernel.Task), req) + if se == nil { + n, e := dst.CopyOut(ctx, res.Data) + return int64(n), e + } + + return 0, se.ToError() +} + +func rpcWrite(t *kernel.Task, req *pb.SyscallRequest_Write) (uint32, *syserr.Error) { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Write).Write.Result + if e, ok := res.(*pb.WriteResponse_ErrorNumber); ok { + return 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.WriteResponse_Length).Length, nil +} + +// Write implements fs.FileOperations.Write. +func (s *socketOperations) Write(ctx context.Context, _ *fs.File, src usermem.IOSequence, _ int64) (int64, error) { + t := ctx.(*kernel.Task) + v := buffer.NewView(int(src.NumBytes())) + + // Copy all the data into the buffer. + if _, err := src.CopyIn(t, v); err != nil { + return 0, err + } + + n, err := rpcWrite(t, &pb.SyscallRequest_Write{&pb.WriteRequest{Fd: s.fd, Data: v}}) + if n > 0 && n < uint32(src.NumBytes()) { + // The FileOperations.Write interface expects us to return ErrWouldBlock in + // the event of a partial write. + return int64(n), syserror.ErrWouldBlock + } + return int64(n), err.ToError() +} + +func rpcConnect(t *kernel.Task, fd uint32, sockaddr []byte) *syserr.Error { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Connect{&pb.ConnectRequest{Fd: uint32(fd), Address: sockaddr}}}, false /* ignoreResult */) + <-c + + if e := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Connect).Connect.ErrorNumber; e != 0 { + return syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +// Connect implements socket.Socket.Connect. +func (s *socketOperations) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error { + if !blocking { + e := rpcConnect(t, s.fd, sockaddr) + if e == nil { + // Reset the shutdown state on new connects. + s.resetShutdownFlags() + } + return e + } + + // Register for notification when the endpoint becomes writable, then + // initiate the connection. + e, ch := waiter.NewChannelEntry(nil) + s.EventRegister(&e, waiter.EventOut|waiter.EventIn|waiter.EventHUp) + defer s.EventUnregister(&e) + for { + if err := rpcConnect(t, s.fd, sockaddr); err == nil || err != syserr.ErrInProgress && err != syserr.ErrAlreadyInProgress { + if err == nil { + // Reset the shutdown state on new connects. + s.resetShutdownFlags() + } + return err + } + + // It's pending, so we have to wait for a notification, and fetch the + // result once the wait completes. + if err := t.Block(ch); err != nil { + return syserr.FromError(err) + } + } +} + +func rpcAccept(t *kernel.Task, fd uint32, peer bool) (*pb.AcceptResponse_ResultPayload, *syserr.Error) { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Accept{&pb.AcceptRequest{Fd: fd, Peer: peer, Flags: syscall.SOCK_NONBLOCK}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Accept).Accept.Result + if e, ok := res.(*pb.AcceptResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + return res.(*pb.AcceptResponse_Payload).Payload, nil +} + +// Accept implements socket.Socket.Accept. +func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, blocking bool) (kdefs.FD, interface{}, uint32, *syserr.Error) { + payload, se := rpcAccept(t, s.fd, peerRequested) + + // Check if we need to block. + if blocking && se == syserr.ErrTryAgain { + // Register for notifications. + e, ch := waiter.NewChannelEntry(nil) + // FIXME(b/119878986): This waiter.EventHUp is a partial + // measure, need to figure out how to translate linux events to + // internal events. + s.EventRegister(&e, waiter.EventIn|waiter.EventHUp) + defer s.EventUnregister(&e) + + // Try to accept the connection again; if it fails, then wait until we + // get a notification. + for { + if payload, se = rpcAccept(t, s.fd, peerRequested); se != syserr.ErrTryAgain { + break + } + + if err := t.Block(ch); err != nil { + return 0, nil, 0, syserr.FromError(err) + } + } + } + + // Handle any error from accept. + if se != nil { + return 0, nil, 0, se + } + + var wq waiter.Queue + s.notifier.AddFD(payload.Fd, &wq) + + dirent := socket.NewDirent(t, socketDevice) + defer dirent.DecRef() + file := fs.NewFile(t, dirent, fs.FileFlags{Read: true, Write: true, NonBlocking: flags&linux.SOCK_NONBLOCK != 0}, &socketOperations{ + wq: &wq, + fd: payload.Fd, + rpcConn: s.rpcConn, + notifier: s.notifier, + }) + defer file.DecRef() + + fdFlags := kernel.FDFlags{ + CloseOnExec: flags&linux.SOCK_CLOEXEC != 0, + } + fd, err := t.FDMap().NewFDFrom(0, file, fdFlags, t.ThreadGroup().Limits()) + if err != nil { + return 0, nil, 0, syserr.FromError(err) + } + t.Kernel().RecordSocket(file, s.family) + + if peerRequested { + return fd, payload.Address.Address, payload.Address.Length, nil + } + + return fd, nil, 0, nil +} + +// Bind implements socket.Socket.Bind. +func (s *socketOperations) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Bind{&pb.BindRequest{Fd: s.fd, Address: sockaddr}}}, false /* ignoreResult */) + <-c + + if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Bind).Bind.ErrorNumber; e != 0 { + return syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +// Listen implements socket.Socket.Listen. +func (s *socketOperations) Listen(t *kernel.Task, backlog int) *syserr.Error { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Listen{&pb.ListenRequest{Fd: s.fd, Backlog: int64(backlog)}}}, false /* ignoreResult */) + <-c + + if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Listen).Listen.ErrorNumber; e != 0 { + return syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +// Shutdown implements socket.Socket.Shutdown. +func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { + // We save the shutdown state because of strange differences on linux + // related to recvs on blocking vs. non-blocking sockets after a SHUT_RD. + // We need to emulate that behavior on the blocking side. + // TODO(b/120096741): There is a possible race that can exist with loopback, + // where data could possibly be lost. + s.setShutdownFlags(how) + + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Shutdown{&pb.ShutdownRequest{Fd: s.fd, How: int64(how)}}}, false /* ignoreResult */) + <-c + + if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Shutdown).Shutdown.ErrorNumber; e != 0 { + return syserr.FromHost(syscall.Errno(e)) + } + + return nil +} + +// GetSockOpt implements socket.Socket.GetSockOpt. +func (s *socketOperations) GetSockOpt(t *kernel.Task, level int, name int, outLen int) (interface{}, *syserr.Error) { + // SO_RCVTIMEO and SO_SNDTIMEO are special because blocking is performed + // within the sentry. + if level == linux.SOL_SOCKET && name == linux.SO_RCVTIMEO { + if outLen < linux.SizeOfTimeval { + return nil, syserr.ErrInvalidArgument + } + + return linux.NsecToTimeval(s.RecvTimeout()), nil + } + if level == linux.SOL_SOCKET && name == linux.SO_SNDTIMEO { + if outLen < linux.SizeOfTimeval { + return nil, syserr.ErrInvalidArgument + } + + return linux.NsecToTimeval(s.SendTimeout()), nil + } + + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_GetSockOpt{&pb.GetSockOptRequest{Fd: s.fd, Level: int64(level), Name: int64(name), Length: uint32(outLen)}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_GetSockOpt).GetSockOpt.Result + if e, ok := res.(*pb.GetSockOptResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.GetSockOptResponse_Opt).Opt, nil +} + +// SetSockOpt implements socket.Socket.SetSockOpt. +func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *syserr.Error { + // Because blocking actually happens within the sentry we need to inspect + // this socket option to determine if it's a SO_RCVTIMEO or SO_SNDTIMEO, + // and if so, we will save it and use it as the deadline for recv(2) + // or send(2) related syscalls. + if level == linux.SOL_SOCKET && name == linux.SO_RCVTIMEO { + if len(opt) < linux.SizeOfTimeval { + return syserr.ErrInvalidArgument + } + + var v linux.Timeval + binary.Unmarshal(opt[:linux.SizeOfTimeval], usermem.ByteOrder, &v) + if v.Usec < 0 || v.Usec >= int64(time.Second/time.Microsecond) { + return syserr.ErrDomain + } + s.SetRecvTimeout(v.ToNsecCapped()) + return nil + } + if level == linux.SOL_SOCKET && name == linux.SO_SNDTIMEO { + if len(opt) < linux.SizeOfTimeval { + return syserr.ErrInvalidArgument + } + + var v linux.Timeval + binary.Unmarshal(opt[:linux.SizeOfTimeval], usermem.ByteOrder, &v) + if v.Usec < 0 || v.Usec >= int64(time.Second/time.Microsecond) { + return syserr.ErrDomain + } + s.SetSendTimeout(v.ToNsecCapped()) + return nil + } + + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_SetSockOpt{&pb.SetSockOptRequest{Fd: s.fd, Level: int64(level), Name: int64(name), Opt: opt}}}, false /* ignoreResult */) + <-c + + if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_SetSockOpt).SetSockOpt.ErrorNumber; e != 0 { + return syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +// GetPeerName implements socket.Socket.GetPeerName. +func (s *socketOperations) GetPeerName(t *kernel.Task) (interface{}, uint32, *syserr.Error) { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_GetPeerName{&pb.GetPeerNameRequest{Fd: s.fd}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_GetPeerName).GetPeerName.Result + if e, ok := res.(*pb.GetPeerNameResponse_ErrorNumber); ok { + return nil, 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + addr := res.(*pb.GetPeerNameResponse_Address).Address + return addr.Address, addr.Length, nil +} + +// GetSockName implements socket.Socket.GetSockName. +func (s *socketOperations) GetSockName(t *kernel.Task) (interface{}, uint32, *syserr.Error) { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_GetSockName{&pb.GetSockNameRequest{Fd: s.fd}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_GetSockName).GetSockName.Result + if e, ok := res.(*pb.GetSockNameResponse_ErrorNumber); ok { + return nil, 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + addr := res.(*pb.GetSockNameResponse_Address).Address + return addr.Address, addr.Length, nil +} + +func rpcIoctl(t *kernel.Task, fd, cmd uint32, arg []byte) ([]byte, error) { + stack := t.NetworkContext().(*Stack) + + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Ioctl{&pb.IOCtlRequest{Fd: fd, Cmd: cmd, Arg: arg}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Ioctl).Ioctl.Result + if e, ok := res.(*pb.IOCtlResponse_ErrorNumber); ok { + return nil, syscall.Errno(e.ErrorNumber) + } + + return res.(*pb.IOCtlResponse_Value).Value, nil +} + +// ifconfIoctlFromStack populates a struct ifconf for the SIOCGIFCONF ioctl. +func ifconfIoctlFromStack(ctx context.Context, io usermem.IO, ifc *linux.IFConf) error { + // If Ptr is NULL, return the necessary buffer size via Len. + // Otherwise, write up to Len bytes starting at Ptr containing ifreq + // structs. + t := ctx.(*kernel.Task) + s := t.NetworkContext().(*Stack) + if s == nil { + return syserr.ErrNoDevice.ToError() + } + + if ifc.Ptr == 0 { + ifc.Len = int32(len(s.Interfaces())) * int32(linux.SizeOfIFReq) + return nil + } + + max := ifc.Len + ifc.Len = 0 + for key, ifaceAddrs := range s.InterfaceAddrs() { + iface := s.Interfaces()[key] + for _, ifaceAddr := range ifaceAddrs { + // Don't write past the end of the buffer. + if ifc.Len+int32(linux.SizeOfIFReq) > max { + break + } + if ifaceAddr.Family != linux.AF_INET { + continue + } + + // Populate ifr.ifr_addr. + ifr := linux.IFReq{} + ifr.SetName(iface.Name) + usermem.ByteOrder.PutUint16(ifr.Data[0:2], uint16(ifaceAddr.Family)) + usermem.ByteOrder.PutUint16(ifr.Data[2:4], 0) + copy(ifr.Data[4:8], ifaceAddr.Addr[:4]) + + // Copy the ifr to userspace. + dst := uintptr(ifc.Ptr) + uintptr(ifc.Len) + ifc.Len += int32(linux.SizeOfIFReq) + if _, err := usermem.CopyObjectOut(ctx, io, usermem.Addr(dst), ifr, usermem.IOOpts{ + AddressSpaceActive: true, + }); err != nil { + return err + } + } + } + return nil +} + +// Ioctl implements fs.FileOperations.Ioctl. +func (s *socketOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { + t := ctx.(*kernel.Task) + + cmd := uint32(args[1].Int()) + arg := args[2].Pointer() + + var buf []byte + switch cmd { + // The following ioctls take 4 byte argument parameters. + case syscall.TIOCINQ, + syscall.TIOCOUTQ: + buf = make([]byte, 4) + // The following ioctls have args which are sizeof(struct ifreq). + case syscall.SIOCGIFADDR, + syscall.SIOCGIFBRDADDR, + syscall.SIOCGIFDSTADDR, + syscall.SIOCGIFFLAGS, + syscall.SIOCGIFHWADDR, + syscall.SIOCGIFINDEX, + syscall.SIOCGIFMAP, + syscall.SIOCGIFMETRIC, + syscall.SIOCGIFMTU, + syscall.SIOCGIFNAME, + syscall.SIOCGIFNETMASK, + syscall.SIOCGIFTXQLEN: + buf = make([]byte, linux.SizeOfIFReq) + case syscall.SIOCGIFCONF: + // SIOCGIFCONF has slightly different behavior than the others, in that it + // will need to populate the array of ifreqs. + var ifc linux.IFConf + if _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &ifc, usermem.IOOpts{ + AddressSpaceActive: true, + }); err != nil { + return 0, err + } + + if err := ifconfIoctlFromStack(ctx, io, &ifc); err != nil { + return 0, err + } + _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), ifc, usermem.IOOpts{ + AddressSpaceActive: true, + }) + + return 0, err + + case linux.SIOCGIFMEM, linux.SIOCGIFPFLAGS, linux.SIOCGMIIPHY, linux.SIOCGMIIREG: + unimpl.EmitUnimplementedEvent(ctx) + + default: + return 0, syserror.ENOTTY + } + + _, err := io.CopyIn(ctx, arg, buf, usermem.IOOpts{ + AddressSpaceActive: true, + }) + + if err != nil { + return 0, err + } + + v, err := rpcIoctl(t, s.fd, cmd, buf) + if err != nil { + return 0, err + } + + if len(v) != len(buf) { + return 0, syserror.EINVAL + } + + _, err = io.CopyOut(ctx, arg, v, usermem.IOOpts{ + AddressSpaceActive: true, + }) + return 0, err +} + +func rpcRecvMsg(t *kernel.Task, req *pb.SyscallRequest_Recvmsg) (*pb.RecvmsgResponse_ResultPayload, *syserr.Error) { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Recvmsg).Recvmsg.Result + if e, ok := res.(*pb.RecvmsgResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.RecvmsgResponse_Payload).Payload, nil +} + +// Because we only support SO_TIMESTAMP we will search control messages for +// that value and set it if so, all other control messages will be ignored. +func (s *socketOperations) extractControlMessages(payload *pb.RecvmsgResponse_ResultPayload) socket.ControlMessages { + c := socket.ControlMessages{} + if len(payload.GetCmsgData()) > 0 { + // Parse the control messages looking for SO_TIMESTAMP. + msgs, e := syscall.ParseSocketControlMessage(payload.GetCmsgData()) + if e != nil { + return socket.ControlMessages{} + } + for _, m := range msgs { + if m.Header.Level != linux.SOL_SOCKET || m.Header.Type != linux.SO_TIMESTAMP { + continue + } + + // Let's parse the time stamp and set it. + if len(m.Data) < linux.SizeOfTimeval { + // Give up on locating the SO_TIMESTAMP option. + return socket.ControlMessages{} + } + + var v linux.Timeval + binary.Unmarshal(m.Data[:linux.SizeOfTimeval], usermem.ByteOrder, &v) + c.IP.HasTimestamp = true + c.IP.Timestamp = v.ToNsecCapped() + break + } + } + return c +} + +// RecvMsg implements socket.Socket.RecvMsg. +func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { + req := &pb.SyscallRequest_Recvmsg{&pb.RecvmsgRequest{ + Fd: s.fd, + Length: uint32(dst.NumBytes()), + Sender: senderRequested, + Trunc: flags&linux.MSG_TRUNC != 0, + Peek: flags&linux.MSG_PEEK != 0, + CmsgLength: uint32(controlDataLen), + }} + + res, err := rpcRecvMsg(t, req) + if err == nil { + var e error + var n int + if len(res.Data) > 0 { + n, e = dst.CopyOut(t, res.Data) + if e == nil && n != len(res.Data) { + panic("CopyOut failed to copy full buffer") + } + } + c := s.extractControlMessages(res) + return int(res.Length), 0, res.Address.GetAddress(), res.Address.GetLength(), c, syserr.FromError(e) + } + if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain || flags&linux.MSG_DONTWAIT != 0 { + return 0, 0, nil, 0, socket.ControlMessages{}, err + } + + // We'll have to block. Register for notifications and keep trying to + // send all the data. + e, ch := waiter.NewChannelEntry(nil) + s.EventRegister(&e, waiter.EventIn) + defer s.EventUnregister(&e) + + for { + res, err := rpcRecvMsg(t, req) + if err == nil { + var e error + var n int + if len(res.Data) > 0 { + n, e = dst.CopyOut(t, res.Data) + if e == nil && n != len(res.Data) { + panic("CopyOut failed to copy full buffer") + } + } + c := s.extractControlMessages(res) + return int(res.Length), 0, res.Address.GetAddress(), res.Address.GetLength(), c, syserr.FromError(e) + } + if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain { + return 0, 0, nil, 0, socket.ControlMessages{}, err + } + + if s.isShutRdSet() { + // Blocking would have caused us to block indefinitely so we return 0, + // this is the same behavior as Linux. + return 0, 0, nil, 0, socket.ControlMessages{}, nil + } + + if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { + if err == syserror.ETIMEDOUT { + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain + } + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) + } + } +} + +func rpcSendMsg(t *kernel.Task, req *pb.SyscallRequest_Sendmsg) (uint32, *syserr.Error) { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Sendmsg).Sendmsg.Result + if e, ok := res.(*pb.SendmsgResponse_ErrorNumber); ok { + return 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.SendmsgResponse_Length).Length, nil +} + +// SendMsg implements socket.Socket.SendMsg. +func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) { + // Whitelist flags. + if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_EOR|syscall.MSG_FASTOPEN|syscall.MSG_MORE|syscall.MSG_NOSIGNAL) != 0 { + return 0, syserr.ErrInvalidArgument + } + + // Reject Unix control messages. + if !controlMessages.Unix.Empty() { + return 0, syserr.ErrInvalidArgument + } + + v := buffer.NewView(int(src.NumBytes())) + + // Copy all the data into the buffer. + if _, err := src.CopyIn(t, v); err != nil { + return 0, syserr.FromError(err) + } + + // TODO(bgeffon): this needs to change to map directly to a SendMsg syscall + // in the RPC. + totalWritten := 0 + n, err := rpcSendMsg(t, &pb.SyscallRequest_Sendmsg{&pb.SendmsgRequest{ + Fd: uint32(s.fd), + Data: v, + Address: to, + More: flags&linux.MSG_MORE != 0, + EndOfRecord: flags&linux.MSG_EOR != 0, + }}) + + if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain || flags&linux.MSG_DONTWAIT != 0 { + return int(n), err + } + + if n > 0 { + totalWritten += int(n) + v.TrimFront(int(n)) + } + + // We'll have to block. Register for notification and keep trying to + // send all the data. + e, ch := waiter.NewChannelEntry(nil) + s.EventRegister(&e, waiter.EventOut) + defer s.EventUnregister(&e) + + for { + n, err := rpcSendMsg(t, &pb.SyscallRequest_Sendmsg{&pb.SendmsgRequest{ + Fd: uint32(s.fd), + Data: v, + Address: to, + More: flags&linux.MSG_MORE != 0, + EndOfRecord: flags&linux.MSG_EOR != 0, + }}) + + if n > 0 { + totalWritten += int(n) + v.TrimFront(int(n)) + + if err == nil && totalWritten < int(src.NumBytes()) { + continue + } + } + + if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain { + // We eat the error in this situation. + return int(totalWritten), nil + } + + if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { + if err == syserror.ETIMEDOUT { + return int(totalWritten), syserr.ErrTryAgain + } + return int(totalWritten), syserr.FromError(err) + } + } +} + +type socketProvider struct { + family int +} + +// Socket implements socket.Provider.Socket. +func (p *socketProvider) Socket(t *kernel.Task, stypeflags transport.SockType, protocol int) (*fs.File, *syserr.Error) { + // Check that we are using the RPC network stack. + stack := t.NetworkContext() + if stack == nil { + return nil, nil + } + + s, ok := stack.(*Stack) + if !ok { + return nil, nil + } + + // Only accept TCP and UDP. + // + // Try to restrict the flags we will accept to minimize backwards + // incompatibility with netstack. + stype := int(stypeflags) & linux.SOCK_TYPE_MASK + switch stype { + case syscall.SOCK_STREAM: + switch protocol { + case 0, syscall.IPPROTO_TCP: + // ok + default: + return nil, nil + } + case syscall.SOCK_DGRAM: + switch protocol { + case 0, syscall.IPPROTO_UDP: + // ok + default: + return nil, nil + } + default: + return nil, nil + } + + return newSocketFile(t, s, p.family, stype, 0) +} + +// Pair implements socket.Provider.Pair. +func (p *socketProvider) Pair(t *kernel.Task, stype transport.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { + // Not supported by AF_INET/AF_INET6. + return nil, nil, nil +} + +func init() { + for _, family := range []int{syscall.AF_INET, syscall.AF_INET6} { + socket.RegisterProvider(family, &socketProvider{family}) + } +} diff --git a/pkg/sentry/socket/rpcinet/stack.go b/pkg/sentry/socket/rpcinet/stack.go new file mode 100644 index 000000000..a1be711df --- /dev/null +++ b/pkg/sentry/socket/rpcinet/stack.go @@ -0,0 +1,135 @@ +// Copyright 2018 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rpcinet + +import ( + "fmt" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/sentry/inet" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/hostinet" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/conn" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/notifier" + "gvisor.googlesource.com/gvisor/pkg/syserr" + "gvisor.googlesource.com/gvisor/pkg/unet" +) + +// Stack implements inet.Stack for RPC backed sockets. +type Stack struct { + interfaces map[int32]inet.Interface + interfaceAddrs map[int32][]inet.InterfaceAddr + rpcConn *conn.RPCConnection + notifier *notifier.Notifier +} + +// NewStack returns a Stack containing the current state of the host network +// stack. +func NewStack(fd int32) (*Stack, error) { + sock, err := unet.NewSocket(int(fd)) + if err != nil { + return nil, err + } + + stack := &Stack{ + interfaces: make(map[int32]inet.Interface), + interfaceAddrs: make(map[int32][]inet.InterfaceAddr), + rpcConn: conn.NewRPCConnection(sock), + } + + var e error + stack.notifier, e = notifier.NewRPCNotifier(stack.rpcConn) + if e != nil { + return nil, e + } + + links, err := stack.DoNetlinkRouteRequest(syscall.RTM_GETLINK) + if err != nil { + return nil, fmt.Errorf("RTM_GETLINK failed: %v", err) + } + + addrs, err := stack.DoNetlinkRouteRequest(syscall.RTM_GETADDR) + if err != nil { + return nil, fmt.Errorf("RTM_GETADDR failed: %v", err) + } + + e = hostinet.ExtractHostInterfaces(links, addrs, stack.interfaces, stack.interfaceAddrs) + if e != nil { + return nil, e + } + + return stack, nil +} + +// RPCReadFile will execute the ReadFile helper RPC method which avoids the +// common pattern of open(2), read(2), close(2) by doing all three operations +// as a single RPC. It will read the entire file or return EFBIG if the file +// was too large. +func (s *Stack) RPCReadFile(path string) ([]byte, *syserr.Error) { + return s.rpcConn.RPCReadFile(path) +} + +// RPCWriteFile will execute the WriteFile helper RPC method which avoids the +// common pattern of open(2), write(2), write(2), close(2) by doing all +// operations as a single RPC. +func (s *Stack) RPCWriteFile(path string, data []byte) (int64, *syserr.Error) { + return s.rpcConn.RPCWriteFile(path, data) +} + +// Interfaces implements inet.Stack.Interfaces. +func (s *Stack) Interfaces() map[int32]inet.Interface { + return s.interfaces +} + +// InterfaceAddrs implements inet.Stack.InterfaceAddrs. +func (s *Stack) InterfaceAddrs() map[int32][]inet.InterfaceAddr { + return s.interfaceAddrs +} + +// SupportsIPv6 implements inet.Stack.SupportsIPv6. +func (s *Stack) SupportsIPv6() bool { + panic("rpcinet handles procfs directly this method should not be called") +} + +// TCPReceiveBufferSize implements inet.Stack.TCPReceiveBufferSize. +func (s *Stack) TCPReceiveBufferSize() (inet.TCPBufferSize, error) { + panic("rpcinet handles procfs directly this method should not be called") +} + +// SetTCPReceiveBufferSize implements inet.Stack.SetTCPReceiveBufferSize. +func (s *Stack) SetTCPReceiveBufferSize(size inet.TCPBufferSize) error { + panic("rpcinet handles procfs directly this method should not be called") + +} + +// TCPSendBufferSize implements inet.Stack.TCPSendBufferSize. +func (s *Stack) TCPSendBufferSize() (inet.TCPBufferSize, error) { + panic("rpcinet handles procfs directly this method should not be called") + +} + +// SetTCPSendBufferSize implements inet.Stack.SetTCPSendBufferSize. +func (s *Stack) SetTCPSendBufferSize(size inet.TCPBufferSize) error { + panic("rpcinet handles procfs directly this method should not be called") +} + +// TCPSACKEnabled implements inet.Stack.TCPSACKEnabled. +func (s *Stack) TCPSACKEnabled() (bool, error) { + panic("rpcinet handles procfs directly this method should not be called") +} + +// SetTCPSACKEnabled implements inet.Stack.SetTCPSACKEnabled. +func (s *Stack) SetTCPSACKEnabled(enabled bool) error { + panic("rpcinet handles procfs directly this method should not be called") +} diff --git a/pkg/sentry/socket/rpcinet/stack_unsafe.go b/pkg/sentry/socket/rpcinet/stack_unsafe.go new file mode 100644 index 000000000..e53f578ba --- /dev/null +++ b/pkg/sentry/socket/rpcinet/stack_unsafe.go @@ -0,0 +1,193 @@ +// Copyright 2018 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rpcinet + +import ( + "syscall" + "unsafe" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/binary" + pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserr" +) + +// NewNetlinkRouteRequest builds a netlink message for getting the RIB, +// the routing information base. +func newNetlinkRouteRequest(proto, seq, family int) []byte { + rr := &syscall.NetlinkRouteRequest{} + rr.Header.Len = uint32(syscall.NLMSG_HDRLEN + syscall.SizeofRtGenmsg) + rr.Header.Type = uint16(proto) + rr.Header.Flags = syscall.NLM_F_DUMP | syscall.NLM_F_REQUEST + rr.Header.Seq = uint32(seq) + rr.Data.Family = uint8(family) + return netlinkRRtoWireFormat(rr) +} + +func netlinkRRtoWireFormat(rr *syscall.NetlinkRouteRequest) []byte { + b := make([]byte, rr.Header.Len) + *(*uint32)(unsafe.Pointer(&b[0:4][0])) = rr.Header.Len + *(*uint16)(unsafe.Pointer(&b[4:6][0])) = rr.Header.Type + *(*uint16)(unsafe.Pointer(&b[6:8][0])) = rr.Header.Flags + *(*uint32)(unsafe.Pointer(&b[8:12][0])) = rr.Header.Seq + *(*uint32)(unsafe.Pointer(&b[12:16][0])) = rr.Header.Pid + b[16] = byte(rr.Data.Family) + return b +} + +func (s *Stack) getNetlinkFd() (uint32, *syserr.Error) { + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Socket{&pb.SocketRequest{Family: int64(syscall.AF_NETLINK), Type: int64(syscall.SOCK_RAW | syscall.SOCK_NONBLOCK), Protocol: int64(syscall.NETLINK_ROUTE)}}}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Socket).Socket.Result + if e, ok := res.(*pb.SocketResponse_ErrorNumber); ok { + return 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + return res.(*pb.SocketResponse_Fd).Fd, nil +} + +func (s *Stack) bindNetlinkFd(fd uint32, sockaddr []byte) *syserr.Error { + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Bind{&pb.BindRequest{Fd: fd, Address: sockaddr}}}, false /* ignoreResult */) + <-c + + if e := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Bind).Bind.ErrorNumber; e != 0 { + return syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +func (s *Stack) closeNetlinkFd(fd uint32) { + _, _ = s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Close{&pb.CloseRequest{Fd: fd}}}, true /* ignoreResult */) +} + +func (s *Stack) rpcSendMsg(req *pb.SyscallRequest_Sendmsg) (uint32, *syserr.Error) { + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Sendmsg).Sendmsg.Result + if e, ok := res.(*pb.SendmsgResponse_ErrorNumber); ok { + return 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.SendmsgResponse_Length).Length, nil +} + +func (s *Stack) sendMsg(fd uint32, buf []byte, to []byte, flags int) (int, *syserr.Error) { + // Whitelist flags. + if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_EOR|syscall.MSG_FASTOPEN|syscall.MSG_MORE|syscall.MSG_NOSIGNAL) != 0 { + return 0, syserr.ErrInvalidArgument + } + + req := &pb.SyscallRequest_Sendmsg{&pb.SendmsgRequest{ + Fd: fd, + Data: buf, + Address: to, + More: flags&linux.MSG_MORE != 0, + EndOfRecord: flags&linux.MSG_EOR != 0, + }} + + n, err := s.rpcSendMsg(req) + return int(n), err +} + +func (s *Stack) rpcRecvMsg(req *pb.SyscallRequest_Recvmsg) (*pb.RecvmsgResponse_ResultPayload, *syserr.Error) { + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Recvmsg).Recvmsg.Result + if e, ok := res.(*pb.RecvmsgResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.RecvmsgResponse_Payload).Payload, nil +} + +func (s *Stack) recvMsg(fd, l, flags uint32) ([]byte, *syserr.Error) { + req := &pb.SyscallRequest_Recvmsg{&pb.RecvmsgRequest{ + Fd: fd, + Length: l, + Sender: false, + Trunc: flags&linux.MSG_TRUNC != 0, + Peek: flags&linux.MSG_PEEK != 0, + }} + + res, err := s.rpcRecvMsg(req) + if err != nil { + return nil, err + } + return res.Data, nil +} + +func (s *Stack) netlinkRequest(proto, family int) ([]byte, error) { + fd, err := s.getNetlinkFd() + if err != nil { + return nil, err.ToError() + } + defer s.closeNetlinkFd(fd) + + lsa := syscall.SockaddrNetlink{Family: syscall.AF_NETLINK} + b := binary.Marshal(nil, usermem.ByteOrder, &lsa) + if err := s.bindNetlinkFd(fd, b); err != nil { + return nil, err.ToError() + } + + wb := newNetlinkRouteRequest(proto, 1, family) + _, err = s.sendMsg(fd, wb, b, 0) + if err != nil { + return nil, err.ToError() + } + + var tab []byte +done: + for { + rb, err := s.recvMsg(fd, uint32(syscall.Getpagesize()), 0) + nr := len(rb) + if err != nil { + return nil, err.ToError() + } + + if nr < syscall.NLMSG_HDRLEN { + return nil, syserr.ErrInvalidArgument.ToError() + } + + tab = append(tab, rb...) + msgs, e := syscall.ParseNetlinkMessage(rb) + if e != nil { + return nil, e + } + + for _, m := range msgs { + if m.Header.Type == syscall.NLMSG_DONE { + break done + } + if m.Header.Type == syscall.NLMSG_ERROR { + return nil, syserr.ErrInvalidArgument.ToError() + } + } + } + + return tab, nil +} + +// DoNetlinkRouteRequest returns routing information base, also known as RIB, +// which consists of network facility information, states and parameters. +func (s *Stack) DoNetlinkRouteRequest(req int) ([]syscall.NetlinkMessage, error) { + data, err := s.netlinkRequest(req, syscall.AF_UNSPEC) + if err != nil { + return nil, err + } + return syscall.ParseNetlinkMessage(data) +} diff --git a/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto/syscall_rpc.pb.go b/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto/syscall_rpc.pb.go new file mode 100755 index 000000000..fb68d5294 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto/syscall_rpc.pb.go @@ -0,0 +1,3938 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// source: pkg/sentry/socket/rpcinet/syscall_rpc.proto + +package syscall_rpc + +import ( + fmt "fmt" + proto "github.com/golang/protobuf/proto" + math "math" +) + +// 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.ProtoPackageIsVersion3 // please upgrade the proto package + +type SendmsgRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Address []byte `protobuf:"bytes,3,opt,name=address,proto3" json:"address,omitempty"` + More bool `protobuf:"varint,4,opt,name=more,proto3" json:"more,omitempty"` + EndOfRecord bool `protobuf:"varint,5,opt,name=end_of_record,json=endOfRecord,proto3" json:"end_of_record,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SendmsgRequest) Reset() { *m = SendmsgRequest{} } +func (m *SendmsgRequest) String() string { return proto.CompactTextString(m) } +func (*SendmsgRequest) ProtoMessage() {} +func (*SendmsgRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{0} +} + +func (m *SendmsgRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SendmsgRequest.Unmarshal(m, b) +} +func (m *SendmsgRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SendmsgRequest.Marshal(b, m, deterministic) +} +func (m *SendmsgRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SendmsgRequest.Merge(m, src) +} +func (m *SendmsgRequest) XXX_Size() int { + return xxx_messageInfo_SendmsgRequest.Size(m) +} +func (m *SendmsgRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SendmsgRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SendmsgRequest proto.InternalMessageInfo + +func (m *SendmsgRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *SendmsgRequest) GetData() []byte { + if m != nil { + return m.Data + } + return nil +} + +func (m *SendmsgRequest) GetAddress() []byte { + if m != nil { + return m.Address + } + return nil +} + +func (m *SendmsgRequest) GetMore() bool { + if m != nil { + return m.More + } + return false +} + +func (m *SendmsgRequest) GetEndOfRecord() bool { + if m != nil { + return m.EndOfRecord + } + return false +} + +type SendmsgResponse struct { + // Types that are valid to be assigned to Result: + // *SendmsgResponse_ErrorNumber + // *SendmsgResponse_Length + Result isSendmsgResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SendmsgResponse) Reset() { *m = SendmsgResponse{} } +func (m *SendmsgResponse) String() string { return proto.CompactTextString(m) } +func (*SendmsgResponse) ProtoMessage() {} +func (*SendmsgResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{1} +} + +func (m *SendmsgResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SendmsgResponse.Unmarshal(m, b) +} +func (m *SendmsgResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SendmsgResponse.Marshal(b, m, deterministic) +} +func (m *SendmsgResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SendmsgResponse.Merge(m, src) +} +func (m *SendmsgResponse) XXX_Size() int { + return xxx_messageInfo_SendmsgResponse.Size(m) +} +func (m *SendmsgResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SendmsgResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SendmsgResponse proto.InternalMessageInfo + +type isSendmsgResponse_Result interface { + isSendmsgResponse_Result() +} + +type SendmsgResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type SendmsgResponse_Length struct { + Length uint32 `protobuf:"varint,2,opt,name=length,proto3,oneof"` +} + +func (*SendmsgResponse_ErrorNumber) isSendmsgResponse_Result() {} + +func (*SendmsgResponse_Length) isSendmsgResponse_Result() {} + +func (m *SendmsgResponse) GetResult() isSendmsgResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *SendmsgResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*SendmsgResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *SendmsgResponse) GetLength() uint32 { + if x, ok := m.GetResult().(*SendmsgResponse_Length); ok { + return x.Length + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*SendmsgResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*SendmsgResponse_ErrorNumber)(nil), + (*SendmsgResponse_Length)(nil), + } +} + +type IOCtlRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Cmd uint32 `protobuf:"varint,2,opt,name=cmd,proto3" json:"cmd,omitempty"` + Arg []byte `protobuf:"bytes,3,opt,name=arg,proto3" json:"arg,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *IOCtlRequest) Reset() { *m = IOCtlRequest{} } +func (m *IOCtlRequest) String() string { return proto.CompactTextString(m) } +func (*IOCtlRequest) ProtoMessage() {} +func (*IOCtlRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{2} +} + +func (m *IOCtlRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_IOCtlRequest.Unmarshal(m, b) +} +func (m *IOCtlRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_IOCtlRequest.Marshal(b, m, deterministic) +} +func (m *IOCtlRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_IOCtlRequest.Merge(m, src) +} +func (m *IOCtlRequest) XXX_Size() int { + return xxx_messageInfo_IOCtlRequest.Size(m) +} +func (m *IOCtlRequest) XXX_DiscardUnknown() { + xxx_messageInfo_IOCtlRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_IOCtlRequest proto.InternalMessageInfo + +func (m *IOCtlRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *IOCtlRequest) GetCmd() uint32 { + if m != nil { + return m.Cmd + } + return 0 +} + +func (m *IOCtlRequest) GetArg() []byte { + if m != nil { + return m.Arg + } + return nil +} + +type IOCtlResponse struct { + // Types that are valid to be assigned to Result: + // *IOCtlResponse_ErrorNumber + // *IOCtlResponse_Value + Result isIOCtlResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *IOCtlResponse) Reset() { *m = IOCtlResponse{} } +func (m *IOCtlResponse) String() string { return proto.CompactTextString(m) } +func (*IOCtlResponse) ProtoMessage() {} +func (*IOCtlResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{3} +} + +func (m *IOCtlResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_IOCtlResponse.Unmarshal(m, b) +} +func (m *IOCtlResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_IOCtlResponse.Marshal(b, m, deterministic) +} +func (m *IOCtlResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_IOCtlResponse.Merge(m, src) +} +func (m *IOCtlResponse) XXX_Size() int { + return xxx_messageInfo_IOCtlResponse.Size(m) +} +func (m *IOCtlResponse) XXX_DiscardUnknown() { + xxx_messageInfo_IOCtlResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_IOCtlResponse proto.InternalMessageInfo + +type isIOCtlResponse_Result interface { + isIOCtlResponse_Result() +} + +type IOCtlResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type IOCtlResponse_Value struct { + Value []byte `protobuf:"bytes,2,opt,name=value,proto3,oneof"` +} + +func (*IOCtlResponse_ErrorNumber) isIOCtlResponse_Result() {} + +func (*IOCtlResponse_Value) isIOCtlResponse_Result() {} + +func (m *IOCtlResponse) GetResult() isIOCtlResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *IOCtlResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*IOCtlResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *IOCtlResponse) GetValue() []byte { + if x, ok := m.GetResult().(*IOCtlResponse_Value); ok { + return x.Value + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*IOCtlResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*IOCtlResponse_ErrorNumber)(nil), + (*IOCtlResponse_Value)(nil), + } +} + +type RecvmsgRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Length uint32 `protobuf:"varint,2,opt,name=length,proto3" json:"length,omitempty"` + Sender bool `protobuf:"varint,3,opt,name=sender,proto3" json:"sender,omitempty"` + Peek bool `protobuf:"varint,4,opt,name=peek,proto3" json:"peek,omitempty"` + Trunc bool `protobuf:"varint,5,opt,name=trunc,proto3" json:"trunc,omitempty"` + CmsgLength uint32 `protobuf:"varint,6,opt,name=cmsg_length,json=cmsgLength,proto3" json:"cmsg_length,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RecvmsgRequest) Reset() { *m = RecvmsgRequest{} } +func (m *RecvmsgRequest) String() string { return proto.CompactTextString(m) } +func (*RecvmsgRequest) ProtoMessage() {} +func (*RecvmsgRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{4} +} + +func (m *RecvmsgRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_RecvmsgRequest.Unmarshal(m, b) +} +func (m *RecvmsgRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_RecvmsgRequest.Marshal(b, m, deterministic) +} +func (m *RecvmsgRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RecvmsgRequest.Merge(m, src) +} +func (m *RecvmsgRequest) XXX_Size() int { + return xxx_messageInfo_RecvmsgRequest.Size(m) +} +func (m *RecvmsgRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RecvmsgRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_RecvmsgRequest proto.InternalMessageInfo + +func (m *RecvmsgRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *RecvmsgRequest) GetLength() uint32 { + if m != nil { + return m.Length + } + return 0 +} + +func (m *RecvmsgRequest) GetSender() bool { + if m != nil { + return m.Sender + } + return false +} + +func (m *RecvmsgRequest) GetPeek() bool { + if m != nil { + return m.Peek + } + return false +} + +func (m *RecvmsgRequest) GetTrunc() bool { + if m != nil { + return m.Trunc + } + return false +} + +func (m *RecvmsgRequest) GetCmsgLength() uint32 { + if m != nil { + return m.CmsgLength + } + return 0 +} + +type OpenRequest struct { + Path []byte `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` + Flags uint32 `protobuf:"varint,2,opt,name=flags,proto3" json:"flags,omitempty"` + Mode uint32 `protobuf:"varint,3,opt,name=mode,proto3" json:"mode,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *OpenRequest) Reset() { *m = OpenRequest{} } +func (m *OpenRequest) String() string { return proto.CompactTextString(m) } +func (*OpenRequest) ProtoMessage() {} +func (*OpenRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{5} +} + +func (m *OpenRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_OpenRequest.Unmarshal(m, b) +} +func (m *OpenRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_OpenRequest.Marshal(b, m, deterministic) +} +func (m *OpenRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_OpenRequest.Merge(m, src) +} +func (m *OpenRequest) XXX_Size() int { + return xxx_messageInfo_OpenRequest.Size(m) +} +func (m *OpenRequest) XXX_DiscardUnknown() { + xxx_messageInfo_OpenRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_OpenRequest proto.InternalMessageInfo + +func (m *OpenRequest) GetPath() []byte { + if m != nil { + return m.Path + } + return nil +} + +func (m *OpenRequest) GetFlags() uint32 { + if m != nil { + return m.Flags + } + return 0 +} + +func (m *OpenRequest) GetMode() uint32 { + if m != nil { + return m.Mode + } + return 0 +} + +type OpenResponse struct { + // Types that are valid to be assigned to Result: + // *OpenResponse_ErrorNumber + // *OpenResponse_Fd + Result isOpenResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *OpenResponse) Reset() { *m = OpenResponse{} } +func (m *OpenResponse) String() string { return proto.CompactTextString(m) } +func (*OpenResponse) ProtoMessage() {} +func (*OpenResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{6} +} + +func (m *OpenResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_OpenResponse.Unmarshal(m, b) +} +func (m *OpenResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_OpenResponse.Marshal(b, m, deterministic) +} +func (m *OpenResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_OpenResponse.Merge(m, src) +} +func (m *OpenResponse) XXX_Size() int { + return xxx_messageInfo_OpenResponse.Size(m) +} +func (m *OpenResponse) XXX_DiscardUnknown() { + xxx_messageInfo_OpenResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_OpenResponse proto.InternalMessageInfo + +type isOpenResponse_Result interface { + isOpenResponse_Result() +} + +type OpenResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type OpenResponse_Fd struct { + Fd uint32 `protobuf:"varint,2,opt,name=fd,proto3,oneof"` +} + +func (*OpenResponse_ErrorNumber) isOpenResponse_Result() {} + +func (*OpenResponse_Fd) isOpenResponse_Result() {} + +func (m *OpenResponse) GetResult() isOpenResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *OpenResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*OpenResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *OpenResponse) GetFd() uint32 { + if x, ok := m.GetResult().(*OpenResponse_Fd); ok { + return x.Fd + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*OpenResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*OpenResponse_ErrorNumber)(nil), + (*OpenResponse_Fd)(nil), + } +} + +type ReadRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Length uint32 `protobuf:"varint,2,opt,name=length,proto3" json:"length,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReadRequest) Reset() { *m = ReadRequest{} } +func (m *ReadRequest) String() string { return proto.CompactTextString(m) } +func (*ReadRequest) ProtoMessage() {} +func (*ReadRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{7} +} + +func (m *ReadRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReadRequest.Unmarshal(m, b) +} +func (m *ReadRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReadRequest.Marshal(b, m, deterministic) +} +func (m *ReadRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReadRequest.Merge(m, src) +} +func (m *ReadRequest) XXX_Size() int { + return xxx_messageInfo_ReadRequest.Size(m) +} +func (m *ReadRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ReadRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ReadRequest proto.InternalMessageInfo + +func (m *ReadRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *ReadRequest) GetLength() uint32 { + if m != nil { + return m.Length + } + return 0 +} + +type ReadResponse struct { + // Types that are valid to be assigned to Result: + // *ReadResponse_ErrorNumber + // *ReadResponse_Data + Result isReadResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReadResponse) Reset() { *m = ReadResponse{} } +func (m *ReadResponse) String() string { return proto.CompactTextString(m) } +func (*ReadResponse) ProtoMessage() {} +func (*ReadResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{8} +} + +func (m *ReadResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReadResponse.Unmarshal(m, b) +} +func (m *ReadResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReadResponse.Marshal(b, m, deterministic) +} +func (m *ReadResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReadResponse.Merge(m, src) +} +func (m *ReadResponse) XXX_Size() int { + return xxx_messageInfo_ReadResponse.Size(m) +} +func (m *ReadResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ReadResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ReadResponse proto.InternalMessageInfo + +type isReadResponse_Result interface { + isReadResponse_Result() +} + +type ReadResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type ReadResponse_Data struct { + Data []byte `protobuf:"bytes,2,opt,name=data,proto3,oneof"` +} + +func (*ReadResponse_ErrorNumber) isReadResponse_Result() {} + +func (*ReadResponse_Data) isReadResponse_Result() {} + +func (m *ReadResponse) GetResult() isReadResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *ReadResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*ReadResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *ReadResponse) GetData() []byte { + if x, ok := m.GetResult().(*ReadResponse_Data); ok { + return x.Data + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*ReadResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*ReadResponse_ErrorNumber)(nil), + (*ReadResponse_Data)(nil), + } +} + +type ReadFileRequest struct { + Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReadFileRequest) Reset() { *m = ReadFileRequest{} } +func (m *ReadFileRequest) String() string { return proto.CompactTextString(m) } +func (*ReadFileRequest) ProtoMessage() {} +func (*ReadFileRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{9} +} + +func (m *ReadFileRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReadFileRequest.Unmarshal(m, b) +} +func (m *ReadFileRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReadFileRequest.Marshal(b, m, deterministic) +} +func (m *ReadFileRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReadFileRequest.Merge(m, src) +} +func (m *ReadFileRequest) XXX_Size() int { + return xxx_messageInfo_ReadFileRequest.Size(m) +} +func (m *ReadFileRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ReadFileRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ReadFileRequest proto.InternalMessageInfo + +func (m *ReadFileRequest) GetPath() string { + if m != nil { + return m.Path + } + return "" +} + +type ReadFileResponse struct { + // Types that are valid to be assigned to Result: + // *ReadFileResponse_ErrorNumber + // *ReadFileResponse_Data + Result isReadFileResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReadFileResponse) Reset() { *m = ReadFileResponse{} } +func (m *ReadFileResponse) String() string { return proto.CompactTextString(m) } +func (*ReadFileResponse) ProtoMessage() {} +func (*ReadFileResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{10} +} + +func (m *ReadFileResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReadFileResponse.Unmarshal(m, b) +} +func (m *ReadFileResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReadFileResponse.Marshal(b, m, deterministic) +} +func (m *ReadFileResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReadFileResponse.Merge(m, src) +} +func (m *ReadFileResponse) XXX_Size() int { + return xxx_messageInfo_ReadFileResponse.Size(m) +} +func (m *ReadFileResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ReadFileResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ReadFileResponse proto.InternalMessageInfo + +type isReadFileResponse_Result interface { + isReadFileResponse_Result() +} + +type ReadFileResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type ReadFileResponse_Data struct { + Data []byte `protobuf:"bytes,2,opt,name=data,proto3,oneof"` +} + +func (*ReadFileResponse_ErrorNumber) isReadFileResponse_Result() {} + +func (*ReadFileResponse_Data) isReadFileResponse_Result() {} + +func (m *ReadFileResponse) GetResult() isReadFileResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *ReadFileResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*ReadFileResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *ReadFileResponse) GetData() []byte { + if x, ok := m.GetResult().(*ReadFileResponse_Data); ok { + return x.Data + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*ReadFileResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*ReadFileResponse_ErrorNumber)(nil), + (*ReadFileResponse_Data)(nil), + } +} + +type WriteRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *WriteRequest) Reset() { *m = WriteRequest{} } +func (m *WriteRequest) String() string { return proto.CompactTextString(m) } +func (*WriteRequest) ProtoMessage() {} +func (*WriteRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{11} +} + +func (m *WriteRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_WriteRequest.Unmarshal(m, b) +} +func (m *WriteRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_WriteRequest.Marshal(b, m, deterministic) +} +func (m *WriteRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteRequest.Merge(m, src) +} +func (m *WriteRequest) XXX_Size() int { + return xxx_messageInfo_WriteRequest.Size(m) +} +func (m *WriteRequest) XXX_DiscardUnknown() { + xxx_messageInfo_WriteRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteRequest proto.InternalMessageInfo + +func (m *WriteRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *WriteRequest) GetData() []byte { + if m != nil { + return m.Data + } + return nil +} + +type WriteResponse struct { + // Types that are valid to be assigned to Result: + // *WriteResponse_ErrorNumber + // *WriteResponse_Length + Result isWriteResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *WriteResponse) Reset() { *m = WriteResponse{} } +func (m *WriteResponse) String() string { return proto.CompactTextString(m) } +func (*WriteResponse) ProtoMessage() {} +func (*WriteResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{12} +} + +func (m *WriteResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_WriteResponse.Unmarshal(m, b) +} +func (m *WriteResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_WriteResponse.Marshal(b, m, deterministic) +} +func (m *WriteResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteResponse.Merge(m, src) +} +func (m *WriteResponse) XXX_Size() int { + return xxx_messageInfo_WriteResponse.Size(m) +} +func (m *WriteResponse) XXX_DiscardUnknown() { + xxx_messageInfo_WriteResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteResponse proto.InternalMessageInfo + +type isWriteResponse_Result interface { + isWriteResponse_Result() +} + +type WriteResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type WriteResponse_Length struct { + Length uint32 `protobuf:"varint,2,opt,name=length,proto3,oneof"` +} + +func (*WriteResponse_ErrorNumber) isWriteResponse_Result() {} + +func (*WriteResponse_Length) isWriteResponse_Result() {} + +func (m *WriteResponse) GetResult() isWriteResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *WriteResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*WriteResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *WriteResponse) GetLength() uint32 { + if x, ok := m.GetResult().(*WriteResponse_Length); ok { + return x.Length + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*WriteResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*WriteResponse_ErrorNumber)(nil), + (*WriteResponse_Length)(nil), + } +} + +type WriteFileRequest struct { + Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` + Content []byte `protobuf:"bytes,2,opt,name=content,proto3" json:"content,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *WriteFileRequest) Reset() { *m = WriteFileRequest{} } +func (m *WriteFileRequest) String() string { return proto.CompactTextString(m) } +func (*WriteFileRequest) ProtoMessage() {} +func (*WriteFileRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{13} +} + +func (m *WriteFileRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_WriteFileRequest.Unmarshal(m, b) +} +func (m *WriteFileRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_WriteFileRequest.Marshal(b, m, deterministic) +} +func (m *WriteFileRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteFileRequest.Merge(m, src) +} +func (m *WriteFileRequest) XXX_Size() int { + return xxx_messageInfo_WriteFileRequest.Size(m) +} +func (m *WriteFileRequest) XXX_DiscardUnknown() { + xxx_messageInfo_WriteFileRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteFileRequest proto.InternalMessageInfo + +func (m *WriteFileRequest) GetPath() string { + if m != nil { + return m.Path + } + return "" +} + +func (m *WriteFileRequest) GetContent() []byte { + if m != nil { + return m.Content + } + return nil +} + +type WriteFileResponse struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3" json:"error_number,omitempty"` + Written uint32 `protobuf:"varint,2,opt,name=written,proto3" json:"written,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *WriteFileResponse) Reset() { *m = WriteFileResponse{} } +func (m *WriteFileResponse) String() string { return proto.CompactTextString(m) } +func (*WriteFileResponse) ProtoMessage() {} +func (*WriteFileResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{14} +} + +func (m *WriteFileResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_WriteFileResponse.Unmarshal(m, b) +} +func (m *WriteFileResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_WriteFileResponse.Marshal(b, m, deterministic) +} +func (m *WriteFileResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteFileResponse.Merge(m, src) +} +func (m *WriteFileResponse) XXX_Size() int { + return xxx_messageInfo_WriteFileResponse.Size(m) +} +func (m *WriteFileResponse) XXX_DiscardUnknown() { + xxx_messageInfo_WriteFileResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteFileResponse proto.InternalMessageInfo + +func (m *WriteFileResponse) GetErrorNumber() uint32 { + if m != nil { + return m.ErrorNumber + } + return 0 +} + +func (m *WriteFileResponse) GetWritten() uint32 { + if m != nil { + return m.Written + } + return 0 +} + +type AddressResponse struct { + Address []byte `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` + Length uint32 `protobuf:"varint,2,opt,name=length,proto3" json:"length,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AddressResponse) Reset() { *m = AddressResponse{} } +func (m *AddressResponse) String() string { return proto.CompactTextString(m) } +func (*AddressResponse) ProtoMessage() {} +func (*AddressResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{15} +} + +func (m *AddressResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_AddressResponse.Unmarshal(m, b) +} +func (m *AddressResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_AddressResponse.Marshal(b, m, deterministic) +} +func (m *AddressResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AddressResponse.Merge(m, src) +} +func (m *AddressResponse) XXX_Size() int { + return xxx_messageInfo_AddressResponse.Size(m) +} +func (m *AddressResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AddressResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_AddressResponse proto.InternalMessageInfo + +func (m *AddressResponse) GetAddress() []byte { + if m != nil { + return m.Address + } + return nil +} + +func (m *AddressResponse) GetLength() uint32 { + if m != nil { + return m.Length + } + return 0 +} + +type RecvmsgResponse struct { + // Types that are valid to be assigned to Result: + // *RecvmsgResponse_ErrorNumber + // *RecvmsgResponse_Payload + Result isRecvmsgResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RecvmsgResponse) Reset() { *m = RecvmsgResponse{} } +func (m *RecvmsgResponse) String() string { return proto.CompactTextString(m) } +func (*RecvmsgResponse) ProtoMessage() {} +func (*RecvmsgResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{16} +} + +func (m *RecvmsgResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_RecvmsgResponse.Unmarshal(m, b) +} +func (m *RecvmsgResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_RecvmsgResponse.Marshal(b, m, deterministic) +} +func (m *RecvmsgResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RecvmsgResponse.Merge(m, src) +} +func (m *RecvmsgResponse) XXX_Size() int { + return xxx_messageInfo_RecvmsgResponse.Size(m) +} +func (m *RecvmsgResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RecvmsgResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_RecvmsgResponse proto.InternalMessageInfo + +type isRecvmsgResponse_Result interface { + isRecvmsgResponse_Result() +} + +type RecvmsgResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type RecvmsgResponse_Payload struct { + Payload *RecvmsgResponse_ResultPayload `protobuf:"bytes,2,opt,name=payload,proto3,oneof"` +} + +func (*RecvmsgResponse_ErrorNumber) isRecvmsgResponse_Result() {} + +func (*RecvmsgResponse_Payload) isRecvmsgResponse_Result() {} + +func (m *RecvmsgResponse) GetResult() isRecvmsgResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *RecvmsgResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*RecvmsgResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *RecvmsgResponse) GetPayload() *RecvmsgResponse_ResultPayload { + if x, ok := m.GetResult().(*RecvmsgResponse_Payload); ok { + return x.Payload + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*RecvmsgResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*RecvmsgResponse_ErrorNumber)(nil), + (*RecvmsgResponse_Payload)(nil), + } +} + +type RecvmsgResponse_ResultPayload struct { + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + Address *AddressResponse `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` + Length uint32 `protobuf:"varint,3,opt,name=length,proto3" json:"length,omitempty"` + CmsgData []byte `protobuf:"bytes,4,opt,name=cmsg_data,json=cmsgData,proto3" json:"cmsg_data,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RecvmsgResponse_ResultPayload) Reset() { *m = RecvmsgResponse_ResultPayload{} } +func (m *RecvmsgResponse_ResultPayload) String() string { return proto.CompactTextString(m) } +func (*RecvmsgResponse_ResultPayload) ProtoMessage() {} +func (*RecvmsgResponse_ResultPayload) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{16, 0} +} + +func (m *RecvmsgResponse_ResultPayload) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_RecvmsgResponse_ResultPayload.Unmarshal(m, b) +} +func (m *RecvmsgResponse_ResultPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_RecvmsgResponse_ResultPayload.Marshal(b, m, deterministic) +} +func (m *RecvmsgResponse_ResultPayload) XXX_Merge(src proto.Message) { + xxx_messageInfo_RecvmsgResponse_ResultPayload.Merge(m, src) +} +func (m *RecvmsgResponse_ResultPayload) XXX_Size() int { + return xxx_messageInfo_RecvmsgResponse_ResultPayload.Size(m) +} +func (m *RecvmsgResponse_ResultPayload) XXX_DiscardUnknown() { + xxx_messageInfo_RecvmsgResponse_ResultPayload.DiscardUnknown(m) +} + +var xxx_messageInfo_RecvmsgResponse_ResultPayload proto.InternalMessageInfo + +func (m *RecvmsgResponse_ResultPayload) GetData() []byte { + if m != nil { + return m.Data + } + return nil +} + +func (m *RecvmsgResponse_ResultPayload) GetAddress() *AddressResponse { + if m != nil { + return m.Address + } + return nil +} + +func (m *RecvmsgResponse_ResultPayload) GetLength() uint32 { + if m != nil { + return m.Length + } + return 0 +} + +func (m *RecvmsgResponse_ResultPayload) GetCmsgData() []byte { + if m != nil { + return m.CmsgData + } + return nil +} + +type BindRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Address []byte `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BindRequest) Reset() { *m = BindRequest{} } +func (m *BindRequest) String() string { return proto.CompactTextString(m) } +func (*BindRequest) ProtoMessage() {} +func (*BindRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{17} +} + +func (m *BindRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_BindRequest.Unmarshal(m, b) +} +func (m *BindRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_BindRequest.Marshal(b, m, deterministic) +} +func (m *BindRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_BindRequest.Merge(m, src) +} +func (m *BindRequest) XXX_Size() int { + return xxx_messageInfo_BindRequest.Size(m) +} +func (m *BindRequest) XXX_DiscardUnknown() { + xxx_messageInfo_BindRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_BindRequest proto.InternalMessageInfo + +func (m *BindRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *BindRequest) GetAddress() []byte { + if m != nil { + return m.Address + } + return nil +} + +type BindResponse struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3" json:"error_number,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BindResponse) Reset() { *m = BindResponse{} } +func (m *BindResponse) String() string { return proto.CompactTextString(m) } +func (*BindResponse) ProtoMessage() {} +func (*BindResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{18} +} + +func (m *BindResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_BindResponse.Unmarshal(m, b) +} +func (m *BindResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_BindResponse.Marshal(b, m, deterministic) +} +func (m *BindResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_BindResponse.Merge(m, src) +} +func (m *BindResponse) XXX_Size() int { + return xxx_messageInfo_BindResponse.Size(m) +} +func (m *BindResponse) XXX_DiscardUnknown() { + xxx_messageInfo_BindResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_BindResponse proto.InternalMessageInfo + +func (m *BindResponse) GetErrorNumber() uint32 { + if m != nil { + return m.ErrorNumber + } + return 0 +} + +type AcceptRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Peer bool `protobuf:"varint,2,opt,name=peer,proto3" json:"peer,omitempty"` + Flags int64 `protobuf:"varint,3,opt,name=flags,proto3" json:"flags,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AcceptRequest) Reset() { *m = AcceptRequest{} } +func (m *AcceptRequest) String() string { return proto.CompactTextString(m) } +func (*AcceptRequest) ProtoMessage() {} +func (*AcceptRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{19} +} + +func (m *AcceptRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_AcceptRequest.Unmarshal(m, b) +} +func (m *AcceptRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_AcceptRequest.Marshal(b, m, deterministic) +} +func (m *AcceptRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AcceptRequest.Merge(m, src) +} +func (m *AcceptRequest) XXX_Size() int { + return xxx_messageInfo_AcceptRequest.Size(m) +} +func (m *AcceptRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AcceptRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_AcceptRequest proto.InternalMessageInfo + +func (m *AcceptRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *AcceptRequest) GetPeer() bool { + if m != nil { + return m.Peer + } + return false +} + +func (m *AcceptRequest) GetFlags() int64 { + if m != nil { + return m.Flags + } + return 0 +} + +type AcceptResponse struct { + // Types that are valid to be assigned to Result: + // *AcceptResponse_ErrorNumber + // *AcceptResponse_Payload + Result isAcceptResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AcceptResponse) Reset() { *m = AcceptResponse{} } +func (m *AcceptResponse) String() string { return proto.CompactTextString(m) } +func (*AcceptResponse) ProtoMessage() {} +func (*AcceptResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{20} +} + +func (m *AcceptResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_AcceptResponse.Unmarshal(m, b) +} +func (m *AcceptResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_AcceptResponse.Marshal(b, m, deterministic) +} +func (m *AcceptResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AcceptResponse.Merge(m, src) +} +func (m *AcceptResponse) XXX_Size() int { + return xxx_messageInfo_AcceptResponse.Size(m) +} +func (m *AcceptResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AcceptResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_AcceptResponse proto.InternalMessageInfo + +type isAcceptResponse_Result interface { + isAcceptResponse_Result() +} + +type AcceptResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type AcceptResponse_Payload struct { + Payload *AcceptResponse_ResultPayload `protobuf:"bytes,2,opt,name=payload,proto3,oneof"` +} + +func (*AcceptResponse_ErrorNumber) isAcceptResponse_Result() {} + +func (*AcceptResponse_Payload) isAcceptResponse_Result() {} + +func (m *AcceptResponse) GetResult() isAcceptResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *AcceptResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*AcceptResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *AcceptResponse) GetPayload() *AcceptResponse_ResultPayload { + if x, ok := m.GetResult().(*AcceptResponse_Payload); ok { + return x.Payload + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*AcceptResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*AcceptResponse_ErrorNumber)(nil), + (*AcceptResponse_Payload)(nil), + } +} + +type AcceptResponse_ResultPayload struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Address *AddressResponse `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AcceptResponse_ResultPayload) Reset() { *m = AcceptResponse_ResultPayload{} } +func (m *AcceptResponse_ResultPayload) String() string { return proto.CompactTextString(m) } +func (*AcceptResponse_ResultPayload) ProtoMessage() {} +func (*AcceptResponse_ResultPayload) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{20, 0} +} + +func (m *AcceptResponse_ResultPayload) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_AcceptResponse_ResultPayload.Unmarshal(m, b) +} +func (m *AcceptResponse_ResultPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_AcceptResponse_ResultPayload.Marshal(b, m, deterministic) +} +func (m *AcceptResponse_ResultPayload) XXX_Merge(src proto.Message) { + xxx_messageInfo_AcceptResponse_ResultPayload.Merge(m, src) +} +func (m *AcceptResponse_ResultPayload) XXX_Size() int { + return xxx_messageInfo_AcceptResponse_ResultPayload.Size(m) +} +func (m *AcceptResponse_ResultPayload) XXX_DiscardUnknown() { + xxx_messageInfo_AcceptResponse_ResultPayload.DiscardUnknown(m) +} + +var xxx_messageInfo_AcceptResponse_ResultPayload proto.InternalMessageInfo + +func (m *AcceptResponse_ResultPayload) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *AcceptResponse_ResultPayload) GetAddress() *AddressResponse { + if m != nil { + return m.Address + } + return nil +} + +type ConnectRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Address []byte `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ConnectRequest) Reset() { *m = ConnectRequest{} } +func (m *ConnectRequest) String() string { return proto.CompactTextString(m) } +func (*ConnectRequest) ProtoMessage() {} +func (*ConnectRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{21} +} + +func (m *ConnectRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ConnectRequest.Unmarshal(m, b) +} +func (m *ConnectRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ConnectRequest.Marshal(b, m, deterministic) +} +func (m *ConnectRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConnectRequest.Merge(m, src) +} +func (m *ConnectRequest) XXX_Size() int { + return xxx_messageInfo_ConnectRequest.Size(m) +} +func (m *ConnectRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ConnectRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ConnectRequest proto.InternalMessageInfo + +func (m *ConnectRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *ConnectRequest) GetAddress() []byte { + if m != nil { + return m.Address + } + return nil +} + +type ConnectResponse struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3" json:"error_number,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ConnectResponse) Reset() { *m = ConnectResponse{} } +func (m *ConnectResponse) String() string { return proto.CompactTextString(m) } +func (*ConnectResponse) ProtoMessage() {} +func (*ConnectResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{22} +} + +func (m *ConnectResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ConnectResponse.Unmarshal(m, b) +} +func (m *ConnectResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ConnectResponse.Marshal(b, m, deterministic) +} +func (m *ConnectResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConnectResponse.Merge(m, src) +} +func (m *ConnectResponse) XXX_Size() int { + return xxx_messageInfo_ConnectResponse.Size(m) +} +func (m *ConnectResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ConnectResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ConnectResponse proto.InternalMessageInfo + +func (m *ConnectResponse) GetErrorNumber() uint32 { + if m != nil { + return m.ErrorNumber + } + return 0 +} + +type ListenRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Backlog int64 `protobuf:"varint,2,opt,name=backlog,proto3" json:"backlog,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ListenRequest) Reset() { *m = ListenRequest{} } +func (m *ListenRequest) String() string { return proto.CompactTextString(m) } +func (*ListenRequest) ProtoMessage() {} +func (*ListenRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{23} +} + +func (m *ListenRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ListenRequest.Unmarshal(m, b) +} +func (m *ListenRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ListenRequest.Marshal(b, m, deterministic) +} +func (m *ListenRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ListenRequest.Merge(m, src) +} +func (m *ListenRequest) XXX_Size() int { + return xxx_messageInfo_ListenRequest.Size(m) +} +func (m *ListenRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ListenRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ListenRequest proto.InternalMessageInfo + +func (m *ListenRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *ListenRequest) GetBacklog() int64 { + if m != nil { + return m.Backlog + } + return 0 +} + +type ListenResponse struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3" json:"error_number,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ListenResponse) Reset() { *m = ListenResponse{} } +func (m *ListenResponse) String() string { return proto.CompactTextString(m) } +func (*ListenResponse) ProtoMessage() {} +func (*ListenResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{24} +} + +func (m *ListenResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ListenResponse.Unmarshal(m, b) +} +func (m *ListenResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ListenResponse.Marshal(b, m, deterministic) +} +func (m *ListenResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ListenResponse.Merge(m, src) +} +func (m *ListenResponse) XXX_Size() int { + return xxx_messageInfo_ListenResponse.Size(m) +} +func (m *ListenResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ListenResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ListenResponse proto.InternalMessageInfo + +func (m *ListenResponse) GetErrorNumber() uint32 { + if m != nil { + return m.ErrorNumber + } + return 0 +} + +type ShutdownRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + How int64 `protobuf:"varint,2,opt,name=how,proto3" json:"how,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ShutdownRequest) Reset() { *m = ShutdownRequest{} } +func (m *ShutdownRequest) String() string { return proto.CompactTextString(m) } +func (*ShutdownRequest) ProtoMessage() {} +func (*ShutdownRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{25} +} + +func (m *ShutdownRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ShutdownRequest.Unmarshal(m, b) +} +func (m *ShutdownRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ShutdownRequest.Marshal(b, m, deterministic) +} +func (m *ShutdownRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShutdownRequest.Merge(m, src) +} +func (m *ShutdownRequest) XXX_Size() int { + return xxx_messageInfo_ShutdownRequest.Size(m) +} +func (m *ShutdownRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ShutdownRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ShutdownRequest proto.InternalMessageInfo + +func (m *ShutdownRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *ShutdownRequest) GetHow() int64 { + if m != nil { + return m.How + } + return 0 +} + +type ShutdownResponse struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3" json:"error_number,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ShutdownResponse) Reset() { *m = ShutdownResponse{} } +func (m *ShutdownResponse) String() string { return proto.CompactTextString(m) } +func (*ShutdownResponse) ProtoMessage() {} +func (*ShutdownResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{26} +} + +func (m *ShutdownResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ShutdownResponse.Unmarshal(m, b) +} +func (m *ShutdownResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ShutdownResponse.Marshal(b, m, deterministic) +} +func (m *ShutdownResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShutdownResponse.Merge(m, src) +} +func (m *ShutdownResponse) XXX_Size() int { + return xxx_messageInfo_ShutdownResponse.Size(m) +} +func (m *ShutdownResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ShutdownResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ShutdownResponse proto.InternalMessageInfo + +func (m *ShutdownResponse) GetErrorNumber() uint32 { + if m != nil { + return m.ErrorNumber + } + return 0 +} + +type CloseRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CloseRequest) Reset() { *m = CloseRequest{} } +func (m *CloseRequest) String() string { return proto.CompactTextString(m) } +func (*CloseRequest) ProtoMessage() {} +func (*CloseRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{27} +} + +func (m *CloseRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_CloseRequest.Unmarshal(m, b) +} +func (m *CloseRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_CloseRequest.Marshal(b, m, deterministic) +} +func (m *CloseRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CloseRequest.Merge(m, src) +} +func (m *CloseRequest) XXX_Size() int { + return xxx_messageInfo_CloseRequest.Size(m) +} +func (m *CloseRequest) XXX_DiscardUnknown() { + xxx_messageInfo_CloseRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_CloseRequest proto.InternalMessageInfo + +func (m *CloseRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +type CloseResponse struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3" json:"error_number,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CloseResponse) Reset() { *m = CloseResponse{} } +func (m *CloseResponse) String() string { return proto.CompactTextString(m) } +func (*CloseResponse) ProtoMessage() {} +func (*CloseResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{28} +} + +func (m *CloseResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_CloseResponse.Unmarshal(m, b) +} +func (m *CloseResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_CloseResponse.Marshal(b, m, deterministic) +} +func (m *CloseResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CloseResponse.Merge(m, src) +} +func (m *CloseResponse) XXX_Size() int { + return xxx_messageInfo_CloseResponse.Size(m) +} +func (m *CloseResponse) XXX_DiscardUnknown() { + xxx_messageInfo_CloseResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_CloseResponse proto.InternalMessageInfo + +func (m *CloseResponse) GetErrorNumber() uint32 { + if m != nil { + return m.ErrorNumber + } + return 0 +} + +type GetSockOptRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Level int64 `protobuf:"varint,2,opt,name=level,proto3" json:"level,omitempty"` + Name int64 `protobuf:"varint,3,opt,name=name,proto3" json:"name,omitempty"` + Length uint32 `protobuf:"varint,4,opt,name=length,proto3" json:"length,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetSockOptRequest) Reset() { *m = GetSockOptRequest{} } +func (m *GetSockOptRequest) String() string { return proto.CompactTextString(m) } +func (*GetSockOptRequest) ProtoMessage() {} +func (*GetSockOptRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{29} +} + +func (m *GetSockOptRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetSockOptRequest.Unmarshal(m, b) +} +func (m *GetSockOptRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetSockOptRequest.Marshal(b, m, deterministic) +} +func (m *GetSockOptRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetSockOptRequest.Merge(m, src) +} +func (m *GetSockOptRequest) XXX_Size() int { + return xxx_messageInfo_GetSockOptRequest.Size(m) +} +func (m *GetSockOptRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetSockOptRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GetSockOptRequest proto.InternalMessageInfo + +func (m *GetSockOptRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *GetSockOptRequest) GetLevel() int64 { + if m != nil { + return m.Level + } + return 0 +} + +func (m *GetSockOptRequest) GetName() int64 { + if m != nil { + return m.Name + } + return 0 +} + +func (m *GetSockOptRequest) GetLength() uint32 { + if m != nil { + return m.Length + } + return 0 +} + +type GetSockOptResponse struct { + // Types that are valid to be assigned to Result: + // *GetSockOptResponse_ErrorNumber + // *GetSockOptResponse_Opt + Result isGetSockOptResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetSockOptResponse) Reset() { *m = GetSockOptResponse{} } +func (m *GetSockOptResponse) String() string { return proto.CompactTextString(m) } +func (*GetSockOptResponse) ProtoMessage() {} +func (*GetSockOptResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{30} +} + +func (m *GetSockOptResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetSockOptResponse.Unmarshal(m, b) +} +func (m *GetSockOptResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetSockOptResponse.Marshal(b, m, deterministic) +} +func (m *GetSockOptResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetSockOptResponse.Merge(m, src) +} +func (m *GetSockOptResponse) XXX_Size() int { + return xxx_messageInfo_GetSockOptResponse.Size(m) +} +func (m *GetSockOptResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetSockOptResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_GetSockOptResponse proto.InternalMessageInfo + +type isGetSockOptResponse_Result interface { + isGetSockOptResponse_Result() +} + +type GetSockOptResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type GetSockOptResponse_Opt struct { + Opt []byte `protobuf:"bytes,2,opt,name=opt,proto3,oneof"` +} + +func (*GetSockOptResponse_ErrorNumber) isGetSockOptResponse_Result() {} + +func (*GetSockOptResponse_Opt) isGetSockOptResponse_Result() {} + +func (m *GetSockOptResponse) GetResult() isGetSockOptResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *GetSockOptResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*GetSockOptResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *GetSockOptResponse) GetOpt() []byte { + if x, ok := m.GetResult().(*GetSockOptResponse_Opt); ok { + return x.Opt + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*GetSockOptResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*GetSockOptResponse_ErrorNumber)(nil), + (*GetSockOptResponse_Opt)(nil), + } +} + +type SetSockOptRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Level int64 `protobuf:"varint,2,opt,name=level,proto3" json:"level,omitempty"` + Name int64 `protobuf:"varint,3,opt,name=name,proto3" json:"name,omitempty"` + Opt []byte `protobuf:"bytes,4,opt,name=opt,proto3" json:"opt,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SetSockOptRequest) Reset() { *m = SetSockOptRequest{} } +func (m *SetSockOptRequest) String() string { return proto.CompactTextString(m) } +func (*SetSockOptRequest) ProtoMessage() {} +func (*SetSockOptRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{31} +} + +func (m *SetSockOptRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SetSockOptRequest.Unmarshal(m, b) +} +func (m *SetSockOptRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SetSockOptRequest.Marshal(b, m, deterministic) +} +func (m *SetSockOptRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SetSockOptRequest.Merge(m, src) +} +func (m *SetSockOptRequest) XXX_Size() int { + return xxx_messageInfo_SetSockOptRequest.Size(m) +} +func (m *SetSockOptRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SetSockOptRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SetSockOptRequest proto.InternalMessageInfo + +func (m *SetSockOptRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *SetSockOptRequest) GetLevel() int64 { + if m != nil { + return m.Level + } + return 0 +} + +func (m *SetSockOptRequest) GetName() int64 { + if m != nil { + return m.Name + } + return 0 +} + +func (m *SetSockOptRequest) GetOpt() []byte { + if m != nil { + return m.Opt + } + return nil +} + +type SetSockOptResponse struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3" json:"error_number,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SetSockOptResponse) Reset() { *m = SetSockOptResponse{} } +func (m *SetSockOptResponse) String() string { return proto.CompactTextString(m) } +func (*SetSockOptResponse) ProtoMessage() {} +func (*SetSockOptResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{32} +} + +func (m *SetSockOptResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SetSockOptResponse.Unmarshal(m, b) +} +func (m *SetSockOptResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SetSockOptResponse.Marshal(b, m, deterministic) +} +func (m *SetSockOptResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SetSockOptResponse.Merge(m, src) +} +func (m *SetSockOptResponse) XXX_Size() int { + return xxx_messageInfo_SetSockOptResponse.Size(m) +} +func (m *SetSockOptResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SetSockOptResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SetSockOptResponse proto.InternalMessageInfo + +func (m *SetSockOptResponse) GetErrorNumber() uint32 { + if m != nil { + return m.ErrorNumber + } + return 0 +} + +type GetSockNameRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetSockNameRequest) Reset() { *m = GetSockNameRequest{} } +func (m *GetSockNameRequest) String() string { return proto.CompactTextString(m) } +func (*GetSockNameRequest) ProtoMessage() {} +func (*GetSockNameRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{33} +} + +func (m *GetSockNameRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetSockNameRequest.Unmarshal(m, b) +} +func (m *GetSockNameRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetSockNameRequest.Marshal(b, m, deterministic) +} +func (m *GetSockNameRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetSockNameRequest.Merge(m, src) +} +func (m *GetSockNameRequest) XXX_Size() int { + return xxx_messageInfo_GetSockNameRequest.Size(m) +} +func (m *GetSockNameRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetSockNameRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GetSockNameRequest proto.InternalMessageInfo + +func (m *GetSockNameRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +type GetSockNameResponse struct { + // Types that are valid to be assigned to Result: + // *GetSockNameResponse_ErrorNumber + // *GetSockNameResponse_Address + Result isGetSockNameResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetSockNameResponse) Reset() { *m = GetSockNameResponse{} } +func (m *GetSockNameResponse) String() string { return proto.CompactTextString(m) } +func (*GetSockNameResponse) ProtoMessage() {} +func (*GetSockNameResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{34} +} + +func (m *GetSockNameResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetSockNameResponse.Unmarshal(m, b) +} +func (m *GetSockNameResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetSockNameResponse.Marshal(b, m, deterministic) +} +func (m *GetSockNameResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetSockNameResponse.Merge(m, src) +} +func (m *GetSockNameResponse) XXX_Size() int { + return xxx_messageInfo_GetSockNameResponse.Size(m) +} +func (m *GetSockNameResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetSockNameResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_GetSockNameResponse proto.InternalMessageInfo + +type isGetSockNameResponse_Result interface { + isGetSockNameResponse_Result() +} + +type GetSockNameResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type GetSockNameResponse_Address struct { + Address *AddressResponse `protobuf:"bytes,2,opt,name=address,proto3,oneof"` +} + +func (*GetSockNameResponse_ErrorNumber) isGetSockNameResponse_Result() {} + +func (*GetSockNameResponse_Address) isGetSockNameResponse_Result() {} + +func (m *GetSockNameResponse) GetResult() isGetSockNameResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *GetSockNameResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*GetSockNameResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *GetSockNameResponse) GetAddress() *AddressResponse { + if x, ok := m.GetResult().(*GetSockNameResponse_Address); ok { + return x.Address + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*GetSockNameResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*GetSockNameResponse_ErrorNumber)(nil), + (*GetSockNameResponse_Address)(nil), + } +} + +type GetPeerNameRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetPeerNameRequest) Reset() { *m = GetPeerNameRequest{} } +func (m *GetPeerNameRequest) String() string { return proto.CompactTextString(m) } +func (*GetPeerNameRequest) ProtoMessage() {} +func (*GetPeerNameRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{35} +} + +func (m *GetPeerNameRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetPeerNameRequest.Unmarshal(m, b) +} +func (m *GetPeerNameRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetPeerNameRequest.Marshal(b, m, deterministic) +} +func (m *GetPeerNameRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetPeerNameRequest.Merge(m, src) +} +func (m *GetPeerNameRequest) XXX_Size() int { + return xxx_messageInfo_GetPeerNameRequest.Size(m) +} +func (m *GetPeerNameRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetPeerNameRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GetPeerNameRequest proto.InternalMessageInfo + +func (m *GetPeerNameRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +type GetPeerNameResponse struct { + // Types that are valid to be assigned to Result: + // *GetPeerNameResponse_ErrorNumber + // *GetPeerNameResponse_Address + Result isGetPeerNameResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetPeerNameResponse) Reset() { *m = GetPeerNameResponse{} } +func (m *GetPeerNameResponse) String() string { return proto.CompactTextString(m) } +func (*GetPeerNameResponse) ProtoMessage() {} +func (*GetPeerNameResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{36} +} + +func (m *GetPeerNameResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetPeerNameResponse.Unmarshal(m, b) +} +func (m *GetPeerNameResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetPeerNameResponse.Marshal(b, m, deterministic) +} +func (m *GetPeerNameResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetPeerNameResponse.Merge(m, src) +} +func (m *GetPeerNameResponse) XXX_Size() int { + return xxx_messageInfo_GetPeerNameResponse.Size(m) +} +func (m *GetPeerNameResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetPeerNameResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_GetPeerNameResponse proto.InternalMessageInfo + +type isGetPeerNameResponse_Result interface { + isGetPeerNameResponse_Result() +} + +type GetPeerNameResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type GetPeerNameResponse_Address struct { + Address *AddressResponse `protobuf:"bytes,2,opt,name=address,proto3,oneof"` +} + +func (*GetPeerNameResponse_ErrorNumber) isGetPeerNameResponse_Result() {} + +func (*GetPeerNameResponse_Address) isGetPeerNameResponse_Result() {} + +func (m *GetPeerNameResponse) GetResult() isGetPeerNameResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *GetPeerNameResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*GetPeerNameResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *GetPeerNameResponse) GetAddress() *AddressResponse { + if x, ok := m.GetResult().(*GetPeerNameResponse_Address); ok { + return x.Address + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*GetPeerNameResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*GetPeerNameResponse_ErrorNumber)(nil), + (*GetPeerNameResponse_Address)(nil), + } +} + +type SocketRequest struct { + Family int64 `protobuf:"varint,1,opt,name=family,proto3" json:"family,omitempty"` + Type int64 `protobuf:"varint,2,opt,name=type,proto3" json:"type,omitempty"` + Protocol int64 `protobuf:"varint,3,opt,name=protocol,proto3" json:"protocol,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SocketRequest) Reset() { *m = SocketRequest{} } +func (m *SocketRequest) String() string { return proto.CompactTextString(m) } +func (*SocketRequest) ProtoMessage() {} +func (*SocketRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{37} +} + +func (m *SocketRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SocketRequest.Unmarshal(m, b) +} +func (m *SocketRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SocketRequest.Marshal(b, m, deterministic) +} +func (m *SocketRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SocketRequest.Merge(m, src) +} +func (m *SocketRequest) XXX_Size() int { + return xxx_messageInfo_SocketRequest.Size(m) +} +func (m *SocketRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SocketRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SocketRequest proto.InternalMessageInfo + +func (m *SocketRequest) GetFamily() int64 { + if m != nil { + return m.Family + } + return 0 +} + +func (m *SocketRequest) GetType() int64 { + if m != nil { + return m.Type + } + return 0 +} + +func (m *SocketRequest) GetProtocol() int64 { + if m != nil { + return m.Protocol + } + return 0 +} + +type SocketResponse struct { + // Types that are valid to be assigned to Result: + // *SocketResponse_ErrorNumber + // *SocketResponse_Fd + Result isSocketResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SocketResponse) Reset() { *m = SocketResponse{} } +func (m *SocketResponse) String() string { return proto.CompactTextString(m) } +func (*SocketResponse) ProtoMessage() {} +func (*SocketResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{38} +} + +func (m *SocketResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SocketResponse.Unmarshal(m, b) +} +func (m *SocketResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SocketResponse.Marshal(b, m, deterministic) +} +func (m *SocketResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SocketResponse.Merge(m, src) +} +func (m *SocketResponse) XXX_Size() int { + return xxx_messageInfo_SocketResponse.Size(m) +} +func (m *SocketResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SocketResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SocketResponse proto.InternalMessageInfo + +type isSocketResponse_Result interface { + isSocketResponse_Result() +} + +type SocketResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type SocketResponse_Fd struct { + Fd uint32 `protobuf:"varint,2,opt,name=fd,proto3,oneof"` +} + +func (*SocketResponse_ErrorNumber) isSocketResponse_Result() {} + +func (*SocketResponse_Fd) isSocketResponse_Result() {} + +func (m *SocketResponse) GetResult() isSocketResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *SocketResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*SocketResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *SocketResponse) GetFd() uint32 { + if x, ok := m.GetResult().(*SocketResponse_Fd); ok { + return x.Fd + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*SocketResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*SocketResponse_ErrorNumber)(nil), + (*SocketResponse_Fd)(nil), + } +} + +type EpollWaitRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + NumEvents uint32 `protobuf:"varint,2,opt,name=num_events,json=numEvents,proto3" json:"num_events,omitempty"` + Msec int64 `protobuf:"zigzag64,3,opt,name=msec,proto3" json:"msec,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EpollWaitRequest) Reset() { *m = EpollWaitRequest{} } +func (m *EpollWaitRequest) String() string { return proto.CompactTextString(m) } +func (*EpollWaitRequest) ProtoMessage() {} +func (*EpollWaitRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{39} +} + +func (m *EpollWaitRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_EpollWaitRequest.Unmarshal(m, b) +} +func (m *EpollWaitRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_EpollWaitRequest.Marshal(b, m, deterministic) +} +func (m *EpollWaitRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_EpollWaitRequest.Merge(m, src) +} +func (m *EpollWaitRequest) XXX_Size() int { + return xxx_messageInfo_EpollWaitRequest.Size(m) +} +func (m *EpollWaitRequest) XXX_DiscardUnknown() { + xxx_messageInfo_EpollWaitRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_EpollWaitRequest proto.InternalMessageInfo + +func (m *EpollWaitRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *EpollWaitRequest) GetNumEvents() uint32 { + if m != nil { + return m.NumEvents + } + return 0 +} + +func (m *EpollWaitRequest) GetMsec() int64 { + if m != nil { + return m.Msec + } + return 0 +} + +type EpollEvent struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Events uint32 `protobuf:"varint,2,opt,name=events,proto3" json:"events,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EpollEvent) Reset() { *m = EpollEvent{} } +func (m *EpollEvent) String() string { return proto.CompactTextString(m) } +func (*EpollEvent) ProtoMessage() {} +func (*EpollEvent) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{40} +} + +func (m *EpollEvent) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_EpollEvent.Unmarshal(m, b) +} +func (m *EpollEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_EpollEvent.Marshal(b, m, deterministic) +} +func (m *EpollEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_EpollEvent.Merge(m, src) +} +func (m *EpollEvent) XXX_Size() int { + return xxx_messageInfo_EpollEvent.Size(m) +} +func (m *EpollEvent) XXX_DiscardUnknown() { + xxx_messageInfo_EpollEvent.DiscardUnknown(m) +} + +var xxx_messageInfo_EpollEvent proto.InternalMessageInfo + +func (m *EpollEvent) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *EpollEvent) GetEvents() uint32 { + if m != nil { + return m.Events + } + return 0 +} + +type EpollEvents struct { + Events []*EpollEvent `protobuf:"bytes,1,rep,name=events,proto3" json:"events,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EpollEvents) Reset() { *m = EpollEvents{} } +func (m *EpollEvents) String() string { return proto.CompactTextString(m) } +func (*EpollEvents) ProtoMessage() {} +func (*EpollEvents) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{41} +} + +func (m *EpollEvents) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_EpollEvents.Unmarshal(m, b) +} +func (m *EpollEvents) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_EpollEvents.Marshal(b, m, deterministic) +} +func (m *EpollEvents) XXX_Merge(src proto.Message) { + xxx_messageInfo_EpollEvents.Merge(m, src) +} +func (m *EpollEvents) XXX_Size() int { + return xxx_messageInfo_EpollEvents.Size(m) +} +func (m *EpollEvents) XXX_DiscardUnknown() { + xxx_messageInfo_EpollEvents.DiscardUnknown(m) +} + +var xxx_messageInfo_EpollEvents proto.InternalMessageInfo + +func (m *EpollEvents) GetEvents() []*EpollEvent { + if m != nil { + return m.Events + } + return nil +} + +type EpollWaitResponse struct { + // Types that are valid to be assigned to Result: + // *EpollWaitResponse_ErrorNumber + // *EpollWaitResponse_Events + Result isEpollWaitResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EpollWaitResponse) Reset() { *m = EpollWaitResponse{} } +func (m *EpollWaitResponse) String() string { return proto.CompactTextString(m) } +func (*EpollWaitResponse) ProtoMessage() {} +func (*EpollWaitResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{42} +} + +func (m *EpollWaitResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_EpollWaitResponse.Unmarshal(m, b) +} +func (m *EpollWaitResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_EpollWaitResponse.Marshal(b, m, deterministic) +} +func (m *EpollWaitResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_EpollWaitResponse.Merge(m, src) +} +func (m *EpollWaitResponse) XXX_Size() int { + return xxx_messageInfo_EpollWaitResponse.Size(m) +} +func (m *EpollWaitResponse) XXX_DiscardUnknown() { + xxx_messageInfo_EpollWaitResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_EpollWaitResponse proto.InternalMessageInfo + +type isEpollWaitResponse_Result interface { + isEpollWaitResponse_Result() +} + +type EpollWaitResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type EpollWaitResponse_Events struct { + Events *EpollEvents `protobuf:"bytes,2,opt,name=events,proto3,oneof"` +} + +func (*EpollWaitResponse_ErrorNumber) isEpollWaitResponse_Result() {} + +func (*EpollWaitResponse_Events) isEpollWaitResponse_Result() {} + +func (m *EpollWaitResponse) GetResult() isEpollWaitResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *EpollWaitResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*EpollWaitResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *EpollWaitResponse) GetEvents() *EpollEvents { + if x, ok := m.GetResult().(*EpollWaitResponse_Events); ok { + return x.Events + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*EpollWaitResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*EpollWaitResponse_ErrorNumber)(nil), + (*EpollWaitResponse_Events)(nil), + } +} + +type EpollCtlRequest struct { + Epfd uint32 `protobuf:"varint,1,opt,name=epfd,proto3" json:"epfd,omitempty"` + Op int64 `protobuf:"varint,2,opt,name=op,proto3" json:"op,omitempty"` + Fd uint32 `protobuf:"varint,3,opt,name=fd,proto3" json:"fd,omitempty"` + Event *EpollEvent `protobuf:"bytes,4,opt,name=event,proto3" json:"event,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EpollCtlRequest) Reset() { *m = EpollCtlRequest{} } +func (m *EpollCtlRequest) String() string { return proto.CompactTextString(m) } +func (*EpollCtlRequest) ProtoMessage() {} +func (*EpollCtlRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{43} +} + +func (m *EpollCtlRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_EpollCtlRequest.Unmarshal(m, b) +} +func (m *EpollCtlRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_EpollCtlRequest.Marshal(b, m, deterministic) +} +func (m *EpollCtlRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_EpollCtlRequest.Merge(m, src) +} +func (m *EpollCtlRequest) XXX_Size() int { + return xxx_messageInfo_EpollCtlRequest.Size(m) +} +func (m *EpollCtlRequest) XXX_DiscardUnknown() { + xxx_messageInfo_EpollCtlRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_EpollCtlRequest proto.InternalMessageInfo + +func (m *EpollCtlRequest) GetEpfd() uint32 { + if m != nil { + return m.Epfd + } + return 0 +} + +func (m *EpollCtlRequest) GetOp() int64 { + if m != nil { + return m.Op + } + return 0 +} + +func (m *EpollCtlRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *EpollCtlRequest) GetEvent() *EpollEvent { + if m != nil { + return m.Event + } + return nil +} + +type EpollCtlResponse struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3" json:"error_number,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EpollCtlResponse) Reset() { *m = EpollCtlResponse{} } +func (m *EpollCtlResponse) String() string { return proto.CompactTextString(m) } +func (*EpollCtlResponse) ProtoMessage() {} +func (*EpollCtlResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{44} +} + +func (m *EpollCtlResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_EpollCtlResponse.Unmarshal(m, b) +} +func (m *EpollCtlResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_EpollCtlResponse.Marshal(b, m, deterministic) +} +func (m *EpollCtlResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_EpollCtlResponse.Merge(m, src) +} +func (m *EpollCtlResponse) XXX_Size() int { + return xxx_messageInfo_EpollCtlResponse.Size(m) +} +func (m *EpollCtlResponse) XXX_DiscardUnknown() { + xxx_messageInfo_EpollCtlResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_EpollCtlResponse proto.InternalMessageInfo + +func (m *EpollCtlResponse) GetErrorNumber() uint32 { + if m != nil { + return m.ErrorNumber + } + return 0 +} + +type EpollCreate1Request struct { + Flag int64 `protobuf:"varint,1,opt,name=flag,proto3" json:"flag,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EpollCreate1Request) Reset() { *m = EpollCreate1Request{} } +func (m *EpollCreate1Request) String() string { return proto.CompactTextString(m) } +func (*EpollCreate1Request) ProtoMessage() {} +func (*EpollCreate1Request) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{45} +} + +func (m *EpollCreate1Request) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_EpollCreate1Request.Unmarshal(m, b) +} +func (m *EpollCreate1Request) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_EpollCreate1Request.Marshal(b, m, deterministic) +} +func (m *EpollCreate1Request) XXX_Merge(src proto.Message) { + xxx_messageInfo_EpollCreate1Request.Merge(m, src) +} +func (m *EpollCreate1Request) XXX_Size() int { + return xxx_messageInfo_EpollCreate1Request.Size(m) +} +func (m *EpollCreate1Request) XXX_DiscardUnknown() { + xxx_messageInfo_EpollCreate1Request.DiscardUnknown(m) +} + +var xxx_messageInfo_EpollCreate1Request proto.InternalMessageInfo + +func (m *EpollCreate1Request) GetFlag() int64 { + if m != nil { + return m.Flag + } + return 0 +} + +type EpollCreate1Response struct { + // Types that are valid to be assigned to Result: + // *EpollCreate1Response_ErrorNumber + // *EpollCreate1Response_Fd + Result isEpollCreate1Response_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EpollCreate1Response) Reset() { *m = EpollCreate1Response{} } +func (m *EpollCreate1Response) String() string { return proto.CompactTextString(m) } +func (*EpollCreate1Response) ProtoMessage() {} +func (*EpollCreate1Response) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{46} +} + +func (m *EpollCreate1Response) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_EpollCreate1Response.Unmarshal(m, b) +} +func (m *EpollCreate1Response) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_EpollCreate1Response.Marshal(b, m, deterministic) +} +func (m *EpollCreate1Response) XXX_Merge(src proto.Message) { + xxx_messageInfo_EpollCreate1Response.Merge(m, src) +} +func (m *EpollCreate1Response) XXX_Size() int { + return xxx_messageInfo_EpollCreate1Response.Size(m) +} +func (m *EpollCreate1Response) XXX_DiscardUnknown() { + xxx_messageInfo_EpollCreate1Response.DiscardUnknown(m) +} + +var xxx_messageInfo_EpollCreate1Response proto.InternalMessageInfo + +type isEpollCreate1Response_Result interface { + isEpollCreate1Response_Result() +} + +type EpollCreate1Response_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type EpollCreate1Response_Fd struct { + Fd uint32 `protobuf:"varint,2,opt,name=fd,proto3,oneof"` +} + +func (*EpollCreate1Response_ErrorNumber) isEpollCreate1Response_Result() {} + +func (*EpollCreate1Response_Fd) isEpollCreate1Response_Result() {} + +func (m *EpollCreate1Response) GetResult() isEpollCreate1Response_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *EpollCreate1Response) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*EpollCreate1Response_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *EpollCreate1Response) GetFd() uint32 { + if x, ok := m.GetResult().(*EpollCreate1Response_Fd); ok { + return x.Fd + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*EpollCreate1Response) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*EpollCreate1Response_ErrorNumber)(nil), + (*EpollCreate1Response_Fd)(nil), + } +} + +type PollRequest struct { + Fd uint32 `protobuf:"varint,1,opt,name=fd,proto3" json:"fd,omitempty"` + Events uint32 `protobuf:"varint,2,opt,name=events,proto3" json:"events,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *PollRequest) Reset() { *m = PollRequest{} } +func (m *PollRequest) String() string { return proto.CompactTextString(m) } +func (*PollRequest) ProtoMessage() {} +func (*PollRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{47} +} + +func (m *PollRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_PollRequest.Unmarshal(m, b) +} +func (m *PollRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_PollRequest.Marshal(b, m, deterministic) +} +func (m *PollRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_PollRequest.Merge(m, src) +} +func (m *PollRequest) XXX_Size() int { + return xxx_messageInfo_PollRequest.Size(m) +} +func (m *PollRequest) XXX_DiscardUnknown() { + xxx_messageInfo_PollRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_PollRequest proto.InternalMessageInfo + +func (m *PollRequest) GetFd() uint32 { + if m != nil { + return m.Fd + } + return 0 +} + +func (m *PollRequest) GetEvents() uint32 { + if m != nil { + return m.Events + } + return 0 +} + +type PollResponse struct { + // Types that are valid to be assigned to Result: + // *PollResponse_ErrorNumber + // *PollResponse_Events + Result isPollResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *PollResponse) Reset() { *m = PollResponse{} } +func (m *PollResponse) String() string { return proto.CompactTextString(m) } +func (*PollResponse) ProtoMessage() {} +func (*PollResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{48} +} + +func (m *PollResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_PollResponse.Unmarshal(m, b) +} +func (m *PollResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_PollResponse.Marshal(b, m, deterministic) +} +func (m *PollResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_PollResponse.Merge(m, src) +} +func (m *PollResponse) XXX_Size() int { + return xxx_messageInfo_PollResponse.Size(m) +} +func (m *PollResponse) XXX_DiscardUnknown() { + xxx_messageInfo_PollResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_PollResponse proto.InternalMessageInfo + +type isPollResponse_Result interface { + isPollResponse_Result() +} + +type PollResponse_ErrorNumber struct { + ErrorNumber uint32 `protobuf:"varint,1,opt,name=error_number,json=errorNumber,proto3,oneof"` +} + +type PollResponse_Events struct { + Events uint32 `protobuf:"varint,2,opt,name=events,proto3,oneof"` +} + +func (*PollResponse_ErrorNumber) isPollResponse_Result() {} + +func (*PollResponse_Events) isPollResponse_Result() {} + +func (m *PollResponse) GetResult() isPollResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *PollResponse) GetErrorNumber() uint32 { + if x, ok := m.GetResult().(*PollResponse_ErrorNumber); ok { + return x.ErrorNumber + } + return 0 +} + +func (m *PollResponse) GetEvents() uint32 { + if x, ok := m.GetResult().(*PollResponse_Events); ok { + return x.Events + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*PollResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*PollResponse_ErrorNumber)(nil), + (*PollResponse_Events)(nil), + } +} + +type SyscallRequest struct { + // Types that are valid to be assigned to Args: + // *SyscallRequest_Socket + // *SyscallRequest_Sendmsg + // *SyscallRequest_Recvmsg + // *SyscallRequest_Bind + // *SyscallRequest_Accept + // *SyscallRequest_Connect + // *SyscallRequest_Listen + // *SyscallRequest_Shutdown + // *SyscallRequest_Close + // *SyscallRequest_GetSockOpt + // *SyscallRequest_SetSockOpt + // *SyscallRequest_GetSockName + // *SyscallRequest_GetPeerName + // *SyscallRequest_EpollWait + // *SyscallRequest_EpollCtl + // *SyscallRequest_EpollCreate1 + // *SyscallRequest_Poll + // *SyscallRequest_Read + // *SyscallRequest_Write + // *SyscallRequest_Open + // *SyscallRequest_Ioctl + // *SyscallRequest_WriteFile + // *SyscallRequest_ReadFile + Args isSyscallRequest_Args `protobuf_oneof:"args"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SyscallRequest) Reset() { *m = SyscallRequest{} } +func (m *SyscallRequest) String() string { return proto.CompactTextString(m) } +func (*SyscallRequest) ProtoMessage() {} +func (*SyscallRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{49} +} + +func (m *SyscallRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SyscallRequest.Unmarshal(m, b) +} +func (m *SyscallRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SyscallRequest.Marshal(b, m, deterministic) +} +func (m *SyscallRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyscallRequest.Merge(m, src) +} +func (m *SyscallRequest) XXX_Size() int { + return xxx_messageInfo_SyscallRequest.Size(m) +} +func (m *SyscallRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SyscallRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SyscallRequest proto.InternalMessageInfo + +type isSyscallRequest_Args interface { + isSyscallRequest_Args() +} + +type SyscallRequest_Socket struct { + Socket *SocketRequest `protobuf:"bytes,1,opt,name=socket,proto3,oneof"` +} + +type SyscallRequest_Sendmsg struct { + Sendmsg *SendmsgRequest `protobuf:"bytes,2,opt,name=sendmsg,proto3,oneof"` +} + +type SyscallRequest_Recvmsg struct { + Recvmsg *RecvmsgRequest `protobuf:"bytes,3,opt,name=recvmsg,proto3,oneof"` +} + +type SyscallRequest_Bind struct { + Bind *BindRequest `protobuf:"bytes,4,opt,name=bind,proto3,oneof"` +} + +type SyscallRequest_Accept struct { + Accept *AcceptRequest `protobuf:"bytes,5,opt,name=accept,proto3,oneof"` +} + +type SyscallRequest_Connect struct { + Connect *ConnectRequest `protobuf:"bytes,6,opt,name=connect,proto3,oneof"` +} + +type SyscallRequest_Listen struct { + Listen *ListenRequest `protobuf:"bytes,7,opt,name=listen,proto3,oneof"` +} + +type SyscallRequest_Shutdown struct { + Shutdown *ShutdownRequest `protobuf:"bytes,8,opt,name=shutdown,proto3,oneof"` +} + +type SyscallRequest_Close struct { + Close *CloseRequest `protobuf:"bytes,9,opt,name=close,proto3,oneof"` +} + +type SyscallRequest_GetSockOpt struct { + GetSockOpt *GetSockOptRequest `protobuf:"bytes,10,opt,name=get_sock_opt,json=getSockOpt,proto3,oneof"` +} + +type SyscallRequest_SetSockOpt struct { + SetSockOpt *SetSockOptRequest `protobuf:"bytes,11,opt,name=set_sock_opt,json=setSockOpt,proto3,oneof"` +} + +type SyscallRequest_GetSockName struct { + GetSockName *GetSockNameRequest `protobuf:"bytes,12,opt,name=get_sock_name,json=getSockName,proto3,oneof"` +} + +type SyscallRequest_GetPeerName struct { + GetPeerName *GetPeerNameRequest `protobuf:"bytes,13,opt,name=get_peer_name,json=getPeerName,proto3,oneof"` +} + +type SyscallRequest_EpollWait struct { + EpollWait *EpollWaitRequest `protobuf:"bytes,14,opt,name=epoll_wait,json=epollWait,proto3,oneof"` +} + +type SyscallRequest_EpollCtl struct { + EpollCtl *EpollCtlRequest `protobuf:"bytes,15,opt,name=epoll_ctl,json=epollCtl,proto3,oneof"` +} + +type SyscallRequest_EpollCreate1 struct { + EpollCreate1 *EpollCreate1Request `protobuf:"bytes,16,opt,name=epoll_create1,json=epollCreate1,proto3,oneof"` +} + +type SyscallRequest_Poll struct { + Poll *PollRequest `protobuf:"bytes,17,opt,name=poll,proto3,oneof"` +} + +type SyscallRequest_Read struct { + Read *ReadRequest `protobuf:"bytes,18,opt,name=read,proto3,oneof"` +} + +type SyscallRequest_Write struct { + Write *WriteRequest `protobuf:"bytes,19,opt,name=write,proto3,oneof"` +} + +type SyscallRequest_Open struct { + Open *OpenRequest `protobuf:"bytes,20,opt,name=open,proto3,oneof"` +} + +type SyscallRequest_Ioctl struct { + Ioctl *IOCtlRequest `protobuf:"bytes,21,opt,name=ioctl,proto3,oneof"` +} + +type SyscallRequest_WriteFile struct { + WriteFile *WriteFileRequest `protobuf:"bytes,22,opt,name=write_file,json=writeFile,proto3,oneof"` +} + +type SyscallRequest_ReadFile struct { + ReadFile *ReadFileRequest `protobuf:"bytes,23,opt,name=read_file,json=readFile,proto3,oneof"` +} + +func (*SyscallRequest_Socket) isSyscallRequest_Args() {} + +func (*SyscallRequest_Sendmsg) isSyscallRequest_Args() {} + +func (*SyscallRequest_Recvmsg) isSyscallRequest_Args() {} + +func (*SyscallRequest_Bind) isSyscallRequest_Args() {} + +func (*SyscallRequest_Accept) isSyscallRequest_Args() {} + +func (*SyscallRequest_Connect) isSyscallRequest_Args() {} + +func (*SyscallRequest_Listen) isSyscallRequest_Args() {} + +func (*SyscallRequest_Shutdown) isSyscallRequest_Args() {} + +func (*SyscallRequest_Close) isSyscallRequest_Args() {} + +func (*SyscallRequest_GetSockOpt) isSyscallRequest_Args() {} + +func (*SyscallRequest_SetSockOpt) isSyscallRequest_Args() {} + +func (*SyscallRequest_GetSockName) isSyscallRequest_Args() {} + +func (*SyscallRequest_GetPeerName) isSyscallRequest_Args() {} + +func (*SyscallRequest_EpollWait) isSyscallRequest_Args() {} + +func (*SyscallRequest_EpollCtl) isSyscallRequest_Args() {} + +func (*SyscallRequest_EpollCreate1) isSyscallRequest_Args() {} + +func (*SyscallRequest_Poll) isSyscallRequest_Args() {} + +func (*SyscallRequest_Read) isSyscallRequest_Args() {} + +func (*SyscallRequest_Write) isSyscallRequest_Args() {} + +func (*SyscallRequest_Open) isSyscallRequest_Args() {} + +func (*SyscallRequest_Ioctl) isSyscallRequest_Args() {} + +func (*SyscallRequest_WriteFile) isSyscallRequest_Args() {} + +func (*SyscallRequest_ReadFile) isSyscallRequest_Args() {} + +func (m *SyscallRequest) GetArgs() isSyscallRequest_Args { + if m != nil { + return m.Args + } + return nil +} + +func (m *SyscallRequest) GetSocket() *SocketRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Socket); ok { + return x.Socket + } + return nil +} + +func (m *SyscallRequest) GetSendmsg() *SendmsgRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Sendmsg); ok { + return x.Sendmsg + } + return nil +} + +func (m *SyscallRequest) GetRecvmsg() *RecvmsgRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Recvmsg); ok { + return x.Recvmsg + } + return nil +} + +func (m *SyscallRequest) GetBind() *BindRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Bind); ok { + return x.Bind + } + return nil +} + +func (m *SyscallRequest) GetAccept() *AcceptRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Accept); ok { + return x.Accept + } + return nil +} + +func (m *SyscallRequest) GetConnect() *ConnectRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Connect); ok { + return x.Connect + } + return nil +} + +func (m *SyscallRequest) GetListen() *ListenRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Listen); ok { + return x.Listen + } + return nil +} + +func (m *SyscallRequest) GetShutdown() *ShutdownRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Shutdown); ok { + return x.Shutdown + } + return nil +} + +func (m *SyscallRequest) GetClose() *CloseRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Close); ok { + return x.Close + } + return nil +} + +func (m *SyscallRequest) GetGetSockOpt() *GetSockOptRequest { + if x, ok := m.GetArgs().(*SyscallRequest_GetSockOpt); ok { + return x.GetSockOpt + } + return nil +} + +func (m *SyscallRequest) GetSetSockOpt() *SetSockOptRequest { + if x, ok := m.GetArgs().(*SyscallRequest_SetSockOpt); ok { + return x.SetSockOpt + } + return nil +} + +func (m *SyscallRequest) GetGetSockName() *GetSockNameRequest { + if x, ok := m.GetArgs().(*SyscallRequest_GetSockName); ok { + return x.GetSockName + } + return nil +} + +func (m *SyscallRequest) GetGetPeerName() *GetPeerNameRequest { + if x, ok := m.GetArgs().(*SyscallRequest_GetPeerName); ok { + return x.GetPeerName + } + return nil +} + +func (m *SyscallRequest) GetEpollWait() *EpollWaitRequest { + if x, ok := m.GetArgs().(*SyscallRequest_EpollWait); ok { + return x.EpollWait + } + return nil +} + +func (m *SyscallRequest) GetEpollCtl() *EpollCtlRequest { + if x, ok := m.GetArgs().(*SyscallRequest_EpollCtl); ok { + return x.EpollCtl + } + return nil +} + +func (m *SyscallRequest) GetEpollCreate1() *EpollCreate1Request { + if x, ok := m.GetArgs().(*SyscallRequest_EpollCreate1); ok { + return x.EpollCreate1 + } + return nil +} + +func (m *SyscallRequest) GetPoll() *PollRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Poll); ok { + return x.Poll + } + return nil +} + +func (m *SyscallRequest) GetRead() *ReadRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Read); ok { + return x.Read + } + return nil +} + +func (m *SyscallRequest) GetWrite() *WriteRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Write); ok { + return x.Write + } + return nil +} + +func (m *SyscallRequest) GetOpen() *OpenRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Open); ok { + return x.Open + } + return nil +} + +func (m *SyscallRequest) GetIoctl() *IOCtlRequest { + if x, ok := m.GetArgs().(*SyscallRequest_Ioctl); ok { + return x.Ioctl + } + return nil +} + +func (m *SyscallRequest) GetWriteFile() *WriteFileRequest { + if x, ok := m.GetArgs().(*SyscallRequest_WriteFile); ok { + return x.WriteFile + } + return nil +} + +func (m *SyscallRequest) GetReadFile() *ReadFileRequest { + if x, ok := m.GetArgs().(*SyscallRequest_ReadFile); ok { + return x.ReadFile + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*SyscallRequest) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*SyscallRequest_Socket)(nil), + (*SyscallRequest_Sendmsg)(nil), + (*SyscallRequest_Recvmsg)(nil), + (*SyscallRequest_Bind)(nil), + (*SyscallRequest_Accept)(nil), + (*SyscallRequest_Connect)(nil), + (*SyscallRequest_Listen)(nil), + (*SyscallRequest_Shutdown)(nil), + (*SyscallRequest_Close)(nil), + (*SyscallRequest_GetSockOpt)(nil), + (*SyscallRequest_SetSockOpt)(nil), + (*SyscallRequest_GetSockName)(nil), + (*SyscallRequest_GetPeerName)(nil), + (*SyscallRequest_EpollWait)(nil), + (*SyscallRequest_EpollCtl)(nil), + (*SyscallRequest_EpollCreate1)(nil), + (*SyscallRequest_Poll)(nil), + (*SyscallRequest_Read)(nil), + (*SyscallRequest_Write)(nil), + (*SyscallRequest_Open)(nil), + (*SyscallRequest_Ioctl)(nil), + (*SyscallRequest_WriteFile)(nil), + (*SyscallRequest_ReadFile)(nil), + } +} + +type SyscallResponse struct { + // Types that are valid to be assigned to Result: + // *SyscallResponse_Socket + // *SyscallResponse_Sendmsg + // *SyscallResponse_Recvmsg + // *SyscallResponse_Bind + // *SyscallResponse_Accept + // *SyscallResponse_Connect + // *SyscallResponse_Listen + // *SyscallResponse_Shutdown + // *SyscallResponse_Close + // *SyscallResponse_GetSockOpt + // *SyscallResponse_SetSockOpt + // *SyscallResponse_GetSockName + // *SyscallResponse_GetPeerName + // *SyscallResponse_EpollWait + // *SyscallResponse_EpollCtl + // *SyscallResponse_EpollCreate1 + // *SyscallResponse_Poll + // *SyscallResponse_Read + // *SyscallResponse_Write + // *SyscallResponse_Open + // *SyscallResponse_Ioctl + // *SyscallResponse_WriteFile + // *SyscallResponse_ReadFile + Result isSyscallResponse_Result `protobuf_oneof:"result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SyscallResponse) Reset() { *m = SyscallResponse{} } +func (m *SyscallResponse) String() string { return proto.CompactTextString(m) } +func (*SyscallResponse) ProtoMessage() {} +func (*SyscallResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_dd04f3a8f0c5288b, []int{50} +} + +func (m *SyscallResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SyscallResponse.Unmarshal(m, b) +} +func (m *SyscallResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SyscallResponse.Marshal(b, m, deterministic) +} +func (m *SyscallResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyscallResponse.Merge(m, src) +} +func (m *SyscallResponse) XXX_Size() int { + return xxx_messageInfo_SyscallResponse.Size(m) +} +func (m *SyscallResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SyscallResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SyscallResponse proto.InternalMessageInfo + +type isSyscallResponse_Result interface { + isSyscallResponse_Result() +} + +type SyscallResponse_Socket struct { + Socket *SocketResponse `protobuf:"bytes,1,opt,name=socket,proto3,oneof"` +} + +type SyscallResponse_Sendmsg struct { + Sendmsg *SendmsgResponse `protobuf:"bytes,2,opt,name=sendmsg,proto3,oneof"` +} + +type SyscallResponse_Recvmsg struct { + Recvmsg *RecvmsgResponse `protobuf:"bytes,3,opt,name=recvmsg,proto3,oneof"` +} + +type SyscallResponse_Bind struct { + Bind *BindResponse `protobuf:"bytes,4,opt,name=bind,proto3,oneof"` +} + +type SyscallResponse_Accept struct { + Accept *AcceptResponse `protobuf:"bytes,5,opt,name=accept,proto3,oneof"` +} + +type SyscallResponse_Connect struct { + Connect *ConnectResponse `protobuf:"bytes,6,opt,name=connect,proto3,oneof"` +} + +type SyscallResponse_Listen struct { + Listen *ListenResponse `protobuf:"bytes,7,opt,name=listen,proto3,oneof"` +} + +type SyscallResponse_Shutdown struct { + Shutdown *ShutdownResponse `protobuf:"bytes,8,opt,name=shutdown,proto3,oneof"` +} + +type SyscallResponse_Close struct { + Close *CloseResponse `protobuf:"bytes,9,opt,name=close,proto3,oneof"` +} + +type SyscallResponse_GetSockOpt struct { + GetSockOpt *GetSockOptResponse `protobuf:"bytes,10,opt,name=get_sock_opt,json=getSockOpt,proto3,oneof"` +} + +type SyscallResponse_SetSockOpt struct { + SetSockOpt *SetSockOptResponse `protobuf:"bytes,11,opt,name=set_sock_opt,json=setSockOpt,proto3,oneof"` +} + +type SyscallResponse_GetSockName struct { + GetSockName *GetSockNameResponse `protobuf:"bytes,12,opt,name=get_sock_name,json=getSockName,proto3,oneof"` +} + +type SyscallResponse_GetPeerName struct { + GetPeerName *GetPeerNameResponse `protobuf:"bytes,13,opt,name=get_peer_name,json=getPeerName,proto3,oneof"` +} + +type SyscallResponse_EpollWait struct { + EpollWait *EpollWaitResponse `protobuf:"bytes,14,opt,name=epoll_wait,json=epollWait,proto3,oneof"` +} + +type SyscallResponse_EpollCtl struct { + EpollCtl *EpollCtlResponse `protobuf:"bytes,15,opt,name=epoll_ctl,json=epollCtl,proto3,oneof"` +} + +type SyscallResponse_EpollCreate1 struct { + EpollCreate1 *EpollCreate1Response `protobuf:"bytes,16,opt,name=epoll_create1,json=epollCreate1,proto3,oneof"` +} + +type SyscallResponse_Poll struct { + Poll *PollResponse `protobuf:"bytes,17,opt,name=poll,proto3,oneof"` +} + +type SyscallResponse_Read struct { + Read *ReadResponse `protobuf:"bytes,18,opt,name=read,proto3,oneof"` +} + +type SyscallResponse_Write struct { + Write *WriteResponse `protobuf:"bytes,19,opt,name=write,proto3,oneof"` +} + +type SyscallResponse_Open struct { + Open *OpenResponse `protobuf:"bytes,20,opt,name=open,proto3,oneof"` +} + +type SyscallResponse_Ioctl struct { + Ioctl *IOCtlResponse `protobuf:"bytes,21,opt,name=ioctl,proto3,oneof"` +} + +type SyscallResponse_WriteFile struct { + WriteFile *WriteFileResponse `protobuf:"bytes,22,opt,name=write_file,json=writeFile,proto3,oneof"` +} + +type SyscallResponse_ReadFile struct { + ReadFile *ReadFileResponse `protobuf:"bytes,23,opt,name=read_file,json=readFile,proto3,oneof"` +} + +func (*SyscallResponse_Socket) isSyscallResponse_Result() {} + +func (*SyscallResponse_Sendmsg) isSyscallResponse_Result() {} + +func (*SyscallResponse_Recvmsg) isSyscallResponse_Result() {} + +func (*SyscallResponse_Bind) isSyscallResponse_Result() {} + +func (*SyscallResponse_Accept) isSyscallResponse_Result() {} + +func (*SyscallResponse_Connect) isSyscallResponse_Result() {} + +func (*SyscallResponse_Listen) isSyscallResponse_Result() {} + +func (*SyscallResponse_Shutdown) isSyscallResponse_Result() {} + +func (*SyscallResponse_Close) isSyscallResponse_Result() {} + +func (*SyscallResponse_GetSockOpt) isSyscallResponse_Result() {} + +func (*SyscallResponse_SetSockOpt) isSyscallResponse_Result() {} + +func (*SyscallResponse_GetSockName) isSyscallResponse_Result() {} + +func (*SyscallResponse_GetPeerName) isSyscallResponse_Result() {} + +func (*SyscallResponse_EpollWait) isSyscallResponse_Result() {} + +func (*SyscallResponse_EpollCtl) isSyscallResponse_Result() {} + +func (*SyscallResponse_EpollCreate1) isSyscallResponse_Result() {} + +func (*SyscallResponse_Poll) isSyscallResponse_Result() {} + +func (*SyscallResponse_Read) isSyscallResponse_Result() {} + +func (*SyscallResponse_Write) isSyscallResponse_Result() {} + +func (*SyscallResponse_Open) isSyscallResponse_Result() {} + +func (*SyscallResponse_Ioctl) isSyscallResponse_Result() {} + +func (*SyscallResponse_WriteFile) isSyscallResponse_Result() {} + +func (*SyscallResponse_ReadFile) isSyscallResponse_Result() {} + +func (m *SyscallResponse) GetResult() isSyscallResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *SyscallResponse) GetSocket() *SocketResponse { + if x, ok := m.GetResult().(*SyscallResponse_Socket); ok { + return x.Socket + } + return nil +} + +func (m *SyscallResponse) GetSendmsg() *SendmsgResponse { + if x, ok := m.GetResult().(*SyscallResponse_Sendmsg); ok { + return x.Sendmsg + } + return nil +} + +func (m *SyscallResponse) GetRecvmsg() *RecvmsgResponse { + if x, ok := m.GetResult().(*SyscallResponse_Recvmsg); ok { + return x.Recvmsg + } + return nil +} + +func (m *SyscallResponse) GetBind() *BindResponse { + if x, ok := m.GetResult().(*SyscallResponse_Bind); ok { + return x.Bind + } + return nil +} + +func (m *SyscallResponse) GetAccept() *AcceptResponse { + if x, ok := m.GetResult().(*SyscallResponse_Accept); ok { + return x.Accept + } + return nil +} + +func (m *SyscallResponse) GetConnect() *ConnectResponse { + if x, ok := m.GetResult().(*SyscallResponse_Connect); ok { + return x.Connect + } + return nil +} + +func (m *SyscallResponse) GetListen() *ListenResponse { + if x, ok := m.GetResult().(*SyscallResponse_Listen); ok { + return x.Listen + } + return nil +} + +func (m *SyscallResponse) GetShutdown() *ShutdownResponse { + if x, ok := m.GetResult().(*SyscallResponse_Shutdown); ok { + return x.Shutdown + } + return nil +} + +func (m *SyscallResponse) GetClose() *CloseResponse { + if x, ok := m.GetResult().(*SyscallResponse_Close); ok { + return x.Close + } + return nil +} + +func (m *SyscallResponse) GetGetSockOpt() *GetSockOptResponse { + if x, ok := m.GetResult().(*SyscallResponse_GetSockOpt); ok { + return x.GetSockOpt + } + return nil +} + +func (m *SyscallResponse) GetSetSockOpt() *SetSockOptResponse { + if x, ok := m.GetResult().(*SyscallResponse_SetSockOpt); ok { + return x.SetSockOpt + } + return nil +} + +func (m *SyscallResponse) GetGetSockName() *GetSockNameResponse { + if x, ok := m.GetResult().(*SyscallResponse_GetSockName); ok { + return x.GetSockName + } + return nil +} + +func (m *SyscallResponse) GetGetPeerName() *GetPeerNameResponse { + if x, ok := m.GetResult().(*SyscallResponse_GetPeerName); ok { + return x.GetPeerName + } + return nil +} + +func (m *SyscallResponse) GetEpollWait() *EpollWaitResponse { + if x, ok := m.GetResult().(*SyscallResponse_EpollWait); ok { + return x.EpollWait + } + return nil +} + +func (m *SyscallResponse) GetEpollCtl() *EpollCtlResponse { + if x, ok := m.GetResult().(*SyscallResponse_EpollCtl); ok { + return x.EpollCtl + } + return nil +} + +func (m *SyscallResponse) GetEpollCreate1() *EpollCreate1Response { + if x, ok := m.GetResult().(*SyscallResponse_EpollCreate1); ok { + return x.EpollCreate1 + } + return nil +} + +func (m *SyscallResponse) GetPoll() *PollResponse { + if x, ok := m.GetResult().(*SyscallResponse_Poll); ok { + return x.Poll + } + return nil +} + +func (m *SyscallResponse) GetRead() *ReadResponse { + if x, ok := m.GetResult().(*SyscallResponse_Read); ok { + return x.Read + } + return nil +} + +func (m *SyscallResponse) GetWrite() *WriteResponse { + if x, ok := m.GetResult().(*SyscallResponse_Write); ok { + return x.Write + } + return nil +} + +func (m *SyscallResponse) GetOpen() *OpenResponse { + if x, ok := m.GetResult().(*SyscallResponse_Open); ok { + return x.Open + } + return nil +} + +func (m *SyscallResponse) GetIoctl() *IOCtlResponse { + if x, ok := m.GetResult().(*SyscallResponse_Ioctl); ok { + return x.Ioctl + } + return nil +} + +func (m *SyscallResponse) GetWriteFile() *WriteFileResponse { + if x, ok := m.GetResult().(*SyscallResponse_WriteFile); ok { + return x.WriteFile + } + return nil +} + +func (m *SyscallResponse) GetReadFile() *ReadFileResponse { + if x, ok := m.GetResult().(*SyscallResponse_ReadFile); ok { + return x.ReadFile + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*SyscallResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*SyscallResponse_Socket)(nil), + (*SyscallResponse_Sendmsg)(nil), + (*SyscallResponse_Recvmsg)(nil), + (*SyscallResponse_Bind)(nil), + (*SyscallResponse_Accept)(nil), + (*SyscallResponse_Connect)(nil), + (*SyscallResponse_Listen)(nil), + (*SyscallResponse_Shutdown)(nil), + (*SyscallResponse_Close)(nil), + (*SyscallResponse_GetSockOpt)(nil), + (*SyscallResponse_SetSockOpt)(nil), + (*SyscallResponse_GetSockName)(nil), + (*SyscallResponse_GetPeerName)(nil), + (*SyscallResponse_EpollWait)(nil), + (*SyscallResponse_EpollCtl)(nil), + (*SyscallResponse_EpollCreate1)(nil), + (*SyscallResponse_Poll)(nil), + (*SyscallResponse_Read)(nil), + (*SyscallResponse_Write)(nil), + (*SyscallResponse_Open)(nil), + (*SyscallResponse_Ioctl)(nil), + (*SyscallResponse_WriteFile)(nil), + (*SyscallResponse_ReadFile)(nil), + } +} + +func init() { + proto.RegisterType((*SendmsgRequest)(nil), "syscall_rpc.SendmsgRequest") + proto.RegisterType((*SendmsgResponse)(nil), "syscall_rpc.SendmsgResponse") + proto.RegisterType((*IOCtlRequest)(nil), "syscall_rpc.IOCtlRequest") + proto.RegisterType((*IOCtlResponse)(nil), "syscall_rpc.IOCtlResponse") + proto.RegisterType((*RecvmsgRequest)(nil), "syscall_rpc.RecvmsgRequest") + proto.RegisterType((*OpenRequest)(nil), "syscall_rpc.OpenRequest") + proto.RegisterType((*OpenResponse)(nil), "syscall_rpc.OpenResponse") + proto.RegisterType((*ReadRequest)(nil), "syscall_rpc.ReadRequest") + proto.RegisterType((*ReadResponse)(nil), "syscall_rpc.ReadResponse") + proto.RegisterType((*ReadFileRequest)(nil), "syscall_rpc.ReadFileRequest") + proto.RegisterType((*ReadFileResponse)(nil), "syscall_rpc.ReadFileResponse") + proto.RegisterType((*WriteRequest)(nil), "syscall_rpc.WriteRequest") + proto.RegisterType((*WriteResponse)(nil), "syscall_rpc.WriteResponse") + proto.RegisterType((*WriteFileRequest)(nil), "syscall_rpc.WriteFileRequest") + proto.RegisterType((*WriteFileResponse)(nil), "syscall_rpc.WriteFileResponse") + proto.RegisterType((*AddressResponse)(nil), "syscall_rpc.AddressResponse") + proto.RegisterType((*RecvmsgResponse)(nil), "syscall_rpc.RecvmsgResponse") + proto.RegisterType((*RecvmsgResponse_ResultPayload)(nil), "syscall_rpc.RecvmsgResponse.ResultPayload") + proto.RegisterType((*BindRequest)(nil), "syscall_rpc.BindRequest") + proto.RegisterType((*BindResponse)(nil), "syscall_rpc.BindResponse") + proto.RegisterType((*AcceptRequest)(nil), "syscall_rpc.AcceptRequest") + proto.RegisterType((*AcceptResponse)(nil), "syscall_rpc.AcceptResponse") + proto.RegisterType((*AcceptResponse_ResultPayload)(nil), "syscall_rpc.AcceptResponse.ResultPayload") + proto.RegisterType((*ConnectRequest)(nil), "syscall_rpc.ConnectRequest") + proto.RegisterType((*ConnectResponse)(nil), "syscall_rpc.ConnectResponse") + proto.RegisterType((*ListenRequest)(nil), "syscall_rpc.ListenRequest") + proto.RegisterType((*ListenResponse)(nil), "syscall_rpc.ListenResponse") + proto.RegisterType((*ShutdownRequest)(nil), "syscall_rpc.ShutdownRequest") + proto.RegisterType((*ShutdownResponse)(nil), "syscall_rpc.ShutdownResponse") + proto.RegisterType((*CloseRequest)(nil), "syscall_rpc.CloseRequest") + proto.RegisterType((*CloseResponse)(nil), "syscall_rpc.CloseResponse") + proto.RegisterType((*GetSockOptRequest)(nil), "syscall_rpc.GetSockOptRequest") + proto.RegisterType((*GetSockOptResponse)(nil), "syscall_rpc.GetSockOptResponse") + proto.RegisterType((*SetSockOptRequest)(nil), "syscall_rpc.SetSockOptRequest") + proto.RegisterType((*SetSockOptResponse)(nil), "syscall_rpc.SetSockOptResponse") + proto.RegisterType((*GetSockNameRequest)(nil), "syscall_rpc.GetSockNameRequest") + proto.RegisterType((*GetSockNameResponse)(nil), "syscall_rpc.GetSockNameResponse") + proto.RegisterType((*GetPeerNameRequest)(nil), "syscall_rpc.GetPeerNameRequest") + proto.RegisterType((*GetPeerNameResponse)(nil), "syscall_rpc.GetPeerNameResponse") + proto.RegisterType((*SocketRequest)(nil), "syscall_rpc.SocketRequest") + proto.RegisterType((*SocketResponse)(nil), "syscall_rpc.SocketResponse") + proto.RegisterType((*EpollWaitRequest)(nil), "syscall_rpc.EpollWaitRequest") + proto.RegisterType((*EpollEvent)(nil), "syscall_rpc.EpollEvent") + proto.RegisterType((*EpollEvents)(nil), "syscall_rpc.EpollEvents") + proto.RegisterType((*EpollWaitResponse)(nil), "syscall_rpc.EpollWaitResponse") + proto.RegisterType((*EpollCtlRequest)(nil), "syscall_rpc.EpollCtlRequest") + proto.RegisterType((*EpollCtlResponse)(nil), "syscall_rpc.EpollCtlResponse") + proto.RegisterType((*EpollCreate1Request)(nil), "syscall_rpc.EpollCreate1Request") + proto.RegisterType((*EpollCreate1Response)(nil), "syscall_rpc.EpollCreate1Response") + proto.RegisterType((*PollRequest)(nil), "syscall_rpc.PollRequest") + proto.RegisterType((*PollResponse)(nil), "syscall_rpc.PollResponse") + proto.RegisterType((*SyscallRequest)(nil), "syscall_rpc.SyscallRequest") + proto.RegisterType((*SyscallResponse)(nil), "syscall_rpc.SyscallResponse") +} + +func init() { + proto.RegisterFile("pkg/sentry/socket/rpcinet/syscall_rpc.proto", fileDescriptor_dd04f3a8f0c5288b) +} + +var fileDescriptor_dd04f3a8f0c5288b = []byte{ + // 1838 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x98, 0xdd, 0x52, 0xe3, 0xc8, + 0x15, 0x80, 0x6d, 0x6c, 0xc0, 0x1c, 0xff, 0xa2, 0x21, 0xac, 0xf8, 0x67, 0x95, 0xa4, 0x8a, 0x4d, + 0x2a, 0xb8, 0xc6, 0x33, 0xec, 0x90, 0xdd, 0xad, 0xdd, 0x2c, 0x04, 0xd6, 0x53, 0x99, 0x1a, 0x88, + 0x5c, 0x09, 0xa9, 0xe4, 0xc2, 0x25, 0xa4, 0xb6, 0x71, 0x21, 0x4b, 0x8a, 0x24, 0x43, 0x71, 0x93, + 0x07, 0xc8, 0x75, 0xee, 0x72, 0x91, 0x87, 0xca, 0x03, 0xe4, 0x09, 0xf2, 0x0e, 0xa9, 0xd3, 0xdd, + 0x92, 0xba, 0x45, 0x8b, 0xc1, 0x29, 0x6a, 0xef, 0xd4, 0xad, 0xf3, 0xd7, 0xe7, 0x74, 0x7f, 0x3a, + 0x2d, 0xf8, 0x65, 0x70, 0x3b, 0xee, 0x46, 0xc4, 0x8b, 0xc3, 0x87, 0x6e, 0xe4, 0xdb, 0xb7, 0x24, + 0xee, 0x86, 0x81, 0x3d, 0xf1, 0x48, 0xdc, 0x8d, 0x1e, 0x22, 0xdb, 0x72, 0xdd, 0x61, 0x18, 0xd8, + 0x87, 0x41, 0xe8, 0xc7, 0xbe, 0x56, 0x17, 0xa6, 0x8c, 0xbf, 0x97, 0xa1, 0x35, 0x20, 0x9e, 0x33, + 0x8d, 0xc6, 0x26, 0xf9, 0xeb, 0x8c, 0x44, 0xb1, 0xd6, 0x82, 0x85, 0x91, 0xa3, 0x97, 0xf7, 0xcb, + 0x07, 0x4d, 0x73, 0x61, 0xe4, 0x68, 0xeb, 0x50, 0x75, 0xac, 0xd8, 0xd2, 0x17, 0xf6, 0xcb, 0x07, + 0x8d, 0x93, 0x85, 0x5a, 0xd9, 0xa4, 0x63, 0x4d, 0x87, 0x65, 0xcb, 0x71, 0x42, 0x12, 0x45, 0x7a, + 0x05, 0x5f, 0x99, 0xc9, 0x50, 0xd3, 0xa0, 0x3a, 0xf5, 0x43, 0xa2, 0x57, 0xf7, 0xcb, 0x07, 0x35, + 0x93, 0x3e, 0x6b, 0x06, 0x34, 0x89, 0xe7, 0x0c, 0xfd, 0xd1, 0x30, 0x24, 0xb6, 0x1f, 0x3a, 0xfa, + 0x22, 0x7d, 0x59, 0x27, 0x9e, 0x73, 0x31, 0x32, 0xe9, 0x94, 0xf1, 0x67, 0x68, 0xa7, 0xb1, 0x44, + 0x81, 0xef, 0x45, 0x44, 0xfb, 0x29, 0x34, 0x48, 0x18, 0xfa, 0xe1, 0xd0, 0x9b, 0x4d, 0xaf, 0x49, + 0xc8, 0xc2, 0xea, 0x97, 0xcc, 0x3a, 0x9d, 0xfd, 0x48, 0x27, 0x35, 0x1d, 0x96, 0x5c, 0xe2, 0x8d, + 0xe3, 0x1b, 0x1a, 0x23, 0xbe, 0xe6, 0xe3, 0x93, 0x1a, 0x2c, 0x85, 0x24, 0x9a, 0xb9, 0xb1, 0x71, + 0x02, 0x8d, 0xf7, 0x17, 0xa7, 0xb1, 0x5b, 0xb4, 0xca, 0x0e, 0x54, 0xec, 0xa9, 0xc3, 0x0c, 0x98, + 0xf8, 0x88, 0x33, 0x56, 0x38, 0xe6, 0x6b, 0xc3, 0x47, 0xe3, 0x8f, 0xd0, 0xe4, 0x36, 0xe6, 0x89, + 0x6e, 0x1d, 0x16, 0xef, 0x2c, 0x77, 0x46, 0x58, 0x02, 0xfb, 0x25, 0x93, 0x0d, 0x85, 0xd8, 0xfe, + 0x59, 0x86, 0x96, 0x49, 0xec, 0xbb, 0x27, 0x8b, 0x20, 0x2d, 0x31, 0x59, 0x20, 0xce, 0x47, 0xc4, + 0x73, 0x48, 0x48, 0xe3, 0xac, 0x99, 0x7c, 0x84, 0x25, 0x08, 0x08, 0xb9, 0x4d, 0x4a, 0x80, 0xcf, + 0xda, 0x1a, 0x2c, 0xc6, 0xe1, 0xcc, 0xb3, 0x79, 0xea, 0xd9, 0x40, 0xdb, 0x83, 0xba, 0x3d, 0x8d, + 0xc6, 0x43, 0x6e, 0x7e, 0x89, 0x9a, 0x07, 0x9c, 0xfa, 0x40, 0x67, 0x8c, 0xdf, 0x41, 0xfd, 0x22, + 0x20, 0x5e, 0x12, 0x19, 0x5a, 0xb6, 0xe2, 0x1b, 0x1a, 0x5b, 0xc3, 0xa4, 0xcf, 0x68, 0x79, 0xe4, + 0x5a, 0xe3, 0x88, 0x07, 0xc7, 0x06, 0x6c, 0x1b, 0x38, 0x84, 0x46, 0xd6, 0x34, 0xe9, 0xb3, 0x71, + 0x01, 0x0d, 0x66, 0x6c, 0x9e, 0x0c, 0x76, 0x68, 0x32, 0x92, 0xda, 0x2e, 0x8c, 0x1c, 0x21, 0x77, + 0x47, 0x50, 0x37, 0x89, 0xe5, 0xcc, 0x99, 0x37, 0xe3, 0x0a, 0x1a, 0x4c, 0x6d, 0xbe, 0x7d, 0x96, + 0x3b, 0x09, 0xfd, 0x12, 0x3b, 0x0b, 0x42, 0x3c, 0x3f, 0x87, 0x36, 0x1a, 0x3e, 0x9f, 0xb8, 0x44, + 0x95, 0xb1, 0x15, 0x96, 0x31, 0xe3, 0x2f, 0xd0, 0xc9, 0xc4, 0x5e, 0x3a, 0x86, 0x2f, 0xa1, 0x71, + 0x15, 0x4e, 0x62, 0x32, 0xe7, 0x89, 0x36, 0xfe, 0x04, 0x4d, 0xae, 0xf7, 0xd2, 0xa7, 0xef, 0x37, + 0xd0, 0xa1, 0x96, 0x3f, 0x91, 0x16, 0x64, 0x8a, 0xed, 0x7b, 0x31, 0xf1, 0x62, 0x16, 0x9c, 0x99, + 0x0c, 0x8d, 0x4b, 0x58, 0x15, 0x2c, 0xf0, 0xf8, 0x3e, 0x57, 0xc5, 0x97, 0x8f, 0x6e, 0xf9, 0x3e, + 0x9c, 0xc4, 0x31, 0xf1, 0xf8, 0x0e, 0x48, 0x86, 0xc6, 0x29, 0xb4, 0xbf, 0x67, 0xc0, 0x4a, 0xed, + 0x09, 0x48, 0x2b, 0xcb, 0x48, 0x2b, 0xda, 0x47, 0xff, 0x5a, 0xc0, 0x7a, 0xf3, 0xa3, 0x3b, 0x4f, + 0xd6, 0xce, 0x61, 0x39, 0xb0, 0x1e, 0x5c, 0xdf, 0x62, 0x1b, 0xbb, 0xde, 0xfb, 0xc5, 0xa1, 0x88, + 0xea, 0x9c, 0xcd, 0x43, 0x93, 0xe6, 0xf1, 0x92, 0x69, 0xf4, 0x4b, 0x66, 0xa2, 0xbc, 0xf9, 0x8f, + 0x32, 0x34, 0xa5, 0x97, 0x69, 0x75, 0xcb, 0x39, 0x5e, 0x7f, 0x99, 0x2d, 0x8e, 0x79, 0xdc, 0x96, + 0x3c, 0xe6, 0x72, 0xa1, 0x5a, 0x7a, 0x45, 0x42, 0xcf, 0x16, 0xac, 0x50, 0x70, 0x50, 0x67, 0x55, + 0x9a, 0xae, 0x1a, 0x4e, 0xfc, 0x56, 0xde, 0x8c, 0xef, 0xa0, 0x7e, 0x32, 0xf1, 0x0a, 0x0f, 0xa8, + 0x2e, 0x47, 0x95, 0xa5, 0xdc, 0x78, 0x0d, 0x0d, 0xa6, 0xf8, 0xec, 0x62, 0x1b, 0xef, 0xa1, 0xf9, + 0xbd, 0x6d, 0x93, 0x20, 0x2e, 0xf2, 0xc6, 0xb0, 0x18, 0x52, 0x57, 0x0c, 0x8b, 0x61, 0x06, 0x2f, + 0x5c, 0x5e, 0x85, 0xc3, 0xcb, 0xf8, 0x4f, 0x19, 0x5a, 0x89, 0xad, 0x79, 0xea, 0x7a, 0x96, 0xaf, + 0xeb, 0x17, 0x72, 0x96, 0x25, 0x93, 0xc5, 0x65, 0xbd, 0xca, 0x57, 0x35, 0xbf, 0x92, 0xff, 0xb3, + 0x9a, 0x42, 0x61, 0xbe, 0x82, 0xd6, 0xa9, 0xef, 0x79, 0xc4, 0x8e, 0xe7, 0xaf, 0xcd, 0x5b, 0x68, + 0xa7, 0xba, 0xcf, 0x2f, 0xcf, 0xaf, 0xa1, 0xf9, 0x61, 0x12, 0xc5, 0xd9, 0xb7, 0x44, 0xe1, 0xf0, + 0xda, 0xb2, 0x6f, 0x5d, 0x7f, 0x4c, 0x1d, 0x56, 0xcc, 0x64, 0x68, 0xbc, 0x81, 0x56, 0xa2, 0xfa, + 0x7c, 0x7f, 0x6f, 0xa0, 0x3d, 0xb8, 0x99, 0xc5, 0x8e, 0x7f, 0xef, 0x3d, 0xf1, 0xd9, 0xbf, 0xf1, + 0xef, 0xb9, 0x37, 0x7c, 0x34, 0x8e, 0xa0, 0x93, 0x29, 0x3d, 0xdf, 0xd7, 0x2e, 0x34, 0x4e, 0x5d, + 0x3f, 0x2a, 0x62, 0xae, 0xd1, 0x83, 0x26, 0x7f, 0xff, 0x7c, 0x9b, 0x04, 0x56, 0x7f, 0x20, 0xf1, + 0xc0, 0xb7, 0x6f, 0x2f, 0x8a, 0xb7, 0xf4, 0x1a, 0x2c, 0xba, 0xe4, 0x8e, 0xb8, 0x7c, 0x0d, 0x6c, + 0x80, 0x1b, 0xdd, 0xb3, 0xa6, 0x84, 0xef, 0x69, 0xfa, 0x2c, 0x1c, 0xe4, 0x6a, 0xee, 0x5b, 0xa8, + 0x89, 0x6e, 0xe6, 0xd9, 0xed, 0x1a, 0x54, 0xfc, 0x20, 0x4e, 0x3b, 0x1b, 0x1c, 0x08, 0x3b, 0x6c, + 0x08, 0xab, 0x83, 0x17, 0x8c, 0xbf, 0xc3, 0x9c, 0x31, 0xd4, 0xe0, 0xa3, 0xf1, 0x0e, 0xb4, 0xc1, + 0xe3, 0xc8, 0x9f, 0x91, 0xd9, 0x9f, 0xa5, 0x4b, 0xfe, 0x68, 0x4d, 0x0b, 0x6b, 0xf6, 0x37, 0x78, + 0x25, 0x49, 0xcd, 0x93, 0x99, 0xe3, 0xb9, 0xce, 0x27, 0x1e, 0xfd, 0xc7, 0x27, 0x94, 0x45, 0x79, + 0x49, 0x48, 0xf8, 0xe9, 0x28, 0x33, 0xa9, 0x1f, 0x3b, 0xca, 0x2b, 0x68, 0x0e, 0xe8, 0x9d, 0x23, + 0x09, 0x70, 0x1d, 0x96, 0x46, 0xd6, 0x74, 0xe2, 0x3e, 0x50, 0x9f, 0x15, 0x93, 0x8f, 0xb0, 0xa6, + 0xf1, 0x43, 0x40, 0x78, 0xa1, 0xe9, 0xb3, 0xb6, 0x09, 0x35, 0x7a, 0x2b, 0xb1, 0x7d, 0x97, 0xd7, + 0x3a, 0x1d, 0x1b, 0xbf, 0x87, 0x56, 0x62, 0xf8, 0xa5, 0xba, 0xc5, 0x3f, 0x40, 0xe7, 0x2c, 0xf0, + 0x5d, 0xf7, 0xca, 0x9a, 0x14, 0x6e, 0xc8, 0x1d, 0x00, 0x6f, 0x36, 0x1d, 0x92, 0x3b, 0xe2, 0xc5, + 0x49, 0x47, 0xbb, 0xe2, 0xcd, 0xa6, 0x67, 0x74, 0x82, 0x76, 0xb5, 0x11, 0xb1, 0x69, 0xb4, 0x9a, + 0x49, 0x9f, 0x8d, 0xb7, 0x00, 0xd4, 0x2c, 0x15, 0x51, 0xf5, 0xa0, 0x92, 0x31, 0x3e, 0x32, 0xbe, + 0x85, 0x7a, 0xa6, 0x15, 0x69, 0xdd, 0x54, 0xac, 0xbc, 0x5f, 0x39, 0xa8, 0xf7, 0x3e, 0x93, 0x4a, + 0x91, 0x49, 0xa6, 0xfa, 0x77, 0xb0, 0x2a, 0x2c, 0x66, 0x9e, 0x14, 0xf5, 0xa4, 0x88, 0xea, 0x3d, + 0xbd, 0xc0, 0x55, 0x84, 0xcd, 0x1c, 0x93, 0x14, 0x92, 0x18, 0x43, 0x9b, 0x8a, 0x08, 0xb7, 0x29, + 0x0d, 0xaa, 0x24, 0x48, 0x17, 0x4d, 0x9f, 0x31, 0x0d, 0x7e, 0xc0, 0x8b, 0xbd, 0xe0, 0x07, 0x3c, + 0x2d, 0x95, 0x34, 0x2d, 0xbf, 0x82, 0x45, 0x6a, 0x9a, 0x1e, 0xe8, 0x27, 0x96, 0xcb, 0xa4, 0x90, + 0xcb, 0x99, 0xd7, 0xe7, 0x9f, 0xf4, 0x2f, 0xe0, 0x15, 0x53, 0x0b, 0x89, 0x15, 0x93, 0xd7, 0x42, + 0xc0, 0xf8, 0x9d, 0xe7, 0x3b, 0x94, 0x3e, 0x1b, 0x57, 0xb0, 0x26, 0x8b, 0xbe, 0xe0, 0x1d, 0xe5, + 0xd2, 0x77, 0xdd, 0x27, 0xee, 0x28, 0xca, 0xfd, 0x71, 0x05, 0x0d, 0xa6, 0x36, 0x67, 0x37, 0x2e, + 0x1a, 0x53, 0x16, 0xf0, 0xdf, 0x00, 0xad, 0x01, 0x4b, 0x76, 0x12, 0xd3, 0x5b, 0x58, 0x62, 0x3f, + 0x0e, 0xa8, 0xd5, 0x7a, 0x6f, 0x53, 0xaa, 0x86, 0x74, 0xbe, 0xd1, 0x24, 0x93, 0xd5, 0xde, 0xc1, + 0x72, 0xc4, 0x2e, 0xec, 0x7c, 0x23, 0x6d, 0xc9, 0x6a, 0xd2, 0x8f, 0x05, 0xa4, 0x07, 0x97, 0x46, + 0xc5, 0x90, 0x75, 0xb8, 0x74, 0x43, 0xe4, 0x15, 0xe5, 0xcb, 0x30, 0x2a, 0x72, 0x69, 0xed, 0x10, + 0xaa, 0xd7, 0x13, 0xcf, 0xe1, 0x7b, 0x46, 0xde, 0xb7, 0x42, 0x9b, 0x89, 0x97, 0x22, 0x94, 0xc3, + 0x75, 0x59, 0xb4, 0xe5, 0xa2, 0x97, 0xde, 0xfc, 0xba, 0xa4, 0x66, 0x11, 0xd7, 0xc5, 0x64, 0x31, + 0x3c, 0x9b, 0xb5, 0x37, 0xf4, 0x3e, 0x9c, 0x0f, 0x4f, 0x6e, 0x9b, 0x30, 0x3c, 0x2e, 0x8d, 0xee, + 0x5c, 0xda, 0xa6, 0xe8, 0xcb, 0x0a, 0x77, 0x52, 0xf3, 0x43, 0xef, 0x49, 0x74, 0x42, 0xfb, 0x0a, + 0x6a, 0x11, 0x6f, 0x39, 0xf4, 0x9a, 0x02, 0xc3, 0xb9, 0x26, 0xa6, 0x5f, 0x32, 0x53, 0x79, 0xed, + 0x35, 0x2c, 0xda, 0xd8, 0x57, 0xe8, 0x2b, 0x54, 0x71, 0x43, 0x0e, 0x54, 0xe8, 0x48, 0xfa, 0x25, + 0x93, 0x49, 0x6a, 0x27, 0xd0, 0x18, 0x93, 0x78, 0x88, 0x35, 0x1c, 0xe2, 0x07, 0x15, 0xa8, 0xe6, + 0xae, 0xa4, 0xf9, 0xa8, 0xef, 0xe8, 0x97, 0x4c, 0x18, 0xa7, 0x93, 0x68, 0x23, 0x12, 0x6d, 0xd4, + 0x15, 0x36, 0x06, 0x2a, 0x1b, 0x51, 0x66, 0xe3, 0x0c, 0x9a, 0x69, 0x1c, 0xf4, 0x63, 0xdf, 0xa0, + 0x46, 0xf6, 0x54, 0x81, 0x08, 0x1f, 0x40, 0xdc, 0xf1, 0xe3, 0x6c, 0x36, 0x31, 0x83, 0xbd, 0x3c, + 0x33, 0xd3, 0x54, 0x9b, 0xc9, 0x7d, 0x47, 0xb9, 0x99, 0x64, 0x56, 0xfb, 0x16, 0x80, 0xe0, 0xe9, + 0x1f, 0xde, 0x5b, 0x93, 0x58, 0x6f, 0x51, 0x1b, 0x3b, 0x8f, 0x99, 0x24, 0x7c, 0x39, 0xfa, 0x25, + 0x73, 0x85, 0x24, 0x73, 0xda, 0xd7, 0xc0, 0x06, 0x43, 0x3b, 0x76, 0xf5, 0xb6, 0xa2, 0x8a, 0x39, + 0x66, 0x62, 0x15, 0x09, 0x9f, 0xd2, 0x7e, 0x80, 0x26, 0x57, 0x66, 0xec, 0xd1, 0x3b, 0xd4, 0xc0, + 0xbe, 0xc2, 0x80, 0xc4, 0xb1, 0x7e, 0xc9, 0x6c, 0x10, 0x61, 0x1a, 0xcf, 0x07, 0x0e, 0xf5, 0x55, + 0xc5, 0xf9, 0x10, 0x18, 0x84, 0xe7, 0x03, 0xe5, 0x50, 0x3e, 0x24, 0x96, 0xa3, 0x6b, 0x0a, 0x79, + 0xe1, 0xbf, 0x0a, 0xca, 0xa3, 0x1c, 0x6e, 0x37, 0xbc, 0x3f, 0x13, 0xfd, 0x95, 0x62, 0xbb, 0x89, + 0x3f, 0x1d, 0x70, 0xbb, 0x51, 0x49, 0x74, 0xe1, 0x07, 0xc4, 0xd3, 0xd7, 0x14, 0x2e, 0x84, 0x1f, + 0x4b, 0xe8, 0x02, 0xe5, 0xd0, 0xc5, 0xc4, 0xc7, 0x24, 0xfe, 0x44, 0xe1, 0x42, 0xfc, 0x87, 0x87, + 0x2e, 0xa8, 0x24, 0xd6, 0x8e, 0xfa, 0x1a, 0x8e, 0x26, 0x2e, 0xd1, 0xd7, 0x15, 0xb5, 0xcb, 0xff, + 0x7d, 0xc0, 0xda, 0xdd, 0x27, 0x73, 0x58, 0x3b, 0x5c, 0x1d, 0x53, 0xff, 0x4c, 0x51, 0xbb, 0xdc, + 0x2f, 0x1d, 0xac, 0x5d, 0xc8, 0xa7, 0x4e, 0x96, 0xa0, 0x6a, 0x85, 0xe3, 0xc8, 0xf8, 0x2f, 0x40, + 0x3b, 0xa5, 0x2a, 0x47, 0xf6, 0x51, 0x0e, 0xab, 0x5b, 0x4a, 0xac, 0xa6, 0xdd, 0x55, 0xc2, 0xd5, + 0xe3, 0x3c, 0x57, 0xb7, 0xd5, 0x5c, 0xcd, 0xda, 0xb2, 0x04, 0xac, 0xc7, 0x79, 0xb0, 0x6e, 0x3f, + 0xf5, 0x5b, 0x41, 0x24, 0x6b, 0x57, 0x22, 0xeb, 0x86, 0x82, 0xac, 0xa9, 0x0e, 0x43, 0xeb, 0x51, + 0x0e, 0xad, 0x5b, 0x4f, 0x5c, 0x74, 0x05, 0xb6, 0x1e, 0xe7, 0xd9, 0xba, 0xad, 0x66, 0x6b, 0x16, + 0x61, 0x02, 0xd7, 0xa3, 0x1c, 0x5c, 0xb7, 0x94, 0x70, 0xcd, 0x1c, 0x72, 0xba, 0x7e, 0xfd, 0x88, + 0xae, 0x3b, 0x05, 0x74, 0x4d, 0x55, 0x33, 0xbc, 0xf6, 0x64, 0xbc, 0x6e, 0xaa, 0xf0, 0x9a, 0xaa, + 0x71, 0xbe, 0x9e, 0x2a, 0xf9, 0xba, 0x57, 0xc8, 0xd7, 0x54, 0x5f, 0x04, 0xec, 0xa9, 0x12, 0xb0, + 0x7b, 0x85, 0x80, 0xcd, 0x8c, 0x08, 0x84, 0x3d, 0x57, 0x13, 0x76, 0xbf, 0x98, 0xb0, 0xa9, 0x19, + 0x09, 0xb1, 0xe7, 0x6a, 0xc4, 0xee, 0x17, 0x23, 0x56, 0xb2, 0x93, 0x32, 0xf6, 0x3b, 0x05, 0x63, + 0x77, 0x8b, 0x18, 0x9b, 0x9a, 0x10, 0x20, 0xfb, 0xcd, 0x63, 0xc8, 0xee, 0x14, 0x40, 0x36, 0x2b, + 0x66, 0x4a, 0xd9, 0xbe, 0x9a, 0xb2, 0x9f, 0x3f, 0x41, 0xd9, 0xd4, 0x8a, 0x8c, 0xd9, 0xae, 0x84, + 0xd9, 0x0d, 0x05, 0x66, 0xb3, 0xc3, 0x42, 0x39, 0xdb, 0x95, 0x38, 0xbb, 0xa1, 0xe0, 0x6c, 0xa6, + 0x40, 0x41, 0xdb, 0x93, 0x41, 0xbb, 0xa9, 0x02, 0x6d, 0xb6, 0xf1, 0x18, 0x69, 0xbb, 0x12, 0x69, + 0x37, 0x14, 0xa4, 0xcd, 0x9c, 0x50, 0xd4, 0xf6, 0x64, 0xd4, 0x6e, 0xaa, 0x50, 0x9b, 0x39, 0x61, + 0xac, 0xfd, 0x4e, 0xc1, 0xda, 0xdd, 0x22, 0xd6, 0x66, 0x35, 0xcc, 0x60, 0xfb, 0xcd, 0x63, 0xd8, + 0xee, 0x14, 0xc0, 0x36, 0xab, 0x61, 0x4a, 0xdb, 0xb4, 0x8b, 0xbd, 0x5e, 0xa2, 0x17, 0xc5, 0x37, + 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xd5, 0x22, 0x29, 0x56, 0xfd, 0x1a, 0x00, 0x00, +} |