TUN-8415: Refactor capnp rpc into a single module
Combines the tunnelrpc and quic/schema capnp files into the same module. To help reduce future issues with capnp id generation, capnpids are provided in the capnp files from the existing capnp struct ids generated in the go files. Reduces the overall interface of the Capnp methods to the rest of the code by providing an interface that will handle the quic protocol selection. Introduces a new `rpc-timeout` config that will allow all of the SessionManager and ConfigurationManager RPC requests to have a timeout. The timeout for these values is set to 5 seconds as non of these operations for the managers should take a long time to complete. Removed the RPC-specific logger as it never provided good debugging value as the RPC method names were not visible in the logs.
This commit is contained in:
parent
7d76ce2d24
commit
eb2e4349e8
14
Makefile
14
Makefile
|
@ -251,17 +251,11 @@ github-windows-upload:
|
|||
python3 github_release.py --path built_artifacts/cloudflared-windows-386.exe --release-version $(VERSION) --name cloudflared-windows-386.exe
|
||||
python3 github_release.py --path built_artifacts/cloudflared-windows-386.msi --release-version $(VERSION) --name cloudflared-windows-386.msi
|
||||
|
||||
.PHONY: tunnelrpc-deps
|
||||
tunnelrpc-deps:
|
||||
.PHONY: capnp
|
||||
capnp:
|
||||
which capnp # https://capnproto.org/install.html
|
||||
which capnpc-go # go install zombiezen.com/go/capnproto2/capnpc-go@latest
|
||||
capnp compile -ogo tunnelrpc/tunnelrpc.capnp
|
||||
|
||||
.PHONY: quic-deps
|
||||
quic-deps:
|
||||
which capnp
|
||||
which capnpc-go
|
||||
capnp compile -ogo quic/schema/quic_metadata_protocol.capnp
|
||||
capnp compile -ogo tunnelrpc/proto/tunnelrpc.capnp tunnelrpc/proto/quic_metadata_protocol.capnp
|
||||
|
||||
.PHONY: vet
|
||||
vet:
|
||||
|
@ -269,4 +263,4 @@ vet:
|
|||
|
||||
.PHONY: fmt
|
||||
fmt:
|
||||
goimports -l -w -local github.com/cloudflare/cloudflared $$(go list -mod=vendor -f '{{.Dir}}' -a ./... | fgrep -v tunnelrpc)
|
||||
goimports -l -w -local github.com/cloudflare/cloudflared $$(go list -mod=vendor -f '{{.Dir}}' -a ./... | fgrep -v tunnelrpc/proto)
|
||||
|
|
|
@ -78,8 +78,8 @@ const (
|
|||
// hostKeyPath is the path of the dir to save SSH host keys too
|
||||
hostKeyPath = "host-key-path"
|
||||
|
||||
// udpUnregisterSessionTimeout is how long we wait before we stop trying to unregister a UDP session from the edge
|
||||
udpUnregisterSessionTimeoutFlag = "udp-unregister-session-timeout"
|
||||
// rpcTimeout is how long to wait for a Capnp RPC request to the edge
|
||||
rpcTimeout = "rpc-timeout"
|
||||
|
||||
// writeStreamTimeout sets if we should have a timeout when writing data to a stream towards the destination (edge/origin).
|
||||
writeStreamTimeout = "write-stream-timeout"
|
||||
|
@ -695,7 +695,7 @@ func tunnelFlags(shouldHide bool) []cli.Flag {
|
|||
Hidden: true,
|
||||
}),
|
||||
altsrc.NewDurationFlag(&cli.DurationFlag{
|
||||
Name: udpUnregisterSessionTimeoutFlag,
|
||||
Name: rpcTimeout,
|
||||
Value: 5 * time.Second,
|
||||
Hidden: true,
|
||||
}),
|
||||
|
|
|
@ -246,7 +246,7 @@ func prepareTunnelConfig(
|
|||
EdgeTLSConfigs: edgeTLSConfigs,
|
||||
FeatureSelector: featureSelector,
|
||||
MaxEdgeAddrRetries: uint8(c.Int("max-edge-addr-retries")),
|
||||
UDPUnregisterSessionTimeout: c.Duration(udpUnregisterSessionTimeoutFlag),
|
||||
RPCTimeout: c.Duration(rpcTimeout),
|
||||
WriteStreamTimeout: c.Duration(writeStreamTimeout),
|
||||
DisableQUICPathMTUDiscovery: c.Bool(quicDisablePathMTUDiscovery),
|
||||
}
|
||||
|
|
|
@ -28,9 +28,11 @@ import (
|
|||
"github.com/cloudflare/cloudflared/ingress"
|
||||
"github.com/cloudflare/cloudflared/management"
|
||||
"github.com/cloudflare/cloudflared/packet"
|
||||
quicpogs "github.com/cloudflare/cloudflared/quic"
|
||||
cfdquic "github.com/cloudflare/cloudflared/quic"
|
||||
"github.com/cloudflare/cloudflared/tracing"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
tunnelpogs "github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
rpcquic "github.com/cloudflare/cloudflared/tunnelrpc/quic"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -59,14 +61,14 @@ type QUICConnection struct {
|
|||
// sessionManager tracks active sessions. It receives datagrams from quic connection via datagramMuxer
|
||||
sessionManager datagramsession.Manager
|
||||
// datagramMuxer mux/demux datagrams from quic connection
|
||||
datagramMuxer *quicpogs.DatagramMuxerV2
|
||||
datagramMuxer *cfdquic.DatagramMuxerV2
|
||||
packetRouter *ingress.PacketRouter
|
||||
controlStreamHandler ControlStreamHandler
|
||||
connOptions *tunnelpogs.ConnectionOptions
|
||||
connIndex uint8
|
||||
|
||||
udpUnregisterTimeout time.Duration
|
||||
streamWriteTimeout time.Duration
|
||||
rpcTimeout time.Duration
|
||||
streamWriteTimeout time.Duration
|
||||
}
|
||||
|
||||
// NewQUICConnection returns a new instance of QUICConnection.
|
||||
|
@ -82,7 +84,7 @@ func NewQUICConnection(
|
|||
controlStreamHandler ControlStreamHandler,
|
||||
logger *zerolog.Logger,
|
||||
packetRouterConfig *ingress.GlobalRouterConfig,
|
||||
udpUnregisterTimeout time.Duration,
|
||||
rpcTimeout time.Duration,
|
||||
streamWriteTimeout time.Duration,
|
||||
) (*QUICConnection, error) {
|
||||
udpConn, err := createUDPConnForConnIndex(connIndex, localAddr, logger)
|
||||
|
@ -104,7 +106,7 @@ func NewQUICConnection(
|
|||
}
|
||||
|
||||
sessionDemuxChan := make(chan *packet.Session, demuxChanCapacity)
|
||||
datagramMuxer := quicpogs.NewDatagramMuxerV2(session, logger, sessionDemuxChan)
|
||||
datagramMuxer := cfdquic.NewDatagramMuxerV2(session, logger, sessionDemuxChan)
|
||||
sessionManager := datagramsession.NewManager(logger, datagramMuxer.SendToSession, sessionDemuxChan)
|
||||
packetRouter := ingress.NewPacketRouter(packetRouterConfig, datagramMuxer, logger)
|
||||
|
||||
|
@ -118,7 +120,7 @@ func NewQUICConnection(
|
|||
controlStreamHandler: controlStreamHandler,
|
||||
connOptions: connOptions,
|
||||
connIndex: connIndex,
|
||||
udpUnregisterTimeout: udpUnregisterTimeout,
|
||||
rpcTimeout: rpcTimeout,
|
||||
streamWriteTimeout: streamWriteTimeout,
|
||||
}, nil
|
||||
}
|
||||
|
@ -198,7 +200,7 @@ func (q *QUICConnection) acceptStream(ctx context.Context) error {
|
|||
|
||||
func (q *QUICConnection) runStream(quicStream quic.Stream) {
|
||||
ctx := quicStream.Context()
|
||||
stream := quicpogs.NewSafeStreamCloser(quicStream, q.streamWriteTimeout, q.logger)
|
||||
stream := cfdquic.NewSafeStreamCloser(quicStream, q.streamWriteTimeout, q.logger)
|
||||
defer stream.Close()
|
||||
|
||||
// we are going to fuse readers/writers from stream <- cloudflared -> origin, and we want to guarantee that
|
||||
|
@ -206,7 +208,8 @@ func (q *QUICConnection) runStream(quicStream quic.Stream) {
|
|||
// So, we wrap the stream with a no-op write closer and only this method can actually close write side of the stream.
|
||||
// A call to close will simulate a close to the read-side, which will fail subsequent reads.
|
||||
noCloseStream := &nopCloserReadWriter{ReadWriteCloser: stream}
|
||||
if err := q.handleStream(ctx, noCloseStream); err != nil {
|
||||
ss := rpcquic.NewCloudflaredServer(q.handleDataStream, q, q, q.rpcTimeout)
|
||||
if err := ss.Serve(ctx, noCloseStream); err != nil {
|
||||
q.logger.Debug().Err(err).Msg("Failed to handle QUIC stream")
|
||||
|
||||
// if we received an error at this level, then close write side of stream with an error, which will result in
|
||||
|
@ -215,30 +218,7 @@ func (q *QUICConnection) runStream(quicStream quic.Stream) {
|
|||
}
|
||||
}
|
||||
|
||||
func (q *QUICConnection) handleStream(ctx context.Context, stream io.ReadWriteCloser) error {
|
||||
signature, err := quicpogs.DetermineProtocol(stream)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
switch signature {
|
||||
case quicpogs.DataStreamProtocolSignature:
|
||||
reqServerStream, err := quicpogs.NewRequestServerStream(stream, signature)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return q.handleDataStream(ctx, reqServerStream)
|
||||
case quicpogs.RPCStreamProtocolSignature:
|
||||
rpcStream, err := quicpogs.NewRPCServerStream(stream, signature)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return q.handleRPCStream(rpcStream)
|
||||
default:
|
||||
return fmt.Errorf("unknown protocol %v", signature)
|
||||
}
|
||||
}
|
||||
|
||||
func (q *QUICConnection) handleDataStream(ctx context.Context, stream *quicpogs.RequestServerStream) error {
|
||||
func (q *QUICConnection) handleDataStream(ctx context.Context, stream *rpcquic.RequestServerStream) error {
|
||||
request, err := stream.ReadConnectRequestData()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -264,22 +244,22 @@ func (q *QUICConnection) handleDataStream(ctx context.Context, stream *quicpogs.
|
|||
// dispatchRequest will dispatch the request depending on the type and returns an error if it occurs.
|
||||
// More importantly, it also tells if the during processing of the request the ConnectResponse metadata was sent downstream.
|
||||
// This is important since it informs
|
||||
func (q *QUICConnection) dispatchRequest(ctx context.Context, stream *quicpogs.RequestServerStream, err error, request *quicpogs.ConnectRequest) (error, bool) {
|
||||
func (q *QUICConnection) dispatchRequest(ctx context.Context, stream *rpcquic.RequestServerStream, err error, request *pogs.ConnectRequest) (error, bool) {
|
||||
originProxy, err := q.orchestrator.GetOriginProxy()
|
||||
if err != nil {
|
||||
return err, false
|
||||
}
|
||||
|
||||
switch request.Type {
|
||||
case quicpogs.ConnectionTypeHTTP, quicpogs.ConnectionTypeWebsocket:
|
||||
case pogs.ConnectionTypeHTTP, pogs.ConnectionTypeWebsocket:
|
||||
tracedReq, err := buildHTTPRequest(ctx, request, stream, q.connIndex, q.logger)
|
||||
if err != nil {
|
||||
return err, false
|
||||
}
|
||||
w := newHTTPResponseAdapter(stream)
|
||||
return originProxy.ProxyHTTP(&w, tracedReq, request.Type == quicpogs.ConnectionTypeWebsocket), w.connectResponseSent
|
||||
return originProxy.ProxyHTTP(&w, tracedReq, request.Type == pogs.ConnectionTypeWebsocket), w.connectResponseSent
|
||||
|
||||
case quicpogs.ConnectionTypeTCP:
|
||||
case pogs.ConnectionTypeTCP:
|
||||
rwa := &streamReadWriteAcker{RequestServerStream: stream}
|
||||
metadata := request.MetadataMap()
|
||||
return originProxy.ProxyTCP(ctx, rwa, &TCPRequest{
|
||||
|
@ -293,14 +273,6 @@ func (q *QUICConnection) dispatchRequest(ctx context.Context, stream *quicpogs.R
|
|||
}
|
||||
}
|
||||
|
||||
func (q *QUICConnection) handleRPCStream(rpcStream *quicpogs.RPCServerStream) error {
|
||||
if err := rpcStream.Serve(q, q, q.logger); err != nil {
|
||||
q.logger.Err(err).Msg("failed handling RPC stream")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// RegisterUdpSession is the RPC method invoked by edge to register and run a session
|
||||
func (q *QUICConnection) RegisterUdpSession(ctx context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeAfterIdleHint time.Duration, traceContext string) (*tunnelpogs.RegisterUdpSessionResponse, error) {
|
||||
traceCtx := tracing.NewTracedContext(ctx, traceContext, q.logger)
|
||||
|
@ -377,9 +349,9 @@ func (q *QUICConnection) closeUDPSession(ctx context.Context, sessionID uuid.UUI
|
|||
return
|
||||
}
|
||||
|
||||
stream := quicpogs.NewSafeStreamCloser(quicStream, q.streamWriteTimeout, q.logger)
|
||||
stream := cfdquic.NewSafeStreamCloser(quicStream, q.streamWriteTimeout, q.logger)
|
||||
defer stream.Close()
|
||||
rpcClientStream, err := quicpogs.NewRPCClientStream(ctx, stream, q.udpUnregisterTimeout, q.logger)
|
||||
rpcClientStream, err := rpcquic.NewSessionClient(ctx, stream, q.rpcTimeout)
|
||||
if err != nil {
|
||||
// Log this at debug because this is not an error if session was closed due to lost connection
|
||||
// with edge
|
||||
|
@ -408,16 +380,16 @@ func (q *QUICConnection) UpdateConfiguration(ctx context.Context, version int32,
|
|||
// streamReadWriteAcker is a light wrapper over QUIC streams with a callback to send response back to
|
||||
// the client.
|
||||
type streamReadWriteAcker struct {
|
||||
*quicpogs.RequestServerStream
|
||||
*rpcquic.RequestServerStream
|
||||
connectResponseSent bool
|
||||
}
|
||||
|
||||
// AckConnection acks response back to the proxy.
|
||||
func (s *streamReadWriteAcker) AckConnection(tracePropagation string) error {
|
||||
metadata := []quicpogs.Metadata{}
|
||||
metadata := []pogs.Metadata{}
|
||||
// Only add tracing if provided by origintunneld
|
||||
if tracePropagation != "" {
|
||||
metadata = append(metadata, quicpogs.Metadata{
|
||||
metadata = append(metadata, pogs.Metadata{
|
||||
Key: tracing.CanonicalCloudflaredTracingHeader,
|
||||
Val: tracePropagation,
|
||||
})
|
||||
|
@ -428,12 +400,12 @@ func (s *streamReadWriteAcker) AckConnection(tracePropagation string) error {
|
|||
|
||||
// httpResponseAdapter translates responses written by the HTTP Proxy into ones that can be used in QUIC.
|
||||
type httpResponseAdapter struct {
|
||||
*quicpogs.RequestServerStream
|
||||
*rpcquic.RequestServerStream
|
||||
headers http.Header
|
||||
connectResponseSent bool
|
||||
}
|
||||
|
||||
func newHTTPResponseAdapter(s *quicpogs.RequestServerStream) httpResponseAdapter {
|
||||
func newHTTPResponseAdapter(s *rpcquic.RequestServerStream) httpResponseAdapter {
|
||||
return httpResponseAdapter{RequestServerStream: s, headers: make(http.Header)}
|
||||
}
|
||||
|
||||
|
@ -442,12 +414,12 @@ func (hrw *httpResponseAdapter) AddTrailer(trailerName, trailerValue string) {
|
|||
}
|
||||
|
||||
func (hrw *httpResponseAdapter) WriteRespHeaders(status int, header http.Header) error {
|
||||
metadata := make([]quicpogs.Metadata, 0)
|
||||
metadata = append(metadata, quicpogs.Metadata{Key: "HttpStatus", Val: strconv.Itoa(status)})
|
||||
metadata := make([]pogs.Metadata, 0)
|
||||
metadata = append(metadata, pogs.Metadata{Key: "HttpStatus", Val: strconv.Itoa(status)})
|
||||
for k, vv := range header {
|
||||
for _, v := range vv {
|
||||
httpHeaderKey := fmt.Sprintf("%s:%s", HTTPHeaderKey, k)
|
||||
metadata = append(metadata, quicpogs.Metadata{Key: httpHeaderKey, Val: v})
|
||||
metadata = append(metadata, pogs.Metadata{Key: httpHeaderKey, Val: v})
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -483,17 +455,17 @@ func (hrw *httpResponseAdapter) Hijack() (net.Conn, *bufio.ReadWriter, error) {
|
|||
}
|
||||
|
||||
func (hrw *httpResponseAdapter) WriteErrorResponse(err error) {
|
||||
hrw.WriteConnectResponseData(err, quicpogs.Metadata{Key: "HttpStatus", Val: strconv.Itoa(http.StatusBadGateway)})
|
||||
hrw.WriteConnectResponseData(err, pogs.Metadata{Key: "HttpStatus", Val: strconv.Itoa(http.StatusBadGateway)})
|
||||
}
|
||||
|
||||
func (hrw *httpResponseAdapter) WriteConnectResponseData(respErr error, metadata ...quicpogs.Metadata) error {
|
||||
func (hrw *httpResponseAdapter) WriteConnectResponseData(respErr error, metadata ...pogs.Metadata) error {
|
||||
hrw.connectResponseSent = true
|
||||
return hrw.RequestServerStream.WriteConnectResponseData(respErr, metadata...)
|
||||
}
|
||||
|
||||
func buildHTTPRequest(
|
||||
ctx context.Context,
|
||||
connectRequest *quicpogs.ConnectRequest,
|
||||
connectRequest *pogs.ConnectRequest,
|
||||
body io.ReadCloser,
|
||||
connIndex uint8,
|
||||
log *zerolog.Logger,
|
||||
|
@ -502,7 +474,7 @@ func buildHTTPRequest(
|
|||
dest := connectRequest.Dest
|
||||
method := metadata[HTTPMethodKey]
|
||||
host := metadata[HTTPHostKey]
|
||||
isWebsocket := connectRequest.Type == quicpogs.ConnectionTypeWebsocket
|
||||
isWebsocket := connectRequest.Type == pogs.ConnectionTypeWebsocket
|
||||
|
||||
req, err := http.NewRequestWithContext(ctx, method, dest, body)
|
||||
if err != nil {
|
||||
|
@ -597,11 +569,11 @@ func (np *nopCloserReadWriter) Close() error {
|
|||
|
||||
// muxerWrapper wraps DatagramMuxerV2 to satisfy the packet.FunnelUniPipe interface
|
||||
type muxerWrapper struct {
|
||||
muxer *quicpogs.DatagramMuxerV2
|
||||
muxer *cfdquic.DatagramMuxerV2
|
||||
}
|
||||
|
||||
func (rp *muxerWrapper) SendPacket(dst netip.Addr, pk packet.RawPacket) error {
|
||||
return rp.muxer.SendPacket(quicpogs.RawPacket(pk))
|
||||
return rp.muxer.SendPacket(cfdquic.RawPacket(pk))
|
||||
}
|
||||
|
||||
func (rp *muxerWrapper) ReceivePacket(ctx context.Context) (packet.RawPacket, error) {
|
||||
|
@ -609,7 +581,7 @@ func (rp *muxerWrapper) ReceivePacket(ctx context.Context) (packet.RawPacket, er
|
|||
if err != nil {
|
||||
return packet.RawPacket{}, err
|
||||
}
|
||||
rawPacket, ok := pk.(quicpogs.RawPacket)
|
||||
rawPacket, ok := pk.(cfdquic.RawPacket)
|
||||
if ok {
|
||||
return packet.RawPacket(rawPacket), nil
|
||||
}
|
||||
|
|
|
@ -3,9 +3,14 @@ package connection
|
|||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"crypto/rand"
|
||||
"crypto/rsa"
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"encoding/pem"
|
||||
"fmt"
|
||||
"io"
|
||||
"math/big"
|
||||
"net"
|
||||
"net/http"
|
||||
"net/url"
|
||||
|
@ -23,14 +28,15 @@ import (
|
|||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/cloudflare/cloudflared/datagramsession"
|
||||
quicpogs "github.com/cloudflare/cloudflared/quic"
|
||||
cfdquic "github.com/cloudflare/cloudflared/quic"
|
||||
"github.com/cloudflare/cloudflared/tracing"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
tunnelpogs "github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
rpcquic "github.com/cloudflare/cloudflared/tunnelrpc/quic"
|
||||
)
|
||||
|
||||
var (
|
||||
testTLSServerConfig = quicpogs.GenerateTLSConfig()
|
||||
testTLSServerConfig = GenerateTLSConfig()
|
||||
testQUICConfig = &quic.Config{
|
||||
KeepAlivePeriod: 5 * time.Second,
|
||||
EnableDatagrams: true,
|
||||
|
@ -50,16 +56,16 @@ func TestQUICServer(t *testing.T) {
|
|||
var tests = []struct {
|
||||
desc string
|
||||
dest string
|
||||
connectionType quicpogs.ConnectionType
|
||||
metadata []quicpogs.Metadata
|
||||
connectionType pogs.ConnectionType
|
||||
metadata []pogs.Metadata
|
||||
message []byte
|
||||
expectedResponse []byte
|
||||
}{
|
||||
{
|
||||
desc: "test http proxy",
|
||||
dest: "/ok",
|
||||
connectionType: quicpogs.ConnectionTypeHTTP,
|
||||
metadata: []quicpogs.Metadata{
|
||||
connectionType: pogs.ConnectionTypeHTTP,
|
||||
metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "HttpHeader:Cf-Ray",
|
||||
Val: "123123123",
|
||||
|
@ -78,8 +84,8 @@ func TestQUICServer(t *testing.T) {
|
|||
{
|
||||
desc: "test http body request streaming",
|
||||
dest: "/slow_echo_body",
|
||||
connectionType: quicpogs.ConnectionTypeHTTP,
|
||||
metadata: []quicpogs.Metadata{
|
||||
connectionType: pogs.ConnectionTypeHTTP,
|
||||
metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "HttpHeader:Cf-Ray",
|
||||
Val: "123123123",
|
||||
|
@ -103,8 +109,8 @@ func TestQUICServer(t *testing.T) {
|
|||
{
|
||||
desc: "test ws proxy",
|
||||
dest: "/ws/echo",
|
||||
connectionType: quicpogs.ConnectionTypeWebsocket,
|
||||
metadata: []quicpogs.Metadata{
|
||||
connectionType: pogs.ConnectionTypeWebsocket,
|
||||
metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "HttpHeader:Cf-Cloudflared-Proxy-Connection-Upgrade",
|
||||
Val: "Websocket",
|
||||
|
@ -127,8 +133,8 @@ func TestQUICServer(t *testing.T) {
|
|||
},
|
||||
{
|
||||
desc: "test tcp proxy",
|
||||
connectionType: quicpogs.ConnectionTypeTCP,
|
||||
metadata: []quicpogs.Metadata{},
|
||||
connectionType: pogs.ConnectionTypeTCP,
|
||||
metadata: []pogs.Metadata{},
|
||||
message: []byte("Here is some tcp data"),
|
||||
expectedResponse: []byte("Here is some tcp data"),
|
||||
},
|
||||
|
@ -175,7 +181,7 @@ type fakeControlStream struct {
|
|||
ControlStreamHandler
|
||||
}
|
||||
|
||||
func (fakeControlStream) ServeControlStream(ctx context.Context, rw io.ReadWriteCloser, connOptions *tunnelpogs.ConnectionOptions, tunnelConfigGetter TunnelConfigJSONGetter) error {
|
||||
func (fakeControlStream) ServeControlStream(ctx context.Context, rw io.ReadWriteCloser, connOptions *pogs.ConnectionOptions, tunnelConfigGetter TunnelConfigJSONGetter) error {
|
||||
<-ctx.Done()
|
||||
return nil
|
||||
}
|
||||
|
@ -188,8 +194,8 @@ func quicServer(
|
|||
t *testing.T,
|
||||
listener *quic.Listener,
|
||||
dest string,
|
||||
connectionType quicpogs.ConnectionType,
|
||||
metadata []quicpogs.Metadata,
|
||||
connectionType pogs.ConnectionType,
|
||||
metadata []pogs.Metadata,
|
||||
message []byte,
|
||||
expectedResponse []byte,
|
||||
) {
|
||||
|
@ -198,9 +204,9 @@ func quicServer(
|
|||
|
||||
quicStream, err := session.OpenStreamSync(context.Background())
|
||||
require.NoError(t, err)
|
||||
stream := quicpogs.NewSafeStreamCloser(quicStream, defaultQUICTimeout, &log)
|
||||
stream := cfdquic.NewSafeStreamCloser(quicStream, defaultQUICTimeout, &log)
|
||||
|
||||
reqClientStream := quicpogs.RequestClientStream{ReadWriteCloser: stream}
|
||||
reqClientStream := rpcquic.RequestClientStream{ReadWriteCloser: stream}
|
||||
err = reqClientStream.WriteConnectRequestData(dest, connectionType, metadata...)
|
||||
require.NoError(t, err)
|
||||
|
||||
|
@ -265,15 +271,15 @@ func (moc *mockOriginProxyWithRequest) ProxyHTTP(w ResponseWriter, tr *tracing.T
|
|||
func TestBuildHTTPRequest(t *testing.T) {
|
||||
var tests = []struct {
|
||||
name string
|
||||
connectRequest *quicpogs.ConnectRequest
|
||||
connectRequest *pogs.ConnectRequest
|
||||
body io.ReadCloser
|
||||
req *http.Request
|
||||
}{
|
||||
{
|
||||
name: "check if http.Request is built correctly with content length",
|
||||
connectRequest: &quicpogs.ConnectRequest{
|
||||
connectRequest: &pogs.ConnectRequest{
|
||||
Dest: "http://test.com",
|
||||
Metadata: []quicpogs.Metadata{
|
||||
Metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "HttpHeader:Cf-Cloudflared-Proxy-Connection-Upgrade",
|
||||
Val: "Websocket",
|
||||
|
@ -317,9 +323,9 @@ func TestBuildHTTPRequest(t *testing.T) {
|
|||
},
|
||||
{
|
||||
name: "if content length isn't part of request headers, then it's not set",
|
||||
connectRequest: &quicpogs.ConnectRequest{
|
||||
connectRequest: &pogs.ConnectRequest{
|
||||
Dest: "http://test.com",
|
||||
Metadata: []quicpogs.Metadata{
|
||||
Metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "HttpHeader:Cf-Cloudflared-Proxy-Connection-Upgrade",
|
||||
Val: "Websocket",
|
||||
|
@ -358,9 +364,9 @@ func TestBuildHTTPRequest(t *testing.T) {
|
|||
},
|
||||
{
|
||||
name: "if content length is 0, but transfer-encoding is chunked, body is not nil",
|
||||
connectRequest: &quicpogs.ConnectRequest{
|
||||
connectRequest: &pogs.ConnectRequest{
|
||||
Dest: "http://test.com",
|
||||
Metadata: []quicpogs.Metadata{
|
||||
Metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "HttpHeader:Another-Header",
|
||||
Val: "Misc",
|
||||
|
@ -400,9 +406,9 @@ func TestBuildHTTPRequest(t *testing.T) {
|
|||
},
|
||||
{
|
||||
name: "if content length is 0, but transfer-encoding is gzip,chunked, body is not nil",
|
||||
connectRequest: &quicpogs.ConnectRequest{
|
||||
connectRequest: &pogs.ConnectRequest{
|
||||
Dest: "http://test.com",
|
||||
Metadata: []quicpogs.Metadata{
|
||||
Metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "HttpHeader:Another-Header",
|
||||
Val: "Misc",
|
||||
|
@ -442,10 +448,10 @@ func TestBuildHTTPRequest(t *testing.T) {
|
|||
},
|
||||
{
|
||||
name: "if content length is 0, and connect request is a websocket, body is not nil",
|
||||
connectRequest: &quicpogs.ConnectRequest{
|
||||
Type: quicpogs.ConnectionTypeWebsocket,
|
||||
connectRequest: &pogs.ConnectRequest{
|
||||
Type: pogs.ConnectionTypeWebsocket,
|
||||
Dest: "http://test.com",
|
||||
Metadata: []quicpogs.Metadata{
|
||||
Metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "HttpHeader:Another-Header",
|
||||
Val: "Misc",
|
||||
|
@ -617,9 +623,9 @@ func serveSession(ctx context.Context, qc *QUICConnection, edgeQUICSession quic.
|
|||
}()
|
||||
|
||||
// Send a message to the quic session on edge side, it should be deumx to this datagram v2 session
|
||||
muxedPayload, err := quicpogs.SuffixSessionID(sessionID, payload)
|
||||
muxedPayload, err := cfdquic.SuffixSessionID(sessionID, payload)
|
||||
require.NoError(t, err)
|
||||
muxedPayload, err = quicpogs.SuffixType(muxedPayload, quicpogs.DatagramTypeUDP)
|
||||
muxedPayload, err = cfdquic.SuffixType(muxedPayload, cfdquic.DatagramTypeUDP)
|
||||
require.NoError(t, err)
|
||||
|
||||
err = edgeQUICSession.SendDatagram(muxedPayload)
|
||||
|
@ -665,7 +671,7 @@ const (
|
|||
closedByTimeout
|
||||
)
|
||||
|
||||
func runRPCServer(ctx context.Context, session quic.Connection, sessionRPCServer tunnelpogs.SessionManager, configRPCServer tunnelpogs.ConfigurationManager, t *testing.T) {
|
||||
func runRPCServer(ctx context.Context, session quic.Connection, sessionRPCServer pogs.SessionManager, configRPCServer pogs.ConfigurationManager, t *testing.T) {
|
||||
stream, err := session.AcceptStream(ctx)
|
||||
require.NoError(t, err)
|
||||
|
||||
|
@ -674,13 +680,15 @@ func runRPCServer(ctx context.Context, session quic.Connection, sessionRPCServer
|
|||
stream, err = session.AcceptStream(ctx)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
protocol, err := quicpogs.DetermineProtocol(stream)
|
||||
assert.NoError(t, err)
|
||||
rpcServerStream, err := quicpogs.NewRPCServerStream(stream, protocol)
|
||||
assert.NoError(t, err)
|
||||
|
||||
log := zerolog.New(os.Stdout)
|
||||
err = rpcServerStream.Serve(sessionRPCServer, configRPCServer, &log)
|
||||
ss := rpcquic.NewCloudflaredServer(
|
||||
func(_ context.Context, _ *rpcquic.RequestServerStream) error {
|
||||
return nil
|
||||
},
|
||||
sessionRPCServer,
|
||||
configRPCServer,
|
||||
10*time.Second,
|
||||
)
|
||||
err = ss.Serve(ctx, stream)
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
|
||||
|
@ -726,7 +734,7 @@ func testQUICConnection(udpListenerAddr net.Addr, t *testing.T, index uint8) *QU
|
|||
fakeControlStream{},
|
||||
&log,
|
||||
nil,
|
||||
5*time.Second,
|
||||
15*time.Second,
|
||||
0*time.Second,
|
||||
)
|
||||
require.NoError(t, err)
|
||||
|
@ -744,3 +752,27 @@ func (m *mockReaderNoopWriter) Write(p []byte) (n int, err error) {
|
|||
func (m *mockReaderNoopWriter) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// GenerateTLSConfig sets up a bare-bones TLS config for a QUIC server
|
||||
func GenerateTLSConfig() *tls.Config {
|
||||
key, err := rsa.GenerateKey(rand.Reader, 1024)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
template := x509.Certificate{SerialNumber: big.NewInt(1)}
|
||||
certDER, err := x509.CreateCertificate(rand.Reader, &template, &template, &key.PublicKey, key)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
keyPEM := pem.EncodeToMemory(&pem.Block{Type: "RSA PRIVATE KEY", Bytes: x509.MarshalPKCS1PrivateKey(key)})
|
||||
certPEM := pem.EncodeToMemory(&pem.Block{Type: "CERTIFICATE", Bytes: certDER})
|
||||
|
||||
tlsCert, err := tls.X509KeyPair(certPEM, keyPEM)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return &tls.Config{
|
||||
Certificates: []tls.Certificate{tlsCert},
|
||||
NextProtos: []string{"argotunnel"},
|
||||
}
|
||||
}
|
||||
|
|
|
@ -9,7 +9,6 @@ import (
|
|||
"github.com/rs/zerolog"
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
tunnelpogs "github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
|
@ -25,11 +24,8 @@ func NewTunnelServerClient(
|
|||
stream io.ReadWriteCloser,
|
||||
log *zerolog.Logger,
|
||||
) *tunnelServerClient {
|
||||
transport := tunnelrpc.NewTransportLogger(log, rpc.StreamTransport(stream))
|
||||
conn := rpc.NewConn(
|
||||
transport,
|
||||
tunnelrpc.ConnLog(log),
|
||||
)
|
||||
transport := rpc.StreamTransport(stream)
|
||||
conn := rpc.NewConn(transport)
|
||||
registrationClient := tunnelpogs.RegistrationServer_PogsClient{Client: conn.Bootstrap(ctx), Conn: conn}
|
||||
return &tunnelServerClient{
|
||||
client: tunnelpogs.TunnelServer_PogsClient{RegistrationServer_PogsClient: registrationClient, Client: conn.Bootstrap(ctx), Conn: conn},
|
||||
|
@ -79,11 +75,8 @@ func newRegistrationRPCClient(
|
|||
stream io.ReadWriteCloser,
|
||||
log *zerolog.Logger,
|
||||
) NamedTunnelRPCClient {
|
||||
transport := tunnelrpc.NewTransportLogger(log, rpc.StreamTransport(stream))
|
||||
conn := rpc.NewConn(
|
||||
transport,
|
||||
tunnelrpc.ConnLog(log),
|
||||
)
|
||||
transport := rpc.StreamTransport(stream)
|
||||
conn := rpc.NewConn(transport)
|
||||
return ®istrationServerClient{
|
||||
client: tunnelpogs.RegistrationServer_PogsClient{Client: conn.Bootstrap(ctx), Conn: conn},
|
||||
transport: transport,
|
||||
|
|
|
@ -0,0 +1,12 @@
|
|||
package quic
|
||||
|
||||
import "time"
|
||||
|
||||
const (
|
||||
HandshakeIdleTimeout = 5 * time.Second
|
||||
MaxIdleTimeout = 5 * time.Second
|
||||
MaxIdlePingPeriod = 1 * time.Second
|
||||
|
||||
// MaxIncomingStreams is 2^60, which is the maximum supported value by Quic-Go
|
||||
MaxIncomingStreams = 1 << 60
|
||||
)
|
|
@ -1,274 +0,0 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
capnp "zombiezen.com/go/capnproto2"
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/rs/zerolog"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
tunnelpogs "github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
// ProtocolSignature defines the first 6 bytes of the stream, which is used to distinguish the type of stream. It
|
||||
// ensures whoever performs a handshake does not write data before writing the metadata.
|
||||
type ProtocolSignature [6]byte
|
||||
|
||||
var (
|
||||
// DataStreamProtocolSignature is a custom protocol signature for data stream
|
||||
DataStreamProtocolSignature = ProtocolSignature{0x0A, 0x36, 0xCD, 0x12, 0xA1, 0x3E}
|
||||
|
||||
// RPCStreamProtocolSignature is a custom protocol signature for RPC stream
|
||||
RPCStreamProtocolSignature = ProtocolSignature{0x52, 0xBB, 0x82, 0x5C, 0xDB, 0x65}
|
||||
)
|
||||
|
||||
type protocolVersion string
|
||||
|
||||
const (
|
||||
protocolV1 protocolVersion = "01"
|
||||
|
||||
protocolVersionLength = 2
|
||||
|
||||
HandshakeIdleTimeout = 5 * time.Second
|
||||
MaxIdleTimeout = 5 * time.Second
|
||||
MaxIdlePingPeriod = 1 * time.Second
|
||||
|
||||
// MaxIncomingStreams is 2^60, which is the maximum supported value by Quic-Go
|
||||
MaxIncomingStreams = 1 << 60
|
||||
)
|
||||
|
||||
// RequestServerStream is a stream to serve requests
|
||||
type RequestServerStream struct {
|
||||
io.ReadWriteCloser
|
||||
}
|
||||
|
||||
func NewRequestServerStream(stream io.ReadWriteCloser, signature ProtocolSignature) (*RequestServerStream, error) {
|
||||
if signature != DataStreamProtocolSignature {
|
||||
return nil, fmt.Errorf("RequestClientStream can only be created from data stream")
|
||||
}
|
||||
return &RequestServerStream{stream}, nil
|
||||
}
|
||||
|
||||
// ReadConnectRequestData reads the handshake data from a QUIC stream.
|
||||
func (rss *RequestServerStream) ReadConnectRequestData() (*ConnectRequest, error) {
|
||||
// This is a NO-OP for now. We could cause a branching if we wanted to use multiple versions.
|
||||
if _, err := readVersion(rss); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
msg, err := capnp.NewDecoder(rss).Decode()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
r := &ConnectRequest{}
|
||||
if err := r.fromPogs(msg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return r, nil
|
||||
}
|
||||
|
||||
// WriteConnectResponseData writes response to a QUIC stream.
|
||||
func (rss *RequestServerStream) WriteConnectResponseData(respErr error, metadata ...Metadata) error {
|
||||
var connectResponse *ConnectResponse
|
||||
if respErr != nil {
|
||||
connectResponse = &ConnectResponse{
|
||||
Error: respErr.Error(),
|
||||
}
|
||||
} else {
|
||||
connectResponse = &ConnectResponse{
|
||||
Metadata: metadata,
|
||||
}
|
||||
}
|
||||
|
||||
msg, err := connectResponse.toPogs()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := writeDataStreamPreamble(rss); err != nil {
|
||||
return err
|
||||
}
|
||||
return capnp.NewEncoder(rss).Encode(msg)
|
||||
}
|
||||
|
||||
type RequestClientStream struct {
|
||||
io.ReadWriteCloser
|
||||
}
|
||||
|
||||
// WriteConnectRequestData writes requestMeta to a stream.
|
||||
func (rcs *RequestClientStream) WriteConnectRequestData(dest string, connectionType ConnectionType, metadata ...Metadata) error {
|
||||
connectRequest := &ConnectRequest{
|
||||
Dest: dest,
|
||||
Type: connectionType,
|
||||
Metadata: metadata,
|
||||
}
|
||||
|
||||
msg, err := connectRequest.toPogs()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := writeDataStreamPreamble(rcs); err != nil {
|
||||
return err
|
||||
}
|
||||
return capnp.NewEncoder(rcs).Encode(msg)
|
||||
}
|
||||
|
||||
// ReadConnectResponseData reads the response to a RequestMeta in a stream.
|
||||
func (rcs *RequestClientStream) ReadConnectResponseData() (*ConnectResponse, error) {
|
||||
signature, err := DetermineProtocol(rcs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if signature != DataStreamProtocolSignature {
|
||||
return nil, fmt.Errorf("wrong protocol signature %v", signature)
|
||||
}
|
||||
|
||||
// This is a NO-OP for now. We could cause a branching if we wanted to use multiple versions.
|
||||
if _, err := readVersion(rcs); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
msg, err := capnp.NewDecoder(rcs).Decode()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
r := &ConnectResponse{}
|
||||
if err := r.fromPogs(msg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return r, nil
|
||||
}
|
||||
|
||||
// RPCServerStream is a stream to serve RPCs. It is closed when the RPC client is done
|
||||
type RPCServerStream struct {
|
||||
io.ReadWriteCloser
|
||||
}
|
||||
|
||||
func NewRPCServerStream(stream io.ReadWriteCloser, protocol ProtocolSignature) (*RPCServerStream, error) {
|
||||
if protocol != RPCStreamProtocolSignature {
|
||||
return nil, fmt.Errorf("RPCStream can only be created from rpc stream")
|
||||
}
|
||||
return &RPCServerStream{stream}, nil
|
||||
}
|
||||
|
||||
func (s *RPCServerStream) Serve(sessionManager tunnelpogs.SessionManager, configManager tunnelpogs.ConfigurationManager, logger *zerolog.Logger) error {
|
||||
// RPC logs are very robust, create a new logger that only logs error to reduce noise
|
||||
rpcLogger := logger.Level(zerolog.ErrorLevel)
|
||||
rpcTransport := tunnelrpc.NewTransportLogger(&rpcLogger, rpc.StreamTransport(s))
|
||||
defer rpcTransport.Close()
|
||||
|
||||
main := tunnelpogs.CloudflaredServer_ServerToClient(sessionManager, configManager)
|
||||
rpcConn := rpc.NewConn(
|
||||
rpcTransport,
|
||||
rpc.MainInterface(main.Client),
|
||||
tunnelrpc.ConnLog(&rpcLogger),
|
||||
)
|
||||
defer rpcConn.Close()
|
||||
|
||||
return rpcConn.Wait()
|
||||
}
|
||||
|
||||
func DetermineProtocol(stream io.Reader) (ProtocolSignature, error) {
|
||||
signature, err := readSignature(stream)
|
||||
if err != nil {
|
||||
return ProtocolSignature{}, err
|
||||
}
|
||||
switch signature {
|
||||
case DataStreamProtocolSignature:
|
||||
return DataStreamProtocolSignature, nil
|
||||
case RPCStreamProtocolSignature:
|
||||
return RPCStreamProtocolSignature, nil
|
||||
default:
|
||||
return ProtocolSignature{}, fmt.Errorf("unknown signature %v", signature)
|
||||
}
|
||||
}
|
||||
|
||||
func writeDataStreamPreamble(stream io.Writer) error {
|
||||
if err := writeSignature(stream, DataStreamProtocolSignature); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return writeVersion(stream)
|
||||
}
|
||||
|
||||
func writeVersion(stream io.Writer) error {
|
||||
_, err := stream.Write([]byte(protocolV1)[:protocolVersionLength])
|
||||
return err
|
||||
}
|
||||
|
||||
func readVersion(stream io.Reader) (string, error) {
|
||||
version := make([]byte, protocolVersionLength)
|
||||
_, err := stream.Read(version)
|
||||
return string(version), err
|
||||
}
|
||||
|
||||
func readSignature(stream io.Reader) (ProtocolSignature, error) {
|
||||
var signature ProtocolSignature
|
||||
if _, err := io.ReadFull(stream, signature[:]); err != nil {
|
||||
return ProtocolSignature{}, err
|
||||
}
|
||||
return signature, nil
|
||||
}
|
||||
|
||||
func writeSignature(stream io.Writer, signature ProtocolSignature) error {
|
||||
_, err := stream.Write(signature[:])
|
||||
return err
|
||||
}
|
||||
|
||||
// RPCClientStream is a stream to call methods of SessionManager
|
||||
type RPCClientStream struct {
|
||||
client tunnelpogs.CloudflaredServer_PogsClient
|
||||
transport rpc.Transport
|
||||
|
||||
// Time we wait for the server to respond to a request before we close the connection.
|
||||
rpcUnregisterUDPSessionDeadline time.Duration
|
||||
}
|
||||
|
||||
func NewRPCClientStream(ctx context.Context, stream io.ReadWriteCloser, rpcUnregisterUDPSessionDeadline time.Duration, logger *zerolog.Logger) (*RPCClientStream, error) {
|
||||
n, err := stream.Write(RPCStreamProtocolSignature[:])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if n != len(RPCStreamProtocolSignature) {
|
||||
return nil, fmt.Errorf("expect to write %d bytes for RPC stream protocol signature, wrote %d", len(RPCStreamProtocolSignature), n)
|
||||
}
|
||||
transport := tunnelrpc.NewTransportLogger(logger, rpc.StreamTransport(stream))
|
||||
conn := rpc.NewConn(
|
||||
transport,
|
||||
tunnelrpc.ConnLog(logger),
|
||||
)
|
||||
return &RPCClientStream{
|
||||
client: tunnelpogs.NewCloudflaredServer_PogsClient(conn.Bootstrap(ctx), conn),
|
||||
transport: transport,
|
||||
rpcUnregisterUDPSessionDeadline: rpcUnregisterUDPSessionDeadline,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (rcs *RPCClientStream) RegisterUdpSession(ctx context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeIdleAfterHint time.Duration, traceContext string) (*tunnelpogs.RegisterUdpSessionResponse, error) {
|
||||
return rcs.client.RegisterUdpSession(ctx, sessionID, dstIP, dstPort, closeIdleAfterHint, traceContext)
|
||||
}
|
||||
|
||||
func (rcs *RPCClientStream) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID, message string) error {
|
||||
ctx, cancel := context.WithTimeout(ctx, rcs.rpcUnregisterUDPSessionDeadline)
|
||||
defer cancel()
|
||||
return rcs.client.UnregisterUdpSession(ctx, sessionID, message)
|
||||
}
|
||||
|
||||
func (rcs *RPCClientStream) UpdateConfiguration(ctx context.Context, version int32, config []byte) (*tunnelpogs.UpdateConfigurationResponse, error) {
|
||||
return rcs.client.UpdateConfiguration(ctx, version, config)
|
||||
}
|
||||
|
||||
func (rcs *RPCClientStream) Close() {
|
||||
_ = rcs.client.Close()
|
||||
_ = rcs.transport.Close()
|
||||
}
|
|
@ -2,8 +2,13 @@ package quic
|
|||
|
||||
import (
|
||||
"context"
|
||||
"crypto/rand"
|
||||
"crypto/rsa"
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"encoding/pem"
|
||||
"io"
|
||||
"math/big"
|
||||
"net"
|
||||
"sync"
|
||||
"testing"
|
||||
|
@ -147,3 +152,27 @@ func serverRoundTrip(t *testing.T, stream io.ReadWriteCloser, mustWork bool) {
|
|||
}
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
// GenerateTLSConfig sets up a bare-bones TLS config for a QUIC server
|
||||
func GenerateTLSConfig() *tls.Config {
|
||||
key, err := rsa.GenerateKey(rand.Reader, 1024)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
template := x509.Certificate{SerialNumber: big.NewInt(1)}
|
||||
certDER, err := x509.CreateCertificate(rand.Reader, &template, &template, &key.PublicKey, key)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
keyPEM := pem.EncodeToMemory(&pem.Block{Type: "RSA PRIVATE KEY", Bytes: x509.MarshalPKCS1PrivateKey(key)})
|
||||
certPEM := pem.EncodeToMemory(&pem.Block{Type: "CERTIFICATE", Bytes: certDER})
|
||||
|
||||
tlsCert, err := tls.X509KeyPair(certPEM, keyPEM)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return &tls.Config{
|
||||
Certificates: []tls.Certificate{tlsCert},
|
||||
NextProtos: []string{"argotunnel"},
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,28 +0,0 @@
|
|||
using Go = import "/go.capnp";
|
||||
@0xb29021ef7421cc32;
|
||||
|
||||
$Go.package("schema");
|
||||
$Go.import("schema");
|
||||
|
||||
|
||||
struct ConnectRequest{
|
||||
dest @0 :Text;
|
||||
type @1 :ConnectionType;
|
||||
metadata @2 :List(Metadata);
|
||||
}
|
||||
|
||||
enum ConnectionType{
|
||||
http @0;
|
||||
websocket @1;
|
||||
tcp @2;
|
||||
}
|
||||
|
||||
struct Metadata {
|
||||
key @0 :Text;
|
||||
val @1 :Text;
|
||||
}
|
||||
|
||||
struct ConnectResponse{
|
||||
error @0 :Text;
|
||||
metadata @1 :List(Metadata);
|
||||
}
|
|
@ -1,34 +0,0 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"crypto/rand"
|
||||
"crypto/rsa"
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"encoding/pem"
|
||||
"math/big"
|
||||
)
|
||||
|
||||
// GenerateTLSConfig sets up a bare-bones TLS config for a QUIC server
|
||||
func GenerateTLSConfig() *tls.Config {
|
||||
key, err := rsa.GenerateKey(rand.Reader, 1024)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
template := x509.Certificate{SerialNumber: big.NewInt(1)}
|
||||
certDER, err := x509.CreateCertificate(rand.Reader, &template, &template, &key.PublicKey, key)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
keyPEM := pem.EncodeToMemory(&pem.Block{Type: "RSA PRIVATE KEY", Bytes: x509.MarshalPKCS1PrivateKey(key)})
|
||||
certPEM := pem.EncodeToMemory(&pem.Block{Type: "CERTIFICATE", Bytes: certDER})
|
||||
|
||||
tlsCert, err := tls.X509KeyPair(certPEM, keyPEM)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return &tls.Config{
|
||||
Certificates: []tls.Certificate{tlsCert},
|
||||
NextProtos: []string{"argotunnel"},
|
||||
}
|
||||
}
|
|
@ -27,8 +27,8 @@ import (
|
|||
quicpogs "github.com/cloudflare/cloudflared/quic"
|
||||
"github.com/cloudflare/cloudflared/retry"
|
||||
"github.com/cloudflare/cloudflared/signal"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
tunnelpogs "github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
"github.com/cloudflare/cloudflared/tunnelstate"
|
||||
)
|
||||
|
||||
|
@ -65,8 +65,8 @@ type TunnelConfig struct {
|
|||
EdgeTLSConfigs map[connection.Protocol]*tls.Config
|
||||
PacketConfig *ingress.GlobalRouterConfig
|
||||
|
||||
UDPUnregisterSessionTimeout time.Duration
|
||||
WriteStreamTimeout time.Duration
|
||||
RPCTimeout time.Duration
|
||||
WriteStreamTimeout time.Duration
|
||||
|
||||
DisableQUICPathMTUDiscovery bool
|
||||
|
||||
|
@ -74,9 +74,9 @@ type TunnelConfig struct {
|
|||
}
|
||||
|
||||
func (c *TunnelConfig) registrationOptions(connectionID uint8, OriginLocalIP string, uuid uuid.UUID) *tunnelpogs.RegistrationOptions {
|
||||
policy := tunnelrpc.ExistingTunnelPolicy_balance
|
||||
policy := proto.ExistingTunnelPolicy_balance
|
||||
if c.HAConnections <= 1 && c.LBPool == "" {
|
||||
policy = tunnelrpc.ExistingTunnelPolicy_disconnect
|
||||
policy = proto.ExistingTunnelPolicy_disconnect
|
||||
}
|
||||
return &tunnelpogs.RegistrationOptions{
|
||||
ClientID: c.ClientID,
|
||||
|
@ -614,7 +614,7 @@ func (e *EdgeTunnelServer) serveQUIC(
|
|||
controlStreamHandler,
|
||||
connLogger.Logger(),
|
||||
e.config.PacketConfig,
|
||||
e.config.UDPUnregisterSessionTimeout,
|
||||
e.config.RPCTimeout,
|
||||
e.config.WriteStreamTimeout,
|
||||
)
|
||||
if err != nil {
|
||||
|
|
|
@ -1,15 +0,0 @@
|
|||
# Generate go.capnp.out with:
|
||||
# capnp compile -o- go.capnp > go.capnp.out
|
||||
# Must run inside this directory to preserve paths.
|
||||
|
||||
@0xd12a1c51fedd6c88;
|
||||
|
||||
annotation package(file) :Text;
|
||||
annotation import(file) :Text;
|
||||
annotation doc(struct, field, enum) :Text;
|
||||
annotation tag(enumerant) :Text;
|
||||
annotation notag(enumerant) :Void;
|
||||
annotation customtype(field) :Text;
|
||||
annotation name(struct, field, union, enum, enumerant, interface, method, param, annotation, const, group) :Text;
|
||||
|
||||
$package("capnp");
|
|
@ -1,43 +0,0 @@
|
|||
package tunnelrpc
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/rs/zerolog"
|
||||
"golang.org/x/net/trace"
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
)
|
||||
|
||||
// ConnLogger wraps a Zerolog Logger for a connection.
|
||||
type ConnLogger struct {
|
||||
Log *zerolog.Logger
|
||||
}
|
||||
|
||||
func (c ConnLogger) Infof(ctx context.Context, format string, args ...interface{}) {
|
||||
c.Log.Info().Msgf(format, args...)
|
||||
}
|
||||
|
||||
func (c ConnLogger) Errorf(ctx context.Context, format string, args ...interface{}) {
|
||||
c.Log.Error().Msgf(format, args...)
|
||||
}
|
||||
|
||||
func ConnLog(log *zerolog.Logger) rpc.ConnOption {
|
||||
return rpc.ConnLog(ConnLogger{log})
|
||||
}
|
||||
|
||||
// ConnTracer wraps a trace.EventLog for a connection.
|
||||
type ConnTracer struct {
|
||||
Events trace.EventLog
|
||||
}
|
||||
|
||||
func (c ConnTracer) Infof(ctx context.Context, format string, args ...interface{}) {
|
||||
c.Events.Printf(format, args...)
|
||||
}
|
||||
|
||||
func (c ConnTracer) Errorf(ctx context.Context, format string, args ...interface{}) {
|
||||
c.Events.Errorf(format, args...)
|
||||
}
|
||||
|
||||
func ConnTrace(events trace.EventLog) rpc.ConnOption {
|
||||
return rpc.ConnLog(ConnTracer{events})
|
||||
}
|
|
@ -1,45 +0,0 @@
|
|||
// Package logtransport provides a transport that logs all of its messages.
|
||||
package tunnelrpc
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
|
||||
"github.com/rs/zerolog"
|
||||
"zombiezen.com/go/capnproto2/encoding/text"
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
rpccapnp "zombiezen.com/go/capnproto2/std/capnp/rpc"
|
||||
)
|
||||
|
||||
type transport struct {
|
||||
rpc.Transport
|
||||
log *zerolog.Logger
|
||||
}
|
||||
|
||||
// NewTransportLogger creates a new logger that proxies messages to and from t and
|
||||
// logs them to log. If log is nil, then the log package's default
|
||||
// logger is used.
|
||||
func NewTransportLogger(log *zerolog.Logger, t rpc.Transport) rpc.Transport {
|
||||
return &transport{Transport: t, log: log}
|
||||
}
|
||||
|
||||
func (t *transport) SendMessage(ctx context.Context, msg rpccapnp.Message) error {
|
||||
t.log.Trace().Msgf("rpc tx: %s", formatMsg(msg))
|
||||
return t.Transport.SendMessage(ctx, msg)
|
||||
}
|
||||
|
||||
func (t *transport) RecvMessage(ctx context.Context) (rpccapnp.Message, error) {
|
||||
msg, err := t.Transport.RecvMessage(ctx)
|
||||
if err != nil {
|
||||
t.log.Debug().Msgf("rpc rx error: %s", err)
|
||||
return msg, err
|
||||
}
|
||||
t.log.Trace().Msgf("rpc rx: %s", formatMsg(msg))
|
||||
return msg, nil
|
||||
}
|
||||
|
||||
func formatMsg(m rpccapnp.Message) string {
|
||||
var buf bytes.Buffer
|
||||
_ = text.NewEncoder(&buf).Encode(0x91b79f1f808db032, m.Struct)
|
||||
return buf.String()
|
||||
}
|
|
@ -6,10 +6,10 @@ import (
|
|||
"zombiezen.com/go/capnproto2/pogs"
|
||||
"zombiezen.com/go/capnproto2/server"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
func (i TunnelServer_PogsImpl) Authenticate(p tunnelrpc.TunnelServer_authenticate) error {
|
||||
func (i TunnelServer_PogsImpl) Authenticate(p proto.TunnelServer_authenticate) error {
|
||||
originCert, err := p.Params.OriginCert()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -39,13 +39,13 @@ func (i TunnelServer_PogsImpl) Authenticate(p tunnelrpc.TunnelServer_authenticat
|
|||
return MarshalAuthenticateResponse(result, resp)
|
||||
}
|
||||
|
||||
func MarshalAuthenticateResponse(s tunnelrpc.AuthenticateResponse, p *AuthenticateResponse) error {
|
||||
return pogs.Insert(tunnelrpc.AuthenticateResponse_TypeID, s.Struct, p)
|
||||
func MarshalAuthenticateResponse(s proto.AuthenticateResponse, p *AuthenticateResponse) error {
|
||||
return pogs.Insert(proto.AuthenticateResponse_TypeID, s.Struct, p)
|
||||
}
|
||||
|
||||
func (c TunnelServer_PogsClient) Authenticate(ctx context.Context, originCert []byte, hostname string, options *RegistrationOptions) (*AuthenticateResponse, error) {
|
||||
client := tunnelrpc.TunnelServer{Client: c.Client}
|
||||
promise := client.Authenticate(ctx, func(p tunnelrpc.TunnelServer_authenticate_Params) error {
|
||||
client := proto.TunnelServer{Client: c.Client}
|
||||
promise := client.Authenticate(ctx, func(p proto.TunnelServer_authenticate_Params) error {
|
||||
err := p.SetOriginCert(originCert)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -71,8 +71,8 @@ func (c TunnelServer_PogsClient) Authenticate(ctx context.Context, originCert []
|
|||
return UnmarshalAuthenticateResponse(retval)
|
||||
}
|
||||
|
||||
func UnmarshalAuthenticateResponse(s tunnelrpc.AuthenticateResponse) (*AuthenticateResponse, error) {
|
||||
func UnmarshalAuthenticateResponse(s proto.AuthenticateResponse) (*AuthenticateResponse, error) {
|
||||
p := new(AuthenticateResponse)
|
||||
err := pogs.Extract(p, tunnelrpc.AuthenticateResponse_TypeID, s.Struct)
|
||||
err := pogs.Extract(p, proto.AuthenticateResponse_TypeID, s.Struct)
|
||||
return p, err
|
||||
}
|
||||
|
|
|
@ -9,7 +9,7 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
capnp "zombiezen.com/go/capnproto2"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
// Ensure the AuthOutcome sum is correct
|
||||
|
@ -119,7 +119,7 @@ func TestSerializeAuthenticationResponse(t *testing.T) {
|
|||
for i, testCase := range tests {
|
||||
_, seg, err := capnp.NewMessage(capnp.SingleSegment(nil))
|
||||
assert.NoError(t, err)
|
||||
capnpEntity, err := tunnelrpc.NewAuthenticateResponse(seg)
|
||||
capnpEntity, err := proto.NewAuthenticateResponse(seg)
|
||||
if !assert.NoError(t, err) {
|
||||
t.Fatal("Couldn't initialize a new message")
|
||||
}
|
||||
|
|
|
@ -1,9 +1,10 @@
|
|||
package pogs
|
||||
|
||||
import (
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
capnp "zombiezen.com/go/capnproto2"
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
type CloudflaredServer interface {
|
||||
|
@ -16,8 +17,8 @@ type CloudflaredServer_PogsImpl struct {
|
|||
ConfigurationManager_PogsImpl
|
||||
}
|
||||
|
||||
func CloudflaredServer_ServerToClient(s SessionManager, c ConfigurationManager) tunnelrpc.CloudflaredServer {
|
||||
return tunnelrpc.CloudflaredServer_ServerToClient(CloudflaredServer_PogsImpl{
|
||||
func CloudflaredServer_ServerToClient(s SessionManager, c ConfigurationManager) proto.CloudflaredServer {
|
||||
return proto.CloudflaredServer_ServerToClient(CloudflaredServer_PogsImpl{
|
||||
SessionManager_PogsImpl: SessionManager_PogsImpl{s},
|
||||
ConfigurationManager_PogsImpl: ConfigurationManager_PogsImpl{c},
|
||||
})
|
||||
|
|
|
@ -4,10 +4,11 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
capnp "zombiezen.com/go/capnproto2"
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
"zombiezen.com/go/capnproto2/server"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
type ConfigurationManager interface {
|
||||
|
@ -18,11 +19,11 @@ type ConfigurationManager_PogsImpl struct {
|
|||
impl ConfigurationManager
|
||||
}
|
||||
|
||||
func ConfigurationManager_ServerToClient(c ConfigurationManager) tunnelrpc.ConfigurationManager {
|
||||
return tunnelrpc.ConfigurationManager_ServerToClient(ConfigurationManager_PogsImpl{c})
|
||||
func ConfigurationManager_ServerToClient(c ConfigurationManager) proto.ConfigurationManager {
|
||||
return proto.ConfigurationManager_ServerToClient(ConfigurationManager_PogsImpl{c})
|
||||
}
|
||||
|
||||
func (i ConfigurationManager_PogsImpl) UpdateConfiguration(p tunnelrpc.ConfigurationManager_updateConfiguration) error {
|
||||
func (i ConfigurationManager_PogsImpl) UpdateConfiguration(p proto.ConfigurationManager_updateConfiguration) error {
|
||||
server.Ack(p.Options)
|
||||
|
||||
version := p.Params.Version()
|
||||
|
@ -51,8 +52,8 @@ func (c ConfigurationManager_PogsClient) Close() error {
|
|||
}
|
||||
|
||||
func (c ConfigurationManager_PogsClient) UpdateConfiguration(ctx context.Context, version int32, config []byte) (*UpdateConfigurationResponse, error) {
|
||||
client := tunnelrpc.ConfigurationManager{Client: c.Client}
|
||||
promise := client.UpdateConfiguration(ctx, func(p tunnelrpc.ConfigurationManager_updateConfiguration_Params) error {
|
||||
client := proto.ConfigurationManager{Client: c.Client}
|
||||
promise := client.UpdateConfiguration(ctx, func(p proto.ConfigurationManager_updateConfiguration_Params) error {
|
||||
p.SetVersion(version)
|
||||
return p.SetConfig(config)
|
||||
})
|
||||
|
@ -74,7 +75,7 @@ type UpdateConfigurationResponse struct {
|
|||
Err error `json:"err"`
|
||||
}
|
||||
|
||||
func (p *UpdateConfigurationResponse) Marshal(s tunnelrpc.UpdateConfigurationResponse) error {
|
||||
func (p *UpdateConfigurationResponse) Marshal(s proto.UpdateConfigurationResponse) error {
|
||||
s.SetLatestAppliedVersion(p.LastAppliedVersion)
|
||||
if p.Err != nil {
|
||||
return s.SetErr(p.Err.Error())
|
||||
|
@ -82,7 +83,7 @@ func (p *UpdateConfigurationResponse) Marshal(s tunnelrpc.UpdateConfigurationRes
|
|||
return nil
|
||||
}
|
||||
|
||||
func (p *UpdateConfigurationResponse) Unmarshal(s tunnelrpc.UpdateConfigurationResponse) error {
|
||||
func (p *UpdateConfigurationResponse) Unmarshal(s proto.UpdateConfigurationResponse) error {
|
||||
p.LastAppliedVersion = s.LatestAppliedVersion()
|
||||
respErr, err := s.Err()
|
||||
if err != nil {
|
||||
|
|
|
@ -12,7 +12,7 @@ import (
|
|||
"zombiezen.com/go/capnproto2/rpc"
|
||||
"zombiezen.com/go/capnproto2/server"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
type RegistrationServer interface {
|
||||
|
@ -25,11 +25,11 @@ type RegistrationServer_PogsImpl struct {
|
|||
impl RegistrationServer
|
||||
}
|
||||
|
||||
func RegistrationServer_ServerToClient(s RegistrationServer) tunnelrpc.RegistrationServer {
|
||||
return tunnelrpc.RegistrationServer_ServerToClient(RegistrationServer_PogsImpl{s})
|
||||
func RegistrationServer_ServerToClient(s RegistrationServer) proto.RegistrationServer {
|
||||
return proto.RegistrationServer_ServerToClient(RegistrationServer_PogsImpl{s})
|
||||
}
|
||||
|
||||
func (i RegistrationServer_PogsImpl) RegisterConnection(p tunnelrpc.RegistrationServer_registerConnection) error {
|
||||
func (i RegistrationServer_PogsImpl) RegisterConnection(p proto.RegistrationServer_registerConnection) error {
|
||||
server.Ack(p.Options)
|
||||
|
||||
auth, err := p.Params.Auth()
|
||||
|
@ -82,14 +82,14 @@ func (i RegistrationServer_PogsImpl) RegisterConnection(p tunnelrpc.Registration
|
|||
}
|
||||
}
|
||||
|
||||
func (i RegistrationServer_PogsImpl) UnregisterConnection(p tunnelrpc.RegistrationServer_unregisterConnection) error {
|
||||
func (i RegistrationServer_PogsImpl) UnregisterConnection(p proto.RegistrationServer_unregisterConnection) error {
|
||||
server.Ack(p.Options)
|
||||
|
||||
i.impl.UnregisterConnection(p.Ctx)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i RegistrationServer_PogsImpl) UpdateLocalConfiguration(c tunnelrpc.RegistrationServer_updateLocalConfiguration) error {
|
||||
func (i RegistrationServer_PogsImpl) UpdateLocalConfiguration(c proto.RegistrationServer_updateLocalConfiguration) error {
|
||||
server.Ack(c.Options)
|
||||
|
||||
configBytes, err := c.Params.Config()
|
||||
|
@ -111,8 +111,8 @@ func (c RegistrationServer_PogsClient) Close() error {
|
|||
}
|
||||
|
||||
func (c RegistrationServer_PogsClient) RegisterConnection(ctx context.Context, auth TunnelAuth, tunnelID uuid.UUID, connIndex byte, options *ConnectionOptions) (*ConnectionDetails, error) {
|
||||
client := tunnelrpc.TunnelServer{Client: c.Client}
|
||||
promise := client.RegisterConnection(ctx, func(p tunnelrpc.RegistrationServer_registerConnection_Params) error {
|
||||
client := proto.TunnelServer{Client: c.Client}
|
||||
promise := client.RegisterConnection(ctx, func(p proto.RegistrationServer_registerConnection_Params) error {
|
||||
tunnelAuth, err := p.NewAuth()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -145,7 +145,7 @@ func (c RegistrationServer_PogsClient) RegisterConnection(ctx context.Context, a
|
|||
}
|
||||
result := response.Result()
|
||||
switch result.Which() {
|
||||
case tunnelrpc.ConnectionResponse_result_Which_error:
|
||||
case proto.ConnectionResponse_result_Which_error:
|
||||
resultError, err := result.Error()
|
||||
if err != nil {
|
||||
return nil, wrapRPCError(err)
|
||||
|
@ -160,7 +160,7 @@ func (c RegistrationServer_PogsClient) RegisterConnection(ctx context.Context, a
|
|||
}
|
||||
return nil, err
|
||||
|
||||
case tunnelrpc.ConnectionResponse_result_Which_connectionDetails:
|
||||
case proto.ConnectionResponse_result_Which_connectionDetails:
|
||||
connDetails, err := result.ConnectionDetails()
|
||||
if err != nil {
|
||||
return nil, wrapRPCError(err)
|
||||
|
@ -176,8 +176,8 @@ func (c RegistrationServer_PogsClient) RegisterConnection(ctx context.Context, a
|
|||
}
|
||||
|
||||
func (c RegistrationServer_PogsClient) SendLocalConfiguration(ctx context.Context, config []byte) error {
|
||||
client := tunnelrpc.TunnelServer{Client: c.Client}
|
||||
promise := client.UpdateLocalConfiguration(ctx, func(p tunnelrpc.RegistrationServer_updateLocalConfiguration_Params) error {
|
||||
client := proto.TunnelServer{Client: c.Client}
|
||||
promise := client.UpdateLocalConfiguration(ctx, func(p proto.RegistrationServer_updateLocalConfiguration_Params) error {
|
||||
if err := p.SetConfig(config); err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -194,8 +194,8 @@ func (c RegistrationServer_PogsClient) SendLocalConfiguration(ctx context.Contex
|
|||
}
|
||||
|
||||
func (c RegistrationServer_PogsClient) UnregisterConnection(ctx context.Context) error {
|
||||
client := tunnelrpc.TunnelServer{Client: c.Client}
|
||||
promise := client.UnregisterConnection(ctx, func(p tunnelrpc.RegistrationServer_unregisterConnection_Params) error {
|
||||
client := proto.TunnelServer{Client: c.Client}
|
||||
promise := client.UnregisterConnection(ctx, func(p proto.RegistrationServer_unregisterConnection_Params) error {
|
||||
return nil
|
||||
})
|
||||
_, err := promise.Struct()
|
||||
|
@ -225,20 +225,20 @@ type TunnelAuth struct {
|
|||
TunnelSecret []byte
|
||||
}
|
||||
|
||||
func (p *ConnectionOptions) MarshalCapnproto(s tunnelrpc.ConnectionOptions) error {
|
||||
return pogs.Insert(tunnelrpc.ConnectionOptions_TypeID, s.Struct, p)
|
||||
func (p *ConnectionOptions) MarshalCapnproto(s proto.ConnectionOptions) error {
|
||||
return pogs.Insert(proto.ConnectionOptions_TypeID, s.Struct, p)
|
||||
}
|
||||
|
||||
func (p *ConnectionOptions) UnmarshalCapnproto(s tunnelrpc.ConnectionOptions) error {
|
||||
return pogs.Extract(p, tunnelrpc.ConnectionOptions_TypeID, s.Struct)
|
||||
func (p *ConnectionOptions) UnmarshalCapnproto(s proto.ConnectionOptions) error {
|
||||
return pogs.Extract(p, proto.ConnectionOptions_TypeID, s.Struct)
|
||||
}
|
||||
|
||||
func (a *TunnelAuth) MarshalCapnproto(s tunnelrpc.TunnelAuth) error {
|
||||
return pogs.Insert(tunnelrpc.TunnelAuth_TypeID, s.Struct, a)
|
||||
func (a *TunnelAuth) MarshalCapnproto(s proto.TunnelAuth) error {
|
||||
return pogs.Insert(proto.TunnelAuth_TypeID, s.Struct, a)
|
||||
}
|
||||
|
||||
func (a *TunnelAuth) UnmarshalCapnproto(s tunnelrpc.TunnelAuth) error {
|
||||
return pogs.Extract(a, tunnelrpc.TunnelAuth_TypeID, s.Struct)
|
||||
func (a *TunnelAuth) UnmarshalCapnproto(s proto.TunnelAuth) error {
|
||||
return pogs.Extract(a, proto.TunnelAuth_TypeID, s.Struct)
|
||||
}
|
||||
|
||||
type ConnectionDetails struct {
|
||||
|
@ -247,7 +247,7 @@ type ConnectionDetails struct {
|
|||
TunnelIsRemotelyManaged bool
|
||||
}
|
||||
|
||||
func (details *ConnectionDetails) MarshalCapnproto(s tunnelrpc.ConnectionDetails) error {
|
||||
func (details *ConnectionDetails) MarshalCapnproto(s proto.ConnectionDetails) error {
|
||||
if err := s.SetUuid(details.UUID[:]); err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -259,7 +259,7 @@ func (details *ConnectionDetails) MarshalCapnproto(s tunnelrpc.ConnectionDetails
|
|||
return nil
|
||||
}
|
||||
|
||||
func (details *ConnectionDetails) UnmarshalCapnproto(s tunnelrpc.ConnectionDetails) error {
|
||||
func (details *ConnectionDetails) UnmarshalCapnproto(s proto.ConnectionDetails) error {
|
||||
uuidBytes, err := s.Uuid()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -277,7 +277,7 @@ func (details *ConnectionDetails) UnmarshalCapnproto(s tunnelrpc.ConnectionDetai
|
|||
return err
|
||||
}
|
||||
|
||||
func MarshalError(s tunnelrpc.ConnectionError, err error) error {
|
||||
func MarshalError(s proto.ConnectionError, err error) error {
|
||||
if err := s.SetCause(err.Error()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -13,7 +13,7 @@ import (
|
|||
capnp "zombiezen.com/go/capnproto2"
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
const testAccountTag = "abc123"
|
||||
|
@ -34,7 +34,7 @@ func TestMarshalConnectionOptions(t *testing.T) {
|
|||
|
||||
_, seg, err := capnp.NewMessage(capnp.SingleSegment(nil))
|
||||
require.NoError(t, err)
|
||||
capnpOpts, err := tunnelrpc.NewConnectionOptions(seg)
|
||||
capnpOpts, err := proto.NewConnectionOptions(seg)
|
||||
require.NoError(t, err)
|
||||
|
||||
err = orig.MarshalCapnproto(capnpOpts)
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package quic
|
||||
package pogs
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
@ -6,7 +6,7 @@ import (
|
|||
capnp "zombiezen.com/go/capnproto2"
|
||||
"zombiezen.com/go/capnproto2/pogs"
|
||||
|
||||
"github.com/cloudflare/cloudflared/quic/schema"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
// ConnectionType indicates the type of underlying connection proxied within the QUIC stream.
|
||||
|
@ -52,26 +52,26 @@ func (r *ConnectRequest) MetadataMap() map[string]string {
|
|||
return metadataMap
|
||||
}
|
||||
|
||||
func (r *ConnectRequest) fromPogs(msg *capnp.Message) error {
|
||||
metadata, err := schema.ReadRootConnectRequest(msg)
|
||||
func (r *ConnectRequest) FromPogs(msg *capnp.Message) error {
|
||||
metadata, err := proto.ReadRootConnectRequest(msg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return pogs.Extract(r, schema.ConnectRequest_TypeID, metadata.Struct)
|
||||
return pogs.Extract(r, proto.ConnectRequest_TypeID, metadata.Struct)
|
||||
}
|
||||
|
||||
func (r *ConnectRequest) toPogs() (*capnp.Message, error) {
|
||||
func (r *ConnectRequest) ToPogs() (*capnp.Message, error) {
|
||||
msg, seg, err := capnp.NewMessage(capnp.SingleSegment(nil))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
root, err := schema.NewRootConnectRequest(seg)
|
||||
root, err := proto.NewRootConnectRequest(seg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := pogs.Insert(schema.ConnectRequest_TypeID, root.Struct, r); err != nil {
|
||||
if err := pogs.Insert(proto.ConnectRequest_TypeID, root.Struct, r); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -84,26 +84,26 @@ type ConnectResponse struct {
|
|||
Metadata []Metadata `capnp:"metadata"`
|
||||
}
|
||||
|
||||
func (r *ConnectResponse) fromPogs(msg *capnp.Message) error {
|
||||
metadata, err := schema.ReadRootConnectResponse(msg)
|
||||
func (r *ConnectResponse) FromPogs(msg *capnp.Message) error {
|
||||
metadata, err := proto.ReadRootConnectResponse(msg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return pogs.Extract(r, schema.ConnectResponse_TypeID, metadata.Struct)
|
||||
return pogs.Extract(r, proto.ConnectResponse_TypeID, metadata.Struct)
|
||||
}
|
||||
|
||||
func (r *ConnectResponse) toPogs() (*capnp.Message, error) {
|
||||
func (r *ConnectResponse) ToPogs() (*capnp.Message, error) {
|
||||
msg, seg, err := capnp.NewMessage(capnp.SingleSegment(nil))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
root, err := schema.NewRootConnectResponse(seg)
|
||||
root, err := proto.NewRootConnectResponse(seg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := pogs.Insert(schema.ConnectResponse_TypeID, root.Struct, r); err != nil {
|
||||
if err := pogs.Insert(proto.ConnectResponse_TypeID, root.Struct, r); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
@ -5,10 +5,10 @@ import (
|
|||
|
||||
"zombiezen.com/go/capnproto2/server"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
func (i TunnelServer_PogsImpl) ReconnectTunnel(p tunnelrpc.TunnelServer_reconnectTunnel) error {
|
||||
func (i TunnelServer_PogsImpl) ReconnectTunnel(p proto.TunnelServer_reconnectTunnel) error {
|
||||
jwt, err := p.Params.Jwt()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -53,8 +53,8 @@ func (c TunnelServer_PogsClient) ReconnectTunnel(
|
|||
hostname string,
|
||||
options *RegistrationOptions,
|
||||
) *TunnelRegistration {
|
||||
client := tunnelrpc.TunnelServer{Client: c.Client}
|
||||
promise := client.ReconnectTunnel(ctx, func(p tunnelrpc.TunnelServer_reconnectTunnel_Params) error {
|
||||
client := proto.TunnelServer{Client: c.Client}
|
||||
promise := client.ReconnectTunnel(ctx, func(p proto.TunnelServer_reconnectTunnel_Params) error {
|
||||
err := p.SetJwt(jwt)
|
||||
if err != nil {
|
||||
return err
|
||||
|
|
|
@ -6,11 +6,12 @@ import (
|
|||
"net"
|
||||
"time"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
"github.com/google/uuid"
|
||||
capnp "zombiezen.com/go/capnproto2"
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
"zombiezen.com/go/capnproto2/server"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
type SessionManager interface {
|
||||
|
@ -26,11 +27,11 @@ type SessionManager_PogsImpl struct {
|
|||
impl SessionManager
|
||||
}
|
||||
|
||||
func SessionManager_ServerToClient(s SessionManager) tunnelrpc.SessionManager {
|
||||
return tunnelrpc.SessionManager_ServerToClient(SessionManager_PogsImpl{s})
|
||||
func SessionManager_ServerToClient(s SessionManager) proto.SessionManager {
|
||||
return proto.SessionManager_ServerToClient(SessionManager_PogsImpl{s})
|
||||
}
|
||||
|
||||
func (i SessionManager_PogsImpl) RegisterUdpSession(p tunnelrpc.SessionManager_registerUdpSession) error {
|
||||
func (i SessionManager_PogsImpl) RegisterUdpSession(p proto.SessionManager_registerUdpSession) error {
|
||||
server.Ack(p.Options)
|
||||
|
||||
sessionIDRaw, err := p.Params.SessionId()
|
||||
|
@ -76,7 +77,7 @@ func (i SessionManager_PogsImpl) RegisterUdpSession(p tunnelrpc.SessionManager_r
|
|||
return resp.Marshal(result)
|
||||
}
|
||||
|
||||
func (i SessionManager_PogsImpl) UnregisterUdpSession(p tunnelrpc.SessionManager_unregisterUdpSession) error {
|
||||
func (i SessionManager_PogsImpl) UnregisterUdpSession(p proto.SessionManager_unregisterUdpSession) error {
|
||||
server.Ack(p.Options)
|
||||
|
||||
sessionIDRaw, err := p.Params.SessionId()
|
||||
|
@ -101,7 +102,7 @@ type RegisterUdpSessionResponse struct {
|
|||
Spans []byte // Spans in protobuf format
|
||||
}
|
||||
|
||||
func (p *RegisterUdpSessionResponse) Marshal(s tunnelrpc.RegisterUdpSessionResponse) error {
|
||||
func (p *RegisterUdpSessionResponse) Marshal(s proto.RegisterUdpSessionResponse) error {
|
||||
if p.Err != nil {
|
||||
return s.SetErr(p.Err.Error())
|
||||
}
|
||||
|
@ -111,7 +112,7 @@ func (p *RegisterUdpSessionResponse) Marshal(s tunnelrpc.RegisterUdpSessionRespo
|
|||
return nil
|
||||
}
|
||||
|
||||
func (p *RegisterUdpSessionResponse) Unmarshal(s tunnelrpc.RegisterUdpSessionResponse) error {
|
||||
func (p *RegisterUdpSessionResponse) Unmarshal(s proto.RegisterUdpSessionResponse) error {
|
||||
respErr, err := s.Err()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -131,14 +132,21 @@ type SessionManager_PogsClient struct {
|
|||
Conn *rpc.Conn
|
||||
}
|
||||
|
||||
func NewSessionManager_PogsClient(client capnp.Client, conn *rpc.Conn) SessionManager_PogsClient {
|
||||
return SessionManager_PogsClient{
|
||||
Client: client,
|
||||
Conn: conn,
|
||||
}
|
||||
}
|
||||
|
||||
func (c SessionManager_PogsClient) Close() error {
|
||||
c.Client.Close()
|
||||
return c.Conn.Close()
|
||||
}
|
||||
|
||||
func (c SessionManager_PogsClient) RegisterUdpSession(ctx context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeAfterIdleHint time.Duration, traceContext string) (*RegisterUdpSessionResponse, error) {
|
||||
client := tunnelrpc.SessionManager{Client: c.Client}
|
||||
promise := client.RegisterUdpSession(ctx, func(p tunnelrpc.SessionManager_registerUdpSession_Params) error {
|
||||
client := proto.SessionManager{Client: c.Client}
|
||||
promise := client.RegisterUdpSession(ctx, func(p proto.SessionManager_registerUdpSession_Params) error {
|
||||
if err := p.SetSessionId(sessionID[:]); err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -164,8 +172,8 @@ func (c SessionManager_PogsClient) RegisterUdpSession(ctx context.Context, sessi
|
|||
}
|
||||
|
||||
func (c SessionManager_PogsClient) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID, message string) error {
|
||||
client := tunnelrpc.SessionManager{Client: c.Client}
|
||||
promise := client.UnregisterUdpSession(ctx, func(p tunnelrpc.SessionManager_unregisterUdpSession_Params) error {
|
||||
client := proto.SessionManager{Client: c.Client}
|
||||
promise := client.UnregisterUdpSession(ctx, func(p proto.SessionManager_unregisterUdpSession_Params) error {
|
||||
if err := p.SetSessionId(sessionID[:]); err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -9,7 +9,7 @@ import (
|
|||
"zombiezen.com/go/capnproto2/rpc"
|
||||
"zombiezen.com/go/capnproto2/server"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -22,13 +22,13 @@ type Authentication struct {
|
|||
OriginCAKey string
|
||||
}
|
||||
|
||||
func MarshalAuthentication(s tunnelrpc.Authentication, p *Authentication) error {
|
||||
return pogs.Insert(tunnelrpc.Authentication_TypeID, s.Struct, p)
|
||||
func MarshalAuthentication(s proto.Authentication, p *Authentication) error {
|
||||
return pogs.Insert(proto.Authentication_TypeID, s.Struct, p)
|
||||
}
|
||||
|
||||
func UnmarshalAuthentication(s tunnelrpc.Authentication) (*Authentication, error) {
|
||||
func UnmarshalAuthentication(s proto.Authentication) (*Authentication, error) {
|
||||
p := new(Authentication)
|
||||
err := pogs.Extract(p, tunnelrpc.Authentication_TypeID, s.Struct)
|
||||
err := pogs.Extract(p, proto.Authentication_TypeID, s.Struct)
|
||||
return p, err
|
||||
}
|
||||
|
||||
|
@ -144,13 +144,13 @@ func (*RetryableRegistrationError) IsPermanent() bool {
|
|||
return false
|
||||
}
|
||||
|
||||
func MarshalTunnelRegistration(s tunnelrpc.TunnelRegistration, p *TunnelRegistration) error {
|
||||
return pogs.Insert(tunnelrpc.TunnelRegistration_TypeID, s.Struct, p)
|
||||
func MarshalTunnelRegistration(s proto.TunnelRegistration, p *TunnelRegistration) error {
|
||||
return pogs.Insert(proto.TunnelRegistration_TypeID, s.Struct, p)
|
||||
}
|
||||
|
||||
func UnmarshalTunnelRegistration(s tunnelrpc.TunnelRegistration) (*TunnelRegistration, error) {
|
||||
func UnmarshalTunnelRegistration(s proto.TunnelRegistration) (*TunnelRegistration, error) {
|
||||
p := new(TunnelRegistration)
|
||||
err := pogs.Extract(p, tunnelrpc.TunnelRegistration_TypeID, s.Struct)
|
||||
err := pogs.Extract(p, proto.TunnelRegistration_TypeID, s.Struct)
|
||||
return p, err
|
||||
}
|
||||
|
||||
|
@ -158,7 +158,7 @@ type RegistrationOptions struct {
|
|||
ClientID string `capnp:"clientId"`
|
||||
Version string
|
||||
OS string `capnp:"os"`
|
||||
ExistingTunnelPolicy tunnelrpc.ExistingTunnelPolicy
|
||||
ExistingTunnelPolicy proto.ExistingTunnelPolicy
|
||||
PoolName string `capnp:"poolName"`
|
||||
Tags []Tag
|
||||
ConnectionID uint8 `capnp:"connectionId"`
|
||||
|
@ -171,13 +171,13 @@ type RegistrationOptions struct {
|
|||
Features []string
|
||||
}
|
||||
|
||||
func MarshalRegistrationOptions(s tunnelrpc.RegistrationOptions, p *RegistrationOptions) error {
|
||||
return pogs.Insert(tunnelrpc.RegistrationOptions_TypeID, s.Struct, p)
|
||||
func MarshalRegistrationOptions(s proto.RegistrationOptions, p *RegistrationOptions) error {
|
||||
return pogs.Insert(proto.RegistrationOptions_TypeID, s.Struct, p)
|
||||
}
|
||||
|
||||
func UnmarshalRegistrationOptions(s tunnelrpc.RegistrationOptions) (*RegistrationOptions, error) {
|
||||
func UnmarshalRegistrationOptions(s proto.RegistrationOptions) (*RegistrationOptions, error) {
|
||||
p := new(RegistrationOptions)
|
||||
err := pogs.Extract(p, tunnelrpc.RegistrationOptions_TypeID, s.Struct)
|
||||
err := pogs.Extract(p, proto.RegistrationOptions_TypeID, s.Struct)
|
||||
return p, err
|
||||
}
|
||||
|
||||
|
@ -190,13 +190,13 @@ type ServerInfo struct {
|
|||
LocationName string
|
||||
}
|
||||
|
||||
func MarshalServerInfo(s tunnelrpc.ServerInfo, p *ServerInfo) error {
|
||||
return pogs.Insert(tunnelrpc.ServerInfo_TypeID, s.Struct, p)
|
||||
func MarshalServerInfo(s proto.ServerInfo, p *ServerInfo) error {
|
||||
return pogs.Insert(proto.ServerInfo_TypeID, s.Struct, p)
|
||||
}
|
||||
|
||||
func UnmarshalServerInfo(s tunnelrpc.ServerInfo) (*ServerInfo, error) {
|
||||
func UnmarshalServerInfo(s proto.ServerInfo) (*ServerInfo, error) {
|
||||
p := new(ServerInfo)
|
||||
err := pogs.Extract(p, tunnelrpc.ServerInfo_TypeID, s.Struct)
|
||||
err := pogs.Extract(p, proto.ServerInfo_TypeID, s.Struct)
|
||||
return p, err
|
||||
}
|
||||
|
||||
|
@ -209,8 +209,8 @@ type TunnelServer interface {
|
|||
ReconnectTunnel(ctx context.Context, jwt, eventDigest, connDigest []byte, hostname string, options *RegistrationOptions) (*TunnelRegistration, error)
|
||||
}
|
||||
|
||||
func TunnelServer_ServerToClient(s TunnelServer) tunnelrpc.TunnelServer {
|
||||
return tunnelrpc.TunnelServer_ServerToClient(TunnelServer_PogsImpl{RegistrationServer_PogsImpl{s}, s})
|
||||
func TunnelServer_ServerToClient(s TunnelServer) proto.TunnelServer {
|
||||
return proto.TunnelServer_ServerToClient(TunnelServer_PogsImpl{RegistrationServer_PogsImpl{s}, s})
|
||||
}
|
||||
|
||||
type TunnelServer_PogsImpl struct {
|
||||
|
@ -218,7 +218,7 @@ type TunnelServer_PogsImpl struct {
|
|||
impl TunnelServer
|
||||
}
|
||||
|
||||
func (i TunnelServer_PogsImpl) RegisterTunnel(p tunnelrpc.TunnelServer_registerTunnel) error {
|
||||
func (i TunnelServer_PogsImpl) RegisterTunnel(p proto.TunnelServer_registerTunnel) error {
|
||||
originCert, err := p.Params.OriginCert()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -245,7 +245,7 @@ func (i TunnelServer_PogsImpl) RegisterTunnel(p tunnelrpc.TunnelServer_registerT
|
|||
return MarshalTunnelRegistration(result, registration)
|
||||
}
|
||||
|
||||
func (i TunnelServer_PogsImpl) GetServerInfo(p tunnelrpc.TunnelServer_getServerInfo) error {
|
||||
func (i TunnelServer_PogsImpl) GetServerInfo(p proto.TunnelServer_getServerInfo) error {
|
||||
server.Ack(p.Options)
|
||||
serverInfo, err := i.impl.GetServerInfo(p.Ctx)
|
||||
if err != nil {
|
||||
|
@ -258,13 +258,13 @@ func (i TunnelServer_PogsImpl) GetServerInfo(p tunnelrpc.TunnelServer_getServerI
|
|||
return MarshalServerInfo(result, serverInfo)
|
||||
}
|
||||
|
||||
func (i TunnelServer_PogsImpl) UnregisterTunnel(p tunnelrpc.TunnelServer_unregisterTunnel) error {
|
||||
func (i TunnelServer_PogsImpl) UnregisterTunnel(p proto.TunnelServer_unregisterTunnel) error {
|
||||
gracePeriodNanoSec := p.Params.GracePeriodNanoSec()
|
||||
server.Ack(p.Options)
|
||||
return i.impl.UnregisterTunnel(p.Ctx, gracePeriodNanoSec)
|
||||
}
|
||||
|
||||
func (i TunnelServer_PogsImpl) ObsoleteDeclarativeTunnelConnect(p tunnelrpc.TunnelServer_obsoleteDeclarativeTunnelConnect) error {
|
||||
func (i TunnelServer_PogsImpl) ObsoleteDeclarativeTunnelConnect(p proto.TunnelServer_obsoleteDeclarativeTunnelConnect) error {
|
||||
return fmt.Errorf("RPC to create declarative tunnel connection has been deprecated")
|
||||
}
|
||||
|
||||
|
@ -280,8 +280,8 @@ func (c TunnelServer_PogsClient) Close() error {
|
|||
}
|
||||
|
||||
func (c TunnelServer_PogsClient) RegisterTunnel(ctx context.Context, originCert []byte, hostname string, options *RegistrationOptions) *TunnelRegistration {
|
||||
client := tunnelrpc.TunnelServer{Client: c.Client}
|
||||
promise := client.RegisterTunnel(ctx, func(p tunnelrpc.TunnelServer_registerTunnel_Params) error {
|
||||
client := proto.TunnelServer{Client: c.Client}
|
||||
promise := client.RegisterTunnel(ctx, func(p proto.TunnelServer_registerTunnel_Params) error {
|
||||
err := p.SetOriginCert(originCert)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -312,8 +312,8 @@ func (c TunnelServer_PogsClient) RegisterTunnel(ctx context.Context, originCert
|
|||
}
|
||||
|
||||
func (c TunnelServer_PogsClient) GetServerInfo(ctx context.Context) (*ServerInfo, error) {
|
||||
client := tunnelrpc.TunnelServer{Client: c.Client}
|
||||
promise := client.GetServerInfo(ctx, func(p tunnelrpc.TunnelServer_getServerInfo_Params) error {
|
||||
client := proto.TunnelServer{Client: c.Client}
|
||||
promise := client.GetServerInfo(ctx, func(p proto.TunnelServer_getServerInfo_Params) error {
|
||||
return nil
|
||||
})
|
||||
retval, err := promise.Result().Struct()
|
||||
|
@ -324,8 +324,8 @@ func (c TunnelServer_PogsClient) GetServerInfo(ctx context.Context) (*ServerInfo
|
|||
}
|
||||
|
||||
func (c TunnelServer_PogsClient) UnregisterTunnel(ctx context.Context, gracePeriodNanoSec int64) error {
|
||||
client := tunnelrpc.TunnelServer{Client: c.Client}
|
||||
promise := client.UnregisterTunnel(ctx, func(p tunnelrpc.TunnelServer_unregisterTunnel_Params) error {
|
||||
client := proto.TunnelServer{Client: c.Client}
|
||||
promise := client.UnregisterTunnel(ctx, func(p proto.TunnelServer_unregisterTunnel_Params) error {
|
||||
p.SetGracePeriodNanoSec(gracePeriodNanoSec)
|
||||
return nil
|
||||
})
|
||||
|
|
|
@ -7,7 +7,7 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
capnp "zombiezen.com/go/capnproto2"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/proto"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -39,7 +39,7 @@ func TestTunnelRegistration(t *testing.T) {
|
|||
for i, testCase := range testCases {
|
||||
_, seg, err := capnp.NewMessage(capnp.SingleSegment(nil))
|
||||
assert.NoError(t, err)
|
||||
capnpEntity, err := tunnelrpc.NewTunnelRegistration(seg)
|
||||
capnpEntity, err := proto.NewTunnelRegistration(seg)
|
||||
if !assert.NoError(t, err) {
|
||||
t.Fatal("Couldn't initialize a new message")
|
||||
}
|
||||
|
|
|
@ -0,0 +1,31 @@
|
|||
# Generate go.capnp.out with:
|
||||
# capnp compile -o- go.capnp > go.capnp.out
|
||||
# Must run inside this directory to preserve paths.
|
||||
|
||||
@0xd12a1c51fedd6c88;
|
||||
|
||||
annotation package(file) :Text;
|
||||
# The Go package name for the generated file.
|
||||
|
||||
annotation import(file) :Text;
|
||||
# The Go import path that the generated file is accessible from.
|
||||
# Used to generate import statements and check if two types are in the
|
||||
# same package.
|
||||
|
||||
annotation doc(struct, field, enum) :Text;
|
||||
# Adds a doc comment to the generated code.
|
||||
|
||||
annotation tag(enumerant) :Text;
|
||||
# Changes the string representation of the enum in the generated code.
|
||||
|
||||
annotation notag(enumerant) :Void;
|
||||
# Removes the string representation of the enum in the generated code.
|
||||
|
||||
annotation customtype(field) :Text;
|
||||
# OBSOLETE, not used by code generator.
|
||||
|
||||
annotation name(struct, field, union, enum, enumerant, interface, method, param, annotation, const, group) :Text;
|
||||
# Used to rename the element in the generated code.
|
||||
|
||||
$package("capnp");
|
||||
$import("zombiezen.com/go/capnproto2");
|
|
@ -0,0 +1,28 @@
|
|||
using Go = import "go.capnp";
|
||||
@0xb29021ef7421cc32;
|
||||
|
||||
$Go.package("proto");
|
||||
$Go.import("github.com/cloudflare/cloudflared/tunnelrpc");
|
||||
|
||||
|
||||
struct ConnectRequest @0xc47116a1045e4061 {
|
||||
dest @0 :Text;
|
||||
type @1 :ConnectionType;
|
||||
metadata @2 :List(Metadata);
|
||||
}
|
||||
|
||||
enum ConnectionType @0xc52e1bac26d379c8 {
|
||||
http @0;
|
||||
websocket @1;
|
||||
tcp @2;
|
||||
}
|
||||
|
||||
struct Metadata @0xe1446b97bfd1cd37 {
|
||||
key @0 :Text;
|
||||
val @1 :Text;
|
||||
}
|
||||
|
||||
struct ConnectResponse @0xb1032ec91cef8727 {
|
||||
error @0 :Text;
|
||||
metadata @1 :List(Metadata);
|
||||
}
|
|
@ -1,6 +1,6 @@
|
|||
// Code generated by capnpc-go. DO NOT EDIT.
|
||||
|
||||
package schema
|
||||
package proto
|
||||
|
||||
import (
|
||||
capnp "zombiezen.com/go/capnproto2"
|
||||
|
@ -357,34 +357,34 @@ func (p ConnectResponse_Promise) Struct() (ConnectResponse, error) {
|
|||
return ConnectResponse{s}, err
|
||||
}
|
||||
|
||||
const schema_b29021ef7421cc32 = "x\xda\xb4\x91\xcfk\x13A\x1c\xc5\xdf\x9bI\xba\x1e\xa2" +
|
||||
"\x9b!\xd5\x8b\x8a\xa4\xf8+ES\xdb(\xa2\xa7\x80\x15" +
|
||||
"TZ\xcc\x14\xcf\x96u;\x98\x92vw\x92\x9dZ\xf2" +
|
||||
"\x17x\x15/\xe2\xd1\xbb \x15<\x0b\xa2\xa0\xa2\x07\x11" +
|
||||
"\xff\x80\xfe\x05=y\xf0\xb42)\xdb@)\x08Bo" +
|
||||
"\xdfy<\xe6}\xbe\xdfW\xfd\xd5\x16\xb3\xe5\xc7\x04t" +
|
||||
"\xb5<\x91_x\xbas\xeaKSnA5\x98\xcf}" +
|
||||
"\xab\xbb\x9d\xfa\xb3\xb7(\x8b\x00\x98}\xf9\x95\xea]\x00" +
|
||||
"\xa8\xadM0\x8f\xda\x0fK\xafN\xf4?B7\xb8\xdf" +
|
||||
"\xda\xaa\xf3\x03k7\x18\x00\xb5k|\x03\xe6\x9f\x87?" +
|
||||
"\xcf\xbf>\xd9\xfc\x04\xd5\x10c3\xd8\xda\xf6\xce?#" +
|
||||
"\xe7o\xde\x07\xf3\xeb\xdf\x7f\xbc\x7f\xd1\x9b\xdf>\x80\xa0" +
|
||||
"uT<g\xed\x9c\x1fku\xe1!\xfa\x1b\xab\xf1L" +
|
||||
"\x16w'\xccz4\xe3\x1f\xcb\xeb\xc6E+\x91\x8b\x96" +
|
||||
"\xed ui\x9c\xae5\xe3\xc8&\xf6\xe6\xad4IL" +
|
||||
"\xec\x96Lf\xd3$3@\x87\xd4Gd\x09(\x11P" +
|
||||
"\x8d9@\x9f\x95\xd4W\x04\x159I/^\xbe\x07\xe8" +
|
||||
"K\x92\xfa\x8e\xe0\x193\x18\xa4\x03V X\x01\xf3\"" +
|
||||
"\x06\x00\x8f\x81\x1dIV\xc7\xe8\xa0\x17\xff\x87\xae\xbfa" +
|
||||
"27b\xab\xec\xb1\xdd\x9e\x06t[R/\x08\x16h" +
|
||||
"w\xbd6/\xa9;\x82Jp\x92\x02P\x8b\x9ewA" +
|
||||
"Rw\x05\xc3\x15\x93\xb9\x027tCk\x18\x8e[\x00" +
|
||||
"\x19\x1e\xda\x16\xabi\xf2`h\xcd\xee\x16#\xb0\xd3\xd3" +
|
||||
"\xfe3u|\x09\xa0Pj\x0a\x08\xbb\xce\xd9|\xd3<" +
|
||||
"\xca\xd2\xb8g@\x17\xb8\xd8\xeeE\x95\xff\x19\xb5\xb8\xab" +
|
||||
"3\xdaW\xe3\xd4A5z\xf1\xa2\xa4\xbe*\x18\xf4\xcc" +
|
||||
"\xb0\xb8J\xf0$Z+\xe6\xbf\x01\x00\x00\xff\xff\xf5\xed" +
|
||||
"\xc9\xfe"
|
||||
const schema_b29021ef7421cc32 = "x\xda\xb4\x911k\x14A\x1c\xc5\xdf\x9b\xcde-\x0e" +
|
||||
"\xf7\x86KlT\xc2\x05Q\x13\xdc\x8b\xc9\x09\xa2 \x1c" +
|
||||
"\x18A%\xc1\x9b`mX7\x83\x09w\xee\xce\xed\xce" +
|
||||
"\x19\xee\x13\xd8\xda\x89\xa5\xbd \x09X\xdb((h!" +
|
||||
"\x16\xd6\x0a66\xf9\x04\xb22\x0b\x9b\x83\x90B\x04\xbb" +
|
||||
"\xe1\xcd\x9by\xbf\xff\xff5\xbeu\xc5r\xed\x11\x01\xd5" +
|
||||
"\xa8M\x17\x17\x9e\x1e\x9c\xf9\xd8\xf6\xf6 C\x16+\x9f" +
|
||||
"Z\xf6\xa0\xf5l\x1f5\xe1\x03\xcb/~Q\xbe\xf1\x01" +
|
||||
"\xb9\xb7\x0b\x16Q\xf7\xc1\xd4\xcbS\xc3wP!\x8fZ" +
|
||||
";-\xfe`\xf3\x06}\xa0y\x8d\xaf\xc1\xe2\xc3\xf8\xeb" +
|
||||
"\xf9W\xa7\xdb\xef!C11\x83\x9d\x9f\xceI\xf7\xa8" +
|
||||
"\xf9\x9b\xf7\xc0\xe2\xea\xe7/o\x9f\xf7W\xbf\x1fC\xd0" +
|
||||
"\x99\x15\xfbl\x86\xa5yA8\x08;J\x12=\xc8\xcc" +
|
||||
"t\xbcd\xb2\xd4\xa6K\xc3\xd1N\xbc\xf9X\xdbh+" +
|
||||
"\xb2\xd1f\xa9\xc5\xe9\xa0\x1dG&1\xd7o\xa6I\xa2" +
|
||||
"c\xbb\xa1s\x13\xa4I\xae{\xa4:\xe1M\x01S\x04" +
|
||||
"\xe4\xc2\x0a\xa0\xceyT\x97\x05%9C'\x86w\x01" +
|
||||
"u\xc9\xa3\xba-8\xa7\xb3,\xcdX\x87`\x1d,\xaa" +
|
||||
"\x14\x00<\x09\xf6<\xb21\xa1\x07\x9d\xf8\xaf\x80\xc3\x91" +
|
||||
"\xafs\xeb\xf8\xea\x87|\xb7\x16\x01\xd5\xf5\xa8\xd6\x04+" +
|
||||
"\xbc;N[\xf5\xa8z\x82Rp\x86\x02\x90\xeb\x8ey" +
|
||||
"\xcd\xa3\xda\x16\x0c\xb6tn+\xe4\xc0\x8e\x8df0)" +
|
||||
"\x03d\xf0_'\xd9I\x93\xfb\xfe\xd8\x94\x9b\xae\x97p" +
|
||||
"g\x17\xdd\x87rv\x03\xa0\x90r\x1e\x08\xb6\xad5\xc5" +
|
||||
"\xae~\x98\xa7q_\x83\xd6\xb7\xb19\x8c\xab\xfdU\xdc" +
|
||||
"\xba\xb6s\xe5\xc5\x91J\xe7\x8f\xab\xd4\x89\x17=\xaa+" +
|
||||
"\x82~_\x8f\xab\xed\xf8O\xa2Au\xfe\x13\x00\x00\xff" +
|
||||
"\xff\x1d\xce\xd1\xb0"
|
||||
|
||||
func init() {
|
||||
schemas.Register(schema_b29021ef7421cc32,
|
|
@ -1,15 +1,15 @@
|
|||
using Go = import "go.capnp";
|
||||
@0xdb8274f9144abc7e;
|
||||
$Go.package("tunnelrpc");
|
||||
$Go.package("proto");
|
||||
$Go.import("github.com/cloudflare/cloudflared/tunnelrpc");
|
||||
|
||||
struct Authentication {
|
||||
struct Authentication @0xc082ef6e0d42ed1d {
|
||||
key @0 :Text;
|
||||
email @1 :Text;
|
||||
originCAKey @2 :Text;
|
||||
}
|
||||
|
||||
struct TunnelRegistration {
|
||||
struct TunnelRegistration @0xf41a0f001ad49e46 {
|
||||
err @0 :Text;
|
||||
# the url to access the tunnel
|
||||
url @1 :Text;
|
||||
|
@ -27,7 +27,7 @@ struct TunnelRegistration {
|
|||
connDigest @7 :Data;
|
||||
}
|
||||
|
||||
struct RegistrationOptions {
|
||||
struct RegistrationOptions @0xc793e50592935b4a {
|
||||
# The tunnel client's unique identifier, used to verify a reconnection.
|
||||
clientId @0 :Text;
|
||||
# Information about the running binary.
|
||||
|
@ -56,29 +56,29 @@ struct RegistrationOptions {
|
|||
features @13 :List(Text);
|
||||
}
|
||||
|
||||
struct Tag {
|
||||
struct Tag @0xcbd96442ae3bb01a {
|
||||
name @0 :Text;
|
||||
value @1 :Text;
|
||||
}
|
||||
|
||||
enum ExistingTunnelPolicy {
|
||||
enum ExistingTunnelPolicy @0x84cb9536a2cf6d3c {
|
||||
ignore @0;
|
||||
disconnect @1;
|
||||
balance @2;
|
||||
}
|
||||
|
||||
struct ServerInfo {
|
||||
struct ServerInfo @0xf2c68e2547ec3866 {
|
||||
locationName @0 :Text;
|
||||
}
|
||||
|
||||
struct AuthenticateResponse {
|
||||
struct AuthenticateResponse @0x82c325a07ad22a65 {
|
||||
permanentErr @0 :Text;
|
||||
retryableErr @1 :Text;
|
||||
jwt @2 :Data;
|
||||
hoursUntilRefresh @3 :UInt8;
|
||||
}
|
||||
|
||||
struct ClientInfo {
|
||||
struct ClientInfo @0x83ced0145b2f114b {
|
||||
# The tunnel client's unique identifier, used to verify a reconnection.
|
||||
clientId @0 :Data;
|
||||
# Set of features this cloudflared knows it supports
|
||||
|
@ -89,7 +89,7 @@ struct ClientInfo {
|
|||
arch @3 :Text;
|
||||
}
|
||||
|
||||
struct ConnectionOptions {
|
||||
struct ConnectionOptions @0xb4bf9861fe035d04 {
|
||||
# client details
|
||||
client @0 :ClientInfo;
|
||||
# origin LAN IP
|
||||
|
@ -102,21 +102,21 @@ struct ConnectionOptions {
|
|||
numPreviousAttempts @4 :UInt8;
|
||||
}
|
||||
|
||||
struct ConnectionResponse {
|
||||
struct ConnectionResponse @0xdbaa9d03d52b62dc {
|
||||
result :union {
|
||||
error @0 :ConnectionError;
|
||||
connectionDetails @1 :ConnectionDetails;
|
||||
}
|
||||
}
|
||||
|
||||
struct ConnectionError {
|
||||
struct ConnectionError @0xf5f383d2785edb86 {
|
||||
cause @0 :Text;
|
||||
# How long should this connection wait to retry in ns
|
||||
retryAfter @1 :Int64;
|
||||
shouldRetry @2 :Bool;
|
||||
}
|
||||
|
||||
struct ConnectionDetails {
|
||||
struct ConnectionDetails @0xb5f39f082b9ac18a {
|
||||
# identifier of this connection
|
||||
uuid @0 :Data;
|
||||
# airport code of the colo where this connection landed
|
||||
|
@ -125,18 +125,18 @@ struct ConnectionDetails {
|
|||
tunnelIsRemotelyManaged @2: Bool;
|
||||
}
|
||||
|
||||
struct TunnelAuth {
|
||||
struct TunnelAuth @0x9496331ab9cd463f {
|
||||
accountTag @0 :Text;
|
||||
tunnelSecret @1 :Data;
|
||||
}
|
||||
|
||||
interface RegistrationServer {
|
||||
interface RegistrationServer @0xf71695ec7fe85497 {
|
||||
registerConnection @0 (auth :TunnelAuth, tunnelId :Data, connIndex :UInt8, options :ConnectionOptions) -> (result :ConnectionResponse);
|
||||
unregisterConnection @1 () -> ();
|
||||
updateLocalConfiguration @2 (config :Data) -> ();
|
||||
}
|
||||
|
||||
interface TunnelServer extends (RegistrationServer) {
|
||||
interface TunnelServer @0xea58385c65416035 extends (RegistrationServer) {
|
||||
registerTunnel @0 (originCert :Data, hostname :Text, options :RegistrationOptions) -> (result :TunnelRegistration);
|
||||
getServerInfo @1 () -> (result :ServerInfo);
|
||||
unregisterTunnel @2 (gracePeriodNanoSec :Int64) -> ();
|
||||
|
@ -146,18 +146,18 @@ interface TunnelServer extends (RegistrationServer) {
|
|||
reconnectTunnel @5 (jwt :Data, eventDigest :Data, connDigest :Data, hostname :Text, options :RegistrationOptions) -> (result :TunnelRegistration);
|
||||
}
|
||||
|
||||
struct RegisterUdpSessionResponse {
|
||||
struct RegisterUdpSessionResponse @0xab6d5210c1f26687 {
|
||||
err @0 :Text;
|
||||
spans @1 :Data;
|
||||
}
|
||||
|
||||
interface SessionManager {
|
||||
interface SessionManager @0x839445a59fb01686 {
|
||||
# Let the edge decide closeAfterIdle to make sure cloudflared doesn't close session before the edge closes its side
|
||||
registerUdpSession @0 (sessionId :Data, dstIp :Data, dstPort :UInt16, closeAfterIdleHint :Int64, traceContext :Text = "") -> (result :RegisterUdpSessionResponse);
|
||||
unregisterUdpSession @1 (sessionId :Data, message :Text) -> ();
|
||||
}
|
||||
|
||||
struct UpdateConfigurationResponse {
|
||||
struct UpdateConfigurationResponse @0xdb58ff694ba05cf9 {
|
||||
# Latest configuration that was applied successfully. The err field might be populated at the same time to indicate
|
||||
# that cloudflared is using an older configuration because the latest cannot be applied
|
||||
latestAppliedVersion @0 :Int32;
|
||||
|
@ -166,8 +166,8 @@ struct UpdateConfigurationResponse {
|
|||
}
|
||||
|
||||
# ConfigurationManager defines RPC to manage cloudflared configuration remotely
|
||||
interface ConfigurationManager {
|
||||
interface ConfigurationManager @0xb48edfbdaa25db04 {
|
||||
updateConfiguration @0 (version :Int32, config :Data) -> (result: UpdateConfigurationResponse);
|
||||
}
|
||||
|
||||
interface CloudflaredServer extends(SessionManager, ConfigurationManager) {}
|
||||
interface CloudflaredServer @0xf548cef9dea2a4a1 extends(SessionManager, ConfigurationManager) {}
|
|
@ -1,6 +1,6 @@
|
|||
// Code generated by capnpc-go. DO NOT EDIT.
|
||||
|
||||
package tunnelrpc
|
||||
package proto
|
||||
|
||||
import (
|
||||
context "golang.org/x/net/context"
|
||||
|
@ -1598,7 +1598,7 @@ func (c RegistrationServer) RegisterConnection(ctx context.Context, params func(
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "registerConnection",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -1618,7 +1618,7 @@ func (c RegistrationServer) UnregisterConnection(ctx context.Context, params fun
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "unregisterConnection",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -1638,7 +1638,7 @@ func (c RegistrationServer) UpdateLocalConfiguration(ctx context.Context, params
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 2,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "updateLocalConfiguration",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -1674,7 +1674,7 @@ func RegistrationServer_Methods(methods []server.Method, s RegistrationServer_Se
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "registerConnection",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -1688,7 +1688,7 @@ func RegistrationServer_Methods(methods []server.Method, s RegistrationServer_Se
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "unregisterConnection",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -1702,7 +1702,7 @@ func RegistrationServer_Methods(methods []server.Method, s RegistrationServer_Se
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 2,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "updateLocalConfiguration",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -2206,7 +2206,7 @@ func (c TunnelServer) RegisterTunnel(ctx context.Context, params func(TunnelServ
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "registerTunnel",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -2226,7 +2226,7 @@ func (c TunnelServer) GetServerInfo(ctx context.Context, params func(TunnelServe
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "getServerInfo",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -2246,7 +2246,7 @@ func (c TunnelServer) UnregisterTunnel(ctx context.Context, params func(TunnelSe
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 2,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "unregisterTunnel",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -2266,7 +2266,7 @@ func (c TunnelServer) ObsoleteDeclarativeTunnelConnect(ctx context.Context, para
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 3,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "obsoleteDeclarativeTunnelConnect",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -2288,7 +2288,7 @@ func (c TunnelServer) Authenticate(ctx context.Context, params func(TunnelServer
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 4,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "authenticate",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -2308,7 +2308,7 @@ func (c TunnelServer) ReconnectTunnel(ctx context.Context, params func(TunnelSer
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 5,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "reconnectTunnel",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -2328,7 +2328,7 @@ func (c TunnelServer) RegisterConnection(ctx context.Context, params func(Regist
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "registerConnection",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -2348,7 +2348,7 @@ func (c TunnelServer) UnregisterConnection(ctx context.Context, params func(Regi
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "unregisterConnection",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -2368,7 +2368,7 @@ func (c TunnelServer) UpdateLocalConfiguration(ctx context.Context, params func(
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 2,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "updateLocalConfiguration",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -2416,7 +2416,7 @@ func TunnelServer_Methods(methods []server.Method, s TunnelServer_Server) []serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "registerTunnel",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -2430,7 +2430,7 @@ func TunnelServer_Methods(methods []server.Method, s TunnelServer_Server) []serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "getServerInfo",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -2444,7 +2444,7 @@ func TunnelServer_Methods(methods []server.Method, s TunnelServer_Server) []serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 2,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "unregisterTunnel",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -2458,7 +2458,7 @@ func TunnelServer_Methods(methods []server.Method, s TunnelServer_Server) []serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 3,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "obsoleteDeclarativeTunnelConnect",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -2472,7 +2472,7 @@ func TunnelServer_Methods(methods []server.Method, s TunnelServer_Server) []serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 4,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "authenticate",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -2486,7 +2486,7 @@ func TunnelServer_Methods(methods []server.Method, s TunnelServer_Server) []serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xea58385c65416035,
|
||||
MethodID: 5,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:TunnelServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:TunnelServer",
|
||||
MethodName: "reconnectTunnel",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -2500,7 +2500,7 @@ func TunnelServer_Methods(methods []server.Method, s TunnelServer_Server) []serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "registerConnection",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -2514,7 +2514,7 @@ func TunnelServer_Methods(methods []server.Method, s TunnelServer_Server) []serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "unregisterConnection",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -2528,7 +2528,7 @@ func TunnelServer_Methods(methods []server.Method, s TunnelServer_Server) []serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xf71695ec7fe85497,
|
||||
MethodID: 2,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:RegistrationServer",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:RegistrationServer",
|
||||
MethodName: "updateLocalConfiguration",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -3689,7 +3689,7 @@ func (c SessionManager) RegisterUdpSession(ctx context.Context, params func(Sess
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0x839445a59fb01686,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:SessionManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:SessionManager",
|
||||
MethodName: "registerUdpSession",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -3709,7 +3709,7 @@ func (c SessionManager) UnregisterUdpSession(ctx context.Context, params func(Se
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0x839445a59fb01686,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:SessionManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:SessionManager",
|
||||
MethodName: "unregisterUdpSession",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -3741,7 +3741,7 @@ func SessionManager_Methods(methods []server.Method, s SessionManager_Server) []
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0x839445a59fb01686,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:SessionManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:SessionManager",
|
||||
MethodName: "registerUdpSession",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -3755,7 +3755,7 @@ func SessionManager_Methods(methods []server.Method, s SessionManager_Server) []
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0x839445a59fb01686,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:SessionManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:SessionManager",
|
||||
MethodName: "unregisterUdpSession",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -4225,7 +4225,7 @@ func (c ConfigurationManager) UpdateConfiguration(ctx context.Context, params fu
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xb48edfbdaa25db04,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:ConfigurationManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:ConfigurationManager",
|
||||
MethodName: "updateConfiguration",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -4255,7 +4255,7 @@ func ConfigurationManager_Methods(methods []server.Method, s ConfigurationManage
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xb48edfbdaa25db04,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:ConfigurationManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:ConfigurationManager",
|
||||
MethodName: "updateConfiguration",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -4451,7 +4451,7 @@ func (c CloudflaredServer) RegisterUdpSession(ctx context.Context, params func(S
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0x839445a59fb01686,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:SessionManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:SessionManager",
|
||||
MethodName: "registerUdpSession",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -4471,7 +4471,7 @@ func (c CloudflaredServer) UnregisterUdpSession(ctx context.Context, params func
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0x839445a59fb01686,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:SessionManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:SessionManager",
|
||||
MethodName: "unregisterUdpSession",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -4491,7 +4491,7 @@ func (c CloudflaredServer) UpdateConfiguration(ctx context.Context, params func(
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xb48edfbdaa25db04,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:ConfigurationManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:ConfigurationManager",
|
||||
MethodName: "updateConfiguration",
|
||||
},
|
||||
Options: capnp.NewCallOptions(opts),
|
||||
|
@ -4525,7 +4525,7 @@ func CloudflaredServer_Methods(methods []server.Method, s CloudflaredServer_Serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0x839445a59fb01686,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:SessionManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:SessionManager",
|
||||
MethodName: "registerUdpSession",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -4539,7 +4539,7 @@ func CloudflaredServer_Methods(methods []server.Method, s CloudflaredServer_Serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0x839445a59fb01686,
|
||||
MethodID: 1,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:SessionManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:SessionManager",
|
||||
MethodName: "unregisterUdpSession",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -4553,7 +4553,7 @@ func CloudflaredServer_Methods(methods []server.Method, s CloudflaredServer_Serv
|
|||
Method: capnp.Method{
|
||||
InterfaceID: 0xb48edfbdaa25db04,
|
||||
MethodID: 0,
|
||||
InterfaceName: "tunnelrpc/tunnelrpc.capnp:ConfigurationManager",
|
||||
InterfaceName: "tunnelrpc/proto/tunnelrpc.capnp:ConfigurationManager",
|
||||
MethodName: "updateConfiguration",
|
||||
},
|
||||
Impl: func(c context.Context, opts capnp.CallOptions, p, r capnp.Struct) error {
|
||||
|
@ -4566,234 +4566,232 @@ func CloudflaredServer_Methods(methods []server.Method, s CloudflaredServer_Serv
|
|||
return methods
|
||||
}
|
||||
|
||||
const schema_db8274f9144abc7e = "x\xda\xccZ{t\x1c\xe5u\xbfwfW#\x19\xc9" +
|
||||
"\xaba\x96H\x96#\xab\xd1\xb1Kqb@v\x9d\x82" +
|
||||
"\x9bF\x92\x91\x1cV\xf8\xa1\xd9\xb5s\xa8\xb1s\x18\xed" +
|
||||
"~\x92F\x9d\x9dYff\x85\xe5\xe0\xd886\x06\x0e" +
|
||||
"\x10 6`'n\x8c\x1d\xda\x83\x09)\x0e\xa6)=" +
|
||||
"\xa4\xc5i\x1c\x12H\x1c\x9c\x03\xa9\x89I\xd3\xc6q[" +
|
||||
"\xfb\x98R\x0c4\xc7m\xcc\xf4\xdc\x99\x9d\x87v\x17=" +
|
||||
" \x7f\xe4\xbf\xd5\x9d\xfb=\xee\xef\xfb\xdd\xc7w?]" +
|
||||
"}\xa0\xae\x8b\xeb\x88oN\x00\xc8\x87\xe35\x0e\x9b\xff" +
|
||||
"\xd3\x8d\xfb\xe6\xfd\xd3V\x90[\x10\x9d/<\xd7\x97\xbc" +
|
||||
"`o=\x09q^\x00X\xf4\x98\xb0\x11\xa5g\x05\x01" +
|
||||
"@zF\xf8\x0f@\xe7\x8e\x8f<\xf5\xb5\xc7zw~" +
|
||||
"\x11\xc4\x16>T\x06\\\xb4\xbb\xb6\x0f\xa5'kI\xf3" +
|
||||
"\xf1\xda\x1dR]\x9d\x00\xe0\xdc ^uS\xf2\xe5c" +
|
||||
"\xa4\x1d\x9d:FS\xbfU;\x1f%$5\xe9b-" +
|
||||
"M\xfd\xa9\xfcO\xf6\x7fr\xd7K\xdb@l\xe1\xc6M" +
|
||||
"}\xa6n#J\x17]\xcd\x0bu\xab\x00\x9d\xb7w6" +
|
||||
"?\xf1\xe8\xb1\x1fl\x07\xf1r\x84\xd2N\xc5\x19?G" +
|
||||
"@i\xde\x8c\xbf\x01t~\xfc\xeeM\xef\x1c\xfe\xfe\xe2" +
|
||||
";@\xbc\x82\x14\x90\x14\x8e\xceh\xe7\x00\xa5_\xcc\xe8" +
|
||||
"\x04t\xce\x9e\xfb\xbf\x1d\x9f\xbfb\xe5\xfd _\x81\x9c" +
|
||||
"?\xc5\xc5\x19-\x1c\xe0\xa2\xcb.iC@\xa7s\xd9" +
|
||||
"\x8f\x9fmY\xf4\xd0\xce\xb2\xbds\xa4ym\xfd|\x94" +
|
||||
"R\xf5\xb4\xa3\xde\xfa[\x01\x9dO\xff\xc9\xd3\xf7\xf5<" +
|
||||
"\xb4e\x17\x88W\x05\x0b>^\xbf\x96\x16<ZO\x0b" +
|
||||
"\xfe\xcf\xcc\xaf\x1c+^\xf7\xed\x87J;rg9]" +
|
||||
"?\x9f\x14.\xba3\xb4\x8f\xce\xbb\xf9\xbbG\x9f~\x18" +
|
||||
"\xe4\x05\x88\xce\xeb\x03\x1f\x7f\x95\xdf{\xf0$\xacA\x81" +
|
||||
"6\xb8h}\xc3~2/\xdf@\xba?\xf9\xc4s\x7f" +
|
||||
"\x7f\xff\xd3;\xbe\x02\xf2\xe5\x88\x00.\x9c\xc7\x1b\xfe\x97" +
|
||||
"\x14N7\xd0j;O|ge\xfe\x81=\xfb=\x80" +
|
||||
"\xdc\xef\x0d39\x0eb\xce\xb6\xd4o\xf2k\x0ed\x0e" +
|
||||
"\x94\xa0\x8b\xd3'\x9cy\x1e\xc9\xee\x99\xae\xdd\x8b\x7f~" +
|
||||
"z\xd5\x8ao\x0d\xfeudlGb#\x8d\xdd1x" +
|
||||
"\xfeHc:\xffD5D\x16$\x0e\xa2\xd4\x9b D" +
|
||||
"\xba\x13\xb4\xc7'\xe7\xdcP\xb7\xe1\xf4\xb2\xa7@\\\xe0" +
|
||||
"O\xb37\x91\xa6iF_<\xf0\x87\xf3^\xbc\xf5\x10" +
|
||||
"\xc8Wa\x00\xd6n\xfa\x86\xd23\xee\xd8\xd8\xc9y\x07" +
|
||||
"\xbf\xf3\xcb\xfb\x0eW\x90Ll\xdc\x88\xd2\xbcFZ\xe5" +
|
||||
"c\x8d\x9f\x91d\xfa\xe5\xc4\xd6\xf3\xef)\x8f\xfc\xe3\xe1" +
|
||||
"r\x02{'\xd58\x80\xd2\x0a\xd2[\x94jt\xed\xbb" +
|
||||
"\xfb\xc8\x9e\x8f\xd7~\xed\xedg\xaa\xaa\xdf\"\x0e\xa0\xb4" +
|
||||
"]\xa4\x05n\x17\x89I\x97\xa5\xf0\xf5\xe7;b\xdf\x8e" +
|
||||
"Rm\xde\xa5g\x09\xeak/%\x85\xd67\x966\xe8" +
|
||||
"on}\xbe\x0c\x14W\xf1\xb5K\xfbPz\xe3R\x9a" +
|
||||
"\xed\x8c\xab\x1c\xfb\xe4\xc8\x0e\xf1\xd4\xcf\x8ez\xa0x\x96" +
|
||||
"o\x97F\xc8\xf2G%:\xb8\xbe\x9b\xbe\xfc`\xfc\xf4" +
|
||||
"\x97_\xa0\xcdE\x9c N~\xb5\xe8\x88d\xa2\xf4\xaa" +
|
||||
"\xe4\x9e\xb6\xd4\xc4\x03:-O\xfd\xe97\x97\xe6^{" +
|
||||
"\xa9\xca\x89Hr\xd3yi}\x13\xfd\xfa\xf3&\x02\xf5" +
|
||||
"\xd4\x82C\x9f?s\xef\xf1WJ\x96\xb8k?\xd3\xe4" +
|
||||
"\x92\xe6\x87M\xb4\xf6\x85u\xfbnP\x9d\x1bO\x96\x03" +
|
||||
"\xe3j\x9ei\xfa\x16J\xd8\xecz\xab;]\xc0\xd0j" +
|
||||
"\xdaJ\xf3\x08JEW\xfb\x96f\x9a\x9b;\xad\xcc\xda" +
|
||||
"\xf2\xb3O\xbf\x1e!U\xb1\xf9W\x081g\xe5go" +
|
||||
"\x1a\xa9\xdbt\xeaTt[j\xb3\x0b\xf0&w\xe8?" +
|
||||
"\xfc\xf3\xc3\xc3\xeb\xbfy\xect\x94H\xcd&\x11\xe9\xbf" +
|
||||
"\xfe\xea\xec\x97\xce\xe5s\xff\xee\xba\x8c\x7f8\xbb\x9b\x97" +
|
||||
"\x10\x9c\x87\x9a)\xa44\xb55\xf4\xb6\x9f\xe8?\x1b\xc5" +
|
||||
"\xfb\xdeYKI\xe1\xb1Y4\xf9\xe2\x9b\xbb\xd9\xbak" +
|
||||
"n<[\xc1\xb4\xa3\xb3\x96\xa0\xf4\xea,\x17\xebY;" +
|
||||
"P:\xd3\xd2\x04\xe0\x8c\xfe\xed\x037>\xf1\xbd\x95\xe7" +
|
||||
"=/v\xf7\xf2Z\xcbB\xda\xcb}_\xe8Yum" +
|
||||
"\xfb\x91\xf3Q3\x8e\xb7\x9cw]\xb2\x85V\x1a\xbc\xe6" +
|
||||
"\xdcg\xe6\xdd\xf7\xfd\xf3eG\xe5*\xc6g\xcfG\xe9" +
|
||||
"\xb2\xd9\x04\x978\xbb\x13\xf0\xcde\x7f\xf9JK\xa2\xe5" +
|
||||
"\x9d2hkHw\xf1\xec\x11\x94R\xa4\xbb\xa8w\xf6" +
|
||||
"\x0bD\xe8G\xbf\xbe\xff_.\x1c\xbb\xfe\xdd\x0a\x1b\xae" +
|
||||
"m%\xee\xb7\xd2\xb4\xa9VAJ\xb5^\x0e\xe0\xdcq" +
|
||||
"\xf2s\x1b~\xfa\xc5\xb7\xdf-g\x98\xbb\x91\xee\xd64" +
|
||||
"Jk\xdc\x11r+\x11\xf6\xe1\xd5\xff\xb9\xf9\xdc\xae\x8f" +
|
||||
"\xfc\xa6b\xee\xb7ZGP\x8a\xcf!M\x9c\xf3\x82\xb4" +
|
||||
"\x8b~9/\x0b\x07:z6\xbft!rT\x9b\xe6" +
|
||||
"\xf4\x11<\x0f\x09_=\xb5\xe5\x97\x9f\xfbm\x14\x9e\xb1" +
|
||||
"9\xbf\"x\xee\x9dC\xf0\xdc\xf6\xe6\xee\xeb\xbf\xb4\xee" +
|
||||
"\x1b\xefE\x08\xf2\xe4\x9c\xad4\xd4.\xea:\xd3\xccB" +
|
||||
",{\x95\xff3{eV)\xe8\x85%\xddE{\x98" +
|
||||
"\xe9\xb6\x9aUl\x96f\x9dV\xc1\xd0-\xd6\x8f(7" +
|
||||
"\xf21\x80\x18\x02\x88\xca\x08\x80|3\x8f\xb2\xc6\xa1\x88" +
|
||||
"\x98$\xa2\x88*\x09\x87y\x94m\x0eE\x8eKR\x84" +
|
||||
"\x15oi\x07\x905\x1e\xe5\x0d\x1c\"\x9fD\x1e@," +
|
||||
">\x08 o\xe0Q\xde\xc6\xa1S`f^\xd1\x99\x0e" +
|
||||
"\x09\xbb\xd74\xb1\x1e8\xac\x07tLf\x9bc\xca\x80" +
|
||||
"\x06\x09\x16\x11\x0b#\xb7\xda\xd8\x00\x1c6\x00:\xc3F" +
|
||||
"\xd1\xb4\xd6\xe86\xaaZ\x9a\x0d\x9a\xcc\xc2a\xac\x01\x0e" +
|
||||
"k\x00'2/\xc3,K5\xf4\x15\x8a\xae\x0c1\x13" +
|
||||
"\x80,\xab\xe5\xe3\x00A\xfaB?\xd1\x89\x1d{\x80\x13" +
|
||||
"\x17\x08\x18f\x1a\xf4\xc9*~\xec pb\xab\xe0\x98" +
|
||||
"lH\xb5lf\xe2\x9a\\\xc1\x9d\x9b7\xf4.t\x8a" +
|
||||
"\xba\xf7\x01\x99\xe9}H\xd0\xaa]\xd8\x8f\xe1\xee\xf8\xca" +
|
||||
"\xdd]\xa7\xa9L\xb7\x13)}\xd0(\x83\xbc\xaf\x1a\xe4" +
|
||||
"}%\xc8\xb7E \xbf})\x80|\x1b\x8f\xf2\x9d\x1c" +
|
||||
"\x8a|\x09\xf3\xed\xf3\x01\xe4-<\xca\xf7p\xe8d\xdd" +
|
||||
"ER9\x00\x08\xd0\x1cd\x8a]4\x99E\xb2\x99\x80" +
|
||||
"\xfd<\xba\xa0\xcf\x04\xdc<\xcaL\xda\xbb\x7f\x08\x09\xc5" +
|
||||
"\xcc\x0e\x07\x075\x01\xd2\xbd\x1bT\xcbV\xf5\xa1\xd5\xae" +
|
||||
"\xbc\xb3\xdf\xd0\xd4\xec\x18YU\xef\xee\xb3u\x09\x00\xa2" +
|
||||
"x\xd9Z\x00\xe4Dq)@\xa7:\xa4\x1b&sr" +
|
||||
"\xaa\x955t\x9d\x01\x9f\xb57\x0f(\x9a\xa2gY\xb0" +
|
||||
"PM\xe5B\xde\x02\x19f\x8e2\xf3J%B\xdf\xb9" +
|
||||
"\xfd\x8a\xa9\xf0yK\xae\x0fp\xec]\x0b \xf7\xf0(" +
|
||||
"\xf7Gp\\A8.\xe7Q\xbe1\x82\xe3\x1a\xc2\xb1" +
|
||||
"\x9fGy\x1d\x87\x8ea\xaaC\xaa~\x1d\x03\xde\x8c2" +
|
||||
"\xd0\xb2u%\xcf\x08\xb3\x12\x1e\x9b\x8d\x82\xad\x1a\xba\x85" +
|
||||
"\x8da\xd6\x01\xc4\xc6\x08R\xc2d\x9c\xbc\xd2\xa7\x94\xcf" +
|
||||
"(C\x9f\x9bfVQ\xd0lK\x8e\x05\x964,\x01" +
|
||||
"\x90ky\x94\x93\x1cv\x9a\xcc*j66\x86\xe5\xc4" +
|
||||
"\xefbU\x1f\xbed\xb0\xe8\xa6t\x84\\>|\xdb\x17" +
|
||||
"\x86\xe4\xc2\x12zw\x11z\xdbx\x94\xef'\x16\xa2\xc7" +
|
||||
"\xc2{\xf7\x00\xc8\xf7\xf3(\x7f\x95C1\xc6%1\x86" +
|
||||
"(\xee\xa6\xb8\xf1\x08\x8f\xf2\xd79t,o\xe5\x14`" +
|
||||
"\xce\x87\xb9-g\xd9\xa9\x82\xff\xd7\xe6\x9ce\xf7\x1b\xa6" +
|
||||
"\x8d\x02p(\x00\x91\xd9\xb0X\xf7 9Z*\xa7\xb1" +
|
||||
"\xebU^\xb71\x0e\x1c\xc6\xc9zS\xc9\xb2\xeb\x0c\x8a" +
|
||||
".l\x83]:$\x10q\x06\xc0D^\xe8\x11*A" +
|
||||
"\x91\xd0\x0b\x0f\xbe\xf9W\x10{\xfe\x88G\xf9\x8f#\xe6" +
|
||||
"w\x90\x01W\xf3(\x7f\x8aCG\xc9f\x8d\xa2n\xaf" +
|
||||
"\x06^\x19*s\x92\x0c\x83D\xd6d!\x7f\xfcek" +
|
||||
"\xab\xc4\x01C\x1fT\x87\x8a\xa6bGN\xa8X\xc8)" +
|
||||
"6\x1b\xf7\xc9%\x86\xc6O\x81\x18A!2mb\xf8" +
|
||||
"\xa1\xac\x8c\x1a\x09S\xc9[Ql\xd2\xd5\xb0!\x1a|" +
|
||||
"\x82G\xf9\x9a\xea\x87\xbb9\xcf,K\x19b\x15\xf1$" +
|
||||
"^\x15\x13\x9de\xc9\xea4\xf3\xb2\xd2\x95&\xb3\x84\xa2" +
|
||||
"f\xd3.\xea\x1d\xc7\xdb\x06\x91q.\x8f\xf2\xd5\x1c6" +
|
||||
"\xe0{\x8e\xb7\x8f\x05\x0f\x86g\xd4\xc6L\xd30\xb11" +
|
||||
"\xcc\xda%H\xb2\xa5\x05\xd0\xd0{\x98\xad\xa8\x1a\x92\x1f" +
|
||||
"\x07\xa5m\x19p\x93\x05\xa2\x106O<\xb7\x93\xdc)" +
|
||||
"?\xee\xa4\xc8\x1f\x1ay\x94?\xca\xa13D\\\xedg" +
|
||||
"&\xaaFn\xa5\xa2\x1b\x19\x9eeC\"\x97V\x9a9" +
|
||||
"\xddE]~\xd8\x16\x04\xa3&\x1eo\xb2\x12\x08\xa5\xe1" +
|
||||
"\xfdm\xde\x9e#\x11\xa0=\xcc\xde\xc11\xdf>\x10F" +
|
||||
"\x80 \x80\xdeE\xcer'\x8f\xf2\xceH\"z\xa0/" +
|
||||
"\x1a\x02bI\x8c\x01\x88\xbb\x89%;y\x94\xf7q\xe3" +
|
||||
"s<\x1be\xba\xdd\xa3\x0e\x81\xc0\xacPJ[\xecQ" +
|
||||
"\x87\x18\xf0\xd6\x87\x0d\xc6\xb5\x93\xe0a\x0cX\x86\xc6l" +
|
||||
"\xd6\xc3\xb2\x9aB.7\xca\xbc\xef%2\xfa\x87:\x11" +
|
||||
"o\xd3\x15\xdeC\xfcM\xf8eU\xc4\x83\xdaC\xea\x06" +
|
||||
"\xd0.X\x18\xba\x95\xc0\xc2Z\xa8\xcd*(\xba5\x95" +
|
||||
"X\xe2\xad\xef\xc5\x8b\x0a\x9a\x84NU\xa2\x0aZ\xbf\x93" +
|
||||
"\xb8\xe4\xc2\x82\xe3\xe2\xc3\xd2\xd0\xba\xc0\xb8%\xa1qA" +
|
||||
"\x9d\x11\x03\x0ec\x80\x9dYw\xc2\x0a\x0bc\x93\xed\xaa" +
|
||||
"\xd3\xdb\x16a\x1bs\x0b;\xffn\x8c~CA\x14\xf7" +
|
||||
"\x03'6\x08\x8e\xbfs\xf4\xc7\x0b\x15EZl\xa2@" +
|
||||
"\xb4\xaa`\xab\x82\xa1[\xb4V\xc4E\x96Ts\x11\xb3" +
|
||||
"J\x92\xdc\x1a\xf5\x90R\x92|`O\xe8\x0c^\x92\x04" +
|
||||
"\x10\xf7\xee\x07\x90\xf7\xf1(\x7f\x83\xc3N\xaf~\xc3\xc6" +
|
||||
"\xb0\x13Tb\xb5W\xa5,7\xa0-\xabha\xcet" +
|
||||
"LV\xd0\x94,\xeb\xc5RE\x06\x88\xc0!\xba\xae\x94" +
|
||||
"/\x98\xcc\xb2P5t\xb9\xa8h*o\x8f\x05U\xb4" +
|
||||
"^\xcc\xf7\x9blTE\xa3hu\xdb6\xcb\x0b\x05\xdb" +
|
||||
"\x9aJ\x8d\x1d\x02DqTP5\x17\xa0H\x11F\xd5" +
|
||||
"h\x17\x8f\xf2\xf2\x08@)J\xa3\xd7\xf3(\xaf\x0e\x01" +
|
||||
"\x92\xbf\x0b \xaf\xe6Q\xbe\x99\xc3D\xb1\xa8\x06y\xc3" +
|
||||
"\xd1\x8c\xac{\xda\x90X\xa9\xe4\xcb\xd3G\xca\xe2\xd2," +
|
||||
"o\xd8L\x1b\xf38\x9a\x0b-\x9ej\xf8.\x8b\xa3~" +
|
||||
"\xbe\xfb}\xaa$'\xbe\xbc\x118P\x86{{5\xdc" +
|
||||
"\x17F\xec\xf0\xb7\xbcb \xb4C\xf8\x0b6\x16D\x1d" +
|
||||
"\x96\xa7\xbc\xe8\xc3]2\xa6\x1b\x84\x1bB\x9d\xe9\xc6\"" +
|
||||
"\xd7\x07\x97\x1bYE+\x0f!\x89\xf2|\x19\xadl\xa6" +
|
||||
"\x1e\x1e\xa2\x8b\xae*\xb4\xb9\xb0\x120\xd7\xf8\x13Kc" +
|
||||
"\xd8\x07\x90\xd9\x80<f\xb6a\x88\x8dt;.\x05\xc8" +
|
||||
"\xdcF\xf2;1\x84G\xda\x8e-\x00\x99-$\xbf\x07" +
|
||||
"\x83\x9b\xadt\x17\x1e\x04\xc8\xdcC\xe2GH=\xc6\xbb" +
|
||||
"\xde+\xedr\xa7\xdfI\xf2}$\x8f\xc7\x92\x18\x07\x90" +
|
||||
"\xf6\xe2|\x80\xcc#$?L\xf2\x1a.\x895\x00\xd2" +
|
||||
"!\x1c\x01\xc8<E\xf2\xe7H.\xc4\x93t\xb9\x97\x9e" +
|
||||
"E\x13 \xf3w$\xff\x1e\xc9k\x9b\x93X\x0b \x1d" +
|
||||
"q\xe5\xcf\x93\xfcG$\xaf\x9b\x95\xc4:\x00\xe9\x87\xb8" +
|
||||
"\x15 \xf3\x03\x92\xbfB\xf2\x19\x98\xa4\xd2W:\x8e{" +
|
||||
"\x002\xaf\x90\xfc_I~IM\x12/\x01\x90~\xe1" +
|
||||
"\xee\xe7\x04\xc9\x7fM\xf2\xfaX\x92\xeae\xe9\xdfp?" +
|
||||
"@\xe6\xd7$\xffo\x927\x08Il\x00\x90\xdep\xed" +
|
||||
":G\xf2Z\xae\xecb\xe9\xd3\xb8\xec\xf6\xc8\x1bV\xc0" +
|
||||
"\x13V\x0aG\xe8\xf9X\xbf\x91\xa0\x1b\"&\xc2\xa65" +
|
||||
" &\x00\x9d\x82ah+\xc7\xbbG\xc2V\x86,\xff" +
|
||||
"\xa6\xda\x18\xf6\xec\x00I\x18\x94r\x900\xf4T.\x88" +
|
||||
"Y\xe5\x01\xd2\xdf\x89ju\x17m\xa3X\x806\xe2b" +
|
||||
".\x08\x16fQ_f\x1a\xf9\xd5\xc8\xcc\xbc\xaa+\xda" +
|
||||
"$\x81\xb3\x0e8\xac\x83R\xa4\xf2\xe7\x9e8\x8a\xbe\xff" +
|
||||
"\xbd;`4W\xce\xe8\xb6\xc2\x92\xd5\xcaPY\xed0" +
|
||||
"\x7f\x92\xda!\xa1G\xe2d\xdb\xa8\xa2\x15+\x8b\xee\x9a" +
|
||||
"iV\x87\xe9N\xaf\xba\x9c\xec\xf2\xe1\xb7\xde\xca\xe2W" +
|
||||
"\x95ZiMe)\x91fV[\xa1J\xb1t0\xbc" +
|
||||
"Y\xf8\xf6.n\x8f\xdc\xc44\xc5f\x96\xdd]\xc0\x82" +
|
||||
"\xa6\xb2\xdcg\x99\x99\x88V\x17\xd1Rjj\xa9l\\" +
|
||||
"\xd1\xe6\x1a\x8c\x91\x07\x062\x9c+\x19<e<\x87\x98" +
|
||||
"\xed\xfdJ\xe9\x83\x06\x95LB\xb4\x94\x9c\xde\xe84\xb3" +
|
||||
"\x12S9\x8b\xb0g:y)<\x9dh\x9dfm." +
|
||||
"\x17&\xbaXV\x99\xafJ\x1d\xea\xdf\x93\"\xcd/\"" +
|
||||
"\xf7:\x1e\xe5\xe1\x08\xb9\x19\xa5\xda\x1c\x8fr!\xac\x17" +
|
||||
"\xf2\xe9\xb0\xdd(\xf2\\\xa9\xdfH\xe9\xb7\xc0\xa3|\x1b" +
|
||||
"\x87\x09\xa5h\x0fcc\xf8\x0e5\x0e\x84\xf1-1\xe2" +
|
||||
"zJ\xcf1\xc0\x0d\xbe\xbbF\x92r\xf0@\xf2\x81`" +
|
||||
"|\xdf\xf2\xdb\x82I\x0f0x$([\xf9}\x1b\x19" +
|
||||
"\x9d\xde\xa2\xc4\xdbf\xb7 \xf6\xdf_\xd0o\xb0\x8b\x87" +
|
||||
"6\x02'>.`\xf8\xa8\x80\xfe\x1b\x82\xb8\xd7\x04N" +
|
||||
"\xdc% \x17<\x91\xa1\xff\x14&\xdeu7p\xe2v" +
|
||||
"\x01\xf9\xe0\x85\x0b\xfd\xa6s\xc7\xd8\x0c\x04N\xdc$`" +
|
||||
",x[D\xbfe-\xde2\x02\x9c\xa8\x0a\x18\x0f\x1e" +
|
||||
"\xcf\xd0\x7fK\x11\xd7o\x05N\\\x13\xb6V\xa1\xd3\xb3" +
|
||||
"\xa3\x0b\x1d\x9f\xf3\xd0\xe6\xb2~|\xa3\xd5\xd3\x02\xe8B" +
|
||||
"\xc7\xbf\xb7\xf1\xefwqs\xb5\xfc^!$\xb2\x8a\xcd" +
|
||||
"\xba\xa8P\xf6\x02\x1c\x96\"\x1ct\xa1\x1c\xc3H\xc7>" +
|
||||
"\xd27\xfa\x80\x8d\x93\x0a?\x99f!\xea\x8f\xff\x801" +
|
||||
"\x97\xaf\xb6kZ'\xe89G\xe6\xa5\x8a\xbc\x9eG\xb9" +
|
||||
"\x99\x9b\xa4\xe0\xae\x1a:\xbd\x0d\xfb\xe4O\xd0`\x9a\xff" +
|
||||
"\x0f\x82\xf9\x8fS\xb8\xfe\x11\x8f\xf2\x89\x88[\xbfJ\xc2" +
|
||||
"\x97y\x94_\x8f\x94\xa3\xaf\x91\xaf\x9f\xe0Q~'|" +
|
||||
"Fx\xebn\x00\xf9\x1d\x1e\xd3\x91JK\xbcH\x8a\xbf" +
|
||||
"\xa5z\xc4\xad\xb3\xd0\xab\xb3\xe2\xf8 @\xa6\x96\xea\x94" +
|
||||
"\xa4[g\xc5\xbc:K\xc4\x01\x80L#\xc9?\x1a\xad" +
|
||||
"\xb3f\xe1Z\x80L3\xc9\xe7\xe2\xf8k\xb7P4\xc3" +
|
||||
"\xf2W3\x86\x96\xabz\xd5\xe4\xed\xbfk\xa0\xbdLQ" +
|
||||
"\xb5\xa2\xc9\xa0\xfc\x0a\x92\xea\x89\x943\xde\x83\x87\xd7\xc6" +
|
||||
"\xcc\x10\x09sh\x05-\xceitA&\xcad\x9aQ" +
|
||||
"\xcc\x0dj\x8a\xc9r\x19f\x0a^@\xe8\xe7\xe3r-" +
|
||||
"F\xfe\x03\x01 |(\x8e\x90}\xc2\xcc\xd8k\x9a\x06" +
|
||||
"\x9aeW\x8d\x85\xe1U#\xb8i\xac\x0dox\"\xd7" +
|
||||
"U\xba\xe2\x0d\x84\x97\xa3\xb6\xacR\xb4X\x05&\xc03" +
|
||||
"3\xe8\x84Y\xc3FQ\xcb\xa5\x19\x08\xb69Vq\xab" +
|
||||
"\x8bM\x16}\x13~$\xacw#\xa1\xff\xda\x89\xfe\xa3" +
|
||||
"\xa6(\xef\x01N\\A\x91\xd0\x7fxC\xff\xd5]\xec" +
|
||||
">\x08\x9c\xf8g\x14\x09\xfd7g\xf4\x1fR\xc5\x8e\x17" +
|
||||
"\x81\x13;\"\xefA>>\x15\xefA\xde\x07\xd7\x1f\xe8" +
|
||||
"C)\xa1r\xe5\x19\x95\"T\xb4\x11\xf1!:;^" +
|
||||
"B\x8d\x1c\xe7\xb4\x1eQ\xa6\xfc\xf6\x10\xfc\xd3KY\xcc" +
|
||||
"\xa9\xfb\xb0M6?5\xfe\x7f\x00\x00\x00\xff\xff\xda\xbc" +
|
||||
"\xea\xa1"
|
||||
const schema_db8274f9144abc7e = "x\xda\xccZ}t\x14\xd7u\xbfw\xde.#\xc9\x12" +
|
||||
"\xbb\xe3\xd9\x04!E^G\x07\xda\x9a\x04cAIm" +
|
||||
"\x9aD\x12\x96\x88\x85\x01k\xb4\x90\xe3\x83q\x8eG\xbb" +
|
||||
"O\xd2\xa8\xbb3\x9b\x99Y\x19\x11;|\x04\x8c\xf1\xb1" +
|
||||
"\x1dC\xc06Jh\x08\x8e\xdbS9IM\x8c\x9b\xa6" +
|
||||
"\xc7nM\x1a\xc7\x89\x1d\x13\xe3cR\x08N\xd3\x94\xd0" +
|
||||
"6>\xa4\xae\xbf\x9a\xc3i\xea\xe9\xb93;\x1f\xda\x15" +
|
||||
"H\x82\xf6\x9c\xfe\x07w\xef\xbc\xf7\xee\xef\xfd\xee}\xbf" +
|
||||
"w\x9f\xae{\xb1\xb6Ch\x8boN\x00(G\xe2\xb3" +
|
||||
"\x1c\xbe\xe0\xd5M\x07\xe7\xff\xfd6P\xaeFt>\xff" +
|
||||
"\xcc\xca\xd4y{\xdbi\x883\x11`\xc9\xe3\xe28\xca" +
|
||||
"\xcf\x8a\"\x80\xfc]\xf1_\x01\x9d{>\xf8\xe4W\x1f" +
|
||||
"\xef\xde\xfb\x05\x90\xaef\xa13\xe0\x92\x035\x9bP>" +
|
||||
"\\C\x9e\xdf\xac\xd9)7\xd4\x8a\x00\xce\xcd\xd2\xa2\xdb" +
|
||||
"R\xaf\x1c#\xef\xe8\xd01\x1a\xfa\xbd\x9a\xf5(\xd7\x92" +
|
||||
"\x9b\x1c\xaf\xa5\xa1?^\xf8\xc9\xa1\x8f\xed{i;H" +
|
||||
"W\x0b\x13\x86~\xabv\x1c\xe5\xda:\xd7\xb3\xee\x16@" +
|
||||
"\xe7\x9d\xbd\x8dO|\xed\xd8\x0fw\x80\xb4\x10\xa1\xbc\xd2" +
|
||||
"\x96\xba:\x01P^Z\xf7\x97\x80\xce\xcb\xef\xdd\xf6\xee" +
|
||||
"\x91\x1f,\xbd\x07\xa4E\xe4\x80\xe4p\xa2\xae\x8f\x1c\xde" +
|
||||
"\xaak\x07t\xde8\xf7_;?w\xcd\x9a\x87@Y" +
|
||||
"\x84\x82?\x84t\xc5J\x01p\xc9\xc2+\xd2\x08\xe8\xb4" +
|
||||
"\xafx\xf9\xbbMK\x1e\xde[\xb1v\x81<\x95\xfa\xf5" +
|
||||
"(\xf3zZ\x91Z\x7f'\xa0\xf3\xc9?z\xea\xc1\xae" +
|
||||
"\x87\xb7\xec\x03ii0\xe1\xf3\xf5\xf7\xd1\x84g\xebi" +
|
||||
"\xc2\xff\x9c\xfd\xe5c\xa5\x1b\xbf\xf3pyE\xee(\xf1" +
|
||||
"\x86\xf5\xe4\xf0\xe1\x06\x1a\xa1ud\xfe\x1d\xdf{\xfe\xa9" +
|
||||
"G@Y\x82\xe8\xbc\xde\xff\x91\x13\xec\xc0\xf8iX\x87" +
|
||||
"\"-p\xc9\xd6\x86W\x11P\xde\xe7\xfa\xfe\xe4\xa3\xcf" +
|
||||
"\xfc\xcdCO\xed\xfc2(\x0b\x11\x01\\8\xcf7," +
|
||||
"\xa0\xc1\xa4\xd94\xdb\xde\x93\xcf\xae)\xec\x1e;\xe4\x01" +
|
||||
"\xe4\xfe~\xc3\xec\xc5\x02\xc4\x9c\xed=\xbf-\xac{," +
|
||||
"\xf3X\x19\xba8\xfd\xd46\xbb\x95\xe2\xee\x99\xed\xc6\xbd" +
|
||||
"\xf4ggoY\xfd\xed\x81?\x8f|\xcb\x13\xe3\xf4\xed" +
|
||||
"\xce\x81\xb7\x8f&\xfb\x0aOL\x86\x08O\xfc\x0c\xe5\xad" +
|
||||
"\x09B\xe4\xee\x04\xad\xf1\x9bW\xdd\\\xbb\xf1\xec\x8a'" +
|
||||
"AZ\xe2\x0fs*\xb1\x8d\x86\x19y\xf1\xb1\xdf\x9b\xff" +
|
||||
"\xe2\x9d\x87AY\x8a\xe1\xee\xd0o(\xbf\xe5~\x1b;" +
|
||||
"=\x7f\xfc\xd9_<x\xa4\x8ad\xab\x93\xe3(\xf3\xa4" +
|
||||
"\x8b{\xf2S\xf2n\xfa\x97\x13\xbb\x9d\xbd\xaf>\xfaw" +
|
||||
"G*\x09\xec\xaek4\xb9\x07=\xbf%\x0f$\xdd\xf8" +
|
||||
"\xee;:\xf6\x91\x9a\xaf\xbe\xf3\xf4\xa4\xeeOK{P" +
|
||||
"~Y\xa2\x09~$\x11\x93>\xd0\x83\xaf?\xd7\x16\xfb" +
|
||||
"N\x94j\x85+\x9bh\xad;\xae$\x87\x96\xdf,o" +
|
||||
"\xd0\xdf\xdc\xf6\\\x05(\xae\xe3|y\x13\xca7\xc84" +
|
||||
"\xdaR\x99\x9cc\x1f\x1b\xde)\x9d\xf9\xe9\xf3\x1e(^" +
|
||||
"\xe4\xc7\xe517r\x996n\xe5m_\xda\x13?\xfb" +
|
||||
"\xa5\x17hq\x91$\x88\xd7\xb8\xfcL\x1dB\xf9\x9a\x94" +
|
||||
";rj\x0e\x03t\x9a\x9e\xfc\xe3o-\xcf\x9dzi" +
|
||||
"\xb2\x1dy|N+\xcaO\xcf\xa1\xc9\x0f\xcf!T\xcf" +
|
||||
",<\xfc\xb9_?p\xfc\xb5r(\xee\xe4\x0d\x8d." +
|
||||
"k\xe67\xd2\xe4\xe77\x1c\xbcYsn=]\x89\x8c" +
|
||||
"\xeb\xd9\xdd\xf8\xcf(\xab\x8d4\xdc\xed\x8d4\\@\xd1" +
|
||||
"\xc9\xbc\x8f6\x8e\xa1|\xca\xf5>\xe1\x8e-\x9cU\xe7" +
|
||||
"n\xf9\xe9'_\x8f\xb0\xeaT\xe3\x95D\x875\x9f\xbe" +
|
||||
"m\xb8\xf6\xee3g\xa2\xcb:\xde\xe8\"\xfck\xf7\xd3" +
|
||||
"\xbf\xfd\x87G\x86n\xff\xd6\xb1\xb3\x11&\xd5\xce=D" +
|
||||
"\x9f\xfe\xfb\x9f\xbd\xf1\xc5s\x85\xdc\xbf\xb89\xe3\xefN" +
|
||||
"\xed\xdca7\xa4\xb9TS\xe6\xa4\x1b\xba[O\xf6\xbe" +
|
||||
"\x11\x05\x1c\x9bLrhi\xa2\xc1\x97\xde\xd1\xc97\\" +
|
||||
"\x7f\xeb\x1bUT\xfbD\xd30\xcaJ\x13}\xb0\xbai" +
|
||||
"'\xca\xbcy\x0e\x803\xf2W\xbbo}\xe2\xfbk\xde" +
|
||||
"\xf6\xd2\xd8]\xcb\xba\xe6~Z\xcb\x83\x9f\xef\xba\xe5\x86" +
|
||||
"\xd6\xa3oG\xc3P\x9a)\xb1d\xad\x99f\x1a\xb8\xfe" +
|
||||
"\xdc\xa7\xe6?\xf8\x83\xb7+\xf6\xcau\xdc\xd5\xbc\x1e\xe5" +
|
||||
"\x03\xcd\x04\xd7~r~s\xc5\x9f\xbe\xd6\x94hz\xb7" +
|
||||
"\x02\xdaY\xe4\xfbl\xf3\x18\xca'\x9a]\x98\x9a_ " +
|
||||
"F\x7f\xed\xeb\x87\xfe\xf1\xfc\xb1\x9b\xde\xab\x8a\xe1\xf9\x96" +
|
||||
"=(\xff\xbc\x85\x86=\xd5\"\xca\xa7Z~\x1f\xc0\xb9" +
|
||||
"\xe7\xf4g6\xbe\xfa\x85w\xde\xab\xa4\x98\x07|\xcb6" +
|
||||
"\x94\xcf\xba_\xfc\xb2\x85\x18\xfb\xc8\xda\x7f\xdb|n\xdf" +
|
||||
"\x07\x7f[5\xf6\xd6\xab\xc6P\xde\x7f\x15y\xee\xbb\xea" +
|
||||
"\x05\xf9\x9a4\xa5\xe2+\xe2cm]\x9b_:\x1f\xd9" +
|
||||
"*)\xbd\x89\xe0yX\xfc\xca\x99-\xbf\xf8\xcc\xef&" +
|
||||
"\x90/}\xa5\xbbSi\x82\xe7\xae7\xf7\xdf\xf4\xc5\x0d" +
|
||||
"\xdfx?B\x90\xee\xf4\xb7\xe9S\xbb\xa4\xeb<o\x16" +
|
||||
"\xe3\xd9EE\xd3\xb0\x8dE\xbe!{mV-\xea\xc5" +
|
||||
"e\x9d%{\x88\xeb\xb6\x96Um\xde\xc7\xadb\xc2\xd0" +
|
||||
"-\xde\x8b\xa8$Y\x0c \x86\x00\x92:\x0c\xa0\xdc\xc1" +
|
||||
"P\xc9\x0b(!\xa6\x88.\x92F\xc6!\x86\x8a-\xa0" +
|
||||
"$\x08)*\xb4\xd2g[\x01\x94<Ce\xa3\x80\xc8" +
|
||||
"R\xc8\x00\xa4\xd2\x1e\x00e#Ce\xbb\x80N\x91\x9b" +
|
||||
"\x05U\xe7:$\xecn\xd3\xc4z\x10\xb0\x1e\xd01\xb9" +
|
||||
"m\x8e\xaa\xfdyH\xf0\x88Y\x1c\xbe\xd3\xc6\x06\x10\xb0" +
|
||||
"\x01\xd0\x192J\xa6\xb5N\xb7Q\xcb\xf7\xf1\x01\x93[" +
|
||||
"8\x84\xb3@\xc0Y\x80A\x90\xb1\x0b\x05\x99\xe1\x96\xa5" +
|
||||
"\x19\xfa\xeavUW\x07\xb9I\xe1\xd5\xb08@p\x94" +
|
||||
"\xa1\x7f\xe8Imc H\x0bE\x0cO\x1d\xf4y+" +
|
||||
"}x\x1c\x04\xa9EtL>\xa8Y67q]\xae" +
|
||||
"\xe8\x0e\xcd\x0c\xbd\x03\x9d\x92\xee\xfd\x80\xdc\xf4~H\xd0" +
|
||||
"\xa4\x1d\xd8\x8b\xd3X\xe2\x8dy\x8d\xebv\x8f\xce\x06\x8c" +
|
||||
"\x0a\xf4WN\x86\xfe\xca2\xfa\xdb#\xe8o]\x0e\xa0" +
|
||||
"\xdc\xc5P\xb9W@\x89\x95\xe1\xdf\xb1\x00@\xd9\xc2P" +
|
||||
"\xb9_@'\xebM\x92\x03\x80\x00\xd8\x01\xae\xda%\x93" +
|
||||
"[d\x9b\x0d\xd8\xcb\xd0\xc5\x7f6\xe0\xe6\x11nR\x04" +
|
||||
"\xfe~$T3;\x14\xec\xd9\x94\xcc\xea\xde\xa8Y\xb6" +
|
||||
"\xa6\x0f\xaeu\xed\xbdF\"\xafeG)\xb6zw\xb5" +
|
||||
"-\xcb\x00\x10\xa5\x0f\xac\x07@A\x92\x96\x03\xb4k\x83" +
|
||||
"\xbaar'\xa7YYC\xd79\xb0\xac\xbd\xb9_\xcd" +
|
||||
"\xabz\x96\x07\xd3\x89\x17\x9a\xce\x9b&\xc3\xcd\x11n^" +
|
||||
"\xabFX=\xafW5\xd5\x82\x05\xa0\xd4\x07\xa0v\xaf" +
|
||||
"\x07P\xba\x18*\xbd\x11PW\x13\xa8\xab\x18*\xb7F" +
|
||||
"@]G\xa0\xf62T6\x08\xe8\x18\xa66\xa8\xe97" +
|
||||
"r`f\x94\x99\x96\xad\xab\x05N\x00\x96\xc1\xd9l\x14" +
|
||||
"m\xcd\xd0-L\x86\x87\x12 &#\xb0\xd5L\xc5U" +
|
||||
"\x8f\xaa\xd7\xfa\\\xf3\xa9f\xe8\xf3\xfa\xb8U\xca\xdbh" +
|
||||
")\xb1 \x9e\x86e\x00J\x0dC%%`\xbb\xe9\xfd" +
|
||||
"\x9e\x0c5\xc7\xff\xde\xdc\x01\x96\xa9`\xee\xbb\xfb\"\xb4" +
|
||||
"\xf3\xb1\xdc\xb18\xa4\x1d\x96\xa1\xdcEPng\xa8<" +
|
||||
"D\xfcD\x8f\x9f\x0f\x8c\x01(\x0f1T\xbe\"\xa0\x14" +
|
||||
"\x13R\x18C\x94\xf6Sqy\x94\xa1\xf2u\x01\x1d\xcb" +
|
||||
"\x9b\xba\x070\xe7c\x9e\xceYvO\xd1\xff\xdf\xe6\x9c" +
|
||||
"e\xf7\x1a\xa6\x8d\"\x08(\x02\xd1\xdc\xb0x\xe7\x00%" +
|
||||
"bO.\xcfo\xd2\x98nc\x1c\x04\x8c\x13\x08\xa6\x9a" +
|
||||
"\xe57\x1aT\x82\xf8F\xbb\xbcc a\x1d\xc0\xd4Y" +
|
||||
"\xea\x91\xac\xb3\xc4\xec!\xaf\x88\xf8 \\C\x84\xfa\x03" +
|
||||
"\x86\xca\x1fF@h\xa30\xaec\xa8|\\@G\xcd" +
|
||||
"f\x8d\x92n\xaf\x05\xa6\x0eV$Q\x86C\"k\xf2" +
|
||||
"\x90R\xfe\xb4\xb5\x17\xac\x16\x86>\xa0\x0d\x96L\xd5\x8e" +
|
||||
"lW\xa9\x98Sm>\xe1\xa72Wh\xc3\xa6\"K" +
|
||||
" `.\x91,~\xf5\xab\xa2\x0b+XQ\xa0\xfa&" +
|
||||
"\x03\x8a\x98\xf1Q\x86\xca\xf5\x93\xef\xf7\xe6\x02\xb7,u" +
|
||||
"\x90W\x15\x9fY\x17\x01H\xe7Y\x82\x80\x0e5:\xd3" +
|
||||
"\xaeu#E\x9b\xd6R\xef8\xdeb\x88\xa5\xf3\x18*" +
|
||||
"\xd7\x09\xd8\x80\xef;\xdej\x16\xee\x09\xb7-\xcdM\xd3" +
|
||||
"01\x19\x9e\xfcex\xb2\xe5\x09\xd0\xd0\xbb\xb8\xadj" +
|
||||
"y\xa4l\x0f\xf4q\x05\x88\xd3\xabZ!\x84\x9ey^" +
|
||||
"\xaf\x9a\xa0t\x8b\xee\x1d\xa5K\x92\xa1\xf2!\x01\x9dA" +
|
||||
"\xa2r/7Q3rkT\xdd\xc80\x9e\x0dy~" +
|
||||
"yS\xf7\xf1\xb4\xcb\x9c\x19\x8ec\xf220A\x04\xa6" +
|
||||
"H\x11D\xcaEk\xa8\x07\x02\x02l\xed\x0f\xcbEP" +
|
||||
"zwQN\xdd\xcbP\xd9\x1b9\xcfv\xaf\x8c\xd6\x8b" +
|
||||
"X\x0ac\x00\xd2~\xe2\xcf^\x86\xcaAa\xa2j\xe0" +
|
||||
"#\\\xb7\xbb\xb4A\x10\xb9\x15Zi\x89]\xda \x07" +
|
||||
"f]n\x19\xaf\x9d\x16*F\xbfe\xe4\xb9\xcd\xbbx" +
|
||||
"6\xafRf\x8ep\xef\xf72M\xfd\x8d\x9e\x9a\xd7}" +
|
||||
"U9\xe6\xf1\x9by\xa2-\x92g\xad!\xb5\x03\x98\x17" +
|
||||
".\x0e\x93O\xe4\xa1\xd2J[EU\xb7\xaa\xca\x8f|" +
|
||||
"\xf1Ux%\xa6\x8a@a\xea\x05\xd5'\xf8\xfer\xcb" +
|
||||
"Y\xf9\xf8\x89\xc6\xb9<\x8c3\x08sY\x18f b" +
|
||||
"b `\x0c\xb0=\xeb\x0eX\x15k|\xbakK\xf8" +
|
||||
"\x0a2\xe6*H\xffB\x8e~\x17C\x92\x0e\x81 5" +
|
||||
"\x88\x8e\xbf~\xf4\xbf\x17\xab\xd4`|\xea\xf2u\x8bK" +
|
||||
"A\xb4h\xc6H\x12-\x9b,\x89\xccI\xce\xdcm\xd1" +
|
||||
"\x1c*\x9f\xb9\xbb\xc7\xc2t\xf1\xce\\\x00\xe9\xc0!\x00" +
|
||||
"\xe5 C\xe5\x1b\x02\xb6{B\x11\x93a\x13\xaa\xcc{" +
|
||||
"O\x01\xad2 \x9dU\xf3\xe1\x11\xec\x98\xbc\x98W\xb3" +
|
||||
"\xbc\x1b\xcb\xa2\x0f\x10A@t\x93\xadP4\xb9e\xa1" +
|
||||
"f\xe8JI\xcdk\xcc\x1e\x0d\x94\xbb^*\xf4\x9a|" +
|
||||
"DC\xa3du\xda6/\x88E\xdb\xaa\xd2\xf5\xd3\x80" +
|
||||
"\xc9\xaf\xc1\xae\xbe\x0ce\x1e\x89\xdf\x0e\x86\xca\xaa\x08L" +
|
||||
"=t*\xdf\xc4PY\x1b\xc2\xa4|\x0f@Y\xcbP" +
|
||||
"\xb9C\xc0D\xa9\xa4\x05'\x8f\x937\xb2\xee\xceCb" +
|
||||
"\x8dZ\xa8<\x80z,\xa1\x8f\x17\x0c\x9b\xe7G=\xd6" +
|
||||
"\xe6\xc2\xb8gZ7+\x0a\xbfwn\xfe\x7fR\xac\xb1" +
|
||||
"\xa9\xae\x90\xed\x1eR\x15[\xd0:\xd9\x16,\x8e\x04\xe3" +
|
||||
"\xaf{u\x7f\x18\x8c\xf8'|4(N\xbc@[\xeb" +
|
||||
"#_\x8e\xa8\x13\xc4\x9bC\x9f\xa9\xeb\xf1d%\xcbM" +
|
||||
"\xd0UFV\xcdWW\x19V\xb8\xa0\xbe\x9ei\x05\x89" +
|
||||
"NM\xe9,\x1a\xba\xcb\xd3\xeb\xfd\xe1\xe5Q\\\x09\x90" +
|
||||
"\xd9\x88\x0c3\xdb1\xc4I\xde\x8a\xcb\x012w\x91\xfd" +
|
||||
"^\x0c\xa1\x92w`\x13@f\x0b\xd9\xef\xc7\xe0\xaa-" +
|
||||
"\xef\xc2q\x80\xcc\xfdd~\x94\xdcc\xccMmy\x9f" +
|
||||
";\xfc^\xb2\x1f${<\x96\xc28\x80|\x00\x17\x00" +
|
||||
"d\x1e%\xfb\x11\xb2\xcf\x12R8\x0b@>\x8c\xc3\x00" +
|
||||
"\x99'\xc9\xfe\x0c\xd9\xc5x\x0a\xdd.6\x9a\x00\x99\xbf" +
|
||||
"&\xfb\xf7\xc9^\xd3\x98\xc2\x1a\x00\xf9\xa8k\x7f\x8e\xec" +
|
||||
"?&{\xed\xdc\x14\xd6\x02\xc8?\xc2m\x00\x99\x1f\x92" +
|
||||
"\xfd5\xb2\xd7a\x8ad\xb6|\x1c\xc7\x002\xaf\x91\xfd" +
|
||||
"\x9f\xc8~\xc5\xac\x14^\x01 \xff\xdc]\xcfI\xb2\xff" +
|
||||
"\x8a\xec\xf5\xb1\x14is\xf9\x97x\x08 \xf3+\xb2\xff" +
|
||||
"\x07\xd9\x1b\xc4\x146\x00\xc8\xbfq\xe3:G\xf6\x1a\xa1" +
|
||||
"\xe2z\xeb\xf3\xba\xe2\x0e\xcb\x0c+\xe0\x0c/\xd7*\x9c" +
|
||||
"pC\xc5D\xd8L\x07\xc4\x04\xa0S4\x8c\xfc\x9a\x89" +
|
||||
"\xf9\x92\xb0\xd5A\xcb\xbf/'\xc3^\" \x19\x03u" +
|
||||
"\x08\x09C\xef\xc9\x05\x05\xad\xb2z\xfa+\xd1\xac\xce\x92" +
|
||||
"m\x94\x8a\x90&F\xe6\x82\x1ab\x96\xf4\x15\xa6QX" +
|
||||
"\x8b\xdc,h\xba\x9a\x9f\xa2\xaa\xd6\x82\x80\xb5P.`" +
|
||||
"\xfe\xd8\x17/\xb1\x17\xbe\xfd\x07\xbcf\x17\xe2\xb5\xb8V" +
|
||||
"\x1d\xac\x10\x1d\x0b\xa6\x10\x1d\x09=RD\xd3#j\xbe" +
|
||||
"T\xad\xe9/Mh\xf6q+AZc\xaa{\x8e\xdf" +
|
||||
"\x1d\xac(n\x17\x14\\\xeb\xaaU\x88\xab\xb8D\xbdJ" +
|
||||
"q\x8d\x87\x97\x18?\xf6\xa5\xad\x91\x1b`^\xb5\xb9e" +
|
||||
"w\x16\xb1\x98\xd7x\xee\xd3\xdcLD\x85IT\x8f\xcd" +
|
||||
"\xe4\xe4\x9b\xa0\xff\xdc\xe01\xf2 B \x08\xe5\xe0g" +
|
||||
"\x88\xf0 \xb7\xbd\x7f\xf5\xe8\x03\x86\xa7\xbc\xd0\xba\xac1" +
|
||||
"\\9\xc8\xa6\xde\xa3\xb0\xdd;]\xb5=\x93\xeaN\xab" +
|
||||
"\x10\xa37\x9b\x9a\x19\x8c:\x89\xc8\xf5/i\x91\x96\x1d" +
|
||||
"\xa5\xc1\x06\x86\xcaP$\x0d8\x9d\xd59\x86J1\x94" +
|
||||
"\x1d\x85\xbe\xb0_*1\xa1\xdc0\xa5\xf3\xbb\xc8P\xb9" +
|
||||
"K\xc0\x84Z\xb2\x870\x19\xbe\xa7M\x00db#\x8f" +
|
||||
"\xf2\xa1G\xcfq\xc0\x8d~zGN\xf5\xe0\xa1g\xba" +
|
||||
"\xd7\xfb\xe9\x05\xef_\x13\xa7\xdc\xd2\xe0\xc5c\xba\xaa\xc2" +
|
||||
"\xe7Q\x82\xa6&n7\xbaz\xdb\x7fSB\xff\xcd@" +
|
||||
":\xbc\x09\x04\xe9/D\x0c\xdfI\xd0\x7f\x16\x91\x0e\x98" +
|
||||
" H\xfbD\x14\x82g?\xf4\x9f\xf7\xa4]\xf7\x81 " +
|
||||
"\xed\x10\x91\x05\xafv\xe8\xf7\xd1\xdbF\xeb\x10\x04\xe9n" +
|
||||
"\x11c\xc1{)\xfa]x\xe9\xb3\xc3 H\x9a\x88\xf1" +
|
||||
"\xe0A\x10\xfd\xe7!\xe9\xf6m H\xeb\xc2\x161\xb4" +
|
||||
"{qt\xa0\xe3\xe7\x02\xa4\xddl\x98\xd80\xf6\xbc\x00" +
|
||||
":\xd0\xf1/\x8b\xecB\xb7E\xd7\xcb\xefpB\"\xab" +
|
||||
"\xda\xbc\x83\x14\xb8W\x10\xb1\\\x11\xa1\x03\x95\x18F\x1e" +
|
||||
"!\"\xfd\xad\xcb\xea\xe9T\xe5\xcf%)\\\x7f\x94K" +
|
||||
"\xac\xd7\x17i\xf5{\xe5\xa6\xdcG\x8f\x8c>\xecv\x80" +
|
||||
"Qi\x14\xa6P\xf5\x17)\xbb\xde\xe2\xc3\xd4`\x9e\xdc" +
|
||||
"\xbd:\x98\xe58\x15\xfc\x1f3TNFR\xff\x04\x19" +
|
||||
"_a\xa8\xbc\x1e\x91\xbb\xa7\xa8\x1e\x9cd\xa8\xbc\x1b\xbe" +
|
||||
"\x95\xbcu\x1f\x80\xf2.\xc3\xbe\x88z\x93\xfe\x9b\x1c\x7f" +
|
||||
"G\x1a\xc7\xd5n\xe8i\xb78\xee\x01\xc8\xd4\x90\xf6I" +
|
||||
"\xb9\xda-\xe6i7\x09\xfb\x012I\xb2\x7f(\xaa\xdd" +
|
||||
"\xe6\xe2z\x80L#\xd9\xe7\xe1\xc4\xdb\xbfX2Cy" +
|
||||
"\x9d7\x06Wi\xfa\xa4\x82\xc0\x7f\xbcA{\x85\xaa\xe5" +
|
||||
"K&\x87\xca\xdbNOWD\"y\xaf:^\x1b6" +
|
||||
"C\xe4\xcc\xa1\x15\xb4hg\xd0\x98\x99\xfa,\xcc\x1b\xa5" +
|
||||
"\xdc@^5y\xce\xdd}\xa4r\xd1\xcb\xe2J\x0dF" +
|
||||
"\xfe\xe6\x02 |\x1a\x8f\xa4\xc24N\xd8n\xd34L" +
|
||||
"\xa8\xb8\xd6,\x0e\xaf5\xc1\xadf}x\xb1\x94\x84\x8e" +
|
||||
"\xf2\xcd\xb2?\xbc\x8d\xa5\xb3j\xc9\xe2U\xf8\x00\xe3f" +
|
||||
"\xd0\xb6\xb3\x86\x8cR>\xd7\xc7A\xb4\xcd\xd1\xaa\xcbd" +
|
||||
"|\xba\xd5\x9ay5\xb3\xde\xad\x99\xfeS/\xfa/\xba" +
|
||||
"\x922\x06\x82\xb4\x9aj\xa6\xff\xea\x88\xfe\xdf\x1cH\x9d" +
|
||||
"\xe3 H\x9f\xa0\x9a\xe9\xbf\xb8\xa3\xff\x8a,\xb5\xbd\x08" +
|
||||
"\x82\xd4\x16y\x01\xf3Q\xaaz\x01\xf3~H\xd8\x9a\xf7" +
|
||||
"C\xf90\x16*Oc\xaae\xd1\x8eH\xcd\xe5\xb6\x9c" +
|
||||
"\xda\xbd\x16\xd1\xe5\xbc\x17M\xfb}%\xf8\xeb\x9f\xff\x9b" +
|
||||
"\xa6\xa0\x7f\xb6\xfeO\x00\x00\x00\xff\xff\xb4\x0bQ\xfc"
|
||||
|
||||
func init() {
|
||||
schemas.Register(schema_db8274f9144abc7e,
|
|
@ -0,0 +1,63 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
|
||||
"github.com/google/uuid"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
// CloudflaredClient calls capnp rpc methods of SessionManager and ConfigurationManager.
|
||||
type CloudflaredClient struct {
|
||||
client pogs.CloudflaredServer_PogsClient
|
||||
transport rpc.Transport
|
||||
requestTimeout time.Duration
|
||||
}
|
||||
|
||||
func NewCloudflaredClient(ctx context.Context, stream io.ReadWriteCloser, requestTimeout time.Duration) (*CloudflaredClient, error) {
|
||||
n, err := stream.Write(rpcStreamProtocolSignature[:])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if n != len(rpcStreamProtocolSignature) {
|
||||
return nil, fmt.Errorf("expect to write %d bytes for RPC stream protocol signature, wrote %d", len(rpcStreamProtocolSignature), n)
|
||||
}
|
||||
transport := rpc.StreamTransport(stream)
|
||||
conn := rpc.NewConn(transport)
|
||||
client := pogs.NewCloudflaredServer_PogsClient(conn.Bootstrap(ctx), conn)
|
||||
return &CloudflaredClient{
|
||||
client: client,
|
||||
transport: transport,
|
||||
requestTimeout: requestTimeout,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (c *CloudflaredClient) RegisterUdpSession(ctx context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeIdleAfterHint time.Duration, traceContext string) (*pogs.RegisterUdpSessionResponse, error) {
|
||||
ctx, cancel := context.WithTimeout(ctx, c.requestTimeout)
|
||||
defer cancel()
|
||||
return c.client.RegisterUdpSession(ctx, sessionID, dstIP, dstPort, closeIdleAfterHint, traceContext)
|
||||
}
|
||||
|
||||
func (c *CloudflaredClient) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID, message string) error {
|
||||
ctx, cancel := context.WithTimeout(ctx, c.requestTimeout)
|
||||
defer cancel()
|
||||
return c.client.UnregisterUdpSession(ctx, sessionID, message)
|
||||
}
|
||||
|
||||
func (c *CloudflaredClient) UpdateConfiguration(ctx context.Context, version int32, config []byte) (*pogs.UpdateConfigurationResponse, error) {
|
||||
ctx, cancel := context.WithTimeout(ctx, c.requestTimeout)
|
||||
defer cancel()
|
||||
return c.client.UpdateConfiguration(ctx, version, config)
|
||||
}
|
||||
|
||||
func (c *CloudflaredClient) Close() {
|
||||
_ = c.client.Close()
|
||||
_ = c.transport.Close()
|
||||
}
|
|
@ -0,0 +1,69 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
// HandleRequestFunc wraps the proxied request from the upstream and also provides methods on the stream to
|
||||
// handle the response back.
|
||||
type HandleRequestFunc = func(ctx context.Context, stream *RequestServerStream) error
|
||||
|
||||
// CloudflaredServer provides a handler interface for a client to provide methods to handle the different types of
|
||||
// requests that can be communicated by the stream.
|
||||
type CloudflaredServer struct {
|
||||
handleRequest HandleRequestFunc
|
||||
sessionManager pogs.SessionManager
|
||||
configManager pogs.ConfigurationManager
|
||||
responseTimeout time.Duration
|
||||
}
|
||||
|
||||
func NewCloudflaredServer(handleRequest HandleRequestFunc, sessionManager pogs.SessionManager, configManager pogs.ConfigurationManager, responseTimeout time.Duration) *CloudflaredServer {
|
||||
return &CloudflaredServer{
|
||||
handleRequest: handleRequest,
|
||||
sessionManager: sessionManager,
|
||||
configManager: configManager,
|
||||
responseTimeout: responseTimeout,
|
||||
}
|
||||
}
|
||||
|
||||
// Serve executes the defined handlers in ServerStream on the provided stream if it is a proper RPC stream with the
|
||||
// correct preamble protocol signature.
|
||||
func (s *CloudflaredServer) Serve(ctx context.Context, stream io.ReadWriteCloser) error {
|
||||
signature, err := determineProtocol(stream)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
switch signature {
|
||||
case dataStreamProtocolSignature:
|
||||
return s.handleRequest(ctx, &RequestServerStream{stream})
|
||||
case rpcStreamProtocolSignature:
|
||||
return s.handleRPC(ctx, stream)
|
||||
default:
|
||||
return fmt.Errorf("unknown protocol %v", signature)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *CloudflaredServer) handleRPC(ctx context.Context, stream io.ReadWriteCloser) error {
|
||||
ctx, cancel := context.WithTimeout(ctx, s.responseTimeout)
|
||||
defer cancel()
|
||||
transport := rpc.StreamTransport(stream)
|
||||
defer transport.Close()
|
||||
|
||||
main := pogs.CloudflaredServer_ServerToClient(s.sessionManager, s.configManager)
|
||||
rpcConn := rpc.NewConn(transport, rpc.MainInterface(main.Client))
|
||||
defer rpcConn.Close()
|
||||
|
||||
// We ignore the errors here because if cloudflared fails to handle a request, we will just move on.
|
||||
select {
|
||||
case <-rpcConn.Done():
|
||||
case <-ctx.Done():
|
||||
}
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
)
|
||||
|
||||
// protocolSignature defines the first 6 bytes of the stream, which is used to distinguish the type of stream. It
|
||||
// ensures whoever performs a handshake does not write data before writing the metadata.
|
||||
type protocolSignature [6]byte
|
||||
|
||||
var (
|
||||
// dataStreamProtocolSignature is a custom protocol signature for data stream
|
||||
dataStreamProtocolSignature = protocolSignature{0x0A, 0x36, 0xCD, 0x12, 0xA1, 0x3E}
|
||||
|
||||
// rpcStreamProtocolSignature is a custom protocol signature for RPC stream
|
||||
rpcStreamProtocolSignature = protocolSignature{0x52, 0xBB, 0x82, 0x5C, 0xDB, 0x65}
|
||||
|
||||
errDataStreamNotSupported = fmt.Errorf("data protocol not supported")
|
||||
errRPCStreamNotSupported = fmt.Errorf("rpc protocol not supported")
|
||||
)
|
||||
|
||||
type protocolVersion string
|
||||
|
||||
const (
|
||||
protocolV1 protocolVersion = "01"
|
||||
|
||||
protocolVersionLength = 2
|
||||
)
|
||||
|
||||
// determineProtocol reads the first 6 bytes from the stream to determine which protocol is spoken by the client.
|
||||
// The protocols are magic byte arrays understood by both sides of the stream.
|
||||
func determineProtocol(stream io.Reader) (protocolSignature, error) {
|
||||
signature, err := readSignature(stream)
|
||||
if err != nil {
|
||||
return protocolSignature{}, err
|
||||
}
|
||||
switch signature {
|
||||
case dataStreamProtocolSignature:
|
||||
return dataStreamProtocolSignature, nil
|
||||
case rpcStreamProtocolSignature:
|
||||
return rpcStreamProtocolSignature, nil
|
||||
default:
|
||||
return protocolSignature{}, fmt.Errorf("unknown signature %v", signature)
|
||||
}
|
||||
}
|
||||
|
||||
func writeDataStreamPreamble(stream io.Writer) error {
|
||||
if err := writeSignature(stream, dataStreamProtocolSignature); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return writeVersion(stream)
|
||||
}
|
||||
|
||||
func writeVersion(stream io.Writer) error {
|
||||
_, err := stream.Write([]byte(protocolV1)[:protocolVersionLength])
|
||||
return err
|
||||
}
|
||||
|
||||
func readVersion(stream io.Reader) (string, error) {
|
||||
version := make([]byte, protocolVersionLength)
|
||||
_, err := stream.Read(version)
|
||||
return string(version), err
|
||||
}
|
||||
|
||||
func readSignature(stream io.Reader) (protocolSignature, error) {
|
||||
var signature protocolSignature
|
||||
if _, err := io.ReadFull(stream, signature[:]); err != nil {
|
||||
return protocolSignature{}, err
|
||||
}
|
||||
return signature, nil
|
||||
}
|
||||
|
||||
func writeSignature(stream io.Writer, signature protocolSignature) error {
|
||||
_, err := stream.Write(signature[:])
|
||||
return err
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
|
||||
capnp "zombiezen.com/go/capnproto2"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
// RequestClientStream is a stream to provide requests to the server. This operation is typically driven by the edge service.
|
||||
type RequestClientStream struct {
|
||||
io.ReadWriteCloser
|
||||
}
|
||||
|
||||
// WriteConnectRequestData writes requestMeta to a stream.
|
||||
func (rcs *RequestClientStream) WriteConnectRequestData(dest string, connectionType pogs.ConnectionType, metadata ...pogs.Metadata) error {
|
||||
connectRequest := &pogs.ConnectRequest{
|
||||
Dest: dest,
|
||||
Type: connectionType,
|
||||
Metadata: metadata,
|
||||
}
|
||||
|
||||
msg, err := connectRequest.ToPogs()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := writeDataStreamPreamble(rcs); err != nil {
|
||||
return err
|
||||
}
|
||||
return capnp.NewEncoder(rcs).Encode(msg)
|
||||
}
|
||||
|
||||
// ReadConnectResponseData reads the response from the rpc stream to a ConnectResponse.
|
||||
func (rcs *RequestClientStream) ReadConnectResponseData() (*pogs.ConnectResponse, error) {
|
||||
signature, err := determineProtocol(rcs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if signature != dataStreamProtocolSignature {
|
||||
return nil, fmt.Errorf("wrong protocol signature %v", signature)
|
||||
}
|
||||
|
||||
// This is a NO-OP for now. We could cause a branching if we wanted to use multiple versions.
|
||||
if _, err := readVersion(rcs); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
msg, err := capnp.NewDecoder(rcs).Decode()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
r := &pogs.ConnectResponse{}
|
||||
if err := r.FromPogs(msg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return r, nil
|
||||
}
|
|
@ -0,0 +1,57 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"io"
|
||||
|
||||
capnp "zombiezen.com/go/capnproto2"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
// RequestServerStream is a stream to serve requests
|
||||
type RequestServerStream struct {
|
||||
io.ReadWriteCloser
|
||||
}
|
||||
|
||||
// ReadConnectRequestData reads the handshake data from a QUIC stream.
|
||||
func (rss *RequestServerStream) ReadConnectRequestData() (*pogs.ConnectRequest, error) {
|
||||
// This is a NO-OP for now. We could cause a branching if we wanted to use multiple versions.
|
||||
if _, err := readVersion(rss); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
msg, err := capnp.NewDecoder(rss).Decode()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
r := &pogs.ConnectRequest{}
|
||||
if err := r.FromPogs(msg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return r, nil
|
||||
}
|
||||
|
||||
// WriteConnectResponseData writes response to a QUIC stream.
|
||||
func (rss *RequestServerStream) WriteConnectResponseData(respErr error, metadata ...pogs.Metadata) error {
|
||||
var connectResponse *pogs.ConnectResponse
|
||||
if respErr != nil {
|
||||
connectResponse = &pogs.ConnectResponse{
|
||||
Error: respErr.Error(),
|
||||
}
|
||||
} else {
|
||||
connectResponse = &pogs.ConnectResponse{
|
||||
Metadata: metadata,
|
||||
}
|
||||
}
|
||||
|
||||
msg, err := connectResponse.ToPogs()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := writeDataStreamPreamble(rss); err != nil {
|
||||
return err
|
||||
}
|
||||
return capnp.NewEncoder(rss).Encode(msg)
|
||||
}
|
|
@ -11,11 +11,10 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/rs/zerolog"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
tunnelpogs "github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -26,14 +25,14 @@ func TestConnectRequestData(t *testing.T) {
|
|||
var tests = []struct {
|
||||
name string
|
||||
hostname string
|
||||
connectionType ConnectionType
|
||||
metadata []Metadata
|
||||
connectionType pogs.ConnectionType
|
||||
metadata []pogs.Metadata
|
||||
}{
|
||||
{
|
||||
name: "Signature verified and request metadata is unmarshaled and read correctly",
|
||||
hostname: "tunnel.com",
|
||||
connectionType: ConnectionTypeHTTP,
|
||||
metadata: []Metadata{
|
||||
connectionType: pogs.ConnectionTypeHTTP,
|
||||
metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "key",
|
||||
Val: "1234",
|
||||
|
@ -47,10 +46,10 @@ func TestConnectRequestData(t *testing.T) {
|
|||
reqClientStream := RequestClientStream{noopCloser{b}}
|
||||
err := reqClientStream.WriteConnectRequestData(test.hostname, test.connectionType, test.metadata...)
|
||||
require.NoError(t, err)
|
||||
protocol, err := DetermineProtocol(b)
|
||||
require.NoError(t, err)
|
||||
reqServerStream, err := NewRequestServerStream(noopCloser{b}, protocol)
|
||||
protocol, err := determineProtocol(b)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, dataStreamProtocolSignature, protocol)
|
||||
reqServerStream := RequestServerStream{&noopCloser{b}}
|
||||
|
||||
reqMeta, err := reqServerStream.ReadConnectRequestData()
|
||||
require.NoError(t, err)
|
||||
|
@ -66,11 +65,11 @@ func TestConnectResponseMeta(t *testing.T) {
|
|||
var tests = []struct {
|
||||
name string
|
||||
err error
|
||||
metadata []Metadata
|
||||
metadata []pogs.Metadata
|
||||
}{
|
||||
{
|
||||
name: "Signature verified and response metadata is unmarshaled and read correctly",
|
||||
metadata: []Metadata{
|
||||
metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "key",
|
||||
Val: "1234",
|
||||
|
@ -80,7 +79,7 @@ func TestConnectResponseMeta(t *testing.T) {
|
|||
{
|
||||
name: "If error is not empty, other fields should be blank",
|
||||
err: errors.New("something happened"),
|
||||
metadata: []Metadata{
|
||||
metadata: []pogs.Metadata{
|
||||
{
|
||||
Key: "key",
|
||||
Val: "1234",
|
||||
|
@ -142,22 +141,18 @@ func TestRegisterUdpSession(t *testing.T) {
|
|||
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
logger := zerolog.Nop()
|
||||
clientStream, serverStream := newMockRPCStreams()
|
||||
sessionRegisteredChan := make(chan struct{})
|
||||
go func() {
|
||||
protocol, err := DetermineProtocol(serverStream)
|
||||
assert.NoError(t, err)
|
||||
rpcServerStream, err := NewRPCServerStream(serverStream, protocol)
|
||||
assert.NoError(t, err)
|
||||
err = rpcServerStream.Serve(test.sessionRPCServer, nil, &logger)
|
||||
ss := NewCloudflaredServer(nil, test.sessionRPCServer, nil, 10*time.Second)
|
||||
err := ss.Serve(context.Background(), serverStream)
|
||||
assert.NoError(t, err)
|
||||
|
||||
serverStream.Close()
|
||||
close(sessionRegisteredChan)
|
||||
}()
|
||||
|
||||
rpcClientStream, err := NewRPCClientStream(context.Background(), clientStream, 5*time.Second, &logger)
|
||||
rpcClientStream, err := NewCloudflaredClient(context.Background(), clientStream, 5*time.Second)
|
||||
assert.NoError(t, err)
|
||||
|
||||
reg, err := rpcClientStream.RegisterUdpSession(context.Background(), test.sessionRPCServer.sessionID, test.sessionRPCServer.dstIP, test.sessionRPCServer.dstPort, testCloseIdleAfterHint, test.sessionRPCServer.traceContext)
|
||||
|
@ -192,14 +187,10 @@ func TestManageConfiguration(t *testing.T) {
|
|||
config: config,
|
||||
}
|
||||
|
||||
logger := zerolog.Nop()
|
||||
updatedChan := make(chan struct{})
|
||||
go func() {
|
||||
protocol, err := DetermineProtocol(serverStream)
|
||||
assert.NoError(t, err)
|
||||
rpcServerStream, err := NewRPCServerStream(serverStream, protocol)
|
||||
assert.NoError(t, err)
|
||||
err = rpcServerStream.Serve(nil, configRPCServer, &logger)
|
||||
server := NewCloudflaredServer(nil, nil, configRPCServer, 10*time.Second)
|
||||
err := server.Serve(context.Background(), serverStream)
|
||||
assert.NoError(t, err)
|
||||
|
||||
serverStream.Close()
|
||||
|
@ -208,7 +199,7 @@ func TestManageConfiguration(t *testing.T) {
|
|||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
|
||||
defer cancel()
|
||||
rpcClientStream, err := NewRPCClientStream(ctx, clientStream, 5*time.Second, &logger)
|
||||
rpcClientStream, err := NewCloudflaredClient(ctx, clientStream, 5*time.Second)
|
||||
assert.NoError(t, err)
|
||||
|
||||
result, err := rpcClientStream.UpdateConfiguration(ctx, version, config)
|
||||
|
@ -230,7 +221,7 @@ type mockSessionRPCServer struct {
|
|||
traceContext string
|
||||
}
|
||||
|
||||
func (s mockSessionRPCServer) RegisterUdpSession(_ context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeIdleAfter time.Duration, traceContext string) (*tunnelpogs.RegisterUdpSessionResponse, error) {
|
||||
func (s mockSessionRPCServer) RegisterUdpSession(_ context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeIdleAfter time.Duration, traceContext string) (*pogs.RegisterUdpSessionResponse, error) {
|
||||
if s.sessionID != sessionID {
|
||||
return nil, fmt.Errorf("expect session ID %s, got %s", s.sessionID, sessionID)
|
||||
}
|
||||
|
@ -246,7 +237,7 @@ func (s mockSessionRPCServer) RegisterUdpSession(_ context.Context, sessionID uu
|
|||
if s.traceContext != traceContext {
|
||||
return nil, fmt.Errorf("expect traceContext %s, got %s", s.traceContext, traceContext)
|
||||
}
|
||||
return &tunnelpogs.RegisterUdpSessionResponse{}, nil
|
||||
return &pogs.RegisterUdpSessionResponse{}, nil
|
||||
}
|
||||
|
||||
func (s mockSessionRPCServer) UnregisterUdpSession(_ context.Context, sessionID uuid.UUID, message string) error {
|
||||
|
@ -264,18 +255,18 @@ type mockConfigRPCServer struct {
|
|||
config []byte
|
||||
}
|
||||
|
||||
func (s mockConfigRPCServer) UpdateConfiguration(_ context.Context, version int32, config []byte) *tunnelpogs.UpdateConfigurationResponse {
|
||||
func (s mockConfigRPCServer) UpdateConfiguration(_ context.Context, version int32, config []byte) *pogs.UpdateConfigurationResponse {
|
||||
if s.version != version {
|
||||
return &tunnelpogs.UpdateConfigurationResponse{
|
||||
return &pogs.UpdateConfigurationResponse{
|
||||
Err: fmt.Errorf("expect version %d, got %d", s.version, version),
|
||||
}
|
||||
}
|
||||
if !bytes.Equal(s.config, config) {
|
||||
return &tunnelpogs.UpdateConfigurationResponse{
|
||||
return &pogs.UpdateConfigurationResponse{
|
||||
Err: fmt.Errorf("expect config %v, got %v", s.config, config),
|
||||
}
|
||||
}
|
||||
return &tunnelpogs.UpdateConfigurationResponse{LastAppliedVersion: version}
|
||||
return &pogs.UpdateConfigurationResponse{LastAppliedVersion: version}
|
||||
}
|
||||
|
||||
type mockRPCStream struct {
|
|
@ -0,0 +1,55 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
// SessionClient calls capnp rpc methods of SessionManager.
|
||||
type SessionClient struct {
|
||||
client pogs.SessionManager_PogsClient
|
||||
transport rpc.Transport
|
||||
requestTimeout time.Duration
|
||||
}
|
||||
|
||||
func NewSessionClient(ctx context.Context, stream io.ReadWriteCloser, requestTimeout time.Duration) (*SessionClient, error) {
|
||||
n, err := stream.Write(rpcStreamProtocolSignature[:])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if n != len(rpcStreamProtocolSignature) {
|
||||
return nil, fmt.Errorf("expect to write %d bytes for RPC stream protocol signature, wrote %d", len(rpcStreamProtocolSignature), n)
|
||||
}
|
||||
transport := rpc.StreamTransport(stream)
|
||||
conn := rpc.NewConn(transport)
|
||||
return &SessionClient{
|
||||
client: pogs.NewSessionManager_PogsClient(conn.Bootstrap(ctx), conn),
|
||||
transport: transport,
|
||||
requestTimeout: requestTimeout,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (c *SessionClient) RegisterUdpSession(ctx context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeIdleAfterHint time.Duration, traceContext string) (*pogs.RegisterUdpSessionResponse, error) {
|
||||
ctx, cancel := context.WithTimeout(ctx, c.requestTimeout)
|
||||
defer cancel()
|
||||
return c.client.RegisterUdpSession(ctx, sessionID, dstIP, dstPort, closeIdleAfterHint, traceContext)
|
||||
}
|
||||
|
||||
func (c *SessionClient) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID, message string) error {
|
||||
ctx, cancel := context.WithTimeout(ctx, c.requestTimeout)
|
||||
defer cancel()
|
||||
return c.client.UnregisterUdpSession(ctx, sessionID, message)
|
||||
}
|
||||
|
||||
func (c *SessionClient) Close() {
|
||||
_ = c.client.Close()
|
||||
_ = c.transport.Close()
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"zombiezen.com/go/capnproto2/rpc"
|
||||
|
||||
"github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
// SessionManagerServer handles streams with the SessionManager RPCs.
|
||||
type SessionManagerServer struct {
|
||||
sessionManager pogs.SessionManager
|
||||
responseTimeout time.Duration
|
||||
}
|
||||
|
||||
func NewSessionManagerServer(sessionManager pogs.SessionManager, responseTimeout time.Duration) *SessionManagerServer {
|
||||
return &SessionManagerServer{
|
||||
sessionManager: sessionManager,
|
||||
responseTimeout: responseTimeout,
|
||||
}
|
||||
}
|
||||
|
||||
func (s *SessionManagerServer) Serve(ctx context.Context, stream io.ReadWriteCloser) error {
|
||||
signature, err := determineProtocol(stream)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
switch signature {
|
||||
case rpcStreamProtocolSignature:
|
||||
break
|
||||
case dataStreamProtocolSignature:
|
||||
return errDataStreamNotSupported
|
||||
default:
|
||||
return fmt.Errorf("unknown protocol %v", signature)
|
||||
}
|
||||
|
||||
// Every new quic.Stream request aligns to a new RPC request, this is why there is a timeout for the server-side
|
||||
// of the RPC request.
|
||||
ctx, cancel := context.WithTimeout(ctx, s.responseTimeout)
|
||||
defer cancel()
|
||||
|
||||
transport := rpc.StreamTransport(stream)
|
||||
defer transport.Close()
|
||||
|
||||
main := pogs.SessionManager_ServerToClient(s.sessionManager)
|
||||
rpcConn := rpc.NewConn(transport, rpc.MainInterface(main.Client))
|
||||
defer rpcConn.Close()
|
||||
|
||||
select {
|
||||
case <-rpcConn.Done():
|
||||
return rpcConn.Err()
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue