lib/model, lib/protocol: Handle request concurrency in model (#5216)

This commit is contained in:
Simon Frei
2018-11-13 08:53:55 +01:00
committed by Jakob Borg
parent 9212303906
commit 4f27bdfc27
13 changed files with 358 additions and 226 deletions

View File

@@ -0,0 +1,50 @@
// Copyright (C) 2018 The Syncthing Authors.
//
// This Source Code Form is subject to the terms of the Mozilla Public
// License, v. 2.0. If a copy of the MPL was not distributed with this file,
// You can obtain one at https://mozilla.org/MPL/2.0/.
package model
import "sync"
type byteSemaphore struct {
max int
available int
mut sync.Mutex
cond *sync.Cond
}
func newByteSemaphore(max int) *byteSemaphore {
s := byteSemaphore{
max: max,
available: max,
}
s.cond = sync.NewCond(&s.mut)
return &s
}
func (s *byteSemaphore) take(bytes int) {
if bytes > s.max {
bytes = s.max
}
s.mut.Lock()
for bytes > s.available {
s.cond.Wait()
}
s.available -= bytes
s.mut.Unlock()
}
func (s *byteSemaphore) give(bytes int) {
if bytes > s.max {
bytes = s.max
}
s.mut.Lock()
if s.available+bytes > s.max {
panic("bug: can never give more than max")
}
s.available += bytes
s.cond.Broadcast()
s.mut.Unlock()
}

View File

@@ -15,7 +15,6 @@ import (
"runtime"
"sort"
"strings"
stdsync "sync"
"time"
"github.com/syncthing/syncthing/lib/config"
@@ -1147,7 +1146,10 @@ func (f *sendReceiveFolder) shortcutFile(file, curFile protocol.FileInfo, dbUpda
// copierRoutine reads copierStates until the in channel closes and performs
// the relevant copies when possible, or passes it to the puller routine.
func (f *sendReceiveFolder) copierRoutine(in <-chan copyBlocksState, pullChan chan<- pullBlockState, out chan<- *sharedPullerState) {
buf := make([]byte, protocol.MinBlockSize)
buf := protocol.BufferPool.Get(protocol.MinBlockSize)
defer func() {
protocol.BufferPool.Put(buf)
}()
for state := range in {
dstFd, err := state.tempFile()
@@ -1223,11 +1225,7 @@ func (f *sendReceiveFolder) copierRoutine(in <-chan copyBlocksState, pullChan ch
continue
}
if s := int(block.Size); s > cap(buf) {
buf = make([]byte, s)
} else {
buf = buf[:s]
}
buf = protocol.BufferPool.Upgrade(buf, int(block.Size))
found, err := weakHashFinder.Iterate(block.WeakHash, buf, func(offset int64) bool {
if verifyBuffer(buf, block) != nil {
@@ -1935,41 +1933,3 @@ func componentCount(name string) int {
}
return count
}
type byteSemaphore struct {
max int
available int
mut stdsync.Mutex
cond *stdsync.Cond
}
func newByteSemaphore(max int) *byteSemaphore {
s := byteSemaphore{
max: max,
available: max,
}
s.cond = stdsync.NewCond(&s.mut)
return &s
}
func (s *byteSemaphore) take(bytes int) {
if bytes > s.max {
panic("bug: more than max bytes will never be available")
}
s.mut.Lock()
for bytes > s.available {
s.cond.Wait()
}
s.available -= bytes
s.mut.Unlock()
}
func (s *byteSemaphore) give(bytes int) {
s.mut.Lock()
if s.available+bytes > s.max {
panic("bug: can never give more than max")
}
s.available += bytes
s.cond.Broadcast()
s.mut.Unlock()
}

View File

@@ -105,6 +105,7 @@ type Model struct {
pmut sync.RWMutex // protects the below
conn map[protocol.DeviceID]connections.Connection
connRequestLimiters map[protocol.DeviceID]*byteSemaphore
closed map[protocol.DeviceID]chan struct{}
helloMessages map[protocol.DeviceID]protocol.HelloResult
deviceDownloads map[protocol.DeviceID]*deviceDownloadState
@@ -158,6 +159,7 @@ func NewModel(cfg *config.Wrapper, id protocol.DeviceID, clientName, clientVersi
folderRunnerTokens: make(map[string][]suture.ServiceToken),
folderStatRefs: make(map[string]*stats.FolderStatisticsReference),
conn: make(map[protocol.DeviceID]connections.Connection),
connRequestLimiters: make(map[protocol.DeviceID]*byteSemaphore),
closed: make(map[protocol.DeviceID]chan struct{}),
helloMessages: make(map[protocol.DeviceID]protocol.HelloResult),
deviceDownloads: make(map[protocol.DeviceID]*deviceDownloadState),
@@ -1281,6 +1283,7 @@ func (m *Model) Closed(conn protocol.Connection, err error) {
m.progressEmitter.temporaryIndexUnsubscribe(conn)
}
delete(m.conn, device)
delete(m.connRequestLimiters, device)
delete(m.helloMessages, device)
delete(m.deviceDownloads, device)
delete(m.remotePausedFolders, device)
@@ -1314,19 +1317,40 @@ func (m *Model) closeLocked(device protocol.DeviceID) {
closeRawConn(conn)
}
// Implements protocol.RequestResponse
type requestResponse struct {
data []byte
closed chan struct{}
once stdsync.Once
}
func newRequestResponse(size int) *requestResponse {
return &requestResponse{
data: protocol.BufferPool.Get(size),
closed: make(chan struct{}),
}
}
func (r *requestResponse) Data() []byte {
return r.data
}
func (r *requestResponse) Close() {
r.once.Do(func() {
protocol.BufferPool.Put(r.data)
close(r.closed)
})
}
func (r *requestResponse) Wait() {
<-r.closed
}
// Request returns the specified data segment by reading it from local disk.
// Implements the protocol.Model interface.
func (m *Model) Request(deviceID protocol.DeviceID, folder, name string, offset int64, hash []byte, weakHash uint32, fromTemporary bool, buf []byte) error {
if offset < 0 {
return protocol.ErrInvalid
}
if cfg, ok := m.cfg.Folder(folder); !ok || !cfg.SharedWith(deviceID) {
l.Warnf("Request from %s for file %s in unshared folder %q", deviceID, name, folder)
return protocol.ErrNoSuchFile
} else if cfg.Paused {
l.Debugf("Request from %s for file %s in paused folder %q", deviceID, name, folder)
return protocol.ErrInvalid
func (m *Model) Request(deviceID protocol.DeviceID, folder, name string, size int32, offset int64, hash []byte, weakHash uint32, fromTemporary bool) (out protocol.RequestResponse, err error) {
if size < 0 || offset < 0 {
return nil, protocol.ErrInvalid
}
m.fmut.RLock()
@@ -1337,35 +1361,69 @@ func (m *Model) Request(deviceID protocol.DeviceID, folder, name string, offset
// The folder might be already unpaused in the config, but not yet
// in the model.
l.Debugf("Request from %s for file %s in unstarted folder %q", deviceID, name, folder)
return protocol.ErrInvalid
return nil, protocol.ErrInvalid
}
if !folderCfg.SharedWith(deviceID) {
l.Warnf("Request from %s for file %s in unshared folder %q", deviceID, name, folder)
return nil, protocol.ErrNoSuchFile
}
if folderCfg.Paused {
l.Debugf("Request from %s for file %s in paused folder %q", deviceID, name, folder)
return nil, protocol.ErrInvalid
}
// Make sure the path is valid and in canonical form
var err error
if name, err = fs.Canonicalize(name); err != nil {
l.Debugf("Request from %s in folder %q for invalid filename %s", deviceID, folder, name)
return protocol.ErrInvalid
return nil, protocol.ErrInvalid
}
if deviceID != protocol.LocalDeviceID {
l.Debugf("%v REQ(in): %s: %q / %q o=%d s=%d t=%v", m, deviceID, folder, name, offset, len(buf), fromTemporary)
l.Debugf("%v REQ(in): %s: %q / %q o=%d s=%d t=%v", m, deviceID, folder, name, offset, size, fromTemporary)
}
if fs.IsInternal(name) {
l.Debugf("%v REQ(in) for internal file: %s: %q / %q o=%d s=%d", m, deviceID, folder, name, offset, size)
return nil, protocol.ErrNoSuchFile
}
if folderIgnores.Match(name).IsIgnored() {
l.Debugf("%v REQ(in) for ignored file: %s: %q / %q o=%d s=%d", m, deviceID, folder, name, offset, size)
return nil, protocol.ErrNoSuchFile
}
folderFs := folderCfg.Filesystem()
if fs.IsInternal(name) {
l.Debugf("%v REQ(in) for internal file: %s: %q / %q o=%d s=%d", m, deviceID, folder, name, offset, len(buf))
return protocol.ErrNoSuchFile
}
if folderIgnores.Match(name).IsIgnored() {
l.Debugf("%v REQ(in) for ignored file: %s: %q / %q o=%d s=%d", m, deviceID, folder, name, offset, len(buf))
return protocol.ErrNoSuchFile
}
if err := osutil.TraversesSymlink(folderFs, filepath.Dir(name)); err != nil {
l.Debugf("%v REQ(in) traversal check: %s - %s: %q / %q o=%d s=%d", m, err, deviceID, folder, name, offset, len(buf))
return protocol.ErrNoSuchFile
l.Debugf("%v REQ(in) traversal check: %s - %s: %q / %q o=%d s=%d", m, err, deviceID, folder, name, offset, size)
return nil, protocol.ErrNoSuchFile
}
// Restrict parallel requests by connection/device
m.pmut.RLock()
limiter := m.connRequestLimiters[deviceID]
m.pmut.RUnlock()
if limiter != nil {
limiter.take(int(size))
}
// The requestResponse releases the bytes to the limiter when its Close method is called.
res := newRequestResponse(int(size))
defer func() {
// Close it ourselves if it isn't returned due to an error
if err != nil {
res.Close()
}
}()
if limiter != nil {
go func() {
res.Wait()
limiter.give(int(size))
}()
}
// Only check temp files if the flag is set, and if we are set to advertise
@@ -1376,11 +1434,12 @@ func (m *Model) Request(deviceID protocol.DeviceID, folder, name string, offset
if info, err := folderFs.Lstat(tempFn); err != nil || !info.IsRegular() {
// Reject reads for anything that doesn't exist or is something
// other than a regular file.
return protocol.ErrNoSuchFile
l.Debugf("%v REQ(in) failed stating temp file (%v): %s: %q / %q o=%d s=%d", m, err, deviceID, folder, name, offset, size)
return nil, protocol.ErrNoSuchFile
}
err := readOffsetIntoBuf(folderFs, tempFn, offset, buf)
if err == nil && scanner.Validate(buf, hash, weakHash) {
return nil
err := readOffsetIntoBuf(folderFs, tempFn, offset, res.data)
if err == nil && scanner.Validate(res.data, hash, weakHash) {
return res, nil
}
// Fall through to reading from a non-temp file, just incase the temp
// file has finished downloading.
@@ -1389,21 +1448,25 @@ func (m *Model) Request(deviceID protocol.DeviceID, folder, name string, offset
if info, err := folderFs.Lstat(name); err != nil || !info.IsRegular() {
// Reject reads for anything that doesn't exist or is something
// other than a regular file.
return protocol.ErrNoSuchFile
l.Debugf("%v REQ(in) failed stating file (%v): %s: %q / %q o=%d s=%d", m, err, deviceID, folder, name, offset, size)
return nil, protocol.ErrNoSuchFile
}
if err = readOffsetIntoBuf(folderFs, name, offset, buf); fs.IsNotExist(err) {
return protocol.ErrNoSuchFile
if err := readOffsetIntoBuf(folderFs, name, offset, res.data); fs.IsNotExist(err) {
l.Debugf("%v REQ(in) file doesn't exist: %s: %q / %q o=%d s=%d", m, deviceID, folder, name, offset, size)
return nil, protocol.ErrNoSuchFile
} else if err != nil {
return protocol.ErrGeneric
l.Debugf("%v REQ(in) failed reading file (%v): %s: %q / %q o=%d s=%d", m, err, deviceID, folder, name, offset, size)
return nil, protocol.ErrGeneric
}
if !scanner.Validate(buf, hash, weakHash) {
m.recheckFile(deviceID, folderFs, folder, name, int(offset)/len(buf), hash)
return protocol.ErrNoSuchFile
if !scanner.Validate(res.data, hash, weakHash) {
m.recheckFile(deviceID, folderFs, folder, name, int(offset)/int(size), hash)
l.Debugf("%v REQ(in) failed validating data (%v): %s: %q / %q o=%d s=%d", m, err, deviceID, folder, name, offset, size)
return nil, protocol.ErrNoSuchFile
}
return nil
return res, nil
}
func (m *Model) recheckFile(deviceID protocol.DeviceID, folderFs fs.Filesystem, folder, name string, blockIndex int, hash []byte) {
@@ -1598,6 +1661,11 @@ func (m *Model) GetHello(id protocol.DeviceID) protocol.HelloIntf {
// folder changes.
func (m *Model) AddConnection(conn connections.Connection, hello protocol.HelloResult) {
deviceID := conn.ID()
device, ok := m.cfg.Device(deviceID)
if !ok {
l.Infoln("Trying to add connection to unknown device")
return
}
m.pmut.Lock()
if oldConn, ok := m.conn[deviceID]; ok {
@@ -1617,6 +1685,13 @@ func (m *Model) AddConnection(conn connections.Connection, hello protocol.HelloR
m.conn[deviceID] = conn
m.closed[deviceID] = make(chan struct{})
m.deviceDownloads[deviceID] = newDeviceDownloadState()
// 0: default, <0: no limiting
switch {
case device.MaxRequestKiB > 0:
m.connRequestLimiters[deviceID] = newByteSemaphore(1024 * device.MaxRequestKiB)
case device.MaxRequestKiB == 0:
m.connRequestLimiters[deviceID] = newByteSemaphore(1024 * defaultPullerPendingKiB)
}
m.helloMessages[deviceID] = hello
@@ -1644,8 +1719,7 @@ func (m *Model) AddConnection(conn connections.Connection, hello protocol.HelloR
cm := m.generateClusterConfig(deviceID)
conn.ClusterConfig(cm)
device, ok := m.cfg.Devices()[deviceID]
if ok && (device.Name == "" || m.cfg.Options().OverwriteRemoteDevNames) && hello.DeviceName != "" {
if (device.Name == "" || m.cfg.Options().OverwriteRemoteDevNames) && hello.DeviceName != "" {
device.Name = hello.DeviceName
m.cfg.SetDevice(device)
m.cfg.Save()

View File

@@ -183,45 +183,42 @@ func TestRequest(t *testing.T) {
defer m.Stop()
m.ScanFolder("default")
bs := make([]byte, protocol.MinBlockSize)
// Existing, shared file
bs = bs[:6]
err := m.Request(device1, "default", "foo", 0, nil, 0, false, bs)
res, err := m.Request(device1, "default", "foo", 6, 0, nil, 0, false)
if err != nil {
t.Error(err)
}
bs := res.Data()
if !bytes.Equal(bs, []byte("foobar")) {
t.Errorf("Incorrect data from request: %q", string(bs))
}
// Existing, nonshared file
err = m.Request(device2, "default", "foo", 0, nil, 0, false, bs)
_, err = m.Request(device2, "default", "foo", 6, 0, nil, 0, false)
if err == nil {
t.Error("Unexpected nil error on insecure file read")
}
// Nonexistent file
err = m.Request(device1, "default", "nonexistent", 0, nil, 0, false, bs)
_, err = m.Request(device1, "default", "nonexistent", 6, 0, nil, 0, false)
if err == nil {
t.Error("Unexpected nil error on insecure file read")
}
// Shared folder, but disallowed file name
err = m.Request(device1, "default", "../walk.go", 0, nil, 0, false, bs)
_, err = m.Request(device1, "default", "../walk.go", 6, 0, nil, 0, false)
if err == nil {
t.Error("Unexpected nil error on insecure file read")
}
// Negative offset
err = m.Request(device1, "default", "foo", -4, nil, 0, false, bs[:0])
_, err = m.Request(device1, "default", "foo", -4, 0, nil, 0, false)
if err == nil {
t.Error("Unexpected nil error on insecure file read")
}
// Larger block than available
bs = bs[:42]
err = m.Request(device1, "default", "foo", 0, nil, 0, false, bs)
_, err = m.Request(device1, "default", "foo", 42, 0, nil, 0, false)
if err == nil {
t.Error("Unexpected nil error on insecure file read")
}
@@ -536,7 +533,7 @@ func BenchmarkRequestInSingleFile(b *testing.B) {
b.ResetTimer()
for i := 0; i < b.N; i++ {
if err := m.Request(device1, "default", "request/for/a/file/in/a/couple/of/dirs/128k", 0, nil, 0, false, buf); err != nil {
if _, err := m.Request(device1, "default", "request/for/a/file/in/a/couple/of/dirs/128k", 128<<10, 0, nil, 0, false); err != nil {
b.Error(err)
}
}
@@ -3667,6 +3664,7 @@ func TestFolderRestartZombies(t *testing.T) {
// would leave more than one folder runner alive.
wrapper := createTmpWrapper(defaultCfg.Copy())
defer os.Remove(wrapper.ConfigPath())
folderCfg, _ := wrapper.Folder("default")
folderCfg.FilesystemType = fs.FilesystemTypeFake
wrapper.SetFolder(folderCfg)
@@ -3759,3 +3757,45 @@ func (c *alwaysChanged) Seen(fs fs.Filesystem, name string) bool {
func (c *alwaysChanged) Changed() bool {
return true
}
func TestRequestLimit(t *testing.T) {
cfg := defaultCfg.Copy()
cfg.Devices = append(cfg.Devices, config.NewDeviceConfiguration(device2, "device2"))
cfg.Devices[1].MaxRequestKiB = 1
cfg.Folders[0].Devices = []config.FolderDeviceConfiguration{
{DeviceID: device1},
{DeviceID: device2},
}
m, _, wrapper := setupModelWithConnectionManual(cfg)
defer m.Stop()
defer os.Remove(wrapper.ConfigPath())
file := "tmpfile"
befReq := time.Now()
first, err := m.Request(device2, "default", file, 2000, 0, nil, 0, false)
if err != nil {
t.Fatalf("First request failed: %v", err)
}
reqDur := time.Since(befReq)
returned := make(chan struct{})
go func() {
second, err := m.Request(device2, "default", file, 2000, 0, nil, 0, false)
if err != nil {
t.Fatalf("Second request failed: %v", err)
}
close(returned)
second.Close()
}()
time.Sleep(10 * reqDur)
select {
case <-returned:
t.Fatalf("Second request returned before first was done")
default:
}
first.Close()
select {
case <-returned:
case <-time.After(time.Second):
t.Fatalf("Second request did not return after first was done")
}
}

View File

@@ -98,9 +98,8 @@ func TestSymlinkTraversalRead(t *testing.T) {
<-done
// Request a file by traversing the symlink
buf := make([]byte, 10)
err := m.Request(device1, "default", "symlink/requests_test.go", 0, nil, 0, false, buf)
if err == nil || !bytes.Equal(buf, make([]byte, 10)) {
res, err := m.Request(device1, "default", "symlink/requests_test.go", 10, 0, nil, 0, false)
if err == nil || res != nil {
t.Error("Managed to traverse symlink")
}
}
@@ -225,6 +224,7 @@ func TestRequestVersioningSymlinkAttack(t *testing.T) {
defer os.RemoveAll(tmpDir)
cfg := defaultCfgWrapper.RawCopy()
cfg.Devices = append(cfg.Devices, config.NewDeviceConfiguration(device2, "device2"))
cfg.Folders[0] = config.NewFolderConfiguration(protocol.LocalDeviceID, "default", "default", fs.FilesystemTypeBasic, tmpDir)
cfg.Folders[0].Devices = []config.FolderDeviceConfiguration{
{DeviceID: device1},
@@ -519,12 +519,11 @@ func TestRescanIfHaveInvalidContent(t *testing.T) {
t.Fatalf("unexpected weak hash: %d != 103547413", f.Blocks[0].WeakHash)
}
buf := make([]byte, len(payload))
err := m.Request(device2, "default", "foo", 0, f.Blocks[0].Hash, f.Blocks[0].WeakHash, false, buf)
res, err := m.Request(device2, "default", "foo", int32(len(payload)), 0, f.Blocks[0].Hash, f.Blocks[0].WeakHash, false)
if err != nil {
t.Fatal(err)
}
buf := res.Data()
if !bytes.Equal(buf, payload) {
t.Errorf("%s != %s", buf, payload)
}
@@ -536,7 +535,7 @@ func TestRescanIfHaveInvalidContent(t *testing.T) {
t.Fatal(err)
}
err = m.Request(device2, "default", "foo", 0, f.Blocks[0].Hash, f.Blocks[0].WeakHash, false, buf)
res, err = m.Request(device2, "default", "foo", int32(len(payload)), 0, f.Blocks[0].Hash, f.Blocks[0].WeakHash, false)
if err == nil {
t.Fatalf("expected failure")
}