summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/socket/rpcinet
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2019-06-02 06:44:55 +0000
committergVisor bot <gvisor-bot@google.com>2019-06-02 06:44:55 +0000
commitceb0d792f328d1fc0692197d8856a43c3936a571 (patch)
tree83155f302eff44a78bcc30a3a08f4efe59a79379 /pkg/sentry/socket/rpcinet
parentdeb7ecf1e46862d54f4b102f2d163cfbcfc37f3b (diff)
parent216da0b733dbed9aad9b2ab92ac75bcb906fd7ee (diff)
Merge 216da0b7 (automated)
Diffstat (limited to 'pkg/sentry/socket/rpcinet')
-rw-r--r--pkg/sentry/socket/rpcinet/conn/conn.go187
-rwxr-xr-xpkg/sentry/socket/rpcinet/conn/conn_state_autogen.go4
-rw-r--r--pkg/sentry/socket/rpcinet/device.go19
-rw-r--r--pkg/sentry/socket/rpcinet/notifier/notifier.go230
-rwxr-xr-xpkg/sentry/socket/rpcinet/notifier/notifier_state_autogen.go4
-rw-r--r--pkg/sentry/socket/rpcinet/rpcinet.go16
-rwxr-xr-xpkg/sentry/socket/rpcinet/rpcinet_state_autogen.go4
-rw-r--r--pkg/sentry/socket/rpcinet/socket.go887
-rw-r--r--pkg/sentry/socket/rpcinet/stack.go135
-rw-r--r--pkg/sentry/socket/rpcinet/stack_unsafe.go193
-rwxr-xr-xpkg/sentry/socket/rpcinet/syscall_rpc_go_proto/syscall_rpc.pb.go3938
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,
+}