TUN-5494: Send a RPC with terminate reason to edge if the session is closed locally
This commit is contained in:
parent
70e675f42c
commit
ebae7a7024
|
@ -168,6 +168,7 @@ func (q *QUICConnection) handleRPCStream(rpcStream *quicpogs.RPCServerStream) er
|
|||
return rpcStream.Serve(q, q.logger)
|
||||
}
|
||||
|
||||
// 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) error {
|
||||
// Each session is a series of datagram from an eyeball to a dstIP:dstPort.
|
||||
// (src port, dst IP, dst port) uniquely identifies a session, so it needs a dedicated connected socket.
|
||||
|
@ -178,22 +179,60 @@ func (q *QUICConnection) RegisterUdpSession(ctx context.Context, sessionID uuid.
|
|||
}
|
||||
session, err := q.sessionManager.RegisterSession(ctx, sessionID, originProxy)
|
||||
if err != nil {
|
||||
q.logger.Err(err).Msgf("Failed to register udp session %s", sessionID)
|
||||
q.logger.Err(err).Str("sessionID", sessionID.String()).Msgf("Failed to register udp session")
|
||||
return err
|
||||
}
|
||||
go func() {
|
||||
defer q.sessionManager.UnregisterSession(q.session.Context(), sessionID)
|
||||
if err := session.Serve(q.session.Context(), closeAfterIdleHint); err != nil {
|
||||
q.logger.Debug().Err(err).Str("sessionID", sessionID.String()).Msg("session terminated")
|
||||
}
|
||||
}()
|
||||
|
||||
go q.serveUDPSession(session, closeAfterIdleHint)
|
||||
|
||||
q.logger.Debug().Msgf("Registered session %v, %v, %v", sessionID, dstIP, dstPort)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (q *QUICConnection) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID) error {
|
||||
q.sessionManager.UnregisterSession(ctx, sessionID)
|
||||
return nil
|
||||
func (q *QUICConnection) serveUDPSession(session *datagramsession.Session, closeAfterIdleHint time.Duration) {
|
||||
ctx := q.session.Context()
|
||||
closedByRemote, err := session.Serve(ctx, closeAfterIdleHint)
|
||||
// If session is terminated by remote, then we know it has been unregistered from session manager and edge
|
||||
if !closedByRemote {
|
||||
if err != nil {
|
||||
q.closeUDPSession(ctx, session.ID, err.Error())
|
||||
} else {
|
||||
q.closeUDPSession(ctx, session.ID, "terminated without error")
|
||||
}
|
||||
q.logger.Debug().Err(err).Str("sessionID", session.ID.String()).Msg("session terminated")
|
||||
return
|
||||
}
|
||||
q.logger.Debug().Err(err).Msg("Session terminated by edge")
|
||||
}
|
||||
|
||||
// closeUDPSession first unregisters the session from session manager, then it tries to unregister from edge
|
||||
func (q *QUICConnection) closeUDPSession(ctx context.Context, sessionID uuid.UUID, message string) {
|
||||
q.sessionManager.UnregisterSession(ctx, sessionID, message, false)
|
||||
stream, err := q.session.OpenStream()
|
||||
if err != nil {
|
||||
// Log this at debug because this is not an error if session was closed due to lost connection
|
||||
// with edge
|
||||
q.logger.Debug().Err(err).Str("sessionID", sessionID.String()).
|
||||
Msgf("Failed to open quic stream to unregister udp session with edge")
|
||||
return
|
||||
}
|
||||
rpcClientStream, err := quicpogs.NewRPCClientStream(ctx, stream, q.logger)
|
||||
if err != nil {
|
||||
// Log this at debug because this is not an error if session was closed due to lost connection
|
||||
// with edge
|
||||
q.logger.Err(err).Str("sessionID", sessionID.String()).
|
||||
Msgf("Failed to open rpc stream to unregister udp session with edge")
|
||||
return
|
||||
}
|
||||
if err := rpcClientStream.UnregisterUdpSession(ctx, sessionID, message); err != nil {
|
||||
q.logger.Err(err).Str("sessionID", sessionID.String()).
|
||||
Msgf("Failed to unregister udp session with edge")
|
||||
}
|
||||
}
|
||||
|
||||
// UnregisterUdpSession is the RPC method invoked by edge to unregister and terminate a sesssion
|
||||
func (q *QUICConnection) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID, message string) error {
|
||||
return q.sessionManager.UnregisterSession(ctx, sessionID, message, true)
|
||||
}
|
||||
|
||||
// streamReadWriteAcker is a light wrapper over QUIC streams with a callback to send response back to
|
||||
|
|
|
@ -17,29 +17,32 @@ import (
|
|||
"os"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/gobwas/ws/wsutil"
|
||||
"github.com/google/uuid"
|
||||
"github.com/lucas-clemente/quic-go"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/rs/zerolog"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/cloudflare/cloudflared/datagramsession"
|
||||
quicpogs "github.com/cloudflare/cloudflared/quic"
|
||||
tunnelpogs "github.com/cloudflare/cloudflared/tunnelrpc/pogs"
|
||||
)
|
||||
|
||||
var (
|
||||
testTLSServerConfig = generateTLSConfig()
|
||||
testQUICConfig = &quic.Config{
|
||||
KeepAlive: true,
|
||||
EnableDatagrams: true,
|
||||
}
|
||||
)
|
||||
|
||||
// TestQUICServer tests if a quic server accepts and responds to a quic client with the acceptance protocol.
|
||||
// It also serves as a demonstration for communication with the QUIC connection started by a cloudflared.
|
||||
func TestQUICServer(t *testing.T) {
|
||||
quicConfig := &quic.Config{
|
||||
KeepAlive: true,
|
||||
EnableDatagrams: true,
|
||||
}
|
||||
|
||||
// Setup test.
|
||||
log := zerolog.New(os.Stdout)
|
||||
|
||||
// Start a UDP Listener for QUIC.
|
||||
udpAddr, err := net.ResolveUDPAddr("udp", "127.0.0.1:0")
|
||||
require.NoError(t, err)
|
||||
|
@ -47,18 +50,6 @@ func TestQUICServer(t *testing.T) {
|
|||
require.NoError(t, err)
|
||||
defer udpListener.Close()
|
||||
|
||||
// Create a simple tls config.
|
||||
tlsConfig := generateTLSConfig()
|
||||
|
||||
// Create a client config
|
||||
tlsClientConfig := &tls.Config{
|
||||
InsecureSkipVerify: true,
|
||||
NextProtos: []string{"argotunnel"},
|
||||
}
|
||||
|
||||
// Start a mock httpProxy
|
||||
originProxy := &mockOriginProxyWithRequest{}
|
||||
|
||||
// This is simply a sample websocket frame message.
|
||||
wsBuf := &bytes.Buffer{}
|
||||
wsutil.WriteClientText(wsBuf, []byte("Hello"))
|
||||
|
@ -158,25 +149,13 @@ func TestQUICServer(t *testing.T) {
|
|||
go func() {
|
||||
defer wg.Done()
|
||||
quicServer(
|
||||
t, udpListener, tlsConfig, quicConfig,
|
||||
t, udpListener, testTLSServerConfig, testQUICConfig,
|
||||
test.dest, test.connectionType, test.metadata, test.message, test.expectedResponse,
|
||||
)
|
||||
}()
|
||||
|
||||
controlStream := fakeControlStream{}
|
||||
|
||||
qC, err := NewQUICConnection(
|
||||
ctx,
|
||||
quicConfig,
|
||||
udpListener.LocalAddr(),
|
||||
tlsClientConfig,
|
||||
originProxy,
|
||||
&tunnelpogs.ConnectionOptions{},
|
||||
controlStream,
|
||||
NewObserver(&log, &log, false),
|
||||
)
|
||||
require.NoError(t, err)
|
||||
go qC.Serve(ctx)
|
||||
qc := testQUICConnection(ctx, udpListener.LocalAddr(), t)
|
||||
go qc.Serve(ctx)
|
||||
|
||||
wg.Wait()
|
||||
cancel()
|
||||
|
@ -531,3 +510,159 @@ func (moc *mockOriginProxyWithRequest) ProxyTCP(ctx context.Context, rwa ReadWri
|
|||
io.Copy(rwa, rwa)
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestServeUDPSession(t *testing.T) {
|
||||
// Start a UDP Listener for QUIC.
|
||||
udpAddr, err := net.ResolveUDPAddr("udp", "127.0.0.1:0")
|
||||
require.NoError(t, err)
|
||||
udpListener, err := net.ListenUDP(udpAddr.Network(), udpAddr)
|
||||
require.NoError(t, err)
|
||||
defer udpListener.Close()
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
// Establish QUIC connection with edge
|
||||
edgeQUICSessionChan := make(chan quic.Session)
|
||||
go func() {
|
||||
earlyListener, err := quic.Listen(udpListener, testTLSServerConfig, testQUICConfig)
|
||||
require.NoError(t, err)
|
||||
|
||||
edgeQUICSession, err := earlyListener.Accept(ctx)
|
||||
require.NoError(t, err)
|
||||
edgeQUICSessionChan <- edgeQUICSession
|
||||
}()
|
||||
|
||||
qc := testQUICConnection(ctx, udpListener.LocalAddr(), t)
|
||||
go qc.Serve(ctx)
|
||||
|
||||
edgeQUICSession := <-edgeQUICSessionChan
|
||||
serveSession(ctx, qc, edgeQUICSession, closedByOrigin, io.EOF.Error(), t)
|
||||
serveSession(ctx, qc, edgeQUICSession, closedByTimeout, datagramsession.SessionIdleErr(time.Millisecond*50).Error(), t)
|
||||
serveSession(ctx, qc, edgeQUICSession, closedByRemote, "eyeball closed connection", t)
|
||||
cancel()
|
||||
}
|
||||
|
||||
func serveSession(ctx context.Context, qc *QUICConnection, edgeQUICSession quic.Session, closeType closeReason, expectedReason string, t *testing.T) {
|
||||
var (
|
||||
payload = []byte(t.Name())
|
||||
)
|
||||
sessionID := uuid.New()
|
||||
cfdConn, originConn := net.Pipe()
|
||||
// Registers and run a new session
|
||||
session, err := qc.sessionManager.RegisterSession(ctx, sessionID, cfdConn)
|
||||
require.NoError(t, err)
|
||||
|
||||
sessionDone := make(chan struct{})
|
||||
go func() {
|
||||
qc.serveUDPSession(session, time.Millisecond*50)
|
||||
close(sessionDone)
|
||||
}()
|
||||
|
||||
// Send a message to the quic session on edge side, it should be deumx to this datagram session
|
||||
muxedPayload, err := quicpogs.SuffixSessionID(sessionID, payload)
|
||||
require.NoError(t, err)
|
||||
err = edgeQUICSession.SendMessage(muxedPayload)
|
||||
require.NoError(t, err)
|
||||
|
||||
readBuffer := make([]byte, len(payload)+1)
|
||||
n, err := originConn.Read(readBuffer)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, len(payload), n)
|
||||
require.True(t, bytes.Equal(payload, readBuffer[:n]))
|
||||
|
||||
// Close connection to terminate session
|
||||
switch closeType {
|
||||
case closedByOrigin:
|
||||
originConn.Close()
|
||||
case closedByRemote:
|
||||
err = qc.UnregisterUdpSession(ctx, sessionID, expectedReason)
|
||||
require.NoError(t, err)
|
||||
case closedByTimeout:
|
||||
}
|
||||
|
||||
if closeType != closedByRemote {
|
||||
// Session was not closed by remote, so closeUDPSession should be invoked to unregister from remote
|
||||
unregisterFromEdgeChan := make(chan struct{})
|
||||
rpcServer := &mockSessionRPCServer{
|
||||
sessionID: sessionID,
|
||||
unregisterReason: expectedReason,
|
||||
calledUnregisterChan: unregisterFromEdgeChan,
|
||||
}
|
||||
go runMockSessionRPCServer(ctx, edgeQUICSession, rpcServer, t)
|
||||
|
||||
<-unregisterFromEdgeChan
|
||||
}
|
||||
|
||||
<-sessionDone
|
||||
}
|
||||
|
||||
type closeReason uint8
|
||||
|
||||
const (
|
||||
closedByOrigin closeReason = iota
|
||||
closedByRemote
|
||||
closedByTimeout
|
||||
)
|
||||
|
||||
func runMockSessionRPCServer(ctx context.Context, session quic.Session, rpcServer *mockSessionRPCServer, t *testing.T) {
|
||||
stream, err := session.AcceptStream(ctx)
|
||||
require.NoError(t, err)
|
||||
|
||||
if stream.StreamID() == 0 {
|
||||
// Skip the first stream, it's the control stream of the QUIC connection
|
||||
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(rpcServer, &log)
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
|
||||
type mockSessionRPCServer struct {
|
||||
sessionID uuid.UUID
|
||||
unregisterReason string
|
||||
calledUnregisterChan chan struct{}
|
||||
}
|
||||
|
||||
func (s mockSessionRPCServer) RegisterUdpSession(ctx context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeIdleAfter time.Duration) error {
|
||||
return fmt.Errorf("mockSessionRPCServer doesn't implement RegisterUdpSession")
|
||||
}
|
||||
|
||||
func (s mockSessionRPCServer) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID, reason string) error {
|
||||
if s.sessionID != sessionID {
|
||||
return fmt.Errorf("expect session ID %s, got %s", s.sessionID, sessionID)
|
||||
}
|
||||
if s.unregisterReason != reason {
|
||||
return fmt.Errorf("expect unregister reason %s, got %s", s.unregisterReason, reason)
|
||||
}
|
||||
close(s.calledUnregisterChan)
|
||||
fmt.Println("unregister from edge")
|
||||
return nil
|
||||
}
|
||||
|
||||
func testQUICConnection(ctx context.Context, udpListenerAddr net.Addr, t *testing.T) *QUICConnection {
|
||||
tlsClientConfig := &tls.Config{
|
||||
InsecureSkipVerify: true,
|
||||
NextProtos: []string{"argotunnel"},
|
||||
}
|
||||
// Start a mock httpProxy
|
||||
originProxy := &mockOriginProxyWithRequest{}
|
||||
log := zerolog.New(os.Stdout)
|
||||
qc, err := NewQUICConnection(
|
||||
ctx,
|
||||
testQUICConfig,
|
||||
udpListenerAddr,
|
||||
tlsClientConfig,
|
||||
originProxy,
|
||||
&tunnelpogs.ConnectionOptions{},
|
||||
fakeControlStream{},
|
||||
NewObserver(&log, &log, false),
|
||||
)
|
||||
require.NoError(t, err)
|
||||
return qc
|
||||
}
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package datagramsession
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
|
||||
"github.com/google/uuid"
|
||||
|
@ -24,6 +25,22 @@ func newRegisterSessionEvent(sessionID uuid.UUID, originProxy io.ReadWriteCloser
|
|||
// unregisterSessionEvent is an event to stop tracking and terminate the session.
|
||||
type unregisterSessionEvent struct {
|
||||
sessionID uuid.UUID
|
||||
err *errClosedSession
|
||||
}
|
||||
|
||||
// ClosedSessionError represent a condition that closes the session other than I/O
|
||||
// I/O error is not included, because the side that closes the session is ambiguous.
|
||||
type errClosedSession struct {
|
||||
message string
|
||||
byRemote bool
|
||||
}
|
||||
|
||||
func (sc *errClosedSession) Error() string {
|
||||
if sc.byRemote {
|
||||
return fmt.Sprintf("session closed by remote due to %s", sc.message)
|
||||
} else {
|
||||
return fmt.Sprintf("session closed by local due to %s", sc.message)
|
||||
}
|
||||
}
|
||||
|
||||
// newDatagram is an event when transport receives new datagram
|
||||
|
|
|
@ -20,7 +20,7 @@ type Manager interface {
|
|||
// RegisterSession starts tracking a session. Caller is responsible for starting the session
|
||||
RegisterSession(ctx context.Context, sessionID uuid.UUID, dstConn io.ReadWriteCloser) (*Session, error)
|
||||
// UnregisterSession stops tracking the session and terminates it
|
||||
UnregisterSession(ctx context.Context, sessionID uuid.UUID) error
|
||||
UnregisterSession(ctx context.Context, sessionID uuid.UUID, message string, byRemote bool) error
|
||||
}
|
||||
|
||||
type manager struct {
|
||||
|
@ -100,8 +100,14 @@ func (m *manager) registerSession(ctx context.Context, registration *registerSes
|
|||
registration.resultChan <- session
|
||||
}
|
||||
|
||||
func (m *manager) UnregisterSession(ctx context.Context, sessionID uuid.UUID) error {
|
||||
event := &unregisterSessionEvent{sessionID: sessionID}
|
||||
func (m *manager) UnregisterSession(ctx context.Context, sessionID uuid.UUID, message string, byRemote bool) error {
|
||||
event := &unregisterSessionEvent{
|
||||
sessionID: sessionID,
|
||||
err: &errClosedSession{
|
||||
message: message,
|
||||
byRemote: byRemote,
|
||||
},
|
||||
}
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
|
@ -114,7 +120,7 @@ func (m *manager) unregisterSession(unregistration *unregisterSessionEvent) {
|
|||
session, ok := m.sessions[unregistration.sessionID]
|
||||
if ok {
|
||||
delete(m.sessions, unregistration.sessionID)
|
||||
session.close()
|
||||
session.close(unregistration.err)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ func TestManagerServe(t *testing.T) {
|
|||
const (
|
||||
sessions = 20
|
||||
msgs = 50
|
||||
remoteUnregisterMsg = "eyeball closed connection"
|
||||
)
|
||||
log := zerolog.Nop()
|
||||
transport := &mockQUICTransport{
|
||||
|
@ -89,7 +90,13 @@ func TestManagerServe(t *testing.T) {
|
|||
|
||||
sessionDone := make(chan struct{})
|
||||
go func() {
|
||||
session.Serve(ctx, time.Minute*2)
|
||||
closedByRemote, err := session.Serve(ctx, time.Minute*2)
|
||||
closeSession := &errClosedSession{
|
||||
message: remoteUnregisterMsg,
|
||||
byRemote: true,
|
||||
}
|
||||
require.Equal(t, closeSession, err)
|
||||
require.True(t, closedByRemote)
|
||||
close(sessionDone)
|
||||
}()
|
||||
|
||||
|
@ -100,7 +107,7 @@ func TestManagerServe(t *testing.T) {
|
|||
// Make sure eyeball and origin have received all messages before unregistering the session
|
||||
require.NoError(t, reqErrGroup.Wait())
|
||||
|
||||
require.NoError(t, mg.UnregisterSession(ctx, sessionID))
|
||||
require.NoError(t, mg.UnregisterSession(ctx, sessionID, remoteUnregisterMsg, true))
|
||||
<-sessionDone
|
||||
|
||||
return nil
|
||||
|
|
|
@ -2,6 +2,7 @@ package datagramsession
|
|||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
|
@ -12,6 +13,10 @@ const (
|
|||
defaultCloseIdleAfter = time.Second * 210
|
||||
)
|
||||
|
||||
func SessionIdleErr(timeout time.Duration) error {
|
||||
return fmt.Errorf("session idle for %v", timeout)
|
||||
}
|
||||
|
||||
// Each Session is a bidirectional pipe of datagrams between transport and dstConn
|
||||
// Currently the only implementation of transport is quic DatagramMuxer
|
||||
// Destination can be a connection with origin or with eyeball
|
||||
|
@ -24,47 +29,53 @@ const (
|
|||
// - Datagrams from cloudflared are read by Manager from the transport. Manager finds the corresponding Session and calls the
|
||||
// write method of the Session to send to eyeball
|
||||
type Session struct {
|
||||
id uuid.UUID
|
||||
ID uuid.UUID
|
||||
transport transport
|
||||
dstConn io.ReadWriteCloser
|
||||
// activeAtChan is used to communicate the last read/write time
|
||||
activeAtChan chan time.Time
|
||||
doneChan chan struct{}
|
||||
closeChan chan error
|
||||
}
|
||||
|
||||
func newSession(id uuid.UUID, transport transport, dstConn io.ReadWriteCloser) *Session {
|
||||
return &Session{
|
||||
id: id,
|
||||
ID: id,
|
||||
transport: transport,
|
||||
dstConn: dstConn,
|
||||
// activeAtChan has low capacity. It can be full when there are many concurrent read/write. markActive() will
|
||||
// drop instead of blocking because last active time only needs to be an approximation
|
||||
activeAtChan: make(chan time.Time, 2),
|
||||
doneChan: make(chan struct{}),
|
||||
// capacity is 2 because close() and dstToTransport routine in Serve() can write to this channel
|
||||
closeChan: make(chan error, 2),
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Session) Serve(ctx context.Context, closeAfterIdle time.Duration) error {
|
||||
serveCtx, cancel := context.WithCancel(ctx)
|
||||
defer cancel()
|
||||
go s.waitForCloseCondition(serveCtx, closeAfterIdle)
|
||||
func (s *Session) Serve(ctx context.Context, closeAfterIdle time.Duration) (closedByRemote bool, err error) {
|
||||
go func() {
|
||||
// QUIC implementation copies data to another buffer before returning https://github.com/lucas-clemente/quic-go/blob/v0.24.0/session.go#L1967-L1975
|
||||
// This makes it safe to share readBuffer between iterations
|
||||
readBuffer := make([]byte, s.transport.MTU())
|
||||
for {
|
||||
if err := s.dstToTransport(readBuffer); err != nil {
|
||||
return err
|
||||
s.closeChan <- err
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
err = s.waitForCloseCondition(ctx, closeAfterIdle)
|
||||
if closeSession, ok := err.(*errClosedSession); ok {
|
||||
closedByRemote = closeSession.byRemote
|
||||
}
|
||||
return closedByRemote, err
|
||||
}
|
||||
|
||||
func (s *Session) waitForCloseCondition(ctx context.Context, closeAfterIdle time.Duration) {
|
||||
func (s *Session) waitForCloseCondition(ctx context.Context, closeAfterIdle time.Duration) error {
|
||||
// Closing dstConn cancels read so dstToTransport routine in Serve() can return
|
||||
defer s.dstConn.Close()
|
||||
if closeAfterIdle == 0 {
|
||||
// provide deafult is caller doesn't specify one
|
||||
closeAfterIdle = defaultCloseIdleAfter
|
||||
}
|
||||
// Closing dstConn cancels read so Serve function can return
|
||||
defer s.dstConn.Close()
|
||||
|
||||
checkIdleFreq := closeAfterIdle / 8
|
||||
checkIdleTicker := time.NewTicker(checkIdleFreq)
|
||||
|
@ -74,14 +85,14 @@ func (s *Session) waitForCloseCondition(ctx context.Context, closeAfterIdle time
|
|||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-s.doneChan:
|
||||
return
|
||||
return ctx.Err()
|
||||
case reason := <-s.closeChan:
|
||||
return reason
|
||||
// TODO: TUN-5423 evaluate if using atomic is more efficient
|
||||
case now := <-checkIdleTicker.C:
|
||||
// The session is considered inactive if current time is after (last active time + allowed idle time)
|
||||
if now.After(activeAt.Add(closeAfterIdle)) {
|
||||
return
|
||||
return SessionIdleErr(closeAfterIdle)
|
||||
}
|
||||
case activeAt = <-s.activeAtChan: // Update last active time
|
||||
}
|
||||
|
@ -92,7 +103,7 @@ func (s *Session) dstToTransport(buffer []byte) error {
|
|||
n, err := s.dstConn.Read(buffer)
|
||||
s.markActive()
|
||||
if n > 0 {
|
||||
if err := s.transport.SendTo(s.id, buffer[:n]); err != nil {
|
||||
if err := s.transport.SendTo(s.ID, buffer[:n]); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
@ -113,6 +124,6 @@ func (s *Session) markActive() {
|
|||
}
|
||||
}
|
||||
|
||||
func (s *Session) close() {
|
||||
close(s.doneChan)
|
||||
func (s *Session) close(err *errClosedSession) {
|
||||
s.closeChan <- err
|
||||
}
|
||||
|
|
|
@ -31,6 +31,12 @@ func TestCloseIdle(t *testing.T) {
|
|||
}
|
||||
|
||||
func testSessionReturns(t *testing.T, closeBy closeMethod, closeAfterIdle time.Duration) {
|
||||
var (
|
||||
localCloseReason = &errClosedSession{
|
||||
message: "connection closed by origin",
|
||||
byRemote: false,
|
||||
}
|
||||
)
|
||||
sessionID := uuid.New()
|
||||
cfdConn, originConn := net.Pipe()
|
||||
payload := testPayload(sessionID)
|
||||
|
@ -43,7 +49,18 @@ func testSessionReturns(t *testing.T, closeBy closeMethod, closeAfterIdle time.D
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
sessionDone := make(chan struct{})
|
||||
go func() {
|
||||
session.Serve(ctx, closeAfterIdle)
|
||||
closedByRemote, err := session.Serve(ctx, closeAfterIdle)
|
||||
switch closeBy {
|
||||
case closeByContext:
|
||||
require.Equal(t, context.Canceled, err)
|
||||
require.False(t, closedByRemote)
|
||||
case closeByCallingClose:
|
||||
require.Equal(t, localCloseReason, err)
|
||||
require.Equal(t, localCloseReason.byRemote, closedByRemote)
|
||||
case closeByTimeout:
|
||||
require.Equal(t, SessionIdleErr(closeAfterIdle), err)
|
||||
require.False(t, closedByRemote)
|
||||
}
|
||||
close(sessionDone)
|
||||
}()
|
||||
|
||||
|
@ -64,7 +81,7 @@ func testSessionReturns(t *testing.T, closeBy closeMethod, closeAfterIdle time.D
|
|||
case closeByContext:
|
||||
cancel()
|
||||
case closeByCallingClose:
|
||||
session.close()
|
||||
session.close(localCloseReason)
|
||||
}
|
||||
|
||||
<-sessionDone
|
||||
|
|
|
@ -251,8 +251,8 @@ func (rcs *RPCClientStream) RegisterUdpSession(ctx context.Context, sessionID uu
|
|||
return resp.Err
|
||||
}
|
||||
|
||||
func (rcs *RPCClientStream) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID) error {
|
||||
return rcs.client.UnregisterUdpSession(ctx, sessionID)
|
||||
func (rcs *RPCClientStream) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID, message string) error {
|
||||
return rcs.client.UnregisterUdpSession(ctx, sessionID, message)
|
||||
}
|
||||
|
||||
func (rcs *RPCClientStream) Close() {
|
||||
|
|
|
@ -114,11 +114,13 @@ func TestRegisterUdpSession(t *testing.T) {
|
|||
clientStream := mockRPCStream{clientReader, clientWriter}
|
||||
serverStream := mockRPCStream{serverReader, serverWriter}
|
||||
|
||||
unregisterMessage := "closed by eyeball"
|
||||
rpcServer := mockRPCServer{
|
||||
sessionID: uuid.New(),
|
||||
dstIP: net.IP{172, 16, 0, 1},
|
||||
dstPort: 8000,
|
||||
closeIdleAfter: testCloseIdleAfterHint,
|
||||
unregisterMessage: unregisterMessage,
|
||||
}
|
||||
logger := zerolog.Nop()
|
||||
sessionRegisteredChan := make(chan struct{})
|
||||
|
@ -142,10 +144,10 @@ func TestRegisterUdpSession(t *testing.T) {
|
|||
// Different sessionID, the RPC server should reject the registraion
|
||||
assert.Error(t, rpcClientStream.RegisterUdpSession(context.Background(), uuid.New(), rpcServer.dstIP, rpcServer.dstPort, testCloseIdleAfterHint))
|
||||
|
||||
assert.NoError(t, rpcClientStream.UnregisterUdpSession(context.Background(), rpcServer.sessionID))
|
||||
assert.NoError(t, rpcClientStream.UnregisterUdpSession(context.Background(), rpcServer.sessionID, unregisterMessage))
|
||||
|
||||
// Different sessionID, the RPC server should reject the unregistraion
|
||||
assert.Error(t, rpcClientStream.UnregisterUdpSession(context.Background(), uuid.New()))
|
||||
assert.Error(t, rpcClientStream.UnregisterUdpSession(context.Background(), uuid.New(), unregisterMessage))
|
||||
|
||||
rpcClientStream.Close()
|
||||
<-sessionRegisteredChan
|
||||
|
@ -156,6 +158,7 @@ type mockRPCServer struct {
|
|||
dstIP net.IP
|
||||
dstPort uint16
|
||||
closeIdleAfter time.Duration
|
||||
unregisterMessage string
|
||||
}
|
||||
|
||||
func (s mockRPCServer) RegisterUdpSession(ctx context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeIdleAfter time.Duration) error {
|
||||
|
@ -174,10 +177,13 @@ func (s mockRPCServer) RegisterUdpSession(ctx context.Context, sessionID uuid.UU
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s mockRPCServer) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID) error {
|
||||
func (s mockRPCServer) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID, message string) error {
|
||||
if s.sessionID != sessionID {
|
||||
return fmt.Errorf("expect session ID %s, got %s", s.sessionID, sessionID)
|
||||
}
|
||||
if s.unregisterMessage != message {
|
||||
return fmt.Errorf("expect unregister message %s, got %s", s.unregisterMessage, message)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -15,7 +15,7 @@ import (
|
|||
|
||||
type SessionManager interface {
|
||||
RegisterUdpSession(ctx context.Context, sessionID uuid.UUID, dstIP net.IP, dstPort uint16, closeAfterIdleHint time.Duration) error
|
||||
UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID) error
|
||||
UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID, message string) error
|
||||
}
|
||||
|
||||
type SessionManager_PogsImpl struct {
|
||||
|
@ -76,7 +76,12 @@ func (i SessionManager_PogsImpl) UnregisterUdpSession(p tunnelrpc.SessionManager
|
|||
return err
|
||||
}
|
||||
|
||||
return i.impl.UnregisterUdpSession(p.Ctx, sessionID)
|
||||
message, err := p.Params.Message()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return i.impl.UnregisterUdpSession(p.Ctx, sessionID, message)
|
||||
}
|
||||
|
||||
type RegisterUdpSessionResponse struct {
|
||||
|
@ -137,12 +142,15 @@ func (c SessionManager_PogsClient) RegisterUdpSession(ctx context.Context, sessi
|
|||
return response, nil
|
||||
}
|
||||
|
||||
func (c SessionManager_PogsClient) UnregisterUdpSession(ctx context.Context, sessionID uuid.UUID) error {
|
||||
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 {
|
||||
if err := p.SetSessionId(sessionID[:]); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := p.SetMessage(message); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
})
|
||||
_, err := promise.Struct()
|
||||
|
|
|
@ -150,5 +150,5 @@ struct RegisterUdpSessionResponse {
|
|||
interface SessionManager {
|
||||
# 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) -> (result :RegisterUdpSessionResponse);
|
||||
unregisterUdpSession @1 (sessionId :Data) -> ();
|
||||
unregisterUdpSession @1 (sessionId :Data, message: Text) -> ();
|
||||
}
|
|
@ -3485,7 +3485,7 @@ func (c SessionManager) UnregisterUdpSession(ctx context.Context, params func(Se
|
|||
Options: capnp.NewCallOptions(opts),
|
||||
}
|
||||
if params != nil {
|
||||
call.ParamsSize = capnp.ObjectSize{DataSize: 0, PointerCount: 1}
|
||||
call.ParamsSize = capnp.ObjectSize{DataSize: 0, PointerCount: 2}
|
||||
call.ParamsFunc = func(s capnp.Struct) error { return params(SessionManager_unregisterUdpSession_Params{Struct: s}) }
|
||||
}
|
||||
return SessionManager_unregisterUdpSession_Results_Promise{Pipeline: capnp.NewPipeline(c.Client.Call(call))}
|
||||
|
@ -3743,12 +3743,12 @@ type SessionManager_unregisterUdpSession_Params struct{ capnp.Struct }
|
|||
const SessionManager_unregisterUdpSession_Params_TypeID = 0x96b74375ce9b0ef6
|
||||
|
||||
func NewSessionManager_unregisterUdpSession_Params(s *capnp.Segment) (SessionManager_unregisterUdpSession_Params, error) {
|
||||
st, err := capnp.NewStruct(s, capnp.ObjectSize{DataSize: 0, PointerCount: 1})
|
||||
st, err := capnp.NewStruct(s, capnp.ObjectSize{DataSize: 0, PointerCount: 2})
|
||||
return SessionManager_unregisterUdpSession_Params{st}, err
|
||||
}
|
||||
|
||||
func NewRootSessionManager_unregisterUdpSession_Params(s *capnp.Segment) (SessionManager_unregisterUdpSession_Params, error) {
|
||||
st, err := capnp.NewRootStruct(s, capnp.ObjectSize{DataSize: 0, PointerCount: 1})
|
||||
st, err := capnp.NewRootStruct(s, capnp.ObjectSize{DataSize: 0, PointerCount: 2})
|
||||
return SessionManager_unregisterUdpSession_Params{st}, err
|
||||
}
|
||||
|
||||
|
@ -3776,12 +3776,31 @@ func (s SessionManager_unregisterUdpSession_Params) SetSessionId(v []byte) error
|
|||
return s.Struct.SetData(0, v)
|
||||
}
|
||||
|
||||
func (s SessionManager_unregisterUdpSession_Params) Message() (string, error) {
|
||||
p, err := s.Struct.Ptr(1)
|
||||
return p.Text(), err
|
||||
}
|
||||
|
||||
func (s SessionManager_unregisterUdpSession_Params) HasMessage() bool {
|
||||
p, err := s.Struct.Ptr(1)
|
||||
return p.IsValid() || err != nil
|
||||
}
|
||||
|
||||
func (s SessionManager_unregisterUdpSession_Params) MessageBytes() ([]byte, error) {
|
||||
p, err := s.Struct.Ptr(1)
|
||||
return p.TextBytes(), err
|
||||
}
|
||||
|
||||
func (s SessionManager_unregisterUdpSession_Params) SetMessage(v string) error {
|
||||
return s.Struct.SetText(1, v)
|
||||
}
|
||||
|
||||
// SessionManager_unregisterUdpSession_Params_List is a list of SessionManager_unregisterUdpSession_Params.
|
||||
type SessionManager_unregisterUdpSession_Params_List struct{ capnp.List }
|
||||
|
||||
// NewSessionManager_unregisterUdpSession_Params creates a new list of SessionManager_unregisterUdpSession_Params.
|
||||
func NewSessionManager_unregisterUdpSession_Params_List(s *capnp.Segment, sz int32) (SessionManager_unregisterUdpSession_Params_List, error) {
|
||||
l, err := capnp.NewCompositeList(s, capnp.ObjectSize{DataSize: 0, PointerCount: 1}, sz)
|
||||
l, err := capnp.NewCompositeList(s, capnp.ObjectSize{DataSize: 0, PointerCount: 2}, sz)
|
||||
return SessionManager_unregisterUdpSession_Params_List{l}, err
|
||||
}
|
||||
|
||||
|
@ -3861,203 +3880,204 @@ func (p SessionManager_unregisterUdpSession_Results_Promise) Struct() (SessionMa
|
|||
return SessionManager_unregisterUdpSession_Results{s}, err
|
||||
}
|
||||
|
||||
const schema_db8274f9144abc7e = "x\xda\xccY{p\x14e\xb6?\xa7{&\x9d@\x86" +
|
||||
"IW\x0f\x04\xa6\xe4\xe6^\x0a\xcaK\x14\x14\xb8\xdeB" +
|
||||
"\xae\xde\x04L\xb8&\xf2H\xcf\x90\xbb\x96\xa0eg\xe6" +
|
||||
"#tv\xa6{\xe8\xee\x89\x04A\x1e\x82\x88\xe5\x0b\x04" +
|
||||
"E\x94\x95\xc5r\xb7D\xdd\x85U\xd7eKke\xd7" +
|
||||
"\x17\xa5\xa8X\xb8\x05\x8a\xb5\xab\xc8>(\\W\xc4\xb5" +
|
||||
"\xdcu\xed\xad\xd3=\xfd\xc8$$A\xf6\x8f\xfdor" +
|
||||
"\xfa\xfb\xcew\xce\xef\xfc\xbes\xcewr\xe9w*\x1b" +
|
||||
"\xb9)\xd1\xda\x18\x80\xbc-Za\xb3\xfaw\x96\xef\x9c" +
|
||||
"\xf0\xab\xb5 '\x11\xed[\x9eoM|e\xad}\x1f" +
|
||||
"\xa2\xbc\x000mi\xc5r\x94\xd6W\x08\x00\xd2\x9a\x8a" +
|
||||
"\xdf\x03\xda\xb7\x8d\xda\xf3\xc8c\xcd[n\x051\xc9\x07" +
|
||||
"\x8b\x01\xa71\xa1\x15\xa5\x1e\x81V\x16\x85\x0d\xd2\xbb\xf4" +
|
||||
"\xcb\xbeF\xbcda\xe2\xed7iuXu\x84T\xbf" +
|
||||
" \xd4\xa3t\xd0\xd9p@ \xd5W\xe4\xdf\xda\xf5\xdf" +
|
||||
"[__\x07b\x92\xeb\xa5\xfa\xd9\xca\xe5(\x1d\xa8\xa4" +
|
||||
"\x95/W\xce\x07\xb4?\xdf2\xfa\x89\xef\xbf\xf9\xdaz" +
|
||||
"\x10/D(Y\xfaA\xe5{\x08(}V\xf9c@" +
|
||||
"\xfb\xe0\x17\x0b\xcf<\xf3\xcae\xb7\x818\x91\x16 -" +
|
||||
"\xd8T5\x8e\x03\x94\x1e\xafj\x00\xb4O\x9e\xfa\xdb\x86" +
|
||||
"\x9b'\xce\xbb\x17\xe4\x89\xc8\x01D9Zq\xa0*I" +
|
||||
"+>\xac\"k\x1af\x1f\xdc\x97\x9cv\xff\x962\xd3" +
|
||||
"\x9d\x85\xfb\x87\xd5\xa3th\x18\x19tp\xd8M\x80\xf6" +
|
||||
"_F<\xf4f\xf1\xaa\xe7\xee\x0f\x9f7ex=i" +
|
||||
"k\x19N\xe7\x8d\xeb\x9ep\xe3/_~\xfa\x01\x90'" +
|
||||
"!\xda\xc7:.z\x97\xdf\xb1\xfb}hG\x81\x8e\x9f" +
|
||||
"\x96\x1f\xbe\x8b\x8c_9\x9c\x94\xbdu\xf1\xf3?\xbf\xf7" +
|
||||
"\xe9\x0d\x0f\x81|!\"\x80\x03\xd6\x87\xc3\xffJ\x0b\xbe" +
|
||||
"p\x94m9\xf2\xc2\xbc\xfc\xa6\xed\xbb\\\xf7\x9d\xefc" +
|
||||
"\xab9\x0e\"\xf6\xba\x96/\xf3\xed\x8f\xa6\x1f-\x01\x13" +
|
||||
"\xa5Ob\xf5i\x04\x9c6\xa1\xba\x0e\x01\xed\xcb\xde;" +
|
||||
"1\x7f\xeeO\x16\xff0\xb4wfl9\xed\xdd\xb0\xf8" +
|
||||
"\xf4\xfe\x9aT\xfe\x892\x87\x1d_\xae\x8c\xedF\xa9=" +
|
||||
"F\x0e\xcb12\xe1\xa9\x7f\xbb\xa6j\xd9\x89\xd9{@" +
|
||||
"\x9c\xe4\xa9Y\x1aK\x91\x9a\xc8\xf5\xfc7\xca\xb6_<" +
|
||||
"SN'\x07\xb8|\xac\x03\xa55\xa4g\xda\xca\x98c" +
|
||||
"\xcf\x1d\xfb\xb7_T\xf9\xc8\xe7\xcf\xf6\x07\xf3c#:" +
|
||||
"P\xda7\x82N}v\x04!3\xb2\x05\x8f\xbd8%" +
|
||||
"\xf2\\8\xee#\xe3'\x09\x99\x89q\x8a\xfb\xd8Of" +
|
||||
"\xc5\xb4O\xd7\xbeX\xa6\xcdYx \xde\x8a\xd2\x07q" +
|
||||
"\xd2v\xd4Y\xdc\xba\xf0\xbe\xcd\xd1\x13\xf7\xbdJ\x96\x86" +
|
||||
"\x08\x17%\xa2M+\xd6\x18(m\xac\xa1\x9f\xebkj" +
|
||||
"y@;\xb9\xe7\x7f~4+{\xf4\xf5~,\x95\xa2" +
|
||||
"\x89\xd3\x92\x98\xa0_\xb1\x04\x19z|\xd2\xde\x9b\xffx" +
|
||||
"\xd7\xa1\xc3%C\x1d\x0c\xd5\x84\x13\xc2\x95\x09\xc2\xcfg" +
|
||||
"@\x19J\xce\xca\x1d\x89.\x94\xf6:\xea\x9erVs" +
|
||||
"'\x941\xab\x7f\xfd\xbf\xc7BA\xdb\x9b\xf8\x08!b" +
|
||||
"\xcf\xfb\xff\x85]U+\x8f\x1f\x0f\x1f\xf4X\xc2Ad" +
|
||||
"\x9f\xb3\xf5O?8y\xcf\xa9|\xf6w\x0e\xf1<\xcc" +
|
||||
"\x8e&f\x105?K\x10\xd1k\xebb\xcd\xe3\x8e\xb4" +
|
||||
"\x9dtC\xe9\xaa88r\x16-81\x92T\\v" +
|
||||
"\xe3L\xb6h\xfa\xb5'\xfb\\\xf9\xe8\xa8\x19(\x8d\x1c" +
|
||||
"\xe5\x90l\xd4\x06\x94&\xd6\xd6\x02\xd8\xdd?\xddt\xed" +
|
||||
"\x13/\xcd;\xed\xde\x05\xc7\xd81\xb5S\x89\x1aw\xdf" +
|
||||
"\xd24\xff\xf2q\xfbO\x87\x8d\x15k\x89\x9d\xd2\x84Z" +
|
||||
":i\xf1\xf4S\xff7\xe1\xeeWN\xf7G\xc1\xe6\xda" +
|
||||
"z\x94\xdak\x1d\x0a\xd2\xe2Og\x7f\xefp2\x9e<" +
|
||||
"S\x06`\x85\x13\xbc\xda.\x946\xd6:\xc1\xab}\x95" +
|
||||
"hv\xdb\xfb7,{\xe7\xd6\xcf\xbf(\x8f\xb5\xa3\xba" +
|
||||
"gL\x0a\xa5\xbb\xc6\x90\xea\x8dc\x88\x19\x0f,\xf8\xc3" +
|
||||
"\xaaS[G}\xd9\xc7\xe3\x89\xc9.\x94\xaeL\xd2\xca" +
|
||||
"\xcb\x93\x1b\xa4\x07\xe9\x97\xfd\xb6\xf0\xe8\x94\xa6U\xaf\x7f" +
|
||||
"\x15\xba\x0bk\x92\xad\xe4\xf0\xfd\xc2\xc3\xc7W\xff\xe6\x86" +
|
||||
"\xaf\xc3\x0e\xafL~D\x0eoJ\x92\xc3+>}\xf0" +
|
||||
"\xea{\x16=\xf9M8\xb0\xc9\xb5\xb4\xd5*j\x1a\xcb" +
|
||||
"\x19\x85H\xe6\x12\xefgfrF)h\x85\x193\x8b" +
|
||||
"\xd6\x12\xa6YjF\xb1X\x8a5\x98\x05]3Y\x1b" +
|
||||
"\xa2\\\xc3G\x00\"\x08 *]\x00\xf2\x8d<\xca9" +
|
||||
"\x0eE\xc4\x04\x85^TI\xb8\x84G\xd9\xe2P\xe4\xb8" +
|
||||
"\x04e\x1eq\xe98\x009\xc7\xa3\xbc\x8cC\xe4\x13\xc8" +
|
||||
"\x03\x88\xc5\xcd\x00\xf22\x1e\xe5u\x1c\xda\x05f\xe4\x15" +
|
||||
"\x8di\x10\xb7\x9a\x0d\x03\xab\x81\xc3j@\xdb`\x96\xd1" +
|
||||
"\xa3t\xe4 \xceBb\xa1\xeb&\x0bc\xc0a\x0c\xd0" +
|
||||
"^\xa2\x17\x0d\xb3]\xb3P\xcd\xa5\xd8b\x83\x99\xb8\x04" +
|
||||
"+\x80\xc3\x0a\xc0\x81\xdcK3\xd3Tum\xae\xa2)" +
|
||||
"\x9d\xcc\x00 \xcf*\xf9(\x80\x9f\xb4\xd1K\xef\xe2\x94" +
|
||||
"\xed\xc0\x89\x93\x04\x0c20z\xf4\x13\xffc7p\xe2" +
|
||||
"X\xc16X\xa7jZ\xcc\xc0\xf6l\xc1\xd1\xcd\xebZ" +
|
||||
"#\xdaE\xcd\xfd\x80\xccp?\xc4\xe9\xd4Fl\xc3\xc0" +
|
||||
":\xbe\xafuW\xe5T\xa6Y\xf1\x16m\xb1^\x06y" +
|
||||
"k\x7f\x90\xb7\x96 _\x17\x82|\xcd,\x00y\x05\x8f" +
|
||||
"\xf2\xed\x1c\x8a|\x09\xf3\xf5\xf5\x00\xf2j\x1e\xe5;9" +
|
||||
"\xb43\xce!-Y\x00\xf0\xd1\\\xcc\x14\xabh0\x93" +
|
||||
"d#\x00\xdbxt@\x1f\x01\xb8\xaa\x9b\x19d\xbb\x17" +
|
||||
"\x84\xb8bd\x96\xf8\x81\x1a\x00\xe9\xe6e\xaai\xa9Z" +
|
||||
"\xe7\x02G\xde\xd0\xa6\xe7\xd4L\x0fyU\xed\xd89v" +
|
||||
"\x06\x00\xa28\xf2:\x00\xe4Dq\x16@\x83\xda\xa9\xe9" +
|
||||
"\x06\xb3\xb3\xaa\x99\xd15\x8d\x01\x9f\xb1Vu(9E" +
|
||||
"\xcb0\xff\xa0\x8a\xbe\x07\xb9\x07\xa4\x99\xd1\xcd\x8c\xc9J" +
|
||||
"\x88\xbe\xe3\xdb\x14C\xe1\xf3\xa6\\\xed\xe3\xd8|\x1d\x80" +
|
||||
"\xdc\xc4\xa3\xdc\x16\xc2q.\xe18\x87G\xf9\xda\x10\x8e" +
|
||||
"\xed\x84c\x1b\x8f\xf2\"\x0em\xddP;U\xed*\x06" +
|
||||
"\xbc\x11f\xa0iiJ\x9e\x11f%<V\xe9\x05K" +
|
||||
"\xd55\x13k\x82\xfc\x0f\x885!\xa4\x84\xc189\xd9" +
|
||||
"\xa3\x94\xc7(]\x1b\x9fbfQ\xc8Y\xa6\x1c\xf1=" +
|
||||
"\x89\xcd\x00\x90+y\x94\x13\x1c6\x18\xcc,\xe6,\xac" +
|
||||
"\x09\xca\xec?\xe3T\x0f\xbe\x10\x0dS\xfd\xd1p*\x80" +
|
||||
"\x9c\xe5Q.p\x88%\xf4\xf2\xb3B\xd9\x80G\x97\x85" +
|
||||
"K\xb7\x03\xc8\x16\x8f\xf2j\x0em\xd3=\xa4\x050\xeb" +
|
||||
"!Z\x975\xad\x96\x82\xf7\xd7\xaa\xaci\xb5\xe9\x86\x85" +
|
||||
"\x02p(\x00\xf1V7\xd9\xcc\xc5t\xa7Z\xb29v" +
|
||||
"\xb5\xcak\x16F\x81\xc3(\x0cx\xa9\\~\xc4)\xb1" +
|
||||
"\xb9\xb7\xdd\xf3f\"\x91\xe1?y\x94\xff+\xe4\xcd\x14" +
|
||||
"\xcac\x97\xf2(_\xc1\xa1\xadd2zQ\xb3\x16\x00" +
|
||||
"\xaft\x96q>\xcd \x9e1X@\x87\xa1C\xed%" +
|
||||
"\x872\xb0\xe3\x86\x92\xef\x15a\x02\xbb\x9aGyt\xff" +
|
||||
"p\xf9'F\xfbI$t\x812\xc4\xc5\x14sS\xf8" +
|
||||
"d\x83\x99B1g\x91\xff\xd5\xb6\xed\x02@\x91\x1b\xcf" +
|
||||
"\xa3|)\x871\xfc\xc6v\x11\x98\xb49@\xa0\x8e\x19" +
|
||||
"\x86n`MP\xe2J\xc4\xca\x94\x0e@]kb\x96" +
|
||||
"\xa2\xe6\x90H\xef\xf7[e\xf4\x1b\xec\xd6\x06\x88\xb8\xe2" +
|
||||
"\xf1\x0d\xc4\xbd\xdeh\x10yjx\x94/\xe0\xd0\xee4" +
|
||||
"\x94\x0ckc\x06\xaazv\x9e\xa2\xe9i\x9ee\xfaP" +
|
||||
"a\xc4\xb9\x1e\x9arn\x91\x09\xfe\xae\x81\xf7\x1b\xac\x04" +
|
||||
"Bi{[\x9dks\xc2\xb7y\xe5\xb8\xa0\xd4\xf9\x04" +
|
||||
"[\xd3\x11\xe4b?\xdbl$*\xde\xce\xa3\xbc%\x94" +
|
||||
"\xb57Q^\xba\x97G\xf9a\x0e\xc5H$\x81\x11\x00" +
|
||||
"\xf1A\xbaY[x\x94wr\xbd\x0b\"\xebf\x9a\xd5" +
|
||||
"\xa4v\x82\xc0\xcc@J&6\xa9\x9d\x0cx\xf3|3" +
|
||||
"W\xe5 x\xe8\x1d\xa6\x9ec\x16kb\x99\x9cb(" +
|
||||
"\x96\xda\xcd\xdc\xef%2zA\x1d\x88\xb7\xa9>\x17\x83" +
|
||||
"\xf8\x1b\xf7z\x90\x10\x1d\xc6\x05\xe9O`\xa1\xd6a\x00" +
|
||||
"k]\xe5d\x99\xae\xf5\xe1@pcJ<@s\xa0" +
|
||||
"\xea\x16,\x9f_\xb0TA\xd7L\xb2/\x14\xfa\x19\xfd" +
|
||||
"\x85\xde\x08B\xefe\xca\x8dk\xc3\x91/e\xcaM\xdb" +
|
||||
"\x83 \x8b\x11\xce\x8d\xfc\x8e]\x00\xf2N\x1e\xe5'9" +
|
||||
"lp\x8b8\xd6\x04\x8f\xe0R\xb4\xdcR5G\x87\xba" +
|
||||
"\x8c\x92\x0b\xb2\xa9m\xb0BN\xc9\xb0f,\x95e@" +
|
||||
"\x04\x0e\xd1\xa1H\xbe`0\xd3DU\xd7\xe4\xa2\x92S" +
|
||||
"y\xab\xc7o\xa5\xb4b\xbe\xcd`\xdd*\xeaEs\xa6" +
|
||||
"e\xb1\xbcP\xb0\xcc\xa14Z\x01@\x94\x1f\x045g" +
|
||||
"\x96%\xdf\xfa \xf7\xf8\x00M\xa2\xe4{1\x8f\xf2t" +
|
||||
"\x0e\xe3\xc5\xa2\x1a\xe4\xba\x9c\x9eq\xe2\x06\xf1yJ\x9e" +
|
||||
"\xf5\x89v\xc5\xa0w\xb5\xd7M\xf7\x92\xed\xbfRc0" +
|
||||
"p/N\xae;\xcdj\xc8d\xba\x02\x8d<\xcasB" +
|
||||
"&\xb7L\x0d\xf9\xe1\x99<\xb7#\xf0C\xf8.\xeb\xf1" +
|
||||
"\xac\xaacy\xca\xdc\x1e\x98%gf\x82pM\xb0f" +
|
||||
" \xfb\xc2\x17j~\xa1\xce\xf1\x90l\x9c\xee\xd9(\xf5" +
|
||||
"`+@z\x19\xf2\x98^\x87\x81\x99\xd2\x1a\x9c\x05\x90" +
|
||||
"^A\xf2\xdb1\xb0TZ\x8fI\x80\xf4j\x92\xdf\x89" +
|
||||
"\xfe\x9bA\xda\x88\xbb\x01\xd2w\x92x\x1b-\x8f\xf0\xce" +
|
||||
"\x95\x90\xb6:\xea\xb7\x90|'\xc9\xa3\x91\x04F\x01\xa4" +
|
||||
"\x1dX\x0f\x90\xdeF\xf2gH^\xc1%\xb0\x02@\xda" +
|
||||
"\x8b]\x00\xe9=$\x7f\x9e\xe4B4A\xcf&i\x1f" +
|
||||
"\x1a\x00\xe9\x9f\x91\xfc%\x92W\x8eN`%\x80\xb4\xdf" +
|
||||
"\x91\xbfH\xf27H^5&\x81U\x00\xd2\x01\\\x0b" +
|
||||
"\x90~\x8d\xe4\x87I>\x0c\x138\x0c@:\x84\xdb\x01" +
|
||||
"\xd2\x87I\xfe[\x92\x0f\xafH\xe0p\x00\xe9\x03\xc7\x9e" +
|
||||
"#$\xff\x98\xe4\xd5\x91\x04V\x03H\x1f\xe2.\x80\xf4" +
|
||||
"\xc7$\xff3\xc9cB\x02c\x00\xd2'\x8e_\xa7H" +
|
||||
"^\xc9\x95\xb5\xec\x1e\xa3\xca\xfar^7\xfd\x90\xb1\xd2" +
|
||||
"\x1dG\x97\xeemz\x9czo\x8c\x07C0@\x8c\x03" +
|
||||
"\xda\x05]\xcf\xcd\xeb\xcd\xd4\xb8\xa5t\x9a\xde\x1b\xa0&" +
|
||||
"\x98K\x00\x92\xd0\xaf\xfb\x10\xd7\xb5\x96\xac\x9f\x08\xca\xb3" +
|
||||
"\x8eg\x89j\xce,Zz\xb1\x00uY\xc5bY?" +
|
||||
"\xe7\x18Em\xb6\xa1\xe7\x17 3\xf2\xaa\xa6\xe4\x06\xc9" +
|
||||
"FU\xc0a\x15\x94R\x82\xa7{\xe0\xd4t\xf6\x17\x8d" +
|
||||
"\xcfh\xae\x9c\xd1u\x85\x19\x0b\x94\xce\xa1\xe4\xa9\xa9A" +
|
||||
"\xe7\x18\xd7B\x09\xa9\xae[\xc9\x15\xbfMz\xea\xddJ" +
|
||||
"\xa4\x1a\xdcVd\xb0~\xdf\x1bS\x0c\x9eJz7\x84" +
|
||||
"\xbd\x0b*\x86&\x88t\x0eW\xd2?d\xf3;\x99\xe5" +
|
||||
"\xfe\xa2\x87+=\x1b\x84p\x99?\xb7\xdd)f\xc6\x87" +
|
||||
"\xe2z0\xce\x19\xfc\xa9\xd3O\xe1\xef\xa7\xec{=g" +
|
||||
"\xe8\xb9C\xb1_\xc4\xa3\xbc$\x14{\xd6\xda\xcfs'" +
|
||||
"\x15\xcc9D\x9e+\x0d:\xa8P\x14x\x94Wp\x18" +
|
||||
"\xa7w)\xd6\x04s\xdf^F\xf7~\x8b\x13\x15Z\xb4" +
|
||||
",\x03\\\xe6\xb19T>\xfc\x09\xe8\xe0\xdd\xd9\xd0\xdc" +
|
||||
"\xf6\xba\xdeA\x01\xf7\xa7\x8ae'\x9f\xf5\xc9\xd5\xe0\x1e" +
|
||||
"J<\x1b\xed\x8cX\xbc\x09+z\xb3:q\xefr\xe0" +
|
||||
"\xc4\xc7\x05\x0c\xa6\x90\xe8\x0d\x1d\xc5\x1d\x06p\xe2V\x01" +
|
||||
"9\x7ff\x8d\xdelZ\xdcx\x07p\xe2z\x01y\x7f" +
|
||||
"\xe4\x8c\xde\xb4kJ\xcf0\x04N\\)`\xc4\x1f\xe5" +
|
||||
"\xa37+\x13\x97v\x01'\xaa\x02F\xfdi6z\xe3" +
|
||||
"T\xf1\xfa\xb5\xc0\x89\xed\xc1L\x07\x1a\\?\x1a\xd1\xf6" +
|
||||
"8\x0au\x0eK{Ox\xdcU\x00\x8dh{=0" +
|
||||
"\x7f\xb6&\xd8Y\xe5\x0d) \x9eQ,\xd6H\xcd\x99" +
|
||||
"{\xff\xb1\x94\x00\xa0\x11\xe5\x08\x86F\x85\x00\xe7\xfb\xbe" +
|
||||
"L\xb1:'\xce\xdf\xb6e\xf2\xf6\x7f\xcb\x94\xc4\xf7g" +
|
||||
"5\x9d\xe3\x0f\xbbBz\xbbB\x0f\xdfA\x1a\xbf\xc8\xd9" +
|
||||
"\xbc\xf0\xc8\x1f\xa7\xcd\xa4\xff\xdf}\xfd\x87\xa8qz\x83" +
|
||||
"G\xf9H\xe8Z\xbfK\xc2\xb7y\x94\x8f\x85\x1a\xa7\xa3" +
|
||||
"t\xd7\x8f\xf0(\x9f\x09\xe6\x97\x9f\xdd\x01 \x9f\xe11" +
|
||||
"\x15jD\xc4\xbf\xd3\xc2\xaf\xa9\\;m\x08\xbamH" +
|
||||
"\x147\x03\xa4+\xa9\x8c'\x9c6$\xe2\xb6!\"v" +
|
||||
"\x00\xa4kH~A\xb8\x0d\x19\x83\xd7\x01\xa4G\x93|" +
|
||||
"<\xf6~\xd7\x08E#h\xd4rz\xe7\x1cU\xeb\xb7" +
|
||||
"\xb6y\x03U\xb4f+j\xaeh0\x08Jk)\xd9" +
|
||||
"4\x85\xaa\xbd;iu\x87*i\"a\x16M\x7f\xe0" +
|
||||
"r\x0e/\xca!U\x9ef\xc3\xd0\xd1(kb\xa7\x06" +
|
||||
"M\xac\xdf\xc3R/~5\x8f\xf2\x02\x0aE\xa3\x1b\x0a" +
|
||||
"\xb9#h\xbb\xeb2J\xd1d}|\x00\x9e\x19\xfe\x14" +
|
||||
"\xc0\\\xa2\x17s\xd9\x14\x03\xc12z\xca \x18\xb4\x99" +
|
||||
"M\xb3\xb8\x97\xb9\xdc\xe1\xb0\xf7\x8f\x0e\xf4\xfe\x9f\x11\x1a" +
|
||||
"\x0e{\x13z\xf4\xfem\xd5w8\xeca\xd0g8\xec" +
|
||||
"~p8\xda{8|\x1e\xcfW\xb7\x8c\x852\xc69" +
|
||||
"\xcdL\x87<j\xf4\xff\xb3[v\xd3\xab\xcewL\xe0" +
|
||||
"\x15\xa4\x7f\x04\x00\x00\xff\xff\xa5\x0ed\xc9"
|
||||
const schema_db8274f9144abc7e = "x\xda\xccY}p\x14e\x9a\x7f\x9e\xee\x99t\x02\x19" +
|
||||
"f\xbaz 0%\x97\x93\xc2\xf2\x88\x82\x06\xce+\x8e" +
|
||||
"\xb3.\x09\x06\xceD>\xd23p\xe5\x09Zvf\xde" +
|
||||
"\x84\xc9\xcdt\x0f\xdd=\x91 \xc8\x87 b\xf9\x05\x82" +
|
||||
"\"\xca\xc9ayW\xa0\xde\xc1\xa9\xe7\xb2%\xb5\xb2+" +
|
||||
"*\xa5\xa8X\xb0\x85\x8a\xb5\x8b\xc8\xeeJ\xc1\xba\"\xac" +
|
||||
"\xe5\xaeko=\xdd\xd3\x1f\x99\x84$\xc8\xfe\xb1\xffM" +
|
||||
"\x9e~\xde\xf7}>~\xcf\xef}\xde'\xd7wT6" +
|
||||
"r\xf5\xe1\x9a\x08\x80\xbc%\\a\xb1\xba\x0f\x97n\xbf" +
|
||||
"\xeag\xabAN Z\xf7\xbc\xd6\x1a\xff\xd6\\\xfd\x09" +
|
||||
"\x84y\x01`\xca\xe2\x8a\xa5(\xad\xad\x10\x00\xa4U\x15" +
|
||||
"\xbf\x06\xb4\xee\x1b\xb5\xfb\x99\xe7fl\xba\x17\xc4\x04\xef" +
|
||||
"+\x03NaB+J=\x02i\x16\x85u\xd2Q\xfa" +
|
||||
"e\xdd\"^\xb7 \xfe\xc1{\xa4\x1d\xdc:D[\xef" +
|
||||
"\x13\xeaP:d/8(\xd0\xd67\xe6\xdf\xdf\xf1\x0f" +
|
||||
"\x9b\xdfY\x03b\x82\xeb\xb5\xf5+\x95KQ:XI" +
|
||||
"\x9a\x07*\xe7\x02Z_o\x1a\xfd\xfc\x7f\xbe\xf7\xf6Z" +
|
||||
"\x10\xafF(Y\xfai\xe5\xc7\x08(}U\xf9\xbf\x80" +
|
||||
"\xd6\xa1\x0b\x0b\xce\xbf\xfc\xe6\x0d\xf7\x818\x81\x14\x90\x14" +
|
||||
"6T\x8d\xe3\x00\xa5\x9dU\x0d\x80\xd6\xe93\x7f\\w" +
|
||||
"\xf7\x849\x8f\x82<\x019\x800G\x1a\x07\xab\x12\xa4" +
|
||||
"q\xa2\x8a\xaci\x98yhob\xca\xe3\x9b\xcaL\xb7" +
|
||||
"\x15\xf7\x0f\xabC\xe9\xf002\xe8\xd0\xb0\xbb\x00\xad\xdf" +
|
||||
"\x8fx\xea\xbd\xe2M\xaf>^:\xcfV\xaa\x1f^G" +
|
||||
"\xbb\xb5\x0c'\x85q\xddW\xdd\xf9\xd3\x03/=\x01\xf2" +
|
||||
"DD\xebx\xfb5G\xf9m\xbb>\x81\xf9(\xd0\xf1" +
|
||||
"Sv\x0e\xdfA\xc6\xef\xb5u\xdf\xbf\xf6\xb5\x1f?\xfa" +
|
||||
"\xd2\xba\xa7@\xbe\x1a\x11\xc0\x0e\xd6\xd8\xea?\x90B}" +
|
||||
"5\x19\xbf\xe9\xd8\xbe9\xf9\x0d[w8\xee\xdb\xdf\xff" +
|
||||
"\xad\x9a\xe3 d\xadi\xf9&?\xff\xd9\xd4\xb3\xa5\xc0" +
|
||||
"\x84\xe9\xd3\xec\xeas\x088E\xa9\xaeE@\xeb\x86\x8f" +
|
||||
"O\xcd\x9d\xfd\x7f\x1d\xff\x1dX\xbb<\xb2\x94\xd6\xae\xeb" +
|
||||
"8\xb7?\x96\xcc?_\xe6\xb0\x1d\xbb\x9e\xc8.\x946" +
|
||||
"D\xc8\xe1\x87\"d\xc2\x8b\x7fsK\xd5\x92S3w" +
|
||||
"\x838\xd1\xdd\xe6\xc5H\x92\xb6\x09\xdd\xce\x7f\xafl\xf9" +
|
||||
"\xc9\xcb\xe5p\xb2c\xb23\xd2\x8e\xd2>\xdag\xca\xde" +
|
||||
"\x88m\xcf\x03\xfb\xb7^S\xf9\xcc\xd7\xaf\xf4\x17\xe6\x13" +
|
||||
"#\xdaQ\xba0\x82N\xfdj\x04Efd\x0b\x1e\x7f" +
|
||||
"\xbd>\xf4j0\xefr\xf44E\x86E)\xefc\xcf" +
|
||||
"N\x8f\xa8_\xae~\xbdl7[1\x1ckEiL" +
|
||||
"\x8cv\x1b\x19#\xe5\xd6\x05\x8fm\x0c\x9fz\xec-\xb2" +
|
||||
"4\x00\xb80\x01m\xca\x9e\x98\x8e\xd2\x81\x98\x9d\xedX" +
|
||||
"\x0d\x0fh%v\xff\xd3\xffL\xcf|\xf4N?\x96J" +
|
||||
"M\xf1s\xd2\xec8\xfdj\x89\x93\xa1''\xee\xb9\xfb" +
|
||||
"\x8b\x87\x0e\x1f)\x19j\xc7\xf0\xb9\xb8\x9d\xc2\xbdq\x8a" +
|
||||
"\x9f\x87\x80\xb2(\xd9\x9a\x1f\xc5\xbbP:ko\xf7\x85" +
|
||||
"\xad\xcd\x9dR\xc6\xac\xfc\xf9?\x1f\x0f$\xedl\xfc3" +
|
||||
"\x84\x905\xe7_\x17tU-?y2x\xd0\x89\xb8" +
|
||||
"\x1d\x91\x0b\xf6\xd2\xdf\xfe\xd7\xe9G\xce\xe43\xbf\xb2\x81" +
|
||||
"\xe7\xc6l\xe4\xc8i\x04\xcd\x89#\x09\xe85\xb5\x91\x19" +
|
||||
"\xe3\x8e\xb5\x9dvR\xe9lQ5j:)\\9\x8a" +
|
||||
"\xb6\xb8\xe1\xce&\xb6p\xea\xad\xa7\xfb\x94|\xd3\xa8i" +
|
||||
"(\xc9\xa3l\x90\x8dZ\x87\x12\xab\xa9\x01\xb0\xba\xff\x7f" +
|
||||
"\xc3\xad\xcf\xbf1\xe7\x9cS\x0b\xb6\xb1\xf3k&\x134" +
|
||||
"\x1e\xbe\xa7y\xee?\x8e\xdb\x7f.h\xec\xec\x1aB\xa7" +
|
||||
"\xa4\xd4\xd0I\x1dS\xcf\xfc\xcbU\x0f\xbfy\xae?\x08" +
|
||||
"\xae\xaa\xa9CiC\x8d\x0dAR\xfer\xe6\x7f\x1cI" +
|
||||
"D\x13\xe7\xcb\x02Xa'\xaf\xa6\x0b\xa5\x035v\xf2" +
|
||||
"j\xde\"\x98\xdd\xf7\xc9\x1dK>\xbc\xf7\xeb\x0b\xe5\xb9" +
|
||||
"\xb6\xb7~eL\x12\xa5\x83cl~\x19C\xc8xb" +
|
||||
"\xdeoV\x9c\xd9<\xea\x9b\xbe$\x97\xe8B\xa9'a" +
|
||||
"\x93\\b\x9dt\x94~Y\x1f\x08\xcf\xd67\xafx\xe7" +
|
||||
"\xdb@-\xecK\xb4\x92\xc3\x8f\x0bO\x9f\\\xf9\x8b;" +
|
||||
"\xbe\x0b:\xbc7\xf1\x199|(A\x0e/\xfb\xf2\xc9" +
|
||||
"\x9b\x1fY\xf8\xc2\xf7\xc1\xc4&V\xd3R\xb3\xa8\xaa," +
|
||||
"\xa7\x17B\xe9\xeb\xdc\x9f\xe9Ii\xa5\xa0\x16\xa65\x15" +
|
||||
"\xcdEL5\xb3i\xc5dI\xd6`\x144\xd5`m" +
|
||||
"\x88r\x8c\x0f\x01\x84\x10@T\xba\x00\xe4;y\x94s" +
|
||||
"\x1c\x8a\x88qJ\xbd\x98%\xe1\"\x1ee\x93C\x91\xe3" +
|
||||
"\xe2\xc4<\xe2\xe2q\x00r\x8eGy\x09\x87\xc8\xc7\x91" +
|
||||
"\x07\x10\x8b\x1b\x01\xe4%<\xcak8\xb4\x0aL\xcf+" +
|
||||
"*S!j\xce\xd0u\xac\x06\x0e\xab\x01-\x9d\x99z" +
|
||||
"\x8f\xd2\x9e\x83(\x0b\x88\x85\xae\xbbL\x8c\x00\x87\x11@" +
|
||||
"k\x91V\xd4\x8d\xf9\xaa\x89\xd9\\\x92u\xe8\xcc\xc0E" +
|
||||
"X\x01\x1cV\x00\x0e\xe4^\x8a\x19FVSg+\xaa" +
|
||||
"\xd2\xc9t\x00\xf2\xac\x92\x0f\x03x\xa4\x8d.\xbd\x8b\xf5" +
|
||||
"[\x81\x13'\x0a\xe830\xba\xf0\x13\xaf\xdc\x05\x9c8" +
|
||||
"V\xb0t\xd6\x995L\xa6\xe3\xfcL\xc1\xde\x9b\xd7\xd4" +
|
||||
"F\xb4\x8a\xaa\xf3\x01\x99\xee|\x88\xd2\xa9\x8d\xd8\x86\xbe" +
|
||||
"u|_\xebn\xcae\x99jF[\xd4\x0e\xad,\xe4" +
|
||||
"\xad\xfd\x85\xbc\xb5\x14\xf25\x81\x90\xaf\x9a\x0e /\xe3" +
|
||||
"Q\xbe\x9fC\x91/\xc5|m\x1d\x80\xbc\x92G\xf9A" +
|
||||
"\x0e\xad\xb4}HK\x06\x00\xbchv0\xc5,\xea\xcc" +
|
||||
" \xd9\x08\xc06\x1e\xed\xa0\x8f\x00\\\xd1\xcdt\xb2\xdd" +
|
||||
"MBT\xd1\xd3\x8b\xbcD\x0d\x10\xe9\x19K\xb2\x86\x99" +
|
||||
"U;\xe7\xd9\xf2\x866-\x97M\xf7\x90W\xd5\xb6\x9d" +
|
||||
"c\xa7\x01 \x8a#o\x03@N\x14\xa7\x034d;" +
|
||||
"UMgV&k\xa45Ue\xc0\xa7\xcd\x15\xedJ" +
|
||||
"NQ\xd3\xcc;\xa8\xa2\xefA\xce\x01)\xa6w3}" +
|
||||
"\x92\x12\x80\xef\xf86EW\xf8\xbc!W{q\x9cq" +
|
||||
"\x1b\x80\xdc\xcc\xa3\xdc\x16\x88\xe3l\x8a\xe3,\x1e\xe5[" +
|
||||
"\x03q\x9cOql\xe3Q^\xc8\xa1\xa5\xe9\xd9\xce\xac" +
|
||||
"z\x13\x03^\x0f\"\xd00U%\xcf(f\xa5x\xac" +
|
||||
"\xd0\x0afVS\x0d\x8c\xf9\xfc\x0f\x88\xb1@\xa4\x84\xc1" +
|
||||
"09\xc9\x85\x94\x8b(M\x1d\x9fdFQ\xc8\x99\x86" +
|
||||
"\x1c\xf2<\x89L\x03\x90+y\x94\xe3\x1c6\xe8\xcc(" +
|
||||
"\xe6L\x8c\xf9\xd7\xec_\xe2T7|\x01\x18&\xfb\x83" +
|
||||
"\xe1d\x009\xc3\xa3\\\xe0\x10K\xd1\xcbO\x0f\xb0\x01" +
|
||||
"\x8f\x0e\x0a\x17o\x05\x90M\x1e\xe5\x95\x1cZ\x86sH" +
|
||||
"\x0b`\xc6\x8dhm\xc60[\x0a\xee_+2\x86\xd9" +
|
||||
"\xa6\xe9&\x0a\xc0\xa1\x00\x84[\xcd`M\x1dTS-" +
|
||||
"\x99\x1c\xbb9\xcb\xab&\x86\x81\xc30\x0cXT\x0e>" +
|
||||
"\xa2DlN\xb5\xbb\xdeL 0\xfc\x1d\x8f\xf2\xdf\x07" +
|
||||
"\xbc\xa9'\x1e\xbb\x9eG\xf9F\x0e-%\x9d\xd6\x8a\xaa" +
|
||||
"9\x0fx\xa5\xb3\x0c\xf3)\x06\xd1\xb4\xce|8\x0c=" +
|
||||
"\xd4.9\x94\x05;\xaa+y#h^\xb2?\xf3(" +
|
||||
"\xb0\xd7\xf2(O\xed?\x86+\xf2\xcc0\x94N\xd6\xa7" +
|
||||
"B\xc3\xfd\xb0\x0dUY\x9a\x00\x9bd\x0e\xcfO\xd2\x99" +
|
||||
"!\x14s&YQmY\x8e\x19\x94\xde\xf1<\xca\xd7" +
|
||||
"s\x18\xc1\xef-\xc7\x8e\x89\x1b\xfd0\xd52]\xd7t" +
|
||||
"\x8c\xf9\xf7`\x09}\xe9\xd2\x01\xa8\xa9\xcd\xccT\xb29" +
|
||||
"\xa4\xca\xf0\x9a\xb22\x8c\x0eV\xda~\xd8\x1c\xf1\xf8\x06" +
|
||||
"\x02h\xbeWQ\x10\xc2b<\xcaWphu\xeaJ" +
|
||||
"\x9a\xb51\x1d\xb3Zf\x8e\xa2j)\x9e\xa5\xfb\xe0e" +
|
||||
"\xc4\xa5\x1e\x9a\xb4K\xcd\x00o\xd5\xc0\xebuV\x0aB" +
|
||||
"iy[\xadcs\xdc\xb3y\xf98\xff>\xf4\xd2\xbc" +
|
||||
"\xaa\xdd'l\x8f\x92\xd6\x13^\xef\xe7Q\xde\x14\xa0\xf6" +
|
||||
"\x0dD^\x8f\xf2(?\xcd\xa1\x18\x0a\xc51\x04 >" +
|
||||
"I(\xd9\xc4\xa3\xbc\x9d\xeb}k\xb2n\xa6\x9a\xcd\xd9" +
|
||||
"N\x10\x98\xe1K\xc9\xc4\xe6l'\x03\xde\xb8\\z\xab" +
|
||||
"\x1c$\x1eZ\xbb\xa1\xe5\x98\xc9\x9aY:\xa7\xe8\x8a\x99" +
|
||||
"\xedf\xce\xf7\x12\x18\xdd\xa4\x0e\x84\xdbd\x9f\xea!\xfc" +
|
||||
"F\xddF%\x00\x87q>G\x0a,\xd0_\x0c`\xad" +
|
||||
"\xb39Y\xa6\xa9}0\xe0WL\x09\x07h\x0ct\x05" +
|
||||
"\xfa\xeas\x0bfV\xd0T\x83\xec\x0b\xa4~Z\x7f\xa9" +
|
||||
"\xd7\xfd\xd4\xbbt\xba~u0\xf3%:\xdd\xb0\xd5O" +
|
||||
"\xb2\x18\xe2\x9c\xcco\xdb\x01 o\xe7Q~\x81\xc3\x06" +
|
||||
"\xe7\xa6\xc7\x98\xffR.e\xcb\xb9\xcffiP\x9bV" +
|
||||
"r>\xe5Z:+\xe4\x944\x9b\x81\xa5\xbb\x1b\x10\x81" +
|
||||
"C\xb4!\x92/\xe8\xcc00\xab\xa9rQ\xc9ey" +
|
||||
"\xb3\xc7\xeb\xb7\xd4b\xbeMg\xddY\xd4\x8aF\x93i" +
|
||||
"\xb2\xbcP0\x8d\xa1tc~\x80\x88\x1f\x84l\xce(" +
|
||||
"c\xe8:\x9f{\xbc\x00M\xec\xf2)0Z,f=" +
|
||||
"\xee\xb3rZ\xda\xce\x1bD\xe7(\xf9\xbe\x14X1h" +
|
||||
"\xad\xf6\xaat\x97\x91\xff\x9a\xba\x87\x81\x1bvr\xdd\xee" +
|
||||
"h\x03&S\x094\xf2(\xcf\x0a\x98\xdc29\xe0\x87" +
|
||||
"k\xf2\xecv\xdf\x0f\xe1\xdfY\x8fkU-\xcb\x13s" +
|
||||
"\xbb\xc1,9\xd3\x04\xc2-\xbe\xce@\xf6\x05\x0bjn" +
|
||||
"\xa1\xd6\xf6\x90l\x9c\xea\xda(\xf5`+@j\x09\xf2" +
|
||||
"\x98Z\x83\xbe\x99\xd2*\x9c\x0e\x90ZF\xf2\xfb\xd1\xb7" +
|
||||
"TZ\x8b\x09\x80\xd4J\x92?\x88\xde\xc3BZ\x8f\xbb" +
|
||||
"\x00R\x0f\x92x\x0b\xa9\x87x\xbb$\xa4\xcd\xf6\xf6\x9b" +
|
||||
"H\xbe\x9d\xe4\xe1P\x1c\xc3\x00\xd26\xac\x03Hm!" +
|
||||
"\xf9\xcb$\xaf\xe0\xe2X\x01 \xed\xc1.\x80\xd4n\x92" +
|
||||
"\xbfFr!\x1c\xa7\xb7\x95\xb4\x17u\x80\xd4\x8fH\xfe" +
|
||||
"\x06\xc9+G\xc7\xb1\x12@\xdao\xcb_'\xf9\xbb$" +
|
||||
"\xaf\x1a\x13\xc7*\x00\xe9 \xae\x06H\xbdM\xf2#$" +
|
||||
"\x1f\x86q\x1c\x06 \x1d\xc6\xad\x00\xa9#$\xff%\xc9" +
|
||||
"\x87W\xc4q8\x80\xf4\xa9m\xcf1\x92\x7fN\xf2\xea" +
|
||||
"P\x1c\xab\x01\xa4\x13\xb8\x03 \xf59\xc9\x7fG\xf2\x88" +
|
||||
"\x10\xc7\x08\x80t\xd6\xf6\xeb\x0c\xc9+\xb9\xb2\xbe\xdeE" +
|
||||
"TY\xf3\xcek\x86\x972V\xaaqt\xe0\xde\xa6E" +
|
||||
"\xa9A\xc7\xa8?)\x03\xc4(\xa0U\xd0\xb4\xdc\x9c\xde" +
|
||||
"H\x8d\x9aJ\xa7\xe1>\x14b\xfe\xf0\x02\x90\x84\xde\xbd" +
|
||||
"\x0fQMm\xc9xDP\xce:\xae%Y\xa3\xa9h" +
|
||||
"j\xc5\x02\xd4f\x14\x93e<\xce\xd1\x8b\xeaL]\xcb" +
|
||||
"\xcfC\xa6\xe7\xb3\xaa\x92\x1b\x84\x8d\xaa\x80\xc3*(Q" +
|
||||
"\x82\xbb\xf7\xc0\xd4t\xf1g\x8f\x87h\xae\x1c\xd1\xb5\x85" +
|
||||
"i\xf3\x94\xce\xa1\xf0\xd4d\xbf\x7f\x8b\xaa\x01B\xaa\xed" +
|
||||
"Vr\xc5\x1fBO\xbd[\x89d\x83\xd3\x8a\x0c\xf6(" +
|
||||
"pg\x19\x83SI\xef\x86\xb0\xf7\x85\x8a\x811#\x9d" +
|
||||
"\xc3\x95\xf6\x1f\xb2\xf9\x9d\xcct~\xd1\xeb\x96\xde\x16B" +
|
||||
"\xf0\x9a\xbf\xb4\xd5IfD\x87\xe2\xba?\xf3\x19\xfc=" +
|
||||
"\xd4\xcf\xc5\xdf\xcf\xb5\xef\xf6\x9c\x817\x11\xe5~!\x8f" +
|
||||
"\xf2\xa2@\xeeYk?o\xa2\xa4?\x0c\x11y\xae4" +
|
||||
"\x0d\xa1\x8b\xa2\xc0\xa3\xbc\x8c\xc3(=^1\xe6\x0f\x87" +
|
||||
"{\x19\xdd\xfb\xc1NPhQ3\x0cp\x89\x8b\xe6\xc0" +
|
||||
"\xf5\xe1\x8dI\x07\xef\xce\x86\xe6\xb6\xdb\xf5\x0e\x1apo" +
|
||||
"\xf4Xv\xf2E\xdfe\x0d\xce\xa1\x84\xb3\xd1\xf6\x1c\xc6" +
|
||||
"\x1d\xc3\xa2;\xd0\x13\xf7,\x05N\xdc)\xa0?\xaaD" +
|
||||
"w2)n\xd3\x81\x137\x0b\xc8y\x83mt\x07\xd8" +
|
||||
"\xe2\xfa\x07\x80\x13\xd7\x0a\xc8{sitGb\xf5=" +
|
||||
"\xc3\x108q\xb9\x80!o\xde\x8f\xee@M\\\xdc\x05" +
|
||||
"\x9c\x98\x150\xec\x8d\xbc\xd1\x9d\xb9\x8a\xb7\xaf\x06N\x9c" +
|
||||
"\xef\x0f~\xa0\xc1\xf1\xa3\x11-\x17\xa3Pk\xa3\xb4\xf7" +
|
||||
"\x18\xc8\xd1\x02hD\xcb\xed\x81\xf9\x8b5\xc1\xb6\x96;" +
|
||||
"\xc9\x80hZ1Y#5gN\xfdc\x89\x00\xa0\x11" +
|
||||
"\xe5\x10\x06\xe6\x89\x00\x97\xfb\x08M\xb2Z;\xcf?\xb4" +
|
||||
"er\xd7\xff@J\xe2\xfb\xb3\x9a\xce\xf1&b\x81}" +
|
||||
"\xa9\x0b\xac\xe6Q\x1e\xcd\x0d\xda\xf8\x85.\xe6\x85\x0b\xfe" +
|
||||
"(-\xa6\xfd\xff\xd6\xdb\xff05N\xef\xf2(\x1f\x0b" +
|
||||
"\x94\xf5Q\x12~\xc0\xa3|<\xd08}D\xb5~\x8c" +
|
||||
"G\xf9\xbc?\xe4\xfc\xea\x01\x00\xf9<\x8f\xc9@#\"" +
|
||||
"\xfe\x89\x14\xbf\xa3\xeb\xdanC\xd0iC\xc2\xb8\x11 " +
|
||||
"UI\xd7x\xdcnCBN\x1b\"b;@*F" +
|
||||
"\xf2+\x82m\xc8\x18\xbc\x0d 5\x9a\xe4\xe3\xb1\xf7\xbb" +
|
||||
"F(\xea~\xa3\x96\xd3:ge\xd5~\xef6w\xea" +
|
||||
"\x8a\xe6L%\x9b+\xea\x0c\xfc\xab\xb5D6\xcd\x81\xdb" +
|
||||
"\xde\x19\xc7:\x93\x97\x14\x810\x83\x867\x95\xb9\x84\x17" +
|
||||
"\xe5\x90n\x9e\x19\xba\xae\xa1^\xd6\xc4N\xf6\x9bX\xaf" +
|
||||
"\x87\xa5^\xfcf\x1e\xe5y\x94\x8aF'\x15r\xbb\xdf" +
|
||||
"v\xd7\xa6\x95\xa2\xc1\xfa\xf8\x00<\xd3\xbd)\x80\xb1H" +
|
||||
"+\xe62I\x06\x82\xa9\xf7\x94\x85`\xd0f6\xc5\xa2" +
|
||||
".s9\x13d\xf7\xbf!\xe8\xfe\xd3#0Av\xc7" +
|
||||
"\xf8\xe8\xfeo\xab\xef\x04\xd9\x8dA\x9f\x09\xb2\xf3\xc1\xc6" +
|
||||
"h\xef\x09\xf2e<_\x9dk,\xc0\x18\x974X\x1d" +
|
||||
"\xf2<\xd2\xfb\xf7oY\xa5W]\xee\x98\xc0\xbd\x90\xfe" +
|
||||
"\x1c\x00\x00\xff\xff\xa1\x1ap\xe9"
|
||||
|
||||
func init() {
|
||||
schemas.Register(schema_db8274f9144abc7e,
|
||||
|
|
Loading…
Reference in New Issue