mirror of
https://github.com/tailscale/tailscale.git
synced 2025-08-14 06:57:31 +00:00
feature/taildrop: move rest of Taildrop out of LocalBackend
Updates #12614 Change-Id: If451dec1d796f6a4216fe485975c87f0c62a53e5 Signed-off-by: Brad Fitzpatrick <bradfitz@tailscale.com> Co-authored-by: Nick Khyl <nickk@tailscale.com>
This commit is contained in:

committed by
Brad Fitzpatrick

parent
cf6a593196
commit
068d5ab655
@@ -4,10 +4,30 @@
|
||||
package taildrop
|
||||
|
||||
import (
|
||||
"cmp"
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"maps"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"slices"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
||||
"tailscale.com/client/tailscale/apitype"
|
||||
"tailscale.com/ipn"
|
||||
"tailscale.com/ipn/ipnext"
|
||||
"tailscale.com/ipn/ipnlocal"
|
||||
"tailscale.com/ipn/ipnstate"
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/taildrop"
|
||||
"tailscale.com/tstime"
|
||||
"tailscale.com/types/empty"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/util/osshare"
|
||||
"tailscale.com/util/set"
|
||||
)
|
||||
|
||||
func init() {
|
||||
@@ -15,38 +35,374 @@ func init() {
|
||||
}
|
||||
|
||||
func newExtension(logf logger.Logf, b ipnext.SafeBackend) (ipnext.Extension, error) {
|
||||
return &extension{
|
||||
logf: logger.WithPrefix(logf, "taildrop: "),
|
||||
}, nil
|
||||
e := &Extension{
|
||||
sb: b,
|
||||
stateStore: b.Sys().StateStore.Get(),
|
||||
logf: logger.WithPrefix(logf, "taildrop: "),
|
||||
}
|
||||
e.setPlatformDefaultDirectFileRoot()
|
||||
return e, nil
|
||||
}
|
||||
|
||||
type extension struct {
|
||||
logf logger.Logf
|
||||
sb ipnext.SafeBackend
|
||||
mgr *taildrop.Manager
|
||||
// Extension implements Taildrop.
|
||||
type Extension struct {
|
||||
logf logger.Logf
|
||||
sb ipnext.SafeBackend
|
||||
stateStore ipn.StateStore
|
||||
host ipnext.Host // from Init
|
||||
|
||||
// directFileRoot, if non-empty, means to write received files
|
||||
// directly to this directory, without staging them in an
|
||||
// intermediate buffered directory for "pick-up" later. If
|
||||
// empty, the files are received in a daemon-owned location
|
||||
// and the localapi is used to enumerate, download, and delete
|
||||
// them. This is used on macOS where the GUI lifetime is the
|
||||
// same as the Network Extension lifetime and we can thus avoid
|
||||
// double-copying files by writing them to the right location
|
||||
// immediately.
|
||||
// It's also used on several NAS platforms (Synology, TrueNAS, etc)
|
||||
// but in that case DoFinalRename is also set true, which moves the
|
||||
// *.partial file to its final name on completion.
|
||||
directFileRoot string
|
||||
|
||||
nodeBackendForTest ipnext.NodeBackend // if non-nil, pretend we're this node state for tests
|
||||
|
||||
mu sync.Mutex // Lock order: lb.mu > e.mu
|
||||
backendState ipn.State
|
||||
selfUID tailcfg.UserID
|
||||
capFileSharing bool
|
||||
fileWaiters set.HandleSet[context.CancelFunc] // of wake-up funcs
|
||||
mgr atomic.Pointer[taildrop.Manager] // mutex held to write; safe to read without lock;
|
||||
// outgoingFiles keeps track of Taildrop outgoing files keyed to their OutgoingFile.ID
|
||||
outgoingFiles map[string]*ipn.OutgoingFile
|
||||
}
|
||||
|
||||
func (e *extension) Name() string {
|
||||
func (e *Extension) Name() string {
|
||||
return "taildrop"
|
||||
}
|
||||
|
||||
func (e *extension) Init(h ipnext.Host) error {
|
||||
// TODO(bradfitz): move init of taildrop.Manager from ipnlocal/peerapi.go to
|
||||
// here
|
||||
e.mgr = nil
|
||||
func (e *Extension) Init(h ipnext.Host) error {
|
||||
e.host = h
|
||||
|
||||
osshare.SetFileSharingEnabled(false, e.logf)
|
||||
|
||||
h.Hooks().ProfileStateChange.Add(e.onChangeProfile)
|
||||
h.Hooks().OnSelfChange.Add(e.onSelfChange)
|
||||
h.Hooks().MutateNotifyLocked.Add(e.setNotifyFilesWaiting)
|
||||
h.Hooks().SetPeerStatus.Add(e.setPeerStatus)
|
||||
h.Hooks().BackendStateChange.Add(e.onBackendStateChange)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *extension) Shutdown() error {
|
||||
lb, ok := e.sb.(*ipnlocal.LocalBackend)
|
||||
if !ok {
|
||||
return nil
|
||||
func (e *Extension) onBackendStateChange(st ipn.State) {
|
||||
e.mu.Lock()
|
||||
defer e.mu.Unlock()
|
||||
e.backendState = st
|
||||
}
|
||||
|
||||
func (e *Extension) onSelfChange(self tailcfg.NodeView) {
|
||||
e.mu.Lock()
|
||||
defer e.mu.Unlock()
|
||||
|
||||
e.selfUID = 0
|
||||
if self.Valid() {
|
||||
e.selfUID = self.User()
|
||||
}
|
||||
if mgr, err := lb.TaildropManager(); err == nil {
|
||||
mgr.Shutdown()
|
||||
} else {
|
||||
e.logf("taildrop: failed to shutdown taildrop manager: %v", err)
|
||||
e.capFileSharing = self.Valid() && self.CapMap().Contains(tailcfg.CapabilityFileSharing)
|
||||
osshare.SetFileSharingEnabled(e.capFileSharing, e.logf)
|
||||
}
|
||||
|
||||
func (e *Extension) setMgrLocked(mgr *taildrop.Manager) {
|
||||
if old := e.mgr.Swap(mgr); old != nil {
|
||||
old.Shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
func (e *Extension) onChangeProfile(profile ipn.LoginProfileView, _ ipn.PrefsView, sameNode bool) {
|
||||
e.mu.Lock()
|
||||
defer e.mu.Unlock()
|
||||
|
||||
uid := profile.UserProfile().ID
|
||||
activeLogin := profile.UserProfile().LoginName
|
||||
|
||||
if uid == 0 {
|
||||
e.setMgrLocked(nil)
|
||||
e.outgoingFiles = nil
|
||||
return
|
||||
}
|
||||
|
||||
if sameNode && e.manager() != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// If we have a netmap, create a taildrop manager.
|
||||
fileRoot, isDirectFileMode := e.fileRoot(uid, activeLogin)
|
||||
if fileRoot == "" {
|
||||
e.logf("no Taildrop directory configured")
|
||||
}
|
||||
e.setMgrLocked(taildrop.ManagerOptions{
|
||||
Logf: e.logf,
|
||||
Clock: tstime.DefaultClock{Clock: e.sb.Clock()},
|
||||
State: e.stateStore,
|
||||
Dir: fileRoot,
|
||||
DirectFileMode: isDirectFileMode,
|
||||
SendFileNotify: e.sendFileNotify,
|
||||
}.New())
|
||||
}
|
||||
|
||||
// fileRoot returns where to store Taildrop files for the given user and whether
|
||||
// to write received files directly to this directory, without staging them in
|
||||
// an intermediate buffered directory for "pick-up" later.
|
||||
//
|
||||
// It is safe to call this with b.mu held but it does not require it or acquire
|
||||
// it itself.
|
||||
func (e *Extension) fileRoot(uid tailcfg.UserID, activeLogin string) (root string, isDirect bool) {
|
||||
if v := e.directFileRoot; v != "" {
|
||||
return v, true
|
||||
}
|
||||
varRoot := e.sb.TailscaleVarRoot()
|
||||
if varRoot == "" {
|
||||
e.logf("Taildrop disabled; no state directory")
|
||||
return "", false
|
||||
}
|
||||
|
||||
if activeLogin == "" {
|
||||
e.logf("taildrop: no active login; can't select a target directory")
|
||||
return "", false
|
||||
}
|
||||
|
||||
baseDir := fmt.Sprintf("%s-uid-%d",
|
||||
strings.ReplaceAll(activeLogin, "@", "-"),
|
||||
uid)
|
||||
dir := filepath.Join(varRoot, "files", baseDir)
|
||||
if err := os.MkdirAll(dir, 0700); err != nil {
|
||||
e.logf("Taildrop disabled; error making directory: %v", err)
|
||||
return "", false
|
||||
}
|
||||
return dir, false
|
||||
}
|
||||
|
||||
// hasCapFileSharing reports whether the current node has the file sharing
|
||||
// capability.
|
||||
func (e *Extension) hasCapFileSharing() bool {
|
||||
e.mu.Lock()
|
||||
defer e.mu.Unlock()
|
||||
return e.capFileSharing
|
||||
}
|
||||
|
||||
// manager returns the active taildrop.Manager, or nil.
|
||||
//
|
||||
// Methods on a nil Manager are safe to call.
|
||||
func (e *Extension) manager() *taildrop.Manager {
|
||||
return e.mgr.Load()
|
||||
}
|
||||
|
||||
func (e *Extension) Clock() tstime.Clock {
|
||||
return e.sb.Clock()
|
||||
}
|
||||
|
||||
func (e *Extension) Shutdown() error {
|
||||
e.manager().Shutdown() // no-op on nil receiver
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *Extension) sendFileNotify() {
|
||||
mgr := e.manager()
|
||||
if mgr == nil {
|
||||
return
|
||||
}
|
||||
|
||||
var n ipn.Notify
|
||||
|
||||
e.mu.Lock()
|
||||
for _, wakeWaiter := range e.fileWaiters {
|
||||
wakeWaiter()
|
||||
}
|
||||
n.IncomingFiles = mgr.IncomingFiles()
|
||||
e.mu.Unlock()
|
||||
|
||||
e.host.SendNotifyAsync(n)
|
||||
}
|
||||
|
||||
func (e *Extension) setNotifyFilesWaiting(n *ipn.Notify) {
|
||||
if e.manager().HasFilesWaiting() {
|
||||
n.FilesWaiting = &empty.Message{}
|
||||
}
|
||||
}
|
||||
|
||||
func (e *Extension) setPeerStatus(ps *ipnstate.PeerStatus, p tailcfg.NodeView, nb ipnext.NodeBackend) {
|
||||
ps.TaildropTarget = e.taildropTargetStatus(p, nb)
|
||||
}
|
||||
|
||||
func (e *Extension) removeFileWaiter(handle set.Handle) {
|
||||
e.mu.Lock()
|
||||
defer e.mu.Unlock()
|
||||
delete(e.fileWaiters, handle)
|
||||
}
|
||||
|
||||
func (e *Extension) addFileWaiter(wakeWaiter context.CancelFunc) set.Handle {
|
||||
e.mu.Lock()
|
||||
defer e.mu.Unlock()
|
||||
return e.fileWaiters.Add(wakeWaiter)
|
||||
}
|
||||
|
||||
func (e *Extension) WaitingFiles() ([]apitype.WaitingFile, error) {
|
||||
return e.manager().WaitingFiles()
|
||||
}
|
||||
|
||||
// AwaitWaitingFiles is like WaitingFiles but blocks while ctx is not done,
|
||||
// waiting for any files to be available.
|
||||
//
|
||||
// On return, exactly one of the results will be non-empty or non-nil,
|
||||
// respectively.
|
||||
func (e *Extension) AwaitWaitingFiles(ctx context.Context) ([]apitype.WaitingFile, error) {
|
||||
if ff, err := e.WaitingFiles(); err != nil || len(ff) > 0 {
|
||||
return ff, err
|
||||
}
|
||||
if err := ctx.Err(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for {
|
||||
gotFile, gotFileCancel := context.WithCancel(context.Background())
|
||||
defer gotFileCancel()
|
||||
|
||||
handle := e.addFileWaiter(gotFileCancel)
|
||||
defer e.removeFileWaiter(handle)
|
||||
|
||||
// Now that we've registered ourselves, check again, in case
|
||||
// of race. Otherwise there's a small window where we could
|
||||
// miss a file arrival and wait forever.
|
||||
if ff, err := e.WaitingFiles(); err != nil || len(ff) > 0 {
|
||||
return ff, err
|
||||
}
|
||||
|
||||
select {
|
||||
case <-gotFile.Done():
|
||||
if ff, err := e.WaitingFiles(); err != nil || len(ff) > 0 {
|
||||
return ff, err
|
||||
}
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (e *Extension) DeleteFile(name string) error {
|
||||
return e.manager().DeleteFile(name)
|
||||
}
|
||||
|
||||
func (e *Extension) OpenFile(name string) (rc io.ReadCloser, size int64, err error) {
|
||||
return e.manager().OpenFile(name)
|
||||
}
|
||||
|
||||
func (e *Extension) nodeBackend() ipnext.NodeBackend {
|
||||
if e.nodeBackendForTest != nil {
|
||||
return e.nodeBackendForTest
|
||||
}
|
||||
return e.host.NodeBackend()
|
||||
}
|
||||
|
||||
// FileTargets lists nodes that the current node can send files to.
|
||||
func (e *Extension) FileTargets() ([]*apitype.FileTarget, error) {
|
||||
var ret []*apitype.FileTarget
|
||||
|
||||
e.mu.Lock()
|
||||
st := e.backendState
|
||||
self := e.selfUID
|
||||
e.mu.Unlock()
|
||||
|
||||
if st != ipn.Running {
|
||||
return nil, errors.New("not connected to the tailnet")
|
||||
}
|
||||
if !e.hasCapFileSharing() {
|
||||
return nil, errors.New("file sharing not enabled by Tailscale admin")
|
||||
}
|
||||
nb := e.nodeBackend()
|
||||
peers := nb.AppendMatchingPeers(nil, func(p tailcfg.NodeView) bool {
|
||||
if !p.Valid() || p.Hostinfo().OS() == "tvOS" {
|
||||
return false
|
||||
}
|
||||
if self == p.User() {
|
||||
return true
|
||||
}
|
||||
if nb.PeerHasCap(p, tailcfg.PeerCapabilityFileSharingTarget) {
|
||||
// Explicitly noted in the netmap ACL caps as a target.
|
||||
return true
|
||||
}
|
||||
return false
|
||||
})
|
||||
for _, p := range peers {
|
||||
peerAPI := nb.PeerAPIBase(p)
|
||||
if peerAPI == "" {
|
||||
continue
|
||||
}
|
||||
ret = append(ret, &apitype.FileTarget{
|
||||
Node: p.AsStruct(),
|
||||
PeerAPIURL: peerAPI,
|
||||
})
|
||||
}
|
||||
slices.SortFunc(ret, func(a, b *apitype.FileTarget) int {
|
||||
return cmp.Compare(a.Node.Name, b.Node.Name)
|
||||
})
|
||||
return ret, nil
|
||||
}
|
||||
|
||||
func (e *Extension) taildropTargetStatus(p tailcfg.NodeView, nb ipnext.NodeBackend) ipnstate.TaildropTargetStatus {
|
||||
e.mu.Lock()
|
||||
st := e.backendState
|
||||
selfUID := e.selfUID
|
||||
capFileSharing := e.capFileSharing
|
||||
e.mu.Unlock()
|
||||
|
||||
if st != ipn.Running {
|
||||
return ipnstate.TaildropTargetIpnStateNotRunning
|
||||
}
|
||||
|
||||
if !capFileSharing {
|
||||
return ipnstate.TaildropTargetMissingCap
|
||||
}
|
||||
if !p.Valid() {
|
||||
return ipnstate.TaildropTargetNoPeerInfo
|
||||
}
|
||||
if !p.Online().Get() {
|
||||
return ipnstate.TaildropTargetOffline
|
||||
}
|
||||
if p.Hostinfo().OS() == "tvOS" {
|
||||
return ipnstate.TaildropTargetUnsupportedOS
|
||||
}
|
||||
if selfUID != p.User() {
|
||||
// Different user must have the explicit file sharing target capability
|
||||
if !nb.PeerHasCap(p, tailcfg.PeerCapabilityFileSharingTarget) {
|
||||
return ipnstate.TaildropTargetOwnedByOtherUser
|
||||
}
|
||||
}
|
||||
if !nb.PeerHasPeerAPI(p) {
|
||||
return ipnstate.TaildropTargetNoPeerAPI
|
||||
}
|
||||
return ipnstate.TaildropTargetAvailable
|
||||
}
|
||||
|
||||
// updateOutgoingFiles updates b.outgoingFiles to reflect the given updates and
|
||||
// sends an ipn.Notify with the full list of outgoingFiles.
|
||||
func (e *Extension) updateOutgoingFiles(updates map[string]*ipn.OutgoingFile) {
|
||||
e.mu.Lock()
|
||||
if e.outgoingFiles == nil {
|
||||
e.outgoingFiles = make(map[string]*ipn.OutgoingFile, len(updates))
|
||||
}
|
||||
maps.Copy(e.outgoingFiles, updates)
|
||||
outgoingFiles := make([]*ipn.OutgoingFile, 0, len(e.outgoingFiles))
|
||||
for _, file := range e.outgoingFiles {
|
||||
outgoingFiles = append(outgoingFiles, file)
|
||||
}
|
||||
e.mu.Unlock()
|
||||
slices.SortFunc(outgoingFiles, func(a, b *ipn.OutgoingFile) int {
|
||||
t := a.Started.Compare(b.Started)
|
||||
if t != 0 {
|
||||
return t
|
||||
}
|
||||
return strings.Compare(a.Name, b.Name)
|
||||
})
|
||||
|
||||
e.host.SendNotifyAsync(ipn.Notify{OutgoingFiles: outgoingFiles})
|
||||
}
|
||||
|
@@ -21,6 +21,7 @@ import (
|
||||
|
||||
"tailscale.com/client/tailscale/apitype"
|
||||
"tailscale.com/ipn"
|
||||
"tailscale.com/ipn/ipnlocal"
|
||||
"tailscale.com/ipn/localapi"
|
||||
"tailscale.com/tailcfg"
|
||||
"tailscale.com/taildrop"
|
||||
@@ -80,9 +81,13 @@ func serveFilePut(h *localapi.Handler, w http.ResponseWriter, r *http.Request) {
|
||||
return
|
||||
}
|
||||
|
||||
lb := h.LocalBackend()
|
||||
ext, ok := ipnlocal.GetExt[*Extension](h.LocalBackend())
|
||||
if !ok {
|
||||
http.Error(w, "misconfigured taildrop extension", http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
|
||||
fts, err := lb.FileTargets()
|
||||
fts, err := ext.FileTargets()
|
||||
if err != nil {
|
||||
http.Error(w, err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
@@ -131,7 +136,7 @@ func serveFilePut(h *localapi.Handler, w http.ResponseWriter, r *http.Request) {
|
||||
|
||||
go func() {
|
||||
defer t.Stop()
|
||||
defer lb.UpdateOutgoingFiles(outgoingFiles)
|
||||
defer ext.updateOutgoingFiles(outgoingFiles)
|
||||
for {
|
||||
select {
|
||||
case u, ok := <-progressUpdates:
|
||||
@@ -140,7 +145,7 @@ func serveFilePut(h *localapi.Handler, w http.ResponseWriter, r *http.Request) {
|
||||
}
|
||||
outgoingFiles[u.ID] = &u
|
||||
case <-t.C:
|
||||
lb.UpdateOutgoingFiles(outgoingFiles)
|
||||
ext.updateOutgoingFiles(outgoingFiles)
|
||||
}
|
||||
}
|
||||
}()
|
||||
@@ -301,7 +306,11 @@ func singleFilePut(
|
||||
fail()
|
||||
return false
|
||||
}
|
||||
switch resp, err := client.Do(req); {
|
||||
resp, err := client.Do(req)
|
||||
if resp != nil {
|
||||
defer resp.Body.Close()
|
||||
}
|
||||
switch {
|
||||
case err != nil:
|
||||
h.Logf("could not fetch remote hashes: %v", err)
|
||||
case resp.StatusCode == http.StatusMethodNotAllowed || resp.StatusCode == http.StatusNotFound:
|
||||
@@ -353,7 +362,13 @@ func serveFiles(h *localapi.Handler, w http.ResponseWriter, r *http.Request) {
|
||||
http.Error(w, "file access denied", http.StatusForbidden)
|
||||
return
|
||||
}
|
||||
lb := h.LocalBackend()
|
||||
|
||||
ext, ok := ipnlocal.GetExt[*Extension](h.LocalBackend())
|
||||
if !ok {
|
||||
http.Error(w, "misconfigured taildrop extension", http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
|
||||
suffix, ok := strings.CutPrefix(r.URL.EscapedPath(), "/localapi/v0/files/")
|
||||
if !ok {
|
||||
http.Error(w, "misconfigured", http.StatusInternalServerError)
|
||||
@@ -376,14 +391,14 @@ func serveFiles(h *localapi.Handler, w http.ResponseWriter, r *http.Request) {
|
||||
var cancel context.CancelFunc
|
||||
ctx, cancel = context.WithDeadline(ctx, deadline)
|
||||
defer cancel()
|
||||
wfs, err = lb.AwaitWaitingFiles(ctx)
|
||||
wfs, err = ext.AwaitWaitingFiles(ctx)
|
||||
if err != nil && ctx.Err() == nil {
|
||||
http.Error(w, err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
} else {
|
||||
var err error
|
||||
wfs, err = lb.WaitingFiles()
|
||||
wfs, err = ext.WaitingFiles()
|
||||
if err != nil {
|
||||
http.Error(w, err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
@@ -399,14 +414,14 @@ func serveFiles(h *localapi.Handler, w http.ResponseWriter, r *http.Request) {
|
||||
return
|
||||
}
|
||||
if r.Method == "DELETE" {
|
||||
if err := lb.DeleteFile(name); err != nil {
|
||||
if err := ext.DeleteFile(name); err != nil {
|
||||
http.Error(w, err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
w.WriteHeader(http.StatusNoContent)
|
||||
return
|
||||
}
|
||||
rc, size, err := lb.OpenFile(name)
|
||||
rc, size, err := ext.OpenFile(name)
|
||||
if err != nil {
|
||||
http.Error(w, err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
@@ -426,7 +441,14 @@ func serveFileTargets(h *localapi.Handler, w http.ResponseWriter, r *http.Reques
|
||||
http.Error(w, "want GET to list targets", http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
fts, err := h.LocalBackend().FileTargets()
|
||||
|
||||
ext, ok := ipnlocal.GetExt[*Extension](h.LocalBackend())
|
||||
if !ok {
|
||||
http.Error(w, "misconfigured taildrop extension", http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
|
||||
fts, err := ext.FileTargets()
|
||||
if err != nil {
|
||||
localapi.WriteErrorJSON(w, err)
|
||||
return
|
||||
|
131
feature/taildrop/paths.go
Normal file
131
feature/taildrop/paths.go
Normal file
@@ -0,0 +1,131 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package taildrop
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path/filepath"
|
||||
|
||||
"tailscale.com/version/distro"
|
||||
)
|
||||
|
||||
// SetDirectFileRoot sets the directory where received files are written.
|
||||
//
|
||||
// This must be called before Tailscale is started.
|
||||
func (e *Extension) SetDirectFileRoot(root string) {
|
||||
e.directFileRoot = root
|
||||
}
|
||||
|
||||
func (e *Extension) setPlatformDefaultDirectFileRoot() {
|
||||
dg := distro.Get()
|
||||
|
||||
switch dg {
|
||||
case distro.Synology, distro.TrueNAS, distro.QNAP, distro.Unraid:
|
||||
// See if they have a "Taildrop" share.
|
||||
// See https://github.com/tailscale/tailscale/issues/2179#issuecomment-982821319
|
||||
path, err := findTaildropDir(dg)
|
||||
if err != nil {
|
||||
e.logf("%s Taildrop support: %v", dg, err)
|
||||
} else {
|
||||
e.logf("%s Taildrop: using %v", dg, path)
|
||||
e.directFileRoot = path
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func findTaildropDir(dg distro.Distro) (string, error) {
|
||||
const name = "Taildrop"
|
||||
switch dg {
|
||||
case distro.Synology:
|
||||
return findSynologyTaildropDir(name)
|
||||
case distro.TrueNAS:
|
||||
return findTrueNASTaildropDir(name)
|
||||
case distro.QNAP:
|
||||
return findQnapTaildropDir(name)
|
||||
case distro.Unraid:
|
||||
return findUnraidTaildropDir(name)
|
||||
}
|
||||
return "", fmt.Errorf("%s is an unsupported distro for Taildrop dir", dg)
|
||||
}
|
||||
|
||||
// findSynologyTaildropDir looks for the first volume containing a
|
||||
// "Taildrop" directory. We'd run "synoshare --get Taildrop" command
|
||||
// but on DSM7 at least, we lack permissions to run that.
|
||||
func findSynologyTaildropDir(name string) (dir string, err error) {
|
||||
for i := 1; i <= 16; i++ {
|
||||
dir = fmt.Sprintf("/volume%v/%s", i, name)
|
||||
if fi, err := os.Stat(dir); err == nil && fi.IsDir() {
|
||||
return dir, nil
|
||||
}
|
||||
}
|
||||
return "", fmt.Errorf("shared folder %q not found", name)
|
||||
}
|
||||
|
||||
// findTrueNASTaildropDir returns the first matching directory of
|
||||
// /mnt/{name} or /mnt/*/{name}
|
||||
func findTrueNASTaildropDir(name string) (dir string, err error) {
|
||||
// If we're running in a jail, a mount point could just be added at /mnt/Taildrop
|
||||
dir = fmt.Sprintf("/mnt/%s", name)
|
||||
if fi, err := os.Stat(dir); err == nil && fi.IsDir() {
|
||||
return dir, nil
|
||||
}
|
||||
|
||||
// but if running on the host, it may be something like /mnt/Primary/Taildrop
|
||||
fis, err := os.ReadDir("/mnt")
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("error reading /mnt: %w", err)
|
||||
}
|
||||
for _, fi := range fis {
|
||||
dir = fmt.Sprintf("/mnt/%s/%s", fi.Name(), name)
|
||||
if fi, err := os.Stat(dir); err == nil && fi.IsDir() {
|
||||
return dir, nil
|
||||
}
|
||||
}
|
||||
return "", fmt.Errorf("shared folder %q not found", name)
|
||||
}
|
||||
|
||||
// findQnapTaildropDir checks if a Shared Folder named "Taildrop" exists.
|
||||
func findQnapTaildropDir(name string) (string, error) {
|
||||
dir := fmt.Sprintf("/share/%s", name)
|
||||
fi, err := os.Stat(dir)
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("shared folder %q not found", name)
|
||||
}
|
||||
if fi.IsDir() {
|
||||
return dir, nil
|
||||
}
|
||||
|
||||
// share/Taildrop is usually a symlink to CACHEDEV1_DATA/Taildrop/ or some such.
|
||||
fullpath, err := filepath.EvalSymlinks(dir)
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("symlink to shared folder %q not found", name)
|
||||
}
|
||||
if fi, err = os.Stat(fullpath); err == nil && fi.IsDir() {
|
||||
return dir, nil // return the symlink, how QNAP set it up
|
||||
}
|
||||
return "", fmt.Errorf("shared folder %q not found", name)
|
||||
}
|
||||
|
||||
// findUnraidTaildropDir looks for a directory linked at
|
||||
// /var/lib/tailscale/Taildrop. This is a symlink to the
|
||||
// path specified by the user in the Unraid Web UI
|
||||
func findUnraidTaildropDir(name string) (string, error) {
|
||||
dir := fmt.Sprintf("/var/lib/tailscale/%s", name)
|
||||
_, err := os.Stat(dir)
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("symlink %q not found", name)
|
||||
}
|
||||
|
||||
fullpath, err := filepath.EvalSymlinks(dir)
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("symlink %q to shared folder not valid", name)
|
||||
}
|
||||
|
||||
fi, err := os.Stat(fullpath)
|
||||
if err == nil && fi.IsDir() {
|
||||
return dir, nil // return the symlink
|
||||
}
|
||||
return "", fmt.Errorf("shared folder %q not found", name)
|
||||
}
|
@@ -38,26 +38,30 @@ func canPutFile(h ipnlocal.PeerAPIHandler) bool {
|
||||
}
|
||||
|
||||
func handlePeerPut(h ipnlocal.PeerAPIHandler, w http.ResponseWriter, r *http.Request) {
|
||||
lb := h.LocalBackend()
|
||||
handlePeerPutWithBackend(h, lb, w, r)
|
||||
ext, ok := ipnlocal.GetExt[*Extension](h.LocalBackend())
|
||||
if !ok {
|
||||
http.Error(w, "miswired", http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
handlePeerPutWithBackend(h, ext, w, r)
|
||||
}
|
||||
|
||||
// localBackend is the subset of ipnlocal.Backend that taildrop
|
||||
// extensionForPut is the subset of taildrop extension that taildrop
|
||||
// file put needs. This is pulled out for testability.
|
||||
type localBackend interface {
|
||||
TaildropManager() (*taildrop.Manager, error)
|
||||
HasCapFileSharing() bool
|
||||
type extensionForPut interface {
|
||||
manager() *taildrop.Manager
|
||||
hasCapFileSharing() bool
|
||||
Clock() tstime.Clock
|
||||
}
|
||||
|
||||
func handlePeerPutWithBackend(h ipnlocal.PeerAPIHandler, lb localBackend, w http.ResponseWriter, r *http.Request) {
|
||||
func handlePeerPutWithBackend(h ipnlocal.PeerAPIHandler, ext extensionForPut, w http.ResponseWriter, r *http.Request) {
|
||||
if r.Method == "PUT" {
|
||||
metricPutCalls.Add(1)
|
||||
}
|
||||
|
||||
taildropMgr, err := lb.TaildropManager()
|
||||
if err != nil {
|
||||
h.Logf("taildropManager: %v", err)
|
||||
taildropMgr := ext.manager()
|
||||
if taildropMgr == nil {
|
||||
h.Logf("taildrop: no taildrop manager")
|
||||
http.Error(w, "failed to get taildrop manager", http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
@@ -66,7 +70,7 @@ func handlePeerPutWithBackend(h ipnlocal.PeerAPIHandler, lb localBackend, w http
|
||||
http.Error(w, taildrop.ErrNoTaildrop.Error(), http.StatusForbidden)
|
||||
return
|
||||
}
|
||||
if !lb.HasCapFileSharing() {
|
||||
if !ext.hasCapFileSharing() {
|
||||
http.Error(w, taildrop.ErrNoTaildrop.Error(), http.StatusForbidden)
|
||||
return
|
||||
}
|
||||
@@ -123,7 +127,7 @@ func handlePeerPutWithBackend(h ipnlocal.PeerAPIHandler, lb localBackend, w http
|
||||
}
|
||||
}
|
||||
case "PUT":
|
||||
t0 := lb.Clock().Now()
|
||||
t0 := ext.Clock().Now()
|
||||
id := taildrop.ClientID(h.Peer().StableID())
|
||||
|
||||
var offset int64
|
||||
@@ -138,7 +142,7 @@ func handlePeerPutWithBackend(h ipnlocal.PeerAPIHandler, lb localBackend, w http
|
||||
n, err := taildropMgr.PutFile(taildrop.ClientID(fmt.Sprint(id)), baseName, r.Body, offset, r.ContentLength)
|
||||
switch err {
|
||||
case nil:
|
||||
d := lb.Clock().Since(t0).Round(time.Second / 10)
|
||||
d := ext.Clock().Since(t0).Round(time.Second / 10)
|
||||
h.Logf("got put of %s in %v from %v/%v", approxSize(n), d, h.RemoteAddr().Addr(), h.Peer().ComputedName)
|
||||
io.WriteString(w, "{}\n")
|
||||
case taildrop.ErrNoTaildrop:
|
||||
|
@@ -50,19 +50,19 @@ func (h *peerAPIHandler) PeerCaps() tailcfg.PeerCapMap {
|
||||
return nil
|
||||
}
|
||||
|
||||
type fakeLocalBackend struct {
|
||||
type fakeExtension struct {
|
||||
logf logger.Logf
|
||||
capFileSharing bool
|
||||
clock tstime.Clock
|
||||
taildrop *taildrop.Manager
|
||||
}
|
||||
|
||||
func (lb *fakeLocalBackend) Clock() tstime.Clock { return lb.clock }
|
||||
func (lb *fakeLocalBackend) HasCapFileSharing() bool {
|
||||
return lb.capFileSharing
|
||||
func (lb *fakeExtension) manager() *taildrop.Manager {
|
||||
return lb.taildrop
|
||||
}
|
||||
func (lb *fakeLocalBackend) TaildropManager() (*taildrop.Manager, error) {
|
||||
return lb.taildrop, nil
|
||||
func (lb *fakeExtension) Clock() tstime.Clock { return lb.clock }
|
||||
func (lb *fakeExtension) hasCapFileSharing() bool {
|
||||
return lb.capFileSharing
|
||||
}
|
||||
|
||||
type peerAPITestEnv struct {
|
||||
@@ -472,16 +472,17 @@ func TestHandlePeerAPI(t *testing.T) {
|
||||
selfNode.CapMap = tailcfg.NodeCapMap{tailcfg.CapabilityDebug: nil}
|
||||
}
|
||||
var rootDir string
|
||||
var e peerAPITestEnv
|
||||
if !tt.omitRoot {
|
||||
rootDir = t.TempDir()
|
||||
e.taildrop = taildrop.ManagerOptions{
|
||||
Logf: e.logBuf.Logf,
|
||||
Dir: rootDir,
|
||||
}.New()
|
||||
}
|
||||
|
||||
lb := &fakeLocalBackend{
|
||||
var e peerAPITestEnv
|
||||
e.taildrop = taildrop.ManagerOptions{
|
||||
Logf: e.logBuf.Logf,
|
||||
Dir: rootDir,
|
||||
}.New()
|
||||
|
||||
ext := &fakeExtension{
|
||||
logf: e.logBuf.Logf,
|
||||
capFileSharing: tt.capSharing,
|
||||
clock: &tstest.Clock{},
|
||||
@@ -499,7 +500,7 @@ func TestHandlePeerAPI(t *testing.T) {
|
||||
if req.Host == "example.com" {
|
||||
req.Host = "100.100.100.101:12345"
|
||||
}
|
||||
handlePeerPutWithBackend(e.ph, lb, e.rr, req)
|
||||
handlePeerPutWithBackend(e.ph, ext, e.rr, req)
|
||||
}
|
||||
for _, f := range tt.checks {
|
||||
f(t, &e)
|
||||
@@ -539,7 +540,7 @@ func TestFileDeleteRace(t *testing.T) {
|
||||
Addresses: []netip.Prefix{netip.MustParsePrefix("100.100.100.101/32")},
|
||||
}).View(),
|
||||
}
|
||||
fakeLB := &fakeLocalBackend{
|
||||
fakeLB := &fakeExtension{
|
||||
logf: t.Logf,
|
||||
capFileSharing: true,
|
||||
clock: &tstest.Clock{},
|
||||
|
73
feature/taildrop/target_test.go
Normal file
73
feature/taildrop/target_test.go
Normal file
@@ -0,0 +1,73 @@
|
||||
// Copyright (c) Tailscale Inc & AUTHORS
|
||||
// SPDX-License-Identifier: BSD-3-Clause
|
||||
|
||||
package taildrop
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"tailscale.com/ipn"
|
||||
"tailscale.com/ipn/ipnext"
|
||||
"tailscale.com/tailcfg"
|
||||
)
|
||||
|
||||
func TestFileTargets(t *testing.T) {
|
||||
e := new(Extension)
|
||||
|
||||
_, err := e.FileTargets()
|
||||
if got, want := fmt.Sprint(err), "not connected to the tailnet"; got != want {
|
||||
t.Errorf("before connect: got %q; want %q", got, want)
|
||||
}
|
||||
|
||||
e.nodeBackendForTest = testNodeBackend{peers: nil}
|
||||
|
||||
_, err = e.FileTargets()
|
||||
if got, want := fmt.Sprint(err), "not connected to the tailnet"; got != want {
|
||||
t.Errorf("non-running netmap: got %q; want %q", got, want)
|
||||
}
|
||||
|
||||
e.backendState = ipn.Running
|
||||
_, err = e.FileTargets()
|
||||
if got, want := fmt.Sprint(err), "file sharing not enabled by Tailscale admin"; got != want {
|
||||
t.Errorf("without cap: got %q; want %q", got, want)
|
||||
}
|
||||
|
||||
e.capFileSharing = true
|
||||
got, err := e.FileTargets()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if len(got) != 0 {
|
||||
t.Fatalf("unexpected %d peers", len(got))
|
||||
}
|
||||
|
||||
var nodeID tailcfg.NodeID = 1234
|
||||
peer := &tailcfg.Node{
|
||||
ID: nodeID,
|
||||
Hostinfo: (&tailcfg.Hostinfo{OS: "tvOS"}).View(),
|
||||
}
|
||||
e.nodeBackendForTest = testNodeBackend{peers: []tailcfg.NodeView{peer.View()}}
|
||||
|
||||
got, err = e.FileTargets()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if len(got) != 0 {
|
||||
t.Fatalf("unexpected %d peers", len(got))
|
||||
}
|
||||
}
|
||||
|
||||
type testNodeBackend struct {
|
||||
ipnext.NodeBackend
|
||||
peers []tailcfg.NodeView
|
||||
}
|
||||
|
||||
func (t testNodeBackend) AppendMatchingPeers(peers []tailcfg.NodeView, f func(tailcfg.NodeView) bool) []tailcfg.NodeView {
|
||||
for _, p := range t.peers {
|
||||
if f(p) {
|
||||
peers = append(peers, p)
|
||||
}
|
||||
}
|
||||
return peers
|
||||
}
|
Reference in New Issue
Block a user