Add back the old tunnel interface
This commit is contained in:
		| @@ -1,79 +1,436 @@ | |||||||
| package tunnel | package tunnel | ||||||
|  |  | ||||||
| import ( | import ( | ||||||
|  | 	"crypto/sha256" | ||||||
|  | 	"errors" | ||||||
|  | 	"fmt" | ||||||
| 	"sync" | 	"sync" | ||||||
|  |  | ||||||
|  | 	"github.com/google/uuid" | ||||||
| 	"github.com/micro/go-micro/transport" | 	"github.com/micro/go-micro/transport" | ||||||
|  | 	"github.com/micro/go-micro/util/log" | ||||||
| ) | ) | ||||||
|  |  | ||||||
|  | // tun represents a network tunnel | ||||||
| type tun struct { | type tun struct { | ||||||
| 	sync.RWMutex |  | ||||||
| 	tr        transport.Transport |  | ||||||
| 	options Options | 	options Options | ||||||
|  |  | ||||||
|  | 	sync.RWMutex | ||||||
|  |  | ||||||
|  | 	// to indicate if we're connected or not | ||||||
| 	connected bool | 	connected bool | ||||||
|  |  | ||||||
|  | 	// the send channel for all messages | ||||||
|  | 	send chan *message | ||||||
|  |  | ||||||
|  | 	// close channel | ||||||
| 	closed chan bool | 	closed chan bool | ||||||
|  |  | ||||||
|  | 	// a map of sockets based on Micro-Tunnel-Id | ||||||
|  | 	sockets map[string]*socket | ||||||
|  |  | ||||||
|  | 	// outbound links | ||||||
|  | 	links map[string]*link | ||||||
|  |  | ||||||
|  | 	// listener | ||||||
|  | 	listener transport.Listener | ||||||
| } | } | ||||||
|  |  | ||||||
| func newTunnel(opts ...Option) Tunnel { | type link struct { | ||||||
| 	// initialize default options | 	transport.Socket | ||||||
| 	options := DefaultOptions() | 	id string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // create new tunnel on top of a link | ||||||
|  | func newTunnel(opts ...Option) *tun { | ||||||
|  | 	options := DefaultOptions() | ||||||
| 	for _, o := range opts { | 	for _, o := range opts { | ||||||
| 		o(&options) | 		o(&options) | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	// tunnel transport | 	return &tun{ | ||||||
| 	tr := newTransport() |  | ||||||
|  |  | ||||||
| 	t := &tun{ |  | ||||||
| 		tr:      tr, |  | ||||||
| 		options: options, | 		options: options, | ||||||
|  | 		send:    make(chan *message, 128), | ||||||
| 		closed:  make(chan bool), | 		closed:  make(chan bool), | ||||||
|  | 		sockets: make(map[string]*socket), | ||||||
|  | 		links:   make(map[string]*link), | ||||||
|  | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
| 	return t | // getSocket returns a socket from the internal socket map. | ||||||
|  | // It does this based on the Micro-Tunnel-Id and Micro-Tunnel-Session | ||||||
|  | func (t *tun) getSocket(id, session string) (*socket, bool) { | ||||||
|  | 	// get the socket | ||||||
|  | 	t.RLock() | ||||||
|  | 	s, ok := t.sockets[id+session] | ||||||
|  | 	t.RUnlock() | ||||||
|  | 	return s, ok | ||||||
| } | } | ||||||
|  |  | ||||||
| // Id returns tunnel id | // newSocket creates a new socket and saves it | ||||||
| func (t *tun) Id() string { | func (t *tun) newSocket(id, session string) (*socket, bool) { | ||||||
| 	return t.options.Id | 	// hash the id | ||||||
|  | 	h := sha256.New() | ||||||
|  | 	h.Write([]byte(id)) | ||||||
|  | 	id = fmt.Sprintf("%x", h.Sum(nil)) | ||||||
|  |  | ||||||
|  | 	// new socket | ||||||
|  | 	s := &socket{ | ||||||
|  | 		id:      id, | ||||||
|  | 		session: session, | ||||||
|  | 		closed:  make(chan bool), | ||||||
|  | 		recv:    make(chan *message, 128), | ||||||
|  | 		send:    t.send, | ||||||
|  | 		wait:    make(chan bool), | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| // Options returns tunnel options | 	// save socket | ||||||
| func (t *tun) Options() Options { | 	t.Lock() | ||||||
| 	return t.options | 	_, ok := t.sockets[id+session] | ||||||
|  | 	if ok { | ||||||
|  | 		// socket already exists | ||||||
|  | 		t.Unlock() | ||||||
|  | 		return nil, false | ||||||
|  | 	} | ||||||
|  | 	t.sockets[id+session] = s | ||||||
|  | 	t.Unlock() | ||||||
|  |  | ||||||
|  | 	// return socket | ||||||
|  | 	return s, true | ||||||
| } | } | ||||||
|  |  | ||||||
| // Address returns tunnel listen address | // TODO: use tunnel id as part of the session | ||||||
| func (t *tun) Address() string { | func (t *tun) newSession() string { | ||||||
| 	return t.options.Address | 	return uuid.New().String() | ||||||
| } | } | ||||||
|  |  | ||||||
| // Transport returns tunnel client transport | // process outgoing messages sent by all local sockets | ||||||
| func (t *tun) Transport() transport.Transport { | func (t *tun) process() { | ||||||
| 	return t.tr | 	// manage the send buffer | ||||||
|  | 	// all pseudo sockets throw everything down this | ||||||
|  | 	for { | ||||||
|  | 		select { | ||||||
|  | 		case msg := <-t.send: | ||||||
|  | 			nmsg := &transport.Message{ | ||||||
|  | 				Header: msg.data.Header, | ||||||
|  | 				Body:   msg.data.Body, | ||||||
| 			} | 			} | ||||||
|  |  | ||||||
| // Connect connects establishes point to point tunnel | 			// set the tunnel id on the outgoing message | ||||||
| func (t *tun) Connect() error { | 			nmsg.Header["Micro-Tunnel-Id"] = msg.id | ||||||
|  |  | ||||||
|  | 			// set the session id | ||||||
|  | 			nmsg.Header["Micro-Tunnel-Session"] = msg.session | ||||||
|  |  | ||||||
|  | 			// send the message via the interface | ||||||
|  | 			t.RLock() | ||||||
|  | 			for _, link := range t.links { | ||||||
|  | 				link.Send(nmsg) | ||||||
|  | 			} | ||||||
|  | 			t.RUnlock() | ||||||
|  | 		case <-t.closed: | ||||||
|  | 			return | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // process incoming messages | ||||||
|  | func (t *tun) listen(link transport.Socket, listener bool) { | ||||||
|  | 	for { | ||||||
|  | 		// process anything via the net interface | ||||||
|  | 		msg := new(transport.Message) | ||||||
|  | 		err := link.Recv(msg) | ||||||
|  | 		if err != nil { | ||||||
|  | 			return | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// first check Micro-Tunnel | ||||||
|  | 		switch msg.Header["Micro-Tunnel"] { | ||||||
|  | 		case "connect": | ||||||
|  | 			// assuming new connection | ||||||
|  | 			// TODO: do something with this | ||||||
|  | 			continue | ||||||
|  | 		case "close": | ||||||
|  | 			// assuming connection closed | ||||||
|  | 			// TODO: do something with this | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// the tunnel id | ||||||
|  | 		id := msg.Header["Micro-Tunnel-Id"] | ||||||
|  |  | ||||||
|  | 		// the session id | ||||||
|  | 		session := msg.Header["Micro-Tunnel-Session"] | ||||||
|  |  | ||||||
|  | 		// if the session id is blank there's nothing we can do | ||||||
|  | 		// TODO: check this is the case, is there any reason | ||||||
|  | 		// why we'd have a blank session? Is the tunnel | ||||||
|  | 		// used for some other purpose? | ||||||
|  | 		if len(id) == 0 || len(session) == 0 { | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		var s *socket | ||||||
|  | 		var exists bool | ||||||
|  |  | ||||||
|  | 		// if its a local listener then we use that as the session id | ||||||
|  | 		// e.g we're using a loopback connecting to ourselves | ||||||
|  | 		if listener { | ||||||
|  | 			s, exists = t.getSocket(id, "listener") | ||||||
|  | 		} else { | ||||||
|  | 			// get the socket based on the tunnel id and session | ||||||
|  | 			// this could be something we dialed in which case | ||||||
|  | 			// we have a session for it otherwise its a listener | ||||||
|  | 			s, exists = t.getSocket(id, session) | ||||||
|  | 			if !exists { | ||||||
|  | 				// try get it based on just the tunnel id | ||||||
|  | 				// the assumption here is that a listener | ||||||
|  | 				// has no session but its set a listener session | ||||||
|  | 				s, exists = t.getSocket(id, "listener") | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// no socket in existence | ||||||
|  | 		if !exists { | ||||||
|  | 			// drop it, we don't care about | ||||||
|  | 			// messages we don't know about | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// is the socket closed? | ||||||
|  | 		select { | ||||||
|  | 		case <-s.closed: | ||||||
|  | 			// closed | ||||||
|  | 			delete(t.sockets, id) | ||||||
|  | 			continue | ||||||
|  | 		default: | ||||||
|  | 			// process | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// is the socket new? | ||||||
|  | 		select { | ||||||
|  | 		// if its new the socket is actually blocked waiting | ||||||
|  | 		// for a connection. so we check if its waiting. | ||||||
|  | 		case <-s.wait: | ||||||
|  | 		// if its waiting e.g its new then we close it | ||||||
|  | 		default: | ||||||
|  | 			// set remote address of the socket | ||||||
|  | 			s.remote = msg.Header["Remote"] | ||||||
|  | 			close(s.wait) | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// construct a new transport message | ||||||
|  | 		tmsg := &transport.Message{ | ||||||
|  | 			Header: msg.Header, | ||||||
|  | 			Body:   msg.Body, | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// construct the internal message | ||||||
|  | 		imsg := &message{ | ||||||
|  | 			id:      id, | ||||||
|  | 			session: session, | ||||||
|  | 			data:    tmsg, | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// append to recv backlog | ||||||
|  | 		// we don't block if we can't pass it on | ||||||
|  | 		select { | ||||||
|  | 		case s.recv <- imsg: | ||||||
|  | 		default: | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (t *tun) connect() error { | ||||||
|  | 	l, err := t.options.Transport.Listen(t.options.Address) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return err | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// save the listener | ||||||
|  | 	t.listener = l | ||||||
|  |  | ||||||
|  | 	go func() { | ||||||
|  | 		// accept inbound connections | ||||||
|  | 		err := l.Accept(func(sock transport.Socket) { | ||||||
|  | 			// save the link | ||||||
|  | 			id := uuid.New().String() | ||||||
|  | 			t.Lock() | ||||||
|  | 			t.links[id] = &link{ | ||||||
|  | 				Socket: sock, | ||||||
|  | 				id:     id, | ||||||
|  | 			} | ||||||
|  | 			t.Unlock() | ||||||
|  |  | ||||||
|  | 			// delete the link | ||||||
|  | 			defer func() { | ||||||
|  | 				t.Lock() | ||||||
|  | 				delete(t.links, id) | ||||||
|  | 				t.Unlock() | ||||||
|  | 			}() | ||||||
|  |  | ||||||
|  | 			// listen for inbound messages | ||||||
|  | 			t.listen(sock, true) | ||||||
|  | 		}) | ||||||
|  |  | ||||||
|  | 		t.Lock() | ||||||
|  | 		defer t.Unlock() | ||||||
|  |  | ||||||
|  | 		// still connected but the tunnel died | ||||||
|  | 		if err != nil && t.connected { | ||||||
|  | 			log.Logf("Tunnel listener died: %v", err) | ||||||
|  | 		} | ||||||
|  | 	}() | ||||||
|  |  | ||||||
|  | 	for _, node := range t.options.Nodes { | ||||||
|  | 		c, err := t.options.Transport.Dial(node) | ||||||
|  | 		if err != nil { | ||||||
|  | 			log.Debugf("Tunnel failed to connect to %s: %v", node, err) | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		err = c.Send(&transport.Message{ | ||||||
|  | 			Header: map[string]string{ | ||||||
|  | 				"Micro-Tunnel": "connect", | ||||||
|  | 			}, | ||||||
|  | 		}) | ||||||
|  |  | ||||||
|  | 		if err != nil { | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		// process incoming messages | ||||||
|  | 		go t.listen(c, false) | ||||||
|  |  | ||||||
|  | 		// save the link | ||||||
|  | 		id := uuid.New().String() | ||||||
|  | 		t.links[id] = &link{ | ||||||
|  | 			Socket: c, | ||||||
|  | 			id:     id, | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// process outbound messages to be sent | ||||||
|  | 	// process sends to all links | ||||||
|  | 	go t.process() | ||||||
|  |  | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
| // Close closes the tunnel | func (t *tun) close() error { | ||||||
|  | 	// close all the links | ||||||
|  | 	for id, link := range t.links { | ||||||
|  | 		link.Send(&transport.Message{ | ||||||
|  | 			Header: map[string]string{ | ||||||
|  | 				"Micro-Tunnel": "close", | ||||||
|  | 			}, | ||||||
|  | 		}) | ||||||
|  | 		link.Close() | ||||||
|  | 		delete(t.links, id) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// close the listener | ||||||
|  | 	return t.listener.Close() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Close the tunnel | ||||||
| func (t *tun) Close() error { | func (t *tun) Close() error { | ||||||
|  | 	t.Lock() | ||||||
|  | 	defer t.Unlock() | ||||||
|  |  | ||||||
|  | 	if !t.connected { | ||||||
| 		return nil | 		return nil | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| // Status returns tunnel status |  | ||||||
| func (t *tun) Status() Status { |  | ||||||
| 	select { | 	select { | ||||||
| 	case <-t.closed: | 	case <-t.closed: | ||||||
| 		return Closed | 		return nil | ||||||
| 	default: | 	default: | ||||||
| 		return Connected | 		// close all the sockets | ||||||
|  | 		for _, s := range t.sockets { | ||||||
|  | 			s.Close() | ||||||
| 		} | 		} | ||||||
|  | 		// close the connection | ||||||
|  | 		close(t.closed) | ||||||
|  | 		t.connected = false | ||||||
|  |  | ||||||
|  | 		// send a close message | ||||||
|  | 		// we don't close the link | ||||||
|  | 		// just the tunnel | ||||||
|  | 		return t.close() | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| func (t *tun) String() string { | 	return nil | ||||||
| 	return "micro" | } | ||||||
|  |  | ||||||
|  | // Connect the tunnel | ||||||
|  | func (t *tun) Connect() error { | ||||||
|  | 	t.Lock() | ||||||
|  | 	defer t.Unlock() | ||||||
|  |  | ||||||
|  | 	// already connected | ||||||
|  | 	if t.connected { | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// send the connect message | ||||||
|  | 	if err := t.connect(); err != nil { | ||||||
|  | 		return err | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// set as connected | ||||||
|  | 	t.connected = true | ||||||
|  | 	// create new close channel | ||||||
|  | 	t.closed = make(chan bool) | ||||||
|  |  | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Dial an address | ||||||
|  | func (t *tun) Dial(addr string) (Conn, error) { | ||||||
|  | 	c, ok := t.newSocket(addr, t.newSession()) | ||||||
|  | 	if !ok { | ||||||
|  | 		return nil, errors.New("error dialing " + addr) | ||||||
|  | 	} | ||||||
|  | 	// set remote | ||||||
|  | 	c.remote = addr | ||||||
|  | 	// set local | ||||||
|  | 	c.local = "local" | ||||||
|  |  | ||||||
|  | 	return c, nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Accept a connection on the address | ||||||
|  | func (t *tun) Listen(addr string) (Listener, error) { | ||||||
|  | 	// create a new socket by hashing the address | ||||||
|  | 	c, ok := t.newSocket(addr, "listener") | ||||||
|  | 	if !ok { | ||||||
|  | 		return nil, errors.New("already listening on " + addr) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// set remote. it will be replaced by the first message received | ||||||
|  | 	c.remote = "remote" | ||||||
|  | 	// set local | ||||||
|  | 	c.local = addr | ||||||
|  |  | ||||||
|  | 	tl := &tunListener{ | ||||||
|  | 		addr: addr, | ||||||
|  | 		// the accept channel | ||||||
|  | 		accept: make(chan *socket, 128), | ||||||
|  | 		// the channel to close | ||||||
|  | 		closed: make(chan bool), | ||||||
|  | 		// the connection | ||||||
|  | 		conn: c, | ||||||
|  | 		// the listener socket | ||||||
|  | 		socket: c, | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// this kicks off the internal message processor | ||||||
|  | 	// for the listener so it can create pseudo sockets | ||||||
|  | 	// per session if they do not exist or pass messages | ||||||
|  | 	// to the existign sessions | ||||||
|  | 	go tl.process() | ||||||
|  |  | ||||||
|  | 	// return the listener | ||||||
|  | 	return tl, nil | ||||||
| } | } | ||||||
|   | |||||||
							
								
								
									
										101
									
								
								tunnel/listener.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										101
									
								
								tunnel/listener.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,101 @@ | |||||||
|  | package tunnel | ||||||
|  |  | ||||||
|  | import ( | ||||||
|  | 	"io" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | type tunListener struct { | ||||||
|  | 	// address of the listener | ||||||
|  | 	addr string | ||||||
|  | 	// the accept channel | ||||||
|  | 	accept chan *socket | ||||||
|  | 	// the channel to close | ||||||
|  | 	closed chan bool | ||||||
|  | 	// the connection | ||||||
|  | 	conn Conn | ||||||
|  | 	// the listener socket | ||||||
|  | 	socket *socket | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (t *tunListener) process() { | ||||||
|  | 	// our connection map for session | ||||||
|  | 	conns := make(map[string]*socket) | ||||||
|  |  | ||||||
|  | 	for { | ||||||
|  | 		select { | ||||||
|  | 		case <-t.closed: | ||||||
|  | 			return | ||||||
|  | 		// receive a new message | ||||||
|  | 		case m := <-t.socket.recv: | ||||||
|  | 			// get a socket | ||||||
|  | 			sock, ok := conns[m.session] | ||||||
|  | 			if !ok { | ||||||
|  | 				// create a new socket session | ||||||
|  | 				sock = &socket{ | ||||||
|  | 					// our tunnel id | ||||||
|  | 					id: m.id, | ||||||
|  | 					// the session id | ||||||
|  | 					session: m.session, | ||||||
|  | 					// close chan | ||||||
|  | 					closed: make(chan bool), | ||||||
|  | 					// recv called by the acceptor | ||||||
|  | 					recv: make(chan *message, 128), | ||||||
|  | 					// use the internal send buffer | ||||||
|  | 					send: t.socket.send, | ||||||
|  | 					// wait | ||||||
|  | 					wait: make(chan bool), | ||||||
|  | 				} | ||||||
|  |  | ||||||
|  | 				// first message | ||||||
|  | 				sock.recv <- m | ||||||
|  |  | ||||||
|  | 				// save the socket | ||||||
|  | 				conns[m.session] = sock | ||||||
|  |  | ||||||
|  | 				// send to accept chan | ||||||
|  | 				select { | ||||||
|  | 				case <-t.closed: | ||||||
|  | 					return | ||||||
|  | 				case t.accept <- sock: | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  |  | ||||||
|  | 			// send this to the accept chan | ||||||
|  | 			select { | ||||||
|  | 			case <-sock.closed: | ||||||
|  | 				delete(conns, m.session) | ||||||
|  | 			case sock.recv <- m: | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (t *tunListener) Addr() string { | ||||||
|  | 	return t.addr | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (t *tunListener) Close() error { | ||||||
|  | 	select { | ||||||
|  | 	case <-t.closed: | ||||||
|  | 		return nil | ||||||
|  | 	default: | ||||||
|  | 		close(t.closed) | ||||||
|  | 	} | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Everytime accept is called we essentially block till we get a new connection | ||||||
|  | func (t *tunListener) Accept() (Conn, error) { | ||||||
|  | 	select { | ||||||
|  | 	// if the socket is closed return | ||||||
|  | 	case <-t.closed: | ||||||
|  | 		return nil, io.EOF | ||||||
|  | 	// wait for a new connection | ||||||
|  | 	case c, ok := <-t.accept: | ||||||
|  | 		if !ok { | ||||||
|  | 			return nil, io.EOF | ||||||
|  | 		} | ||||||
|  | 		return c, nil | ||||||
|  | 	} | ||||||
|  | 	return nil, nil | ||||||
|  | } | ||||||
| @@ -3,6 +3,7 @@ package tunnel | |||||||
| import ( | import ( | ||||||
| 	"github.com/google/uuid" | 	"github.com/google/uuid" | ||||||
| 	"github.com/micro/go-micro/transport" | 	"github.com/micro/go-micro/transport" | ||||||
|  | 	"github.com/micro/go-micro/transport/quic" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| var ( | var ( | ||||||
| @@ -39,7 +40,7 @@ func Address(a string) Option { | |||||||
| } | } | ||||||
|  |  | ||||||
| // Nodes specify remote network nodes | // Nodes specify remote network nodes | ||||||
| func Nodes(n []string) Option { | func Nodes(n ...string) Option { | ||||||
| 	return func(o *Options) { | 	return func(o *Options) { | ||||||
| 		o.Nodes = n | 		o.Nodes = n | ||||||
| 	} | 	} | ||||||
| @@ -57,7 +58,6 @@ func DefaultOptions() Options { | |||||||
| 	return Options{ | 	return Options{ | ||||||
| 		Id:        uuid.New().String(), | 		Id:        uuid.New().String(), | ||||||
| 		Address:   DefaultAddress, | 		Address:   DefaultAddress, | ||||||
| 		Nodes:     make([]string, 0), | 		Transport: quic.NewTransport(), | ||||||
| 		Transport: transport.DefaultTransport, |  | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
|   | |||||||
| @@ -1,25 +1,90 @@ | |||||||
| package tunnel | package tunnel | ||||||
|  |  | ||||||
| import "github.com/micro/go-micro/transport" | import ( | ||||||
|  | 	"errors" | ||||||
|  |  | ||||||
| type tunSocket struct{} | 	"github.com/micro/go-micro/transport" | ||||||
|  | ) | ||||||
|  |  | ||||||
| func (s *tunSocket) Recv(m *transport.Message) error { | // socket is our pseudo socket for transport.Socket | ||||||
|  | type socket struct { | ||||||
|  | 	// socket id based on Micro-Tunnel | ||||||
|  | 	id string | ||||||
|  | 	// the session id based on Micro.Tunnel-Session | ||||||
|  | 	session string | ||||||
|  | 	// closed | ||||||
|  | 	closed chan bool | ||||||
|  | 	// remote addr | ||||||
|  | 	remote string | ||||||
|  | 	// local addr | ||||||
|  | 	local string | ||||||
|  | 	// send chan | ||||||
|  | 	send chan *message | ||||||
|  | 	// recv chan | ||||||
|  | 	recv chan *message | ||||||
|  | 	// wait until we have a connection | ||||||
|  | 	wait chan bool | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // message is sent over the send channel | ||||||
|  | type message struct { | ||||||
|  | 	// tunnel id | ||||||
|  | 	id string | ||||||
|  | 	// the session id | ||||||
|  | 	session string | ||||||
|  | 	// transport data | ||||||
|  | 	data *transport.Message | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (s *socket) Remote() string { | ||||||
|  | 	return s.remote | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (s *socket) Local() string { | ||||||
|  | 	return s.local | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (s *socket) Id() string { | ||||||
|  | 	return s.id | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (s *socket) Session() string { | ||||||
|  | 	return s.session | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (s *socket) Send(m *transport.Message) error { | ||||||
|  | 	select { | ||||||
|  | 	case <-s.closed: | ||||||
|  | 		return errors.New("socket is closed") | ||||||
|  | 	default: | ||||||
|  | 		// no op | ||||||
|  | 	} | ||||||
|  | 	// append to backlog | ||||||
|  | 	s.send <- &message{id: s.id, session: s.session, data: m} | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
| func (s *tunSocket) Send(m *transport.Message) error { | func (s *socket) Recv(m *transport.Message) error { | ||||||
|  | 	select { | ||||||
|  | 	case <-s.closed: | ||||||
|  | 		return errors.New("socket is closed") | ||||||
|  | 	default: | ||||||
|  | 		// no op | ||||||
|  | 	} | ||||||
|  | 	// recv from backlog | ||||||
|  | 	msg := <-s.recv | ||||||
|  | 	// set message | ||||||
|  | 	*m = *msg.data | ||||||
|  | 	// return nil | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
| func (s *tunSocket) Close() error { | func (s *socket) Close() error { | ||||||
|  | 	select { | ||||||
|  | 	case <-s.closed: | ||||||
|  | 		// no op | ||||||
|  | 	default: | ||||||
|  | 		close(s.closed) | ||||||
|  | 	} | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
| func (s *tunSocket) Local() string { |  | ||||||
| 	return "" |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (s *tunSocket) Remote() string { |  | ||||||
| 	return "" |  | ||||||
| } |  | ||||||
|   | |||||||
| @@ -1,51 +0,0 @@ | |||||||
| package tunnel |  | ||||||
|  |  | ||||||
| import "github.com/micro/go-micro/transport" |  | ||||||
|  |  | ||||||
| type tunTransport struct { |  | ||||||
| 	options transport.Options |  | ||||||
| } |  | ||||||
|  |  | ||||||
| type tunClient struct { |  | ||||||
| 	*tunSocket |  | ||||||
| 	options transport.DialOptions |  | ||||||
| } |  | ||||||
|  |  | ||||||
| type tunListener struct { |  | ||||||
| 	conn chan *tunSocket |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func newTransport(opts ...transport.Option) transport.Transport { |  | ||||||
| 	var options transport.Options |  | ||||||
|  |  | ||||||
| 	for _, o := range opts { |  | ||||||
| 		o(&options) |  | ||||||
| 	} |  | ||||||
|  |  | ||||||
| 	return &tunTransport{ |  | ||||||
| 		options: options, |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (t *tunTransport) Init(opts ...transport.Option) error { |  | ||||||
| 	for _, o := range opts { |  | ||||||
| 		o(&t.options) |  | ||||||
| 	} |  | ||||||
| 	return nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (t *tunTransport) Options() transport.Options { |  | ||||||
| 	return t.options |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (t *tunTransport) Dial(addr string, opts ...transport.DialOption) (transport.Client, error) { |  | ||||||
| 	return nil, nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (t *tunTransport) Listen(addr string, opts ...transport.ListenOption) (transport.Listener, error) { |  | ||||||
| 	return nil, nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (t *tunTransport) String() string { |  | ||||||
| 	return "micro" |  | ||||||
| } |  | ||||||
| @@ -1,39 +1,43 @@ | |||||||
| // Package tunnel provides micro network tunnelling | // Package tunnel provides gre network tunnelling | ||||||
| package tunnel | package tunnel | ||||||
|  |  | ||||||
| import ( | import ( | ||||||
| 	"github.com/micro/go-micro/transport" | 	"github.com/micro/go-micro/transport" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| // Status is tunnel status | // Tunnel creates a gre network tunnel on top of a link. | ||||||
| type Status int | // It establishes multiple streams using the Micro-Tunnel-Id header | ||||||
|  | // and Micro-Tunnel-Session header. The tunnel id is a hash of | ||||||
| const ( | // the address being requested. | ||||||
| 	// Connected means the tunnel is alive |  | ||||||
| 	Connected Status = iota |  | ||||||
| 	// Closed meands the tunnel has been disconnected |  | ||||||
| 	Closed |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| // Tunnel creates a p2p network tunnel. |  | ||||||
| type Tunnel interface { | type Tunnel interface { | ||||||
| 	// Id returns tunnel id |  | ||||||
| 	Id() string |  | ||||||
| 	// Options returns the tunnel options |  | ||||||
| 	Options() Options |  | ||||||
| 	// Address returns tunnel address |  | ||||||
| 	Address() string |  | ||||||
| 	// Transport to use by tunne clients |  | ||||||
| 	Transport() transport.Transport |  | ||||||
| 	// Connect connects the tunnel | 	// Connect connects the tunnel | ||||||
| 	Connect() error | 	Connect() error | ||||||
| 	// Close closes the tunnel | 	// Close closes the tunnel | ||||||
| 	Close() error | 	Close() error | ||||||
| 	// Status returns tunnel status | 	// Dial an endpoint | ||||||
| 	Status() Status | 	Dial(addr string) (Conn, error) | ||||||
|  | 	// Accept connections | ||||||
|  | 	Listen(addr string) (Listener, error) | ||||||
| } | } | ||||||
|  |  | ||||||
| // NewTunnel creates a new tunnel on top of a link | // The listener provides similar constructs to the transport.Listener | ||||||
|  | type Listener interface { | ||||||
|  | 	Addr() string | ||||||
|  | 	Close() error | ||||||
|  | 	Accept() (Conn, error) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Conn is a connection dialed or accepted which includes the tunnel id and session | ||||||
|  | type Conn interface { | ||||||
|  | 	// Specifies the tunnel id | ||||||
|  | 	Id() string | ||||||
|  | 	// The session | ||||||
|  | 	Session() string | ||||||
|  | 	// a transport socket | ||||||
|  | 	transport.Socket | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NewTunnel creates a new tunnel | ||||||
| func NewTunnel(opts ...Option) Tunnel { | func NewTunnel(opts ...Option) Tunnel { | ||||||
| 	return newTunnel(opts...) | 	return newTunnel(opts...) | ||||||
| } | } | ||||||
|   | |||||||
							
								
								
									
										73
									
								
								tunnel/tunnel_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										73
									
								
								tunnel/tunnel_test.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,73 @@ | |||||||
|  | package tunnel | ||||||
|  |  | ||||||
|  | import ( | ||||||
|  | 	"testing" | ||||||
|  |  | ||||||
|  | 	"github.com/micro/go-micro/transport" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // testAccept will accept connections on the transport, create a new link and tunnel on top | ||||||
|  | func testAccept(t *testing.T, tun Tunnel, wait chan bool) { | ||||||
|  | 	// listen on some virtual address | ||||||
|  | 	tl, err := tun.Listen("test-tunnel") | ||||||
|  | 	if err != nil { | ||||||
|  | 		t.Fatal(err) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// accept a connection | ||||||
|  | 	c, err := tl.Accept() | ||||||
|  | 	if err != nil { | ||||||
|  | 		t.Fatal(err) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// get a message | ||||||
|  | 	for { | ||||||
|  | 		m := new(transport.Message) | ||||||
|  | 		if err := c.Recv(m); err != nil { | ||||||
|  | 			t.Fatal(err) | ||||||
|  | 		} | ||||||
|  | 		close(wait) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // testSend will create a new link to an address and then a tunnel on top | ||||||
|  | func testSend(t *testing.T, tun Tunnel) { | ||||||
|  | 	// dial a new session | ||||||
|  | 	c, err := tun.Dial("test-tunnel") | ||||||
|  | 	if err != nil { | ||||||
|  | 		t.Fatal(err) | ||||||
|  | 	} | ||||||
|  | 	//defer c.Close() | ||||||
|  |  | ||||||
|  | 	m := transport.Message{ | ||||||
|  | 		Header: map[string]string{ | ||||||
|  | 			"test": "header", | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	if err := c.Send(&m); err != nil { | ||||||
|  | 		t.Fatal(err) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func TestTunnel(t *testing.T) { | ||||||
|  | 	// create a new listener | ||||||
|  | 	tun := NewTunnel(Nodes(":9096")) | ||||||
|  | 	err := tun.Connect() | ||||||
|  | 	if err != nil { | ||||||
|  | 		t.Fatal(err) | ||||||
|  | 	} | ||||||
|  | 	//defer tun.Close() | ||||||
|  |  | ||||||
|  | 	wait := make(chan bool) | ||||||
|  |  | ||||||
|  | 	// start accepting connections | ||||||
|  | 	go testAccept(t, tun, wait) | ||||||
|  |  | ||||||
|  | 	// send a message | ||||||
|  | 	testSend(t, tun) | ||||||
|  |  | ||||||
|  | 	// wait until message is received | ||||||
|  | 	<-wait | ||||||
|  | } | ||||||
		Reference in New Issue
	
	Block a user