mirror of
https://github.com/tailscale/tailscale.git
synced 2025-08-11 13:18:53 +00:00
tailfs: initial implementation
Add a WebDAV-based folder sharing mechanism that is exposed to local clients at 100.100.100.100:8080 and to remote peers via a new peerapi endpoint at /v0/tailfs. Add the ability to manage folder sharing via the new 'share' CLI sub-command. Updates tailscale/corp#16827 Signed-off-by: Percy Wegmann <percy@tailscale.com>
This commit is contained in:

committed by
Percy Wegmann

parent
2e404b769d
commit
993acf4475
@@ -67,6 +67,7 @@ import (
|
||||
"tailscale.com/syncs"
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/taildrop"
|
||||
"tailscale.com/tailfs"
|
||||
"tailscale.com/tka"
|
||||
"tailscale.com/tsd"
|
||||
"tailscale.com/tstime"
|
||||
@@ -287,6 +288,9 @@ type LocalBackend struct {
|
||||
serveListeners map[netip.AddrPort]*localListener // listeners for local serve traffic
|
||||
serveProxyHandlers sync.Map // string (HTTPHandler.Proxy) => *reverseProxy
|
||||
|
||||
tailfsListeners map[netip.AddrPort]*localListener // listeners for local tailfs traffic
|
||||
tailfsForRemote *tailfs.FileSystemForRemote
|
||||
|
||||
// statusLock must be held before calling statusChanged.Wait() or
|
||||
// statusChanged.Broadcast().
|
||||
statusLock sync.Mutex
|
||||
@@ -428,6 +432,15 @@ func NewLocalBackend(logf logger.Logf, logID logid.PublicID, sys *tsd.System, lo
|
||||
}
|
||||
}
|
||||
|
||||
// initialize Tailfs shares from saved state
|
||||
b.mu.Lock()
|
||||
b.tailfsForRemote = tailfs.NewFileSystemForRemote(logf)
|
||||
shares, err := b.tailfsGetSharesLocked()
|
||||
b.mu.Unlock()
|
||||
if err == nil && len(shares) > 0 {
|
||||
b.tailfsForRemote.SetShares(shares)
|
||||
}
|
||||
|
||||
return b, nil
|
||||
}
|
||||
|
||||
@@ -915,6 +928,7 @@ func (b *LocalBackend) WhoIs(ipp netip.AddrPort) (n tailcfg.NodeView, u tailcfg.
|
||||
var zero tailcfg.NodeView
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
|
||||
nid, ok := b.nodeByAddr[ipp.Addr()]
|
||||
if !ok {
|
||||
var ip netip.Addr
|
||||
@@ -2254,7 +2268,7 @@ func (b *LocalBackend) WatchNotifications(ctx context.Context, mask ipn.NotifyWa
|
||||
b.mu.Lock()
|
||||
b.activeWatchSessions.Add(sessionID)
|
||||
|
||||
const initialBits = ipn.NotifyInitialState | ipn.NotifyInitialPrefs | ipn.NotifyInitialNetMap
|
||||
const initialBits = ipn.NotifyInitialState | ipn.NotifyInitialPrefs | ipn.NotifyInitialNetMap | ipn.NotifyInitialTailfsShares
|
||||
if mask&initialBits != 0 {
|
||||
ini = &ipn.Notify{Version: version.Long()}
|
||||
if mask&ipn.NotifyInitialState != 0 {
|
||||
@@ -2270,6 +2284,17 @@ func (b *LocalBackend) WatchNotifications(ctx context.Context, mask ipn.NotifyWa
|
||||
if mask&ipn.NotifyInitialNetMap != 0 {
|
||||
ini.NetMap = b.netMap
|
||||
}
|
||||
if mask&ipn.NotifyInitialTailfsShares != 0 && b.tailfsSharingEnabledLocked() {
|
||||
shares, err := b.tailfsGetSharesLocked()
|
||||
if err != nil {
|
||||
b.logf("unable to notify initial tailfs shares: %v", err)
|
||||
} else {
|
||||
ini.TailfsShares = make(map[string]string, len(shares))
|
||||
for _, share := range shares {
|
||||
ini.TailfsShares[share.Name] = share.Path
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
handle := b.notifyWatchers.Add(&watchSession{ch, sessionID})
|
||||
@@ -3312,6 +3337,14 @@ func (b *LocalBackend) TCPHandlerForDst(src, dst netip.AddrPort) (handler func(c
|
||||
if dst.Port() == webClientPort && b.ShouldRunWebClient() {
|
||||
return b.handleWebClientConn, opts
|
||||
}
|
||||
if dst.Port() == TailfsLocalPort {
|
||||
fs, ok := b.sys.TailfsForLocal.GetOK()
|
||||
if ok {
|
||||
return func(conn net.Conn) error {
|
||||
return fs.HandleConn(conn, conn.RemoteAddr())
|
||||
}, opts
|
||||
}
|
||||
}
|
||||
if port, ok := b.GetPeerAPIPort(dst.Addr()); ok && dst.Port() == port {
|
||||
return func(c net.Conn) error {
|
||||
b.handlePeerAPIConn(src, dst, c)
|
||||
@@ -4608,6 +4641,11 @@ func (b *LocalBackend) setNetMapLocked(nm *netmap.NetworkMap) {
|
||||
delete(b.nodeByAddr, k)
|
||||
}
|
||||
}
|
||||
|
||||
if b.tailfsSharingEnabledLocked() {
|
||||
b.updateTailfsPeersLocked(nm)
|
||||
b.tailfsNotifyCurrentSharesLocked()
|
||||
}
|
||||
}
|
||||
|
||||
func (b *LocalBackend) updatePeersFromNetmapLocked(nm *netmap.NetworkMap) {
|
||||
@@ -4615,14 +4653,17 @@ func (b *LocalBackend) updatePeersFromNetmapLocked(nm *netmap.NetworkMap) {
|
||||
b.peers = nil
|
||||
return
|
||||
}
|
||||
|
||||
// First pass, mark everything unwanted.
|
||||
for k := range b.peers {
|
||||
b.peers[k] = tailcfg.NodeView{}
|
||||
}
|
||||
|
||||
// Second pass, add everything wanted.
|
||||
for _, p := range nm.Peers {
|
||||
mak.Set(&b.peers, p.ID(), p)
|
||||
}
|
||||
|
||||
// Third pass, remove deleted things.
|
||||
for k, v := range b.peers {
|
||||
if !v.Valid() {
|
||||
@@ -4631,6 +4672,28 @@ func (b *LocalBackend) updatePeersFromNetmapLocked(nm *netmap.NetworkMap) {
|
||||
}
|
||||
}
|
||||
|
||||
// tailfsTransport is an http.RoundTripper that uses the latest value of
|
||||
// b.Dialer().PeerAPITransport() for each round trip and imposes a short
|
||||
// dial timeout to avoid hanging on connecting to offline/unreachable hosts.
|
||||
type tailfsTransport struct {
|
||||
b *LocalBackend
|
||||
}
|
||||
|
||||
func (t *tailfsTransport) RoundTrip(req *http.Request) (*http.Response, error) {
|
||||
// dialTimeout is fairly aggressive to avoid hangs on contacting offline or
|
||||
// unreachable hosts.
|
||||
dialTimeout := 1 * time.Second // TODO(oxtoacart): tune this
|
||||
|
||||
tr := t.b.Dialer().PeerAPITransport().Clone()
|
||||
dialContext := tr.DialContext
|
||||
tr.DialContext = func(ctx context.Context, network, addr string) (net.Conn, error) {
|
||||
ctxWithTimeout, cancel := context.WithTimeout(ctx, dialTimeout)
|
||||
defer cancel()
|
||||
return dialContext(ctxWithTimeout, network, addr)
|
||||
}
|
||||
return tr.RoundTrip(req)
|
||||
}
|
||||
|
||||
// setDebugLogsByCapabilityLocked sets debug logging based on the self node's
|
||||
// capabilities in the provided NetMap.
|
||||
func (b *LocalBackend) setDebugLogsByCapabilityLocked(nm *netmap.NetworkMap) {
|
||||
@@ -4703,6 +4766,10 @@ func (b *LocalBackend) setTCPPortsInterceptedFromNetmapAndPrefsLocked(prefs ipn.
|
||||
}
|
||||
}
|
||||
|
||||
if !b.sys.IsNetstack() {
|
||||
b.updateTailfsListenersLocked()
|
||||
}
|
||||
|
||||
b.reloadServeConfigLocked(prefs)
|
||||
if b.serveConfig.Valid() {
|
||||
servePorts := make([]uint16, 0, 3)
|
||||
|
@@ -803,7 +803,7 @@ func TestWatchNotificationsCallbacks(t *testing.T) {
|
||||
|
||||
// tests LocalBackend.updateNetmapDeltaLocked
|
||||
func TestUpdateNetmapDelta(t *testing.T) {
|
||||
var b LocalBackend
|
||||
b := newTestLocalBackend(t)
|
||||
if b.updateNetmapDeltaLocked(nil) {
|
||||
t.Errorf("updateNetmapDeltaLocked() = true, want false with nil netmap")
|
||||
}
|
||||
|
@@ -38,12 +38,17 @@ import (
|
||||
"tailscale.com/net/sockstats"
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/taildrop"
|
||||
"tailscale.com/tailfs"
|
||||
"tailscale.com/types/views"
|
||||
"tailscale.com/util/clientmetric"
|
||||
"tailscale.com/util/httphdr"
|
||||
"tailscale.com/wgengine/filter"
|
||||
)
|
||||
|
||||
const (
|
||||
tailfsPrefix = "/v0/tailfs"
|
||||
)
|
||||
|
||||
var initListenConfig func(*net.ListenConfig, netip.Addr, *interfaces.State, string) error
|
||||
|
||||
// addH2C is non-nil on platforms where we want to add H2C
|
||||
@@ -317,6 +322,10 @@ func (h *peerAPIHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||
h.handleDNSQuery(w, r)
|
||||
return
|
||||
}
|
||||
if strings.HasPrefix(r.URL.Path, tailfsPrefix) {
|
||||
h.handleServeTailfs(w, r)
|
||||
return
|
||||
}
|
||||
switch r.URL.Path {
|
||||
case "/v0/goroutines":
|
||||
h.handleServeGoroutines(w, r)
|
||||
@@ -626,7 +635,11 @@ func (h *peerAPIHandler) canIngress() bool {
|
||||
}
|
||||
|
||||
func (h *peerAPIHandler) peerHasCap(wantCap tailcfg.PeerCapability) bool {
|
||||
return h.ps.b.PeerCaps(h.remoteAddr.Addr()).HasCapability(wantCap)
|
||||
return h.peerCaps().HasCapability(wantCap)
|
||||
}
|
||||
|
||||
func (h *peerAPIHandler) peerCaps() tailcfg.PeerCapMap {
|
||||
return h.ps.b.PeerCaps(h.remoteAddr.Addr())
|
||||
}
|
||||
|
||||
func (h *peerAPIHandler) handlePeerPut(w http.ResponseWriter, r *http.Request) {
|
||||
@@ -1090,6 +1103,41 @@ func writePrettyDNSReply(w io.Writer, res []byte) (err error) {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *peerAPIHandler) handleServeTailfs(w http.ResponseWriter, r *http.Request) {
|
||||
if !h.ps.b.TailfsSharingEnabled() {
|
||||
http.Error(w, "tailfs not enabled", http.StatusNotFound)
|
||||
return
|
||||
}
|
||||
|
||||
capsMap := h.peerCaps()
|
||||
tailfsCaps, ok := capsMap[tailcfg.PeerCapabilityTailfs]
|
||||
if !ok {
|
||||
http.Error(w, "tailfs not permitted", http.StatusForbidden)
|
||||
return
|
||||
}
|
||||
|
||||
rawPerms := make([][]byte, 0, len(tailfsCaps))
|
||||
for _, cap := range tailfsCaps {
|
||||
rawPerms = append(rawPerms, []byte(cap))
|
||||
}
|
||||
|
||||
p, err := tailfs.ParsePermissions(rawPerms)
|
||||
if err != nil {
|
||||
http.Error(w, err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
|
||||
h.ps.b.mu.Lock()
|
||||
fs := h.ps.b.tailfsForRemote
|
||||
h.ps.b.mu.Unlock()
|
||||
if fs == nil {
|
||||
http.Error(w, "tailfs not enabled", http.StatusNotFound)
|
||||
return
|
||||
}
|
||||
r.URL.Path = strings.TrimPrefix(r.URL.Path, tailfsPrefix)
|
||||
fs.ServeHTTPWithPerms(p, w, r)
|
||||
}
|
||||
|
||||
// newFakePeerAPIListener creates a new net.Listener that acts like
|
||||
// it's listening on the provided IP address and on TCP port 1.
|
||||
//
|
||||
|
@@ -62,7 +62,7 @@ type serveHTTPContext struct {
|
||||
//
|
||||
// This is not used in userspace-networking mode.
|
||||
//
|
||||
// localListener is used by tailscale serve (TCP only) as well as the built-in web client.
|
||||
// localListener is used by tailscale serve (TCP only), the built-in web client and tailfs.
|
||||
// Most serve traffic and peer traffic for the web client are intercepted by netstack.
|
||||
// This listener exists purely for connections from the machine itself, as that goes via the kernel,
|
||||
// so we need to be in the kernel's listening/routing tables.
|
||||
|
318
ipn/ipnlocal/tailfs.go
Normal file
318
ipn/ipnlocal/tailfs.go
Normal file
@@ -0,0 +1,318 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package ipnlocal
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/netip"
|
||||
"os"
|
||||
"regexp"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"tailscale.com/ipn"
|
||||
"tailscale.com/logtail/backoff"
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/tailfs"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/types/netmap"
|
||||
)
|
||||
|
||||
const (
|
||||
// TailfsLocalPort is the port on which the Tailfs listens for location
|
||||
// connections on quad 100.
|
||||
TailfsLocalPort = 8080
|
||||
|
||||
tailfsSharesStateKey = ipn.StateKey("_tailfs-shares")
|
||||
)
|
||||
|
||||
var (
|
||||
shareNameRegex = regexp.MustCompile(`^[a-z0-9_\(\) ]+$`)
|
||||
errInvalidShareName = errors.New("Share names may only contain the letters a-z, underscore _, parentheses (), or spaces")
|
||||
)
|
||||
|
||||
// TailfsSharingEnabled reports whether sharing to remote nodes via tailfs is
|
||||
// enabled. This is currently based on checking for the tailfs:share node
|
||||
// attribute.
|
||||
func (b *LocalBackend) TailfsSharingEnabled() bool {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
return b.tailfsSharingEnabledLocked()
|
||||
}
|
||||
|
||||
func (b *LocalBackend) tailfsSharingEnabledLocked() bool {
|
||||
return b.netMap != nil && b.netMap.SelfNode.HasCap(tailcfg.NodeAttrsTailfsSharingEnabled)
|
||||
}
|
||||
|
||||
// TailfsSetFileServerAddr tells tailfs to use the given address for connecting
|
||||
// to the tailfs.FileServer that's exposing local files as an unprivileged
|
||||
// user.
|
||||
func (b *LocalBackend) TailfsSetFileServerAddr(addr string) error {
|
||||
b.mu.Lock()
|
||||
fs := b.tailfsForRemote
|
||||
b.mu.Unlock()
|
||||
if fs == nil {
|
||||
return errors.New("tailfs not enabled")
|
||||
}
|
||||
|
||||
fs.SetFileServerAddr(addr)
|
||||
return nil
|
||||
}
|
||||
|
||||
// TailfsAddShare adds the given share if no share with that name exists, or
|
||||
// replaces the existing share if one with the same name already exists.
|
||||
// To avoid potential incompatibilities across file systems, share names are
|
||||
// limited to alphanumeric characters and the underscore _.
|
||||
func (b *LocalBackend) TailfsAddShare(share *tailfs.Share) error {
|
||||
var err error
|
||||
share.Name, err = normalizeShareName(share.Name)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
b.mu.Lock()
|
||||
shares, err := b.tailfsAddShareLocked(share)
|
||||
b.mu.Unlock()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
b.tailfsNotifyShares(shares)
|
||||
return nil
|
||||
}
|
||||
|
||||
// normalizeShareName normalizes the given share name and returns an error if
|
||||
// it contains any disallowed characters.
|
||||
func normalizeShareName(name string) (string, error) {
|
||||
// Force all share names to lowercase to avoid potential incompatibilities
|
||||
// with clients that don't support case-sensitive filenames.
|
||||
name = strings.ToLower(name)
|
||||
|
||||
// Trim whitespace
|
||||
name = strings.TrimSpace(name)
|
||||
|
||||
if !shareNameRegex.MatchString(name) {
|
||||
return "", errInvalidShareName
|
||||
}
|
||||
|
||||
return name, nil
|
||||
}
|
||||
|
||||
func (b *LocalBackend) tailfsAddShareLocked(share *tailfs.Share) (map[string]string, error) {
|
||||
if b.tailfsForRemote == nil {
|
||||
return nil, errors.New("tailfs not enabled")
|
||||
}
|
||||
|
||||
shares, err := b.tailfsGetSharesLocked()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
shares[share.Name] = share
|
||||
data, err := json.Marshal(shares)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("marshal: %w", err)
|
||||
}
|
||||
err = b.store.WriteState(tailfsSharesStateKey, data)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("write state: %w", err)
|
||||
}
|
||||
b.tailfsForRemote.SetShares(shares)
|
||||
|
||||
return shareNameMap(shares), nil
|
||||
}
|
||||
|
||||
// TailfsRemoveShare removes the named share. Share names are forced to
|
||||
// lowercase.
|
||||
func (b *LocalBackend) TailfsRemoveShare(name string) error {
|
||||
// Force all share names to lowercase to avoid potential incompatibilities
|
||||
// with clients that don't support case-sensitive filenames.
|
||||
name = strings.ToLower(name)
|
||||
|
||||
b.mu.Lock()
|
||||
shares, err := b.tailfsRemoveShareLocked(name)
|
||||
b.mu.Unlock()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
b.tailfsNotifyShares(shares)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *LocalBackend) tailfsRemoveShareLocked(name string) (map[string]string, error) {
|
||||
if b.tailfsForRemote == nil {
|
||||
return nil, errors.New("tailfs not enabled")
|
||||
}
|
||||
|
||||
shares, err := b.tailfsGetSharesLocked()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
_, shareExists := shares[name]
|
||||
if !shareExists {
|
||||
return nil, os.ErrNotExist
|
||||
}
|
||||
delete(shares, name)
|
||||
data, err := json.Marshal(shares)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("marshal: %w", err)
|
||||
}
|
||||
err = b.store.WriteState(tailfsSharesStateKey, data)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("write state: %w", err)
|
||||
}
|
||||
b.tailfsForRemote.SetShares(shares)
|
||||
|
||||
return shareNameMap(shares), nil
|
||||
}
|
||||
|
||||
func shareNameMap(sharesByName map[string]*tailfs.Share) map[string]string {
|
||||
sharesMap := make(map[string]string, len(sharesByName))
|
||||
for _, share := range sharesByName {
|
||||
sharesMap[share.Name] = share.Path
|
||||
}
|
||||
return sharesMap
|
||||
}
|
||||
|
||||
// tailfsNotifyShares notifies IPN bus listeners (e.g. Mac Application process)
|
||||
// about the latest set of shares, supplied as a map of name -> directory.
|
||||
func (b *LocalBackend) tailfsNotifyShares(shares map[string]string) {
|
||||
b.send(ipn.Notify{TailfsShares: shares})
|
||||
}
|
||||
|
||||
// tailfsNotifyCurrentSharesLocked sends an ipn.Notify with the current set of
|
||||
// tailfs shares.
|
||||
func (b *LocalBackend) tailfsNotifyCurrentSharesLocked() {
|
||||
shares, err := b.tailfsGetSharesLocked()
|
||||
if err != nil {
|
||||
b.logf("error notifying current tailfs shares: %v", err)
|
||||
return
|
||||
}
|
||||
// Do the below on a goroutine to avoid deadlocking on b.mu in b.send().
|
||||
go b.tailfsNotifyShares(shareNameMap(shares))
|
||||
}
|
||||
|
||||
// TailfsGetShares() returns the current set of shares from the state store.
|
||||
func (b *LocalBackend) TailfsGetShares() (map[string]*tailfs.Share, error) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
|
||||
return b.tailfsGetSharesLocked()
|
||||
}
|
||||
|
||||
func (b *LocalBackend) tailfsGetSharesLocked() (map[string]*tailfs.Share, error) {
|
||||
data, err := b.store.ReadState(tailfsSharesStateKey)
|
||||
if err != nil {
|
||||
if errors.Is(err, ipn.ErrStateNotExist) {
|
||||
return make(map[string]*tailfs.Share), nil
|
||||
}
|
||||
return nil, fmt.Errorf("read state: %w", err)
|
||||
}
|
||||
|
||||
var shares map[string]*tailfs.Share
|
||||
err = json.Unmarshal(data, &shares)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("unmarshal: %w", err)
|
||||
}
|
||||
|
||||
return shares, nil
|
||||
}
|
||||
|
||||
// updateTailfsListenersLocked creates listeners on the local Tailfs port.
|
||||
// This is needed to properly route local traffic when using kernel networking
|
||||
// mode.
|
||||
func (b *LocalBackend) updateTailfsListenersLocked() {
|
||||
if b.netMap == nil {
|
||||
return
|
||||
}
|
||||
|
||||
addrs := b.netMap.GetAddresses()
|
||||
oldListeners := b.tailfsListeners
|
||||
newListeners := make(map[netip.AddrPort]*localListener, addrs.Len())
|
||||
for i := range addrs.LenIter() {
|
||||
if fs, ok := b.sys.TailfsForLocal.GetOK(); ok {
|
||||
addrPort := netip.AddrPortFrom(addrs.At(i).Addr(), TailfsLocalPort)
|
||||
if sl, ok := b.tailfsListeners[addrPort]; ok {
|
||||
newListeners[addrPort] = sl
|
||||
delete(oldListeners, addrPort)
|
||||
continue // already listening
|
||||
}
|
||||
|
||||
sl := b.newTailfsListener(context.Background(), fs, addrPort, b.logf)
|
||||
newListeners[addrPort] = sl
|
||||
go sl.Run()
|
||||
}
|
||||
}
|
||||
|
||||
// At this point, anything left in oldListeners can be stopped.
|
||||
for _, sl := range oldListeners {
|
||||
sl.cancel()
|
||||
}
|
||||
}
|
||||
|
||||
// newTailfsListener returns a listener for local connections to a tailfs
|
||||
// WebDAV FileSystem.
|
||||
func (b *LocalBackend) newTailfsListener(ctx context.Context, fs *tailfs.FileSystemForLocal, ap netip.AddrPort, logf logger.Logf) *localListener {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
return &localListener{
|
||||
b: b,
|
||||
ap: ap,
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
logf: logf,
|
||||
|
||||
handler: func(conn net.Conn) error {
|
||||
return fs.HandleConn(conn, conn.RemoteAddr())
|
||||
},
|
||||
bo: backoff.NewBackoff(fmt.Sprintf("tailfs-listener-%d", ap.Port()), logf, 30*time.Second),
|
||||
}
|
||||
}
|
||||
|
||||
// updateTailfsPeersLocked sets all applicable peers from the netmap as tailfs
|
||||
// remotes.
|
||||
func (b *LocalBackend) updateTailfsPeersLocked(nm *netmap.NetworkMap) {
|
||||
fs, ok := b.sys.TailfsForLocal.GetOK()
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
tailfsRemotes := make([]*tailfs.Remote, 0, len(nm.Peers))
|
||||
for _, p := range nm.Peers {
|
||||
peerID := p.ID()
|
||||
url := fmt.Sprintf("%s/%s", peerAPIBase(nm, p), tailfsPrefix[1:])
|
||||
tailfsRemotes = append(tailfsRemotes, &tailfs.Remote{
|
||||
Name: p.DisplayName(false),
|
||||
URL: url,
|
||||
Available: func() bool {
|
||||
// TODO(oxtoacart): need to figure out a performant and reliable way to only
|
||||
// show the peers that have shares to which we have access
|
||||
// This will require work on the control server to transmit the inverse
|
||||
// of the "tailscale.com/cap/tailfs" capability.
|
||||
// For now, at least limit it only to nodes that are online.
|
||||
// Note, we have to iterate the latest netmap because the peer we got from the first iteration may not be it
|
||||
b.mu.Lock()
|
||||
latestNetMap := b.netMap
|
||||
b.mu.Unlock()
|
||||
|
||||
for _, candidate := range latestNetMap.Peers {
|
||||
if candidate.ID() == peerID {
|
||||
online := candidate.Online()
|
||||
// TODO(oxtoacart): for some reason, this correctly
|
||||
// catches when a node goes from offline to online,
|
||||
// but not the other way around...
|
||||
return online != nil && *online
|
||||
}
|
||||
}
|
||||
|
||||
// peer not found, must not be available
|
||||
return false
|
||||
},
|
||||
})
|
||||
}
|
||||
fs.SetRemotes(b.netMap.Domain, tailfsRemotes, &tailfsTransport{b: b})
|
||||
}
|
40
ipn/ipnlocal/tailfs_test.go
Normal file
40
ipn/ipnlocal/tailfs_test.go
Normal file
@@ -0,0 +1,40 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package ipnlocal
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestNormalizeShareName(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
want string
|
||||
err error
|
||||
}{
|
||||
{
|
||||
name: " (_this is A 5 nAme )_ ",
|
||||
want: "(_this is a 5 name )_",
|
||||
},
|
||||
{
|
||||
name: "",
|
||||
err: errInvalidShareName,
|
||||
},
|
||||
{
|
||||
name: "generally good except for .",
|
||||
err: errInvalidShareName,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(fmt.Sprintf("name %q", tt.name), func(t *testing.T) {
|
||||
got, err := normalizeShareName(tt.name)
|
||||
if tt.err != nil && err != tt.err {
|
||||
t.Errorf("wanted error %v, got %v", tt.err, err)
|
||||
} else if got != tt.want {
|
||||
t.Errorf("wanted %q, got %q", tt.want, got)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user