lib/protocol, lib/discover, lib/db: Use protocol buffer serialization (fixes #3080)

This changes the BEP protocol to use protocol buffer serialization
instead of XDR, and therefore also the database format. The local
discovery protocol is also updated to be protocol buffer format.

GitHub-Pull-Request: https://github.com/syncthing/syncthing/pull/3276
LGTM: AudriusButkevicius
This commit is contained in:
Jakob Borg
2016-07-04 10:40:29 +00:00
committed by Audrius Butkevicius
parent 21f5b16e47
commit fa0101bd60
269 changed files with 477296 additions and 4175 deletions

View File

@@ -63,8 +63,8 @@ func benchmarkRequestsConnPair(b *testing.B, conn0, conn1 net.Conn) {
c1.Start()
// Satisfy the assertions in the protocol by sending an initial cluster config
c0.ClusterConfig(ClusterConfigMessage{})
c1.ClusterConfig(ClusterConfigMessage{})
c0.ClusterConfig(ClusterConfig{})
c1.ClusterConfig(ClusterConfig{})
// Report some useful stats and reset the timer for the actual test
b.ReportAllocs()
@@ -164,13 +164,13 @@ func negotiateTLS(cert tls.Certificate, conn0, conn1 net.Conn) (net.Conn, net.Co
type fakeModel struct{}
func (m *fakeModel) Index(deviceID DeviceID, folder string, files []FileInfo, flags uint32, options []Option) {
func (m *fakeModel) Index(deviceID DeviceID, folder string, files []FileInfo) {
}
func (m *fakeModel) IndexUpdate(deviceID DeviceID, folder string, files []FileInfo, flags uint32, options []Option) {
func (m *fakeModel) IndexUpdate(deviceID DeviceID, folder string, files []FileInfo) {
}
func (m *fakeModel) Request(deviceID DeviceID, folder string, name string, offset int64, hash []byte, flags uint32, options []Option, buf []byte) error {
func (m *fakeModel) Request(deviceID DeviceID, folder string, name string, offset int64, hash []byte, fromTemporary bool, buf []byte) error {
// We write the offset to the end of the buffer, so the receiver
// can verify that it did in fact get some data back over the
// connection.
@@ -178,11 +178,11 @@ func (m *fakeModel) Request(deviceID DeviceID, folder string, name string, offse
return nil
}
func (m *fakeModel) ClusterConfig(deviceID DeviceID, config ClusterConfigMessage) {
func (m *fakeModel) ClusterConfig(deviceID DeviceID, config ClusterConfig) {
}
func (m *fakeModel) Close(deviceID DeviceID, err error) {
}
func (m *fakeModel) DownloadProgress(deviceID DeviceID, folder string, updates []FileDownloadProgressUpdate, flags uint32, options []Option) {
func (m *fakeModel) DownloadProgress(deviceID DeviceID, folder string, updates []FileDownloadProgressUpdate) {
}

3999
lib/protocol/bep.pb.go Normal file

File diff suppressed because it is too large Load Diff

189
lib/protocol/bep.proto Normal file
View File

@@ -0,0 +1,189 @@
// protoc --proto_path=../../../../../:../../../../gogo/protobuf/protobuf:. --gogofast_out=. message.proto
syntax = "proto3";
package protocol;
import "github.com/gogo/protobuf/gogoproto/gogo.proto";
option (gogoproto.goproto_getters_all) = false;
option (gogoproto.sizer_all) = false;
option (gogoproto.protosizer_all) = true;
option (gogoproto.goproto_enum_stringer_all) = true;
option (gogoproto.goproto_enum_prefix_all) = false;
// --- Pre-auth ---
message Hello {
string device_name = 1;
string client_name = 2;
string client_version = 3;
}
// --- Header ---
message Header {
MessageType type = 1;
MessageCompression compression = 2;
}
enum MessageType {
CLUSTER_CONFIG = 0 [(gogoproto.enumvalue_customname) = "messageTypeClusterConfig"];
INDEX = 1 [(gogoproto.enumvalue_customname) = "messageTypeIndex"];
INDEX_UPDATE = 2 [(gogoproto.enumvalue_customname) = "messageTypeIndexUpdate"];
REQUEST = 3 [(gogoproto.enumvalue_customname) = "messageTypeRequest"];
RESPONSE = 4 [(gogoproto.enumvalue_customname) = "messageTypeResponse"];
DOWNLOAD_PROGRESS = 5 [(gogoproto.enumvalue_customname) = "messageTypeDownloadProgress"];
PING = 6 [(gogoproto.enumvalue_customname) = "messageTypePing"];
CLOSE = 7 [(gogoproto.enumvalue_customname) = "messageTypeClose"];
}
enum MessageCompression {
NONE = 0 [(gogoproto.enumvalue_customname) = "MessageCompressionNone"];
LZ4 = 1 [(gogoproto.enumvalue_customname) = "MessageCompressionLZ4"];
}
// --- Actual messages ---
// Cluster Config
message ClusterConfig {
repeated Folder folders = 1 [(gogoproto.nullable) = false];
}
message Folder {
string id = 1 [(gogoproto.customname) = "ID"];
string label = 2;
bool read_only = 3;
bool ignore_permissions = 4;
bool ignore_delete = 5;
bool disable_temp_indexes = 6;
repeated Device devices = 16 [(gogoproto.nullable) = false];
}
message Device {
bytes id = 1 [(gogoproto.customname) = "ID"];
string name = 2;
repeated string addresses = 3;
Compression compression = 4;
string cert_name = 5;
int64 max_local_version = 6;
bool introducer = 7;
}
enum Compression {
METADATA = 0 [(gogoproto.enumvalue_customname) = "CompressMetadata"];
NEVER = 1 [(gogoproto.enumvalue_customname) = "CompressNever"];
ALWAYS = 2 [(gogoproto.enumvalue_customname) = "CompressAlways"];
}
// Index and Index Update
message Index {
string folder = 1;
repeated FileInfo files = 2 [(gogoproto.nullable) = false];
}
message IndexUpdate {
string folder = 1;
repeated FileInfo files = 2 [(gogoproto.nullable) = false];
}
message FileInfo {
option (gogoproto.goproto_stringer) = false;
string name = 1;
FileInfoType type = 2;
int64 size = 3;
uint32 permissions = 4;
int64 modified = 5;
bool deleted = 6;
bool invalid = 7;
bool no_permissions = 8;
Vector version = 9 [(gogoproto.nullable) = false];
int64 local_version = 10;
repeated BlockInfo Blocks = 16 [(gogoproto.nullable) = false];
}
enum FileInfoType {
FILE = 0 [(gogoproto.enumvalue_customname) = "FileInfoTypeFile"];
DIRECTORY = 1 [(gogoproto.enumvalue_customname) = "FileInfoTypeDirectory"];
SYMLINK_FILE = 2 [(gogoproto.enumvalue_customname) = "FileInfoTypeSymlinkFile"];
SYMLINK_DIRECTORY = 3 [(gogoproto.enumvalue_customname) = "FileInfoTypeSymlinkDirectory"];
SYMLINK_UNKNOWN = 4 [(gogoproto.enumvalue_customname) = "FileInfoTypeSymlinkUnknown"];
}
message BlockInfo {
option (gogoproto.goproto_stringer) = false;
int64 offset = 1;
int32 size = 2;
bytes hash = 3;
}
message Vector {
repeated Counter counters = 1 [(gogoproto.nullable) = false];
}
message Counter {
uint64 id = 1 [(gogoproto.customname) = "ID", (gogoproto.customtype) = "ShortID", (gogoproto.nullable) = false];
uint64 value = 2;
}
// Request
message Request {
int32 id = 1 [(gogoproto.customname) = "ID"];
string folder = 2;
string name = 3;
int64 offset = 4;
int32 size = 5;
bytes hash = 6;
bool from_temporary = 7;
}
// Response
message Response {
int32 id = 1 [(gogoproto.customname) = "ID"];
bytes data = 2;
ErrorCode code = 3;
}
enum ErrorCode {
NO_ERROR = 0 [(gogoproto.enumvalue_customname) = "ErrorCodeNoError"];
GENERIC = 1 [(gogoproto.enumvalue_customname) = "ErrorCodeGeneric"];
NO_SUCH_FILE = 2 [(gogoproto.enumvalue_customname) = "ErrorCodeNoSuchFile"];
INVALID_FILE = 3 [(gogoproto.enumvalue_customname) = "ErrorCodeInvalidFile"];
}
// DownloadProgress
message DownloadProgress {
string folder = 1;
repeated FileDownloadProgressUpdate updates = 2 [(gogoproto.nullable) = false];
}
message FileDownloadProgressUpdate {
FileDownloadProgressUpdateType update_type = 1;
string name = 2;
Vector version = 3 [(gogoproto.nullable) = false];
repeated int32 block_indexes = 4;
}
enum FileDownloadProgressUpdateType {
APPEND = 0 [(gogoproto.enumvalue_customname) = "UpdateTypeAppend"];
FORGET = 1 [(gogoproto.enumvalue_customname) = "UpdateTypeForget"];
}
// Ping
message Ping {
}
// Close
message Close {
string reason = 1;
}

View File

@@ -0,0 +1,96 @@
// Copyright (C) 2014 The Protocol Authors.
//go:generate go run ../../script/protofmt.go bep.proto
//go:generate protoc --proto_path=../../../../../:../../../../gogo/protobuf/protobuf:. --gogofast_out=. bep.proto
package protocol
import (
"bytes"
"crypto/sha256"
"fmt"
)
var (
sha256OfEmptyBlock = sha256.Sum256(make([]byte, BlockSize))
HelloMessageMagic = uint32(0x2EA7D90B)
)
func (m Hello) Magic() uint32 {
return HelloMessageMagic
}
func (f FileInfo) String() string {
return fmt.Sprintf("File{Name:%q, Permissions:0%o, Modified:%d, Version:%v, Length:%d, Deleted:%v, Invalid:%v, NoPermissions:%v, Blocks:%v}",
f.Name, f.Permissions, f.Modified, f.Version, f.Size, f.Deleted, f.Invalid, f.NoPermissions, f.Blocks)
}
func (f FileInfo) IsDeleted() bool {
return f.Deleted
}
func (f FileInfo) IsInvalid() bool {
return f.Invalid
}
func (f FileInfo) IsDirectory() bool {
return f.Type == FileInfoTypeDirectory
}
func (f FileInfo) IsSymlink() bool {
switch f.Type {
case FileInfoTypeSymlinkDirectory, FileInfoTypeSymlinkFile, FileInfoTypeSymlinkUnknown:
return true
default:
return false
}
}
func (f FileInfo) HasPermissionBits() bool {
return !f.NoPermissions
}
func (f FileInfo) FileSize() int64 {
if f.IsDirectory() || f.IsDeleted() {
return 128
}
return f.Size
}
func (f FileInfo) FileName() string {
return f.Name
}
// WinsConflict returns true if "f" is the one to choose when it is in
// conflict with "other".
func (f FileInfo) WinsConflict(other FileInfo) bool {
// If a modification is in conflict with a delete, we pick the
// modification.
if !f.IsDeleted() && other.IsDeleted() {
return true
}
if f.IsDeleted() && !other.IsDeleted() {
return false
}
// The one with the newer modification time wins.
if f.Modified > other.Modified {
return true
}
if f.Modified < other.Modified {
return false
}
// The modification times were equal. Use the device ID in the version
// vector as tie breaker.
return f.Version.Compare(other.Version) == ConcurrentGreater
}
func (b BlockInfo) String() string {
return fmt.Sprintf("Block{%d/%d/%x}", b.Offset, b.Size, b.Hash)
}
// IsEmpty returns true if the block is a full block of zeroes.
func (b BlockInfo) IsEmpty() bool {
return b.Size == BlockSize && bytes.Equal(b.Hash, sha256OfEmptyBlock[:])
}

View File

@@ -0,0 +1,59 @@
// Copyright (C) 2016 The Protocol Authors.
package protocol
import "sync"
type bufferPool struct {
minSize int
pool sync.Pool
}
// get returns a new buffer of the requested size
func (p *bufferPool) get(size int) []byte {
intf := p.pool.Get()
if intf == nil {
// Pool is empty, must allocate.
return p.new(size)
}
bs := intf.([]byte)
if cap(bs) < size {
// Buffer was too small, leave it for someone else and allocate.
p.put(bs)
return p.new(size)
}
return bs[:size]
}
// upgrade grows the buffer to the requested size, while attempting to reuse
// it if possible.
func (p *bufferPool) upgrade(bs []byte, size int) []byte {
if cap(bs) >= size {
// Reslicing is enough, lets go!
return bs[:size]
}
// It was too small. But it pack into the pool and try to get another
// buffer.
p.put(bs)
return p.get(size)
}
// put returns the buffer to the pool
func (p *bufferPool) put(bs []byte) {
p.pool.Put(bs)
}
// new creates a new buffer of the requested size, taking the minimum
// allocation count into account. For internal use only.
func (p *bufferPool) new(size int) []byte {
allocSize := size
if allocSize < p.minSize {
// Avoid allocating tiny buffers that we won't be able to reuse for
// anything useful.
allocSize = p.minSize
}
return make([]byte, allocSize)[:size]
}

View File

@@ -5,16 +5,15 @@ package protocol
import "time"
type TestModel struct {
data []byte
folder string
name string
offset int64
size int
hash []byte
flags uint32
options []Option
closedCh chan struct{}
closedErr error
data []byte
folder string
name string
offset int64
size int
hash []byte
fromTemporary bool
closedCh chan struct{}
closedErr error
}
func newTestModel() *TestModel {
@@ -23,20 +22,19 @@ func newTestModel() *TestModel {
}
}
func (t *TestModel) Index(deviceID DeviceID, folder string, files []FileInfo, flags uint32, options []Option) {
func (t *TestModel) Index(deviceID DeviceID, folder string, files []FileInfo) {
}
func (t *TestModel) IndexUpdate(deviceID DeviceID, folder string, files []FileInfo, flags uint32, options []Option) {
func (t *TestModel) IndexUpdate(deviceID DeviceID, folder string, files []FileInfo) {
}
func (t *TestModel) Request(deviceID DeviceID, folder, name string, offset int64, hash []byte, flags uint32, options []Option, buf []byte) error {
func (t *TestModel) Request(deviceID DeviceID, folder, name string, offset int64, hash []byte, fromTemporary bool, buf []byte) error {
t.folder = folder
t.name = name
t.offset = offset
t.size = len(buf)
t.hash = hash
t.flags = flags
t.options = options
t.fromTemporary = fromTemporary
copy(buf, t.data)
return nil
}
@@ -46,10 +44,10 @@ func (t *TestModel) Close(deviceID DeviceID, err error) {
close(t.closedCh)
}
func (t *TestModel) ClusterConfig(deviceID DeviceID, config ClusterConfigMessage) {
func (t *TestModel) ClusterConfig(deviceID DeviceID, config ClusterConfig) {
}
func (t *TestModel) DownloadProgress(DeviceID, string, []FileDownloadProgressUpdate, uint32, []Option) {
func (t *TestModel) DownloadProgress(DeviceID, string, []FileDownloadProgressUpdate) {
}
func (t *TestModel) closedError() error {

View File

@@ -4,13 +4,7 @@ package protocol
import "fmt"
type Compression int
const (
CompressMetadata Compression = iota // zero value is the default, default should be "metadata"
CompressNever
CompressAlways
compressionThreshold = 128 // don't bother compressing messages smaller than this many bytes
)
@@ -31,14 +25,6 @@ var compressionUnmarshal = map[string]Compression{
"always": CompressAlways,
}
func (c Compression) String() string {
s, ok := compressionMarshal[c]
if !ok {
return fmt.Sprintf("unknown:%d", c)
}
return s
}
func (c Compression) GoString() string {
return fmt.Sprintf("%q", c.String())
}

View File

@@ -8,8 +8,8 @@ func TestWinsConflict(t *testing.T) {
testcases := [][2]FileInfo{
// The first should always win over the second
{{Modified: 42}, {Modified: 41}},
{{Modified: 41}, {Modified: 42, Flags: FlagDeleted}},
{{Modified: 41, Version: Vector{{42, 2}, {43, 1}}}, {Modified: 41, Version: Vector{{42, 1}, {43, 2}}}},
{{Modified: 41}, {Modified: 42, Deleted: true}},
{{Modified: 41, Version: Vector{[]Counter{{42, 2}, {43, 1}}}}, {Modified: 41, Version: Vector{[]Counter{{42, 1}, {43, 2}}}}},
}
for _, tc := range testcases {

View File

@@ -16,7 +16,3 @@ var (
func init() {
l.SetDebug("protocol", strings.Contains(os.Getenv("STTRACE"), "protocol") || os.Getenv("STTRACE") == "all")
}
func shouldDebug() bool {
return l.ShouldDebug("protocol")
}

View File

@@ -6,13 +6,6 @@ import (
"errors"
)
const (
ecNoError int32 = iota
ecGeneric
ecNoSuchFile
ecInvalid
)
var (
ErrNoError error
ErrGeneric = errors.New("generic error")
@@ -20,32 +13,32 @@ var (
ErrInvalid = errors.New("file is invalid")
)
var lookupError = map[int32]error{
ecNoError: ErrNoError,
ecGeneric: ErrGeneric,
ecNoSuchFile: ErrNoSuchFile,
ecInvalid: ErrInvalid,
var lookupError = map[ErrorCode]error{
ErrorCodeNoError: ErrNoError,
ErrorCodeGeneric: ErrGeneric,
ErrorCodeNoSuchFile: ErrNoSuchFile,
ErrorCodeInvalidFile: ErrInvalid,
}
var lookupCode = map[error]int32{
ErrNoError: ecNoError,
ErrGeneric: ecGeneric,
ErrNoSuchFile: ecNoSuchFile,
ErrInvalid: ecInvalid,
var lookupCode = map[error]ErrorCode{
ErrNoError: ErrorCodeNoError,
ErrGeneric: ErrorCodeGeneric,
ErrNoSuchFile: ErrorCodeNoSuchFile,
ErrInvalid: ErrorCodeInvalidFile,
}
func codeToError(errcode int32) error {
err, ok := lookupError[errcode]
func codeToError(code ErrorCode) error {
err, ok := lookupError[code]
if !ok {
return ErrGeneric
}
return err
}
func errorToCode(err error) int32 {
func errorToCode(err error) ErrorCode {
code, ok := lookupCode[err]
if !ok {
return ecGeneric
return ErrorCodeGeneric
}
return code
}

View File

@@ -1,70 +0,0 @@
// Copyright (C) 2015 The Protocol Authors.
// +build gofuzz
package protocol
import (
"bytes"
"encoding/binary"
"encoding/hex"
"fmt"
"reflect"
"sync"
)
func Fuzz(data []byte) int {
// Regenerate the length, or we'll most commonly exit quickly due to an
// unexpected eof which is unintestering.
if len(data) > 8 {
binary.BigEndian.PutUint32(data[4:], uint32(len(data))-8)
}
// Setup a rawConnection we'll use to parse the message.
c := rawConnection{
cr: &countingReader{Reader: bytes.NewReader(data)},
closed: make(chan struct{}),
pool: sync.Pool{
New: func() interface{} {
return make([]byte, BlockSize)
},
},
}
// Attempt to parse the message.
hdr, msg, err := c.readMessage()
if err != nil {
return 0
}
// If parsing worked, attempt to encode it again.
newBs, err := msg.AppendXDR(nil)
if err != nil {
panic("not encodable")
}
// Create an appriate header for the re-encoding.
newMsg := make([]byte, 8)
binary.BigEndian.PutUint32(newMsg, encodeHeader(hdr))
binary.BigEndian.PutUint32(newMsg[4:], uint32(len(newBs)))
newMsg = append(newMsg, newBs...)
// Use the rawConnection to parse the re-encoding.
c.cr = &countingReader{Reader: bytes.NewReader(newMsg)}
hdr2, msg2, err := c.readMessage()
if err != nil {
fmt.Println("Initial:\n" + hex.Dump(data))
fmt.Println("New:\n" + hex.Dump(newMsg))
panic("not parseable after re-encode: " + err.Error())
}
// Make sure the data is the same as it was before.
if hdr != hdr2 {
panic("headers differ")
}
if !reflect.DeepEqual(msg, msg2) {
panic("contents differ")
}
return 1
}

View File

@@ -1,89 +0,0 @@
// Copyright (C) 2015 The Protocol Authors.
// +build gofuzz
package protocol
import (
"encoding/binary"
"fmt"
"io/ioutil"
"os"
"strings"
"testing"
"testing/quick"
)
// This can be used to generate a corpus of valid messages as a starting point
// for the fuzzer.
func TestGenerateCorpus(t *testing.T) {
t.Skip("Use to generate initial corpus only")
n := 0
check := func(idx IndexMessage) bool {
for i := range idx.Options {
if len(idx.Options[i].Key) > 64 {
idx.Options[i].Key = idx.Options[i].Key[:64]
}
}
hdr := header{
version: 0,
msgID: 42,
msgType: messageTypeIndex,
compression: false,
}
msgBs := idx.MustMarshalXDR()
buf := make([]byte, 8)
binary.BigEndian.PutUint32(buf, encodeHeader(hdr))
binary.BigEndian.PutUint32(buf[4:], uint32(len(msgBs)))
buf = append(buf, msgBs...)
ioutil.WriteFile(fmt.Sprintf("testdata/corpus/test-%03d.xdr", n), buf, 0644)
n++
return true
}
if err := quick.Check(check, &quick.Config{MaxCount: 1000}); err != nil {
t.Fatal(err)
}
}
// Tests any crashers found by the fuzzer, for closer investigation.
func TestCrashers(t *testing.T) {
testFiles(t, "testdata/crashers")
}
// Tests the entire corpus, which should PASS before the fuzzer starts
// fuzzing.
func TestCorpus(t *testing.T) {
testFiles(t, "testdata/corpus")
}
func testFiles(t *testing.T, dir string) {
fd, err := os.Open(dir)
if err != nil {
t.Fatal(err)
}
crashers, err := fd.Readdirnames(-1)
if err != nil {
t.Fatal(err)
}
for _, name := range crashers {
if strings.HasSuffix(name, ".output") {
continue
}
if strings.HasSuffix(name, ".quoted") {
continue
}
t.Log(name)
crasher, err := ioutil.ReadFile(dir + "/" + name)
if err != nil {
t.Fatal(err)
}
Fuzz(crasher)
}
}

View File

@@ -1,44 +0,0 @@
// Copyright (C) 2014 The Protocol Authors.
package protocol
import "github.com/calmh/xdr"
type header struct {
version int
msgID int
msgType int
compression bool
}
func (h header) MarshalXDRInto(m *xdr.Marshaller) error {
v := encodeHeader(h)
m.MarshalUint32(v)
return m.Error
}
func (h *header) UnmarshalXDRFrom(u *xdr.Unmarshaller) error {
v := u.UnmarshalUint32()
*h = decodeHeader(v)
return u.Error
}
func encodeHeader(h header) uint32 {
var isComp uint32
if h.compression {
isComp = 1 << 0 // the zeroth bit is the compression bit
}
return uint32(h.version&0xf)<<28 +
uint32(h.msgID&0xfff)<<16 +
uint32(h.msgType&0xff)<<8 +
isComp
}
func decodeHeader(u uint32) header {
return header{
version: int(u>>28) & 0xf,
msgID: int(u>>16) & 0xfff,
msgType: int(u>>8) & 0xff,
compression: u&1 == 1,
}
}

View File

@@ -9,10 +9,10 @@ import (
"io"
)
// The HelloMessage interface is implemented by the version specific hello
// The HelloIntf interface is implemented by the version specific hello
// message. It knows its magic number and how to serialize itself to a byte
// buffer.
type HelloMessage interface {
type HelloIntf interface {
Magic() uint32
Marshal() ([]byte, error)
}
@@ -29,12 +29,15 @@ var (
// ErrTooOldVersion12 is returned by ExchangeHello when the other side
// speaks the older, incompatible version 0.12 of the protocol.
ErrTooOldVersion12 = errors.New("the remote device speaks an older version of the protocol (v0.12) not compatible with this version")
// ErrTooOldVersion13 is returned by ExchangeHello when the other side
// speaks the older, incompatible version 0.12 of the protocol.
ErrTooOldVersion13 = errors.New("the remote device speaks an older version of the protocol (v0.13) not compatible with this version")
// ErrUnknownMagic is returned by ExchangeHellow when the other side
// speaks something entirely unknown.
ErrUnknownMagic = errors.New("the remote device speaks an unknown (newer?) version of the protocol")
)
func ExchangeHello(c io.ReadWriter, h HelloMessage) (HelloResult, error) {
func ExchangeHello(c io.ReadWriter, h HelloIntf) (HelloResult, error) {
if err := writeHello(c, h); err != nil {
return HelloResult{}, err
}
@@ -45,7 +48,7 @@ func ExchangeHello(c io.ReadWriter, h HelloMessage) (HelloResult, error) {
// version mismatch that we might want to alert the user about.
func IsVersionMismatch(err error) bool {
switch err {
case ErrTooOldVersion12, ErrUnknownMagic:
case ErrTooOldVersion12, ErrTooOldVersion13, ErrUnknownMagic:
return true
default:
return false
@@ -59,6 +62,28 @@ func readHello(c io.Reader) (HelloResult, error) {
}
switch binary.BigEndian.Uint32(header[:4]) {
case HelloMessageMagic:
// This is a v0.14 Hello message in proto format
msgSize := binary.BigEndian.Uint32(header[4:])
if msgSize > 1024 {
return HelloResult{}, fmt.Errorf("hello message too big")
}
buf := make([]byte, msgSize)
if _, err := io.ReadFull(c, buf); err != nil {
return HelloResult{}, err
}
var hello Hello
if err := hello.Unmarshal(buf); err != nil {
return HelloResult{}, err
}
res := HelloResult{
DeviceName: hello.DeviceName,
ClientName: hello.ClientName,
ClientVersion: hello.ClientVersion,
}
return res, nil
case Version13HelloMagic:
// This is a v0.13 Hello message in XDR format
msgSize := binary.BigEndian.Uint32(header[4:])
@@ -79,7 +104,7 @@ func readHello(c io.Reader) (HelloResult, error) {
ClientName: hello.ClientName,
ClientVersion: hello.ClientVersion,
}
return res, nil
return res, ErrTooOldVersion13
case 0x00010001, 0x00010000:
// This is the first word of a v0.12 cluster config message.
@@ -90,7 +115,7 @@ func readHello(c io.Reader) (HelloResult, error) {
return HelloResult{}, ErrUnknownMagic
}
func writeHello(c io.Writer, h HelloMessage) error {
func writeHello(c io.Writer, h HelloIntf) error {
msg, err := h.Marshal()
if err != nil {
return err

View File

@@ -13,6 +13,53 @@ import (
var spaceRe = regexp.MustCompile(`\s`)
func TestVersion14Hello(t *testing.T) {
// Tests that we can send and receive a version 0.14 hello message.
expected := Hello{
DeviceName: "test device",
ClientName: "syncthing",
ClientVersion: "v0.14.5",
}
msgBuf, err := expected.Marshal()
if err != nil {
t.Fatal(err)
}
hdrBuf := make([]byte, 8)
binary.BigEndian.PutUint32(hdrBuf, HelloMessageMagic)
binary.BigEndian.PutUint32(hdrBuf[4:], uint32(len(msgBuf)))
outBuf := new(bytes.Buffer)
outBuf.Write(hdrBuf)
outBuf.Write(msgBuf)
inBuf := new(bytes.Buffer)
conn := &readWriter{outBuf, inBuf}
send := &Hello{
DeviceName: "this device",
ClientName: "other client",
ClientVersion: "v0.14.6",
}
res, err := ExchangeHello(conn, send)
if err != nil {
t.Fatal(err)
}
if res.ClientName != expected.ClientName {
t.Errorf("incorrect ClientName %q != expected %q", res.ClientName, expected.ClientName)
}
if res.ClientVersion != expected.ClientVersion {
t.Errorf("incorrect ClientVersion %q != expected %q", res.ClientVersion, expected.ClientVersion)
}
if res.DeviceName != expected.DeviceName {
t.Errorf("incorrect DeviceName %q != expected %q", res.DeviceName, expected.DeviceName)
}
}
func TestVersion13Hello(t *testing.T) {
// Tests that we can send and receive a version 0.13 hello message.
@@ -42,8 +89,8 @@ func TestVersion13Hello(t *testing.T) {
}
res, err := ExchangeHello(conn, send)
if err != nil {
t.Fatal(err)
if err != ErrTooOldVersion13 {
t.Errorf("unexpected error %v != ErrTooOldVersion13", err)
}
if res.ClientName != expected.ClientName {
@@ -94,7 +141,7 @@ func TestVersion12Hello(t *testing.T) {
_, err := ExchangeHello(conn, send)
if err != ErrTooOldVersion12 {
t.Errorf("unexpected error %v != ErrTooOld", err)
t.Errorf("unexpected error %v != ErrTooOldVersion12", err)
}
}

View File

@@ -1,186 +0,0 @@
// Copyright (C) 2014 The Protocol Authors.
//go:generate -command genxdr go run ../../vendor/github.com/calmh/xdr/cmd/genxdr/main.go
//go:generate genxdr -o message_xdr.go message.go
package protocol
import (
"bytes"
"crypto/sha256"
"fmt"
)
var (
sha256OfEmptyBlock = sha256.Sum256(make([]byte, BlockSize))
)
type IndexMessage struct {
Folder string // max:256
Files []FileInfo // max:1000000
Flags uint32
Options []Option // max:64
}
type FileInfo struct {
Name string // max:8192
Flags uint32
Modified int64
Version Vector
LocalVersion int64
CachedSize int64 // noencode (cache only)
Blocks []BlockInfo // max:10000000
}
func (f FileInfo) String() string {
return fmt.Sprintf("File{Name:%q, Flags:0%o, Modified:%d, Version:%v, Size:%d, Blocks:%v}",
f.Name, f.Flags, f.Modified, f.Version, f.Size(), f.Blocks)
}
func (f FileInfo) Size() (bytes int64) {
if f.IsDeleted() || f.IsDirectory() {
return 128
}
if f.CachedSize > 0 {
return f.CachedSize
}
for _, b := range f.Blocks {
bytes += int64(b.Size)
}
f.CachedSize = bytes
return
}
func (f FileInfo) IsDeleted() bool {
return f.Flags&FlagDeleted != 0
}
func (f FileInfo) IsInvalid() bool {
return f.Flags&FlagInvalid != 0
}
func (f FileInfo) IsDirectory() bool {
return f.Flags&FlagDirectory != 0
}
func (f FileInfo) IsSymlink() bool {
return f.Flags&FlagSymlink != 0
}
func (f FileInfo) HasPermissionBits() bool {
return f.Flags&FlagNoPermBits == 0
}
// WinsConflict returns true if "f" is the one to choose when it is in
// conflict with "other".
func (f FileInfo) WinsConflict(other FileInfo) bool {
// If a modification is in conflict with a delete, we pick the
// modification.
if !f.IsDeleted() && other.IsDeleted() {
return true
}
if f.IsDeleted() && !other.IsDeleted() {
return false
}
// The one with the newer modification time wins.
if f.Modified > other.Modified {
return true
}
if f.Modified < other.Modified {
return false
}
// The modification times were equal. Use the device ID in the version
// vector as tie breaker.
return f.Version.Compare(other.Version) == ConcurrentGreater
}
type BlockInfo struct {
Offset int64 // noencode (cache only)
Size int32
Hash []byte // max:64
}
func (b BlockInfo) String() string {
return fmt.Sprintf("Block{%d/%d/%x}", b.Offset, b.Size, b.Hash)
}
// IsEmpty returns true if the block is a full block of zeroes.
func (b BlockInfo) IsEmpty() bool {
return b.Size == BlockSize && bytes.Equal(b.Hash, sha256OfEmptyBlock[:])
}
type RequestMessage struct {
Folder string // max:256
Name string // max:8192
Offset int64
Size int32
Hash []byte // max:64
Flags uint32
Options []Option // max:64
}
type ResponseMessage struct {
Data []byte
Code int32
}
type ClusterConfigMessage struct {
Folders []Folder // max:1000000
Options []Option // max:64
}
type DownloadProgressMessage struct {
Folder string // max:64
Updates []FileDownloadProgressUpdate // max:1000000
Flags uint32
Options []Option // max:64
}
func (o *ClusterConfigMessage) GetOption(key string) string {
for _, option := range o.Options {
if option.Key == key {
return option.Value
}
}
return ""
}
type Folder struct {
ID string // max:256
Label string // max:256
Devices []Device // max:1000000
Flags uint32
Options []Option // max:64
}
type Device struct {
ID []byte // max:32
Name string // max:64
Addresses []string // max:64,2083
Compression uint32
CertName string // max:64
MaxLocalVersion int64
Flags uint32
Options []Option // max:64
}
type FileDownloadProgressUpdate struct {
UpdateType uint32
Name string // max:8192
Version Vector
BlockIndexes []int32 // max:1000000
}
type Option struct {
Key string // max:64
Value string // max:1024
}
type CloseMessage struct {
Reason string // max:1024
Code int32
}
type EmptyMessage struct{}

File diff suppressed because it is too large Load Diff

View File

@@ -12,21 +12,21 @@ type nativeModel struct {
Model
}
func (m nativeModel) Index(deviceID DeviceID, folder string, files []FileInfo, flags uint32, options []Option) {
func (m nativeModel) Index(deviceID DeviceID, folder string, files []FileInfo) {
for i := range files {
files[i].Name = norm.NFD.String(files[i].Name)
}
m.Model.Index(deviceID, folder, files, flags, options)
m.Model.Index(deviceID, folder, files)
}
func (m nativeModel) IndexUpdate(deviceID DeviceID, folder string, files []FileInfo, flags uint32, options []Option) {
func (m nativeModel) IndexUpdate(deviceID DeviceID, folder string, files []FileInfo) {
for i := range files {
files[i].Name = norm.NFD.String(files[i].Name)
}
m.Model.IndexUpdate(deviceID, folder, files, flags, options)
m.Model.IndexUpdate(deviceID, folder, files)
}
func (m nativeModel) Request(deviceID DeviceID, folder string, name string, offset int64, hash []byte, flags uint32, options []Option, buf []byte) error {
func (m nativeModel) Request(deviceID DeviceID, folder string, name string, offset int64, hash []byte, fromTemporary bool, buf []byte) error {
name = norm.NFD.String(name)
return m.Model.Request(deviceID, folder, name, offset, hash, flags, options, buf)
return m.Model.Request(deviceID, folder, name, offset, hash, fromTemporary, buf)
}

View File

@@ -24,19 +24,19 @@ type nativeModel struct {
Model
}
func (m nativeModel) Index(deviceID DeviceID, folder string, files []FileInfo, flags uint32, options []Option) {
func (m nativeModel) Index(deviceID DeviceID, folder string, files []FileInfo) {
fixupFiles(folder, files)
m.Model.Index(deviceID, folder, files, flags, options)
m.Model.Index(deviceID, folder, files)
}
func (m nativeModel) IndexUpdate(deviceID DeviceID, folder string, files []FileInfo, flags uint32, options []Option) {
func (m nativeModel) IndexUpdate(deviceID DeviceID, folder string, files []FileInfo) {
fixupFiles(folder, files)
m.Model.IndexUpdate(deviceID, folder, files, flags, options)
m.Model.IndexUpdate(deviceID, folder, files)
}
func (m nativeModel) Request(deviceID DeviceID, folder string, name string, offset int64, hash []byte, flags uint32, options []Option, buf []byte) error {
func (m nativeModel) Request(deviceID DeviceID, folder string, name string, offset int64, hash []byte, fromTemporary bool, buf []byte) error {
name = filepath.FromSlash(name)
return m.Model.Request(deviceID, folder, name, offset, hash, flags, options, buf)
return m.Model.Request(deviceID, folder, name, offset, hash, fromTemporary, buf)
}
func fixupFiles(folder string, files []FileInfo) {
@@ -47,7 +47,7 @@ func fixupFiles(folder string, files []FileInfo) {
// can't possibly exist here anyway.
continue
}
files[i].Flags |= FlagInvalid
files[i].Invalid = true
l.Warnf("File name %q (folder %q) contains invalid characters; marked as invalid.", f.Name, folder)
}
files[i].Name = filepath.FromSlash(files[i].Name)

File diff suppressed because it is too large Load Diff

View File

@@ -3,21 +3,17 @@
package protocol
import (
"encoding/binary"
"encoding/hex"
"bytes"
"encoding/json"
"errors"
"fmt"
"io"
"io/ioutil"
"os"
"reflect"
"strings"
"testing"
"testing/quick"
"time"
"github.com/calmh/xdr"
"github.com/syncthing/syncthing/lib/rand"
)
var (
@@ -26,64 +22,6 @@ var (
quickCfg = &quick.Config{}
)
func TestHeaderEncodeDecode(t *testing.T) {
f := func(ver, id, typ int) bool {
ver = int(uint(ver) % 16)
id = int(uint(id) % 4096)
typ = int(uint(typ) % 256)
h0 := header{version: ver, msgID: id, msgType: typ}
h1 := decodeHeader(encodeHeader(h0))
return h0 == h1
}
if err := quick.Check(f, nil); err != nil {
t.Error(err)
}
}
func TestHeaderMarshalUnmarshal(t *testing.T) {
f := func(ver, id, typ int) bool {
ver = int(uint(ver) % 16)
id = int(uint(id) % 4096)
typ = int(uint(typ) % 256)
buf := make([]byte, 4)
h0 := header{version: ver, msgID: id, msgType: typ}
h0.MarshalXDRInto(&xdr.Marshaller{Data: buf})
var h1 header
h1.UnmarshalXDRFrom(&xdr.Unmarshaller{Data: buf})
return h0 == h1
}
if err := quick.Check(f, nil); err != nil {
t.Error(err)
}
}
func TestHeaderLayout(t *testing.T) {
var e, a uint32
// Version are the first four bits
e = 0xf0000000
a = encodeHeader(header{version: 0xf})
if a != e {
t.Errorf("Header layout incorrect; %08x != %08x", a, e)
}
// Message ID are the following 12 bits
e = 0x0fff0000
a = encodeHeader(header{msgID: 0xfff})
if a != e {
t.Errorf("Header layout incorrect; %08x != %08x", a, e)
}
// Type are the last 8 bits before reserved
e = 0x0000ff00
a = encodeHeader(header{msgType: 0xff})
if a != e {
t.Errorf("Header layout incorrect; %08x != %08x", a, e)
}
}
func TestPing(t *testing.T) {
ar, aw := io.Pipe()
br, bw := io.Pipe()
@@ -92,8 +30,8 @@ func TestPing(t *testing.T) {
c0.Start()
c1 := NewConnection(c1ID, br, aw, newTestModel(), "name", CompressAlways).(wireFormatConnection).Connection.(*rawConnection)
c1.Start()
c0.ClusterConfig(ClusterConfigMessage{})
c1.ClusterConfig(ClusterConfigMessage{})
c0.ClusterConfig(ClusterConfig{})
c1.ClusterConfig(ClusterConfig{})
if ok := c0.ping(); !ok {
t.Error("c0 ping failed")
@@ -103,56 +41,6 @@ func TestPing(t *testing.T) {
}
}
func TestVersionErr(t *testing.T) {
m0 := newTestModel()
m1 := newTestModel()
ar, aw := io.Pipe()
br, bw := io.Pipe()
c0 := NewConnection(c0ID, ar, bw, m0, "name", CompressAlways).(wireFormatConnection).Connection.(*rawConnection)
c0.Start()
c1 := NewConnection(c1ID, br, aw, m1, "name", CompressAlways)
c1.Start()
c0.ClusterConfig(ClusterConfigMessage{})
c1.ClusterConfig(ClusterConfigMessage{})
timeoutWriteHeader(c0.cw, header{
version: 2, // higher than supported
msgID: 0,
msgType: messageTypeIndex,
})
if err := m1.closedError(); err == nil || !strings.Contains(err.Error(), "unknown protocol version") {
t.Error("Connection should close due to unknown version, not", err)
}
}
func TestTypeErr(t *testing.T) {
m0 := newTestModel()
m1 := newTestModel()
ar, aw := io.Pipe()
br, bw := io.Pipe()
c0 := NewConnection(c0ID, ar, bw, m0, "name", CompressAlways).(wireFormatConnection).Connection.(*rawConnection)
c0.Start()
c1 := NewConnection(c1ID, br, aw, m1, "name", CompressAlways)
c1.Start()
c0.ClusterConfig(ClusterConfigMessage{})
c1.ClusterConfig(ClusterConfigMessage{})
timeoutWriteHeader(c0.cw, header{
version: 0,
msgID: 0,
msgType: 42, // unknown type
})
if err := m1.closedError(); err == nil || !strings.Contains(err.Error(), "unknown message type") {
t.Error("Connection should close due to unknown message type, not", err)
}
}
func TestClose(t *testing.T) {
m0 := newTestModel()
m1 := newTestModel()
@@ -164,8 +52,8 @@ func TestClose(t *testing.T) {
c0.Start()
c1 := NewConnection(c1ID, br, aw, m1, "name", CompressAlways)
c1.Start()
c0.ClusterConfig(ClusterConfigMessage{})
c1.ClusterConfig(ClusterConfigMessage{})
c0.ClusterConfig(ClusterConfig{})
c1.ClusterConfig(ClusterConfig{})
c0.close(errors.New("manual close"))
@@ -180,8 +68,8 @@ func TestClose(t *testing.T) {
t.Error("Ping should not return true")
}
c0.Index("default", nil, 0, nil)
c0.Index("default", nil, 0, nil)
c0.Index("default", nil)
c0.Index("default", nil)
if _, err := c0.Request("default", "foo", 0, 0, nil, false); err == nil {
t.Error("Request should return an error")
@@ -193,15 +81,11 @@ func TestMarshalIndexMessage(t *testing.T) {
quickCfg.MaxCount = 10
}
f := func(m1 IndexMessage) bool {
if len(m1.Options) == 0 {
m1.Options = nil
}
f := func(m1 Index) bool {
if len(m1.Files) == 0 {
m1.Files = nil
}
for i, f := range m1.Files {
m1.Files[i].CachedSize = 0
if len(f.Blocks) == 0 {
m1.Files[i].Blocks = nil
} else {
@@ -212,9 +96,12 @@ func TestMarshalIndexMessage(t *testing.T) {
}
}
}
if len(f.Version.Counters) == 0 {
m1.Files[i].Version.Counters = nil
}
}
return testMarshal(t, "index", &m1, &IndexMessage{})
return testMarshal(t, "index", &m1, &Index{})
}
if err := quick.Check(f, quickCfg); err != nil {
@@ -227,14 +114,11 @@ func TestMarshalRequestMessage(t *testing.T) {
quickCfg.MaxCount = 10
}
f := func(m1 RequestMessage) bool {
if len(m1.Options) == 0 {
m1.Options = nil
}
f := func(m1 Request) bool {
if len(m1.Hash) == 0 {
m1.Hash = nil
}
return testMarshal(t, "request", &m1, &RequestMessage{})
return testMarshal(t, "request", &m1, &Request{})
}
if err := quick.Check(f, quickCfg); err != nil {
@@ -247,11 +131,11 @@ func TestMarshalResponseMessage(t *testing.T) {
quickCfg.MaxCount = 10
}
f := func(m1 ResponseMessage) bool {
f := func(m1 Response) bool {
if len(m1.Data) == 0 {
m1.Data = nil
}
return testMarshal(t, "response", &m1, &ResponseMessage{})
return testMarshal(t, "response", &m1, &Response{})
}
if err := quick.Check(f, quickCfg); err != nil {
@@ -264,10 +148,7 @@ func TestMarshalClusterConfigMessage(t *testing.T) {
quickCfg.MaxCount = 10
}
f := func(m1 ClusterConfigMessage) bool {
if len(m1.Options) == 0 {
m1.Options = nil
}
f := func(m1 ClusterConfig) bool {
if len(m1.Folders) == 0 {
m1.Folders = nil
}
@@ -275,11 +156,8 @@ func TestMarshalClusterConfigMessage(t *testing.T) {
if len(m1.Folders[i].Devices) == 0 {
m1.Folders[i].Devices = nil
}
if len(m1.Folders[i].Options) == 0 {
m1.Folders[i].Options = nil
}
}
return testMarshal(t, "clusterconfig", &m1, &ClusterConfigMessage{})
return testMarshal(t, "clusterconfig", &m1, &ClusterConfig{})
}
if err := quick.Check(f, quickCfg); err != nil {
@@ -292,8 +170,8 @@ func TestMarshalCloseMessage(t *testing.T) {
quickCfg.MaxCount = 10
}
f := func(m1 CloseMessage) bool {
return testMarshal(t, "close", &m1, &CloseMessage{})
f := func(m1 Close) bool {
return testMarshal(t, "close", &m1, &Close{})
}
if err := quick.Check(f, quickCfg); err != nil {
@@ -301,108 +179,97 @@ func TestMarshalCloseMessage(t *testing.T) {
}
}
type message interface {
MarshalXDR() ([]byte, error)
UnmarshalXDR([]byte) error
}
func testMarshal(t *testing.T, prefix string, m1, m2 message) bool {
failed := func(bc []byte) {
bs, _ := json.MarshalIndent(m1, "", " ")
ioutil.WriteFile(prefix+"-1.txt", bs, 0644)
bs, _ = json.MarshalIndent(m2, "", " ")
ioutil.WriteFile(prefix+"-2.txt", bs, 0644)
if len(bc) > 0 {
f, _ := os.Create(prefix + "-data.txt")
fmt.Fprint(f, hex.Dump(bc))
f.Close()
buf, err := m1.Marshal()
if err != nil {
t.Fatal(err)
}
err = m2.Unmarshal(buf)
if err != nil {
t.Fatal(err)
}
bs1, _ := json.MarshalIndent(m1, "", " ")
bs2, _ := json.MarshalIndent(m2, "", " ")
if !bytes.Equal(bs1, bs2) {
ioutil.WriteFile(prefix+"-1.txt", bs1, 0644)
ioutil.WriteFile(prefix+"-2.txt", bs1, 0644)
return false
}
return true
}
func TestMarshalledIndexMessageSize(t *testing.T) {
// We should be able to handle a 1 TiB file without
// blowing the default max message size.
if testing.Short() {
t.Skip("this test requires a lot of memory")
return
}
const (
maxMessageSize = MaxMessageLen
fileSize = 1 << 40
blockSize = BlockSize
)
f := FileInfo{
Name: "a normal length file name withoout any weird stuff.txt",
Type: FileInfoTypeFile,
Size: fileSize,
Permissions: 0666,
Modified: time.Now().Unix(),
Version: Vector{Counters: []Counter{{ID: 1 << 60, Value: 1}, {ID: 2 << 60, Value: 1}}},
Blocks: make([]BlockInfo, fileSize/blockSize),
}
for i := 0; i < fileSize/blockSize; i++ {
f.Blocks[i].Offset = int64(i) * blockSize
f.Blocks[i].Size = blockSize
f.Blocks[i].Hash = []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 1, 2, 3, 4, 5, 6, 7, 8, 9, 20, 1, 2, 3, 4, 5, 6, 7, 8, 9, 30, 1, 2}
}
idx := Index{
Folder: "some folder ID",
Files: []FileInfo{f},
}
msgSize := idx.ProtoSize()
if msgSize > maxMessageSize {
t.Errorf("Message size %d bytes is larger than max %d", msgSize, maxMessageSize)
}
}
func TestLZ4Compression(t *testing.T) {
c := new(rawConnection)
for i := 0; i < 10; i++ {
dataLen := 150 + rand.Intn(150)
data := make([]byte, dataLen)
_, err := io.ReadFull(rand.Reader, data[100:])
if err != nil {
t.Fatal(err)
}
comp, err := c.lz4Compress(data)
if err != nil {
t.Errorf("compressing %d bytes: %v", dataLen, err)
continue
}
}
buf, err := m1.MarshalXDR()
if err != nil && strings.Contains(err.Error(), "exceeds size") {
return true
}
if err != nil {
failed(nil)
t.Fatal(err)
}
err = m2.UnmarshalXDR(buf)
if err != nil {
failed(buf)
t.Fatal(err)
}
ok := reflect.DeepEqual(m1, m2)
if !ok {
failed(buf)
}
return ok
}
func timeoutWriteHeader(w io.Writer, hdr header) {
// This tries to write a message header to w, but times out after a while.
// This is useful because in testing, with a PipeWriter, it will block
// forever if the other side isn't reading any more. On the other hand we
// can't just "go" it into the background, because if the other side is
// still there we should wait for the write to complete. Yay.
var buf [8]byte // header and message length
binary.BigEndian.PutUint32(buf[:], encodeHeader(hdr))
binary.BigEndian.PutUint32(buf[4:], 0) // zero message length, explicitly
done := make(chan struct{})
go func() {
w.Write(buf[:])
close(done)
}()
select {
case <-done:
case <-time.After(250 * time.Millisecond):
}
}
func TestFileInfoSize(t *testing.T) {
fi := FileInfo{
Blocks: []BlockInfo{
{Size: 42},
{Offset: 42, Size: 23},
{Offset: 42 + 23, Size: 34},
},
}
size := fi.Size()
want := int64(42 + 23 + 34)
if size != want {
t.Errorf("Incorrect size reported, got %d, want %d", size, want)
}
size = fi.Size() // Cached, this time
if size != want {
t.Errorf("Incorrect cached size reported, got %d, want %d", size, want)
}
fi.CachedSize = 8
want = 8
size = fi.Size() // Ensure it came from the cache
if size != want {
t.Errorf("Incorrect cached size reported, got %d, want %d", size, want)
}
fi.CachedSize = 0
fi.Flags = FlagDirectory
want = 128
size = fi.Size() // Directories are 128 bytes large
if size != want {
t.Errorf("Incorrect cached size reported, got %d, want %d", size, want)
}
fi.CachedSize = 0
fi.Flags = FlagDeleted
want = 128
size = fi.Size() // Also deleted files
if size != want {
t.Errorf("Incorrect cached size reported, got %d, want %d", size, want)
res, err := c.lz4Decompress(comp)
if err != nil {
t.Errorf("decompressing %d bytes to %d: %v", len(comp), dataLen, err)
continue
}
if len(res) != len(data) {
t.Errorf("Incorrect len %d != expected %d", len(res), len(data))
}
if !bytes.Equal(data, res) {
t.Error("Incorrect decompressed data")
}
t.Logf("OK #%d, %d -> %d -> %d", i, dataLen, len(comp), dataLen)
}
}

View File

@@ -6,35 +6,30 @@ package protocol
// version vector. The vector has slice semantics and some operations on it
// are "append-like" in that they may return the same vector modified, or v
// new allocated Vector with the modified contents.
type Vector []Counter
// Counter represents a single counter in the version vector.
type Counter struct {
ID ShortID
Value uint64
}
// Update returns a Vector with the index for the specific ID incremented by
// one. If it is possible, the vector v is updated and returned. If it is not,
// a copy will be created, updated and returned.
func (v Vector) Update(id ShortID) Vector {
for i := range v {
if v[i].ID == id {
for i := range v.Counters {
if v.Counters[i].ID == id {
// Update an existing index
v[i].Value++
v.Counters[i].Value++
return v
} else if v[i].ID > id {
} else if v.Counters[i].ID > id {
// Insert a new index
nv := make(Vector, len(v)+1)
copy(nv, v[:i])
nv := make([]Counter, len(v.Counters)+1)
copy(nv, v.Counters[:i])
nv[i].ID = id
nv[i].Value = 1
copy(nv[i+1:], v[i:])
return nv
copy(nv[i+1:], v.Counters[i:])
return Vector{nv}
}
}
// Append a new index
return append(v, Counter{id, 1})
return Vector{append(v.Counters, Counter{id, 1})}
}
// Merge returns the vector containing the maximum indexes from v and b. If it
@@ -42,28 +37,28 @@ func (v Vector) Update(id ShortID) Vector {
// will be created, updated and returned.
func (v Vector) Merge(b Vector) Vector {
var vi, bi int
for bi < len(b) {
if vi == len(v) {
for bi < len(b.Counters) {
if vi == len(v.Counters) {
// We've reach the end of v, all that remains are appends
return append(v, b[bi:]...)
return Vector{append(v.Counters, b.Counters[bi:]...)}
}
if v[vi].ID > b[bi].ID {
if v.Counters[vi].ID > b.Counters[bi].ID {
// The index from b should be inserted here
n := make(Vector, len(v)+1)
copy(n, v[:vi])
n[vi] = b[bi]
copy(n[vi+1:], v[vi:])
v = n
n := make([]Counter, len(v.Counters)+1)
copy(n, v.Counters[:vi])
n[vi] = b.Counters[bi]
copy(n[vi+1:], v.Counters[vi:])
v.Counters = n
}
if v[vi].ID == b[bi].ID {
if val := b[bi].Value; val > v[vi].Value {
v[vi].Value = val
if v.Counters[vi].ID == b.Counters[bi].ID {
if val := b.Counters[bi].Value; val > v.Counters[vi].Value {
v.Counters[vi].Value = val
}
}
if bi < len(b) && v[vi].ID == b[bi].ID {
if bi < len(b.Counters) && v.Counters[vi].ID == b.Counters[bi].ID {
bi++
}
vi++
@@ -74,9 +69,9 @@ func (v Vector) Merge(b Vector) Vector {
// Copy returns an identical vector that is not shared with v.
func (v Vector) Copy() Vector {
nv := make(Vector, len(v))
copy(nv, v)
return nv
nv := make([]Counter, len(v.Counters))
copy(nv, v.Counters)
return Vector{nv}
}
// Equal returns true when the two vectors are equivalent.
@@ -106,10 +101,96 @@ func (v Vector) Concurrent(b Vector) bool {
// Counter returns the current value of the given counter ID.
func (v Vector) Counter(id ShortID) uint64 {
for _, c := range v {
for _, c := range v.Counters {
if c.ID == id {
return c.Value
}
}
return 0
}
// Ordering represents the relationship between two Vectors.
type Ordering int
const (
Equal Ordering = iota
Greater
Lesser
ConcurrentLesser
ConcurrentGreater
)
// There's really no such thing as "concurrent lesser" and "concurrent
// greater" in version vectors, just "concurrent". But it's useful to be able
// to get a strict ordering between versions for stable sorts and so on, so we
// return both variants. The convenience method Concurrent() can be used to
// check for either case.
// Compare returns the Ordering that describes a's relation to b.
func (v Vector) Compare(b Vector) Ordering {
var ai, bi int // index into a and b
var av, bv Counter // value at current index
result := Equal
for ai < len(v.Counters) || bi < len(b.Counters) {
var aMissing, bMissing bool
if ai < len(v.Counters) {
av = v.Counters[ai]
} else {
av = Counter{}
aMissing = true
}
if bi < len(b.Counters) {
bv = b.Counters[bi]
} else {
bv = Counter{}
bMissing = true
}
switch {
case av.ID == bv.ID:
// We have a counter value for each side
if av.Value > bv.Value {
if result == Lesser {
return ConcurrentLesser
}
result = Greater
} else if av.Value < bv.Value {
if result == Greater {
return ConcurrentGreater
}
result = Lesser
}
case !aMissing && av.ID < bv.ID || bMissing:
// Value is missing on the b side
if av.Value > 0 {
if result == Lesser {
return ConcurrentLesser
}
result = Greater
}
case !bMissing && bv.ID < av.ID || aMissing:
// Value is missing on the a side
if bv.Value > 0 {
if result == Greater {
return ConcurrentGreater
}
result = Lesser
}
}
if ai < len(v.Counters) && (av.ID <= bv.ID || bMissing) {
ai++
}
if bi < len(b.Counters) && (bv.ID <= av.ID || aMissing) {
bi++
}
}
return result
}

View File

@@ -1,89 +0,0 @@
// Copyright (C) 2015 The Protocol Authors.
package protocol
// Ordering represents the relationship between two Vectors.
type Ordering int
const (
Equal Ordering = iota
Greater
Lesser
ConcurrentLesser
ConcurrentGreater
)
// There's really no such thing as "concurrent lesser" and "concurrent
// greater" in version vectors, just "concurrent". But it's useful to be able
// to get a strict ordering between versions for stable sorts and so on, so we
// return both variants. The convenience method Concurrent() can be used to
// check for either case.
// Compare returns the Ordering that describes a's relation to b.
func (a Vector) Compare(b Vector) Ordering {
var ai, bi int // index into a and b
var av, bv Counter // value at current index
result := Equal
for ai < len(a) || bi < len(b) {
var aMissing, bMissing bool
if ai < len(a) {
av = a[ai]
} else {
av = Counter{}
aMissing = true
}
if bi < len(b) {
bv = b[bi]
} else {
bv = Counter{}
bMissing = true
}
switch {
case av.ID == bv.ID:
// We have a counter value for each side
if av.Value > bv.Value {
if result == Lesser {
return ConcurrentLesser
}
result = Greater
} else if av.Value < bv.Value {
if result == Greater {
return ConcurrentGreater
}
result = Lesser
}
case !aMissing && av.ID < bv.ID || bMissing:
// Value is missing on the b side
if av.Value > 0 {
if result == Lesser {
return ConcurrentLesser
}
result = Greater
}
case !bMissing && bv.ID < av.ID || aMissing:
// Value is missing on the a side
if bv.Value > 0 {
if result == Greater {
return ConcurrentGreater
}
result = Lesser
}
}
if ai < len(a) && (av.ID <= bv.ID || bMissing) {
ai++
}
if bi < len(b) && (bv.ID <= av.ID || aMissing) {
bi++
}
}
return result
}

View File

@@ -1,249 +0,0 @@
// Copyright (C) 2015 The Protocol Authors.
package protocol
import (
"math"
"testing"
)
func TestCompare(t *testing.T) {
testcases := []struct {
a, b Vector
r Ordering
}{
// Empty vectors are identical
{Vector{}, Vector{}, Equal},
{Vector{}, nil, Equal},
{nil, Vector{}, Equal},
{nil, Vector{Counter{42, 0}}, Equal},
{Vector{}, Vector{Counter{42, 0}}, Equal},
{Vector{Counter{42, 0}}, nil, Equal},
{Vector{Counter{42, 0}}, Vector{}, Equal},
// Zero is the implied value for a missing Counter
{
Vector{Counter{42, 0}},
Vector{Counter{77, 0}},
Equal,
},
// Equal vectors are equal
{
Vector{Counter{42, 33}},
Vector{Counter{42, 33}},
Equal,
},
{
Vector{Counter{42, 33}, Counter{77, 24}},
Vector{Counter{42, 33}, Counter{77, 24}},
Equal,
},
// These a-vectors are all greater than the b-vector
{
Vector{Counter{42, 1}},
nil,
Greater,
},
{
Vector{Counter{42, 1}},
Vector{},
Greater,
},
{
Vector{Counter{0, 1}},
Vector{Counter{0, 0}},
Greater,
},
{
Vector{Counter{42, 1}},
Vector{Counter{42, 0}},
Greater,
},
{
Vector{Counter{math.MaxUint64, 1}},
Vector{Counter{math.MaxUint64, 0}},
Greater,
},
{
Vector{Counter{0, math.MaxUint64}},
Vector{Counter{0, 0}},
Greater,
},
{
Vector{Counter{42, math.MaxUint64}},
Vector{Counter{42, 0}},
Greater,
},
{
Vector{Counter{math.MaxUint64, math.MaxUint64}},
Vector{Counter{math.MaxUint64, 0}},
Greater,
},
{
Vector{Counter{0, math.MaxUint64}},
Vector{Counter{0, math.MaxUint64 - 1}},
Greater,
},
{
Vector{Counter{42, math.MaxUint64}},
Vector{Counter{42, math.MaxUint64 - 1}},
Greater,
},
{
Vector{Counter{math.MaxUint64, math.MaxUint64}},
Vector{Counter{math.MaxUint64, math.MaxUint64 - 1}},
Greater,
},
{
Vector{Counter{42, 2}},
Vector{Counter{42, 1}},
Greater,
},
{
Vector{Counter{22, 22}, Counter{42, 2}},
Vector{Counter{22, 22}, Counter{42, 1}},
Greater,
},
{
Vector{Counter{42, 2}, Counter{77, 3}},
Vector{Counter{42, 1}, Counter{77, 3}},
Greater,
},
{
Vector{Counter{22, 22}, Counter{42, 2}, Counter{77, 3}},
Vector{Counter{22, 22}, Counter{42, 1}, Counter{77, 3}},
Greater,
},
{
Vector{Counter{22, 23}, Counter{42, 2}, Counter{77, 4}},
Vector{Counter{22, 22}, Counter{42, 1}, Counter{77, 3}},
Greater,
},
// These a-vectors are all lesser than the b-vector
{nil, Vector{Counter{42, 1}}, Lesser},
{Vector{}, Vector{Counter{42, 1}}, Lesser},
{
Vector{Counter{42, 0}},
Vector{Counter{42, 1}},
Lesser,
},
{
Vector{Counter{42, 1}},
Vector{Counter{42, 2}},
Lesser,
},
{
Vector{Counter{22, 22}, Counter{42, 1}},
Vector{Counter{22, 22}, Counter{42, 2}},
Lesser,
},
{
Vector{Counter{42, 1}, Counter{77, 3}},
Vector{Counter{42, 2}, Counter{77, 3}},
Lesser,
},
{
Vector{Counter{22, 22}, Counter{42, 1}, Counter{77, 3}},
Vector{Counter{22, 22}, Counter{42, 2}, Counter{77, 3}},
Lesser,
},
{
Vector{Counter{22, 22}, Counter{42, 1}, Counter{77, 3}},
Vector{Counter{22, 23}, Counter{42, 2}, Counter{77, 4}},
Lesser,
},
// These are all in conflict
{
Vector{Counter{42, 2}},
Vector{Counter{43, 1}},
ConcurrentGreater,
},
{
Vector{Counter{43, 1}},
Vector{Counter{42, 2}},
ConcurrentLesser,
},
{
Vector{Counter{22, 23}, Counter{42, 1}},
Vector{Counter{22, 22}, Counter{42, 2}},
ConcurrentGreater,
},
{
Vector{Counter{22, 21}, Counter{42, 2}},
Vector{Counter{22, 22}, Counter{42, 1}},
ConcurrentLesser,
},
{
Vector{Counter{22, 21}, Counter{42, 2}, Counter{43, 1}},
Vector{Counter{20, 1}, Counter{22, 22}, Counter{42, 1}},
ConcurrentLesser,
},
}
for i, tc := range testcases {
// Test real Compare
if r := tc.a.Compare(tc.b); r != tc.r {
t.Errorf("%d: %+v.Compare(%+v) == %v (expected %v)", i, tc.a, tc.b, r, tc.r)
}
// Test convenience functions
switch tc.r {
case Greater:
if tc.a.Equal(tc.b) {
t.Errorf("%+v == %+v", tc.a, tc.b)
}
if tc.a.Concurrent(tc.b) {
t.Errorf("%+v concurrent %+v", tc.a, tc.b)
}
if !tc.a.GreaterEqual(tc.b) {
t.Errorf("%+v not >= %+v", tc.a, tc.b)
}
if tc.a.LesserEqual(tc.b) {
t.Errorf("%+v <= %+v", tc.a, tc.b)
}
case Lesser:
if tc.a.Concurrent(tc.b) {
t.Errorf("%+v concurrent %+v", tc.a, tc.b)
}
if tc.a.Equal(tc.b) {
t.Errorf("%+v == %+v", tc.a, tc.b)
}
if tc.a.GreaterEqual(tc.b) {
t.Errorf("%+v >= %+v", tc.a, tc.b)
}
if !tc.a.LesserEqual(tc.b) {
t.Errorf("%+v not <= %+v", tc.a, tc.b)
}
case Equal:
if tc.a.Concurrent(tc.b) {
t.Errorf("%+v concurrent %+v", tc.a, tc.b)
}
if !tc.a.Equal(tc.b) {
t.Errorf("%+v not == %+v", tc.a, tc.b)
}
if !tc.a.GreaterEqual(tc.b) {
t.Errorf("%+v not <= %+v", tc.a, tc.b)
}
if !tc.a.LesserEqual(tc.b) {
t.Errorf("%+v not <= %+v", tc.a, tc.b)
}
case ConcurrentLesser, ConcurrentGreater:
if !tc.a.Concurrent(tc.b) {
t.Errorf("%+v not concurrent %+v", tc.a, tc.b)
}
if tc.a.Equal(tc.b) {
t.Errorf("%+v == %+v", tc.a, tc.b)
}
if tc.a.GreaterEqual(tc.b) {
t.Errorf("%+v >= %+v", tc.a, tc.b)
}
if tc.a.LesserEqual(tc.b) {
t.Errorf("%+v <= %+v", tc.a, tc.b)
}
}
}
}

View File

@@ -2,7 +2,10 @@
package protocol
import "testing"
import (
"math"
"testing"
)
func TestUpdate(t *testing.T) {
var v Vector
@@ -10,7 +13,7 @@ func TestUpdate(t *testing.T) {
// Append
v = v.Update(42)
expected := Vector{Counter{42, 1}}
expected := Vector{[]Counter{{42, 1}}}
if v.Compare(expected) != Equal {
t.Errorf("Update error, %+v != %+v", v, expected)
@@ -19,7 +22,7 @@ func TestUpdate(t *testing.T) {
// Insert at front
v = v.Update(36)
expected = Vector{Counter{36, 1}, Counter{42, 1}}
expected = Vector{[]Counter{{36, 1}, {42, 1}}}
if v.Compare(expected) != Equal {
t.Errorf("Update error, %+v != %+v", v, expected)
@@ -28,7 +31,7 @@ func TestUpdate(t *testing.T) {
// Insert in moddle
v = v.Update(37)
expected = Vector{Counter{36, 1}, Counter{37, 1}, Counter{42, 1}}
expected = Vector{[]Counter{{36, 1}, {37, 1}, {42, 1}}}
if v.Compare(expected) != Equal {
t.Errorf("Update error, %+v != %+v", v, expected)
@@ -37,7 +40,7 @@ func TestUpdate(t *testing.T) {
// Update existing
v = v.Update(37)
expected = Vector{Counter{36, 1}, Counter{37, 2}, Counter{42, 1}}
expected = Vector{[]Counter{{36, 1}, {37, 2}, {42, 1}}}
if v.Compare(expected) != Equal {
t.Errorf("Update error, %+v != %+v", v, expected)
@@ -45,7 +48,7 @@ func TestUpdate(t *testing.T) {
}
func TestCopy(t *testing.T) {
v0 := Vector{Counter{42, 1}}
v0 := Vector{[]Counter{{42, 1}}}
v1 := v0.Copy()
v1.Update(42)
if v0.Compare(v1) != Lesser {
@@ -64,52 +67,52 @@ func TestMerge(t *testing.T) {
Vector{},
},
{
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{[]Counter{{22, 1}, {42, 1}}},
Vector{[]Counter{{22, 1}, {42, 1}}},
Vector{[]Counter{{22, 1}, {42, 1}}},
},
// Appends
{
Vector{},
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{[]Counter{{22, 1}, {42, 1}}},
Vector{[]Counter{{22, 1}, {42, 1}}},
},
{
Vector{Counter{22, 1}},
Vector{Counter{42, 1}},
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{[]Counter{{22, 1}}},
Vector{[]Counter{{42, 1}}},
Vector{[]Counter{{22, 1}, {42, 1}}},
},
{
Vector{Counter{22, 1}},
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{[]Counter{{22, 1}}},
Vector{[]Counter{{22, 1}, {42, 1}}},
Vector{[]Counter{{22, 1}, {42, 1}}},
},
// Insert
{
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{Counter{22, 1}, Counter{23, 2}, Counter{42, 1}},
Vector{Counter{22, 1}, Counter{23, 2}, Counter{42, 1}},
Vector{[]Counter{{22, 1}, {42, 1}}},
Vector{[]Counter{{22, 1}, {23, 2}, {42, 1}}},
Vector{[]Counter{{22, 1}, {23, 2}, {42, 1}}},
},
{
Vector{Counter{42, 1}},
Vector{Counter{22, 1}},
Vector{Counter{22, 1}, Counter{42, 1}},
Vector{[]Counter{{42, 1}}},
Vector{[]Counter{{22, 1}}},
Vector{[]Counter{{22, 1}, {42, 1}}},
},
// Update
{
Vector{Counter{22, 1}, Counter{42, 2}},
Vector{Counter{22, 2}, Counter{42, 1}},
Vector{Counter{22, 2}, Counter{42, 2}},
Vector{[]Counter{{22, 1}, {42, 2}}},
Vector{[]Counter{{22, 2}, {42, 1}}},
Vector{[]Counter{{22, 2}, {42, 2}}},
},
// All of the above
{
Vector{Counter{10, 1}, Counter{20, 2}, Counter{30, 1}},
Vector{Counter{5, 1}, Counter{10, 2}, Counter{15, 1}, Counter{20, 1}, Counter{25, 1}, Counter{35, 1}},
Vector{Counter{5, 1}, Counter{10, 2}, Counter{15, 1}, Counter{20, 2}, Counter{25, 1}, Counter{30, 1}, Counter{35, 1}},
Vector{[]Counter{{10, 1}, {20, 2}, {30, 1}}},
Vector{[]Counter{{5, 1}, {10, 2}, {15, 1}, {20, 1}, {25, 1}, {35, 1}}},
Vector{[]Counter{{5, 1}, {10, 2}, {15, 1}, {20, 2}, {25, 1}, {30, 1}, {35, 1}}},
},
}
@@ -121,7 +124,7 @@ func TestMerge(t *testing.T) {
}
func TestCounterValue(t *testing.T) {
v0 := Vector{Counter{42, 1}, Counter{64, 5}}
v0 := Vector{[]Counter{{42, 1}, {64, 5}}}
if v0.Counter(42) != 1 {
t.Errorf("Counter error, %d != %d", v0.Counter(42), 1)
}
@@ -132,3 +135,234 @@ func TestCounterValue(t *testing.T) {
t.Errorf("Counter error, %d != %d", v0.Counter(72), 0)
}
}
func TestCompare(t *testing.T) {
testcases := []struct {
a, b Vector
r Ordering
}{
// Empty vectors are identical
{Vector{}, Vector{}, Equal},
{Vector{}, Vector{[]Counter{{42, 0}}}, Equal},
{Vector{[]Counter{Counter{42, 0}}}, Vector{}, Equal},
// Zero is the implied value for a missing Counter
{
Vector{[]Counter{{42, 0}}},
Vector{[]Counter{{77, 0}}},
Equal,
},
// Equal vectors are equal
{
Vector{[]Counter{{42, 33}}},
Vector{[]Counter{{42, 33}}},
Equal,
},
{
Vector{[]Counter{{42, 33}, {77, 24}}},
Vector{[]Counter{{42, 33}, {77, 24}}},
Equal,
},
// These a-vectors are all greater than the b-vector
{
Vector{[]Counter{{42, 1}}},
Vector{},
Greater,
},
{
Vector{[]Counter{{0, 1}}},
Vector{[]Counter{{0, 0}}},
Greater,
},
{
Vector{[]Counter{{42, 1}}},
Vector{[]Counter{{42, 0}}},
Greater,
},
{
Vector{[]Counter{{math.MaxUint64, 1}}},
Vector{[]Counter{{math.MaxUint64, 0}}},
Greater,
},
{
Vector{[]Counter{{0, math.MaxUint64}}},
Vector{[]Counter{{0, 0}}},
Greater,
},
{
Vector{[]Counter{{42, math.MaxUint64}}},
Vector{[]Counter{{42, 0}}},
Greater,
},
{
Vector{[]Counter{{math.MaxUint64, math.MaxUint64}}},
Vector{[]Counter{{math.MaxUint64, 0}}},
Greater,
},
{
Vector{[]Counter{{0, math.MaxUint64}}},
Vector{[]Counter{{0, math.MaxUint64 - 1}}},
Greater,
},
{
Vector{[]Counter{{42, math.MaxUint64}}},
Vector{[]Counter{{42, math.MaxUint64 - 1}}},
Greater,
},
{
Vector{[]Counter{{math.MaxUint64, math.MaxUint64}}},
Vector{[]Counter{{math.MaxUint64, math.MaxUint64 - 1}}},
Greater,
},
{
Vector{[]Counter{{42, 2}}},
Vector{[]Counter{{42, 1}}},
Greater,
},
{
Vector{[]Counter{{22, 22}, {42, 2}}},
Vector{[]Counter{{22, 22}, {42, 1}}},
Greater,
},
{
Vector{[]Counter{{42, 2}, {77, 3}}},
Vector{[]Counter{{42, 1}, {77, 3}}},
Greater,
},
{
Vector{[]Counter{{22, 22}, {42, 2}, {77, 3}}},
Vector{[]Counter{{22, 22}, {42, 1}, {77, 3}}},
Greater,
},
{
Vector{[]Counter{{22, 23}, {42, 2}, {77, 4}}},
Vector{[]Counter{{22, 22}, {42, 1}, {77, 3}}},
Greater,
},
// These a-vectors are all lesser than the b-vector
{Vector{}, Vector{[]Counter{{42, 1}}}, Lesser},
{
Vector{[]Counter{{42, 0}}},
Vector{[]Counter{{42, 1}}},
Lesser,
},
{
Vector{[]Counter{{42, 1}}},
Vector{[]Counter{{42, 2}}},
Lesser,
},
{
Vector{[]Counter{{22, 22}, {42, 1}}},
Vector{[]Counter{{22, 22}, {42, 2}}},
Lesser,
},
{
Vector{[]Counter{{42, 1}, {77, 3}}},
Vector{[]Counter{{42, 2}, {77, 3}}},
Lesser,
},
{
Vector{[]Counter{{22, 22}, {42, 1}, {77, 3}}},
Vector{[]Counter{{22, 22}, {42, 2}, {77, 3}}},
Lesser,
},
{
Vector{[]Counter{{22, 22}, {42, 1}, {77, 3}}},
Vector{[]Counter{{22, 23}, {42, 2}, {77, 4}}},
Lesser,
},
// These are all in conflict
{
Vector{[]Counter{{42, 2}}},
Vector{[]Counter{{43, 1}}},
ConcurrentGreater,
},
{
Vector{[]Counter{{43, 1}}},
Vector{[]Counter{{42, 2}}},
ConcurrentLesser,
},
{
Vector{[]Counter{{22, 23}, {42, 1}}},
Vector{[]Counter{{22, 22}, {42, 2}}},
ConcurrentGreater,
},
{
Vector{[]Counter{{22, 21}, {42, 2}}},
Vector{[]Counter{{22, 22}, {42, 1}}},
ConcurrentLesser,
},
{
Vector{[]Counter{{22, 21}, {42, 2}, {43, 1}}},
Vector{[]Counter{{20, 1}, {22, 22}, {42, 1}}},
ConcurrentLesser,
},
}
for i, tc := range testcases {
// Test real Compare
if r := tc.a.Compare(tc.b); r != tc.r {
t.Errorf("%d: %+v.Compare(%+v) == %v (expected %v)", i, tc.a, tc.b, r, tc.r)
}
// Test convenience functions
switch tc.r {
case Greater:
if tc.a.Equal(tc.b) {
t.Errorf("%+v == %+v", tc.a, tc.b)
}
if tc.a.Concurrent(tc.b) {
t.Errorf("%+v concurrent %+v", tc.a, tc.b)
}
if !tc.a.GreaterEqual(tc.b) {
t.Errorf("%+v not >= %+v", tc.a, tc.b)
}
if tc.a.LesserEqual(tc.b) {
t.Errorf("%+v <= %+v", tc.a, tc.b)
}
case Lesser:
if tc.a.Concurrent(tc.b) {
t.Errorf("%+v concurrent %+v", tc.a, tc.b)
}
if tc.a.Equal(tc.b) {
t.Errorf("%+v == %+v", tc.a, tc.b)
}
if tc.a.GreaterEqual(tc.b) {
t.Errorf("%+v >= %+v", tc.a, tc.b)
}
if !tc.a.LesserEqual(tc.b) {
t.Errorf("%+v not <= %+v", tc.a, tc.b)
}
case Equal:
if tc.a.Concurrent(tc.b) {
t.Errorf("%+v concurrent %+v", tc.a, tc.b)
}
if !tc.a.Equal(tc.b) {
t.Errorf("%+v not == %+v", tc.a, tc.b)
}
if !tc.a.GreaterEqual(tc.b) {
t.Errorf("%+v not <= %+v", tc.a, tc.b)
}
if !tc.a.LesserEqual(tc.b) {
t.Errorf("%+v not <= %+v", tc.a, tc.b)
}
case ConcurrentLesser, ConcurrentGreater:
if !tc.a.Concurrent(tc.b) {
t.Errorf("%+v not concurrent %+v", tc.a, tc.b)
}
if tc.a.Equal(tc.b) {
t.Errorf("%+v == %+v", tc.a, tc.b)
}
if tc.a.GreaterEqual(tc.b) {
t.Errorf("%+v >= %+v", tc.a, tc.b)
}
if tc.a.LesserEqual(tc.b) {
t.Errorf("%+v <= %+v", tc.a, tc.b)
}
}
}
}

View File

@@ -1,35 +0,0 @@
// Copyright (C) 2015 The Protocol Authors.
package protocol
import "github.com/calmh/xdr"
// This stuff is hacked up manually because genxdr doesn't support 'type
// Vector []Counter' declarations and it was tricky when I tried to add it...
func (v Vector) MarshalXDRInto(m *xdr.Marshaller) error {
m.MarshalUint32(uint32(len(v)))
for i := range v {
m.MarshalUint64(uint64(v[i].ID))
m.MarshalUint64(v[i].Value)
}
return m.Error
}
func (v *Vector) UnmarshalXDRFrom(u *xdr.Unmarshaller) error {
l := int(u.UnmarshalUint32())
if l > 1e6 {
return xdr.ElementSizeExceeded("number of counters", l, 1e6)
}
n := make(Vector, l)
for i := range n {
n[i].ID = ShortID(u.UnmarshalUint64())
n[i].Value = u.UnmarshalUint64()
}
*v = n
return u.Error
}
func (v Vector) XDRSize() int {
return 4 + 16*len(v)
}

View File

@@ -12,7 +12,7 @@ type wireFormatConnection struct {
Connection
}
func (c wireFormatConnection) Index(folder string, fs []FileInfo, flags uint32, options []Option) error {
func (c wireFormatConnection) Index(folder string, fs []FileInfo) error {
var myFs = make([]FileInfo, len(fs))
copy(myFs, fs)
@@ -20,10 +20,10 @@ func (c wireFormatConnection) Index(folder string, fs []FileInfo, flags uint32,
myFs[i].Name = norm.NFC.String(filepath.ToSlash(myFs[i].Name))
}
return c.Connection.Index(folder, myFs, flags, options)
return c.Connection.Index(folder, myFs)
}
func (c wireFormatConnection) IndexUpdate(folder string, fs []FileInfo, flags uint32, options []Option) error {
func (c wireFormatConnection) IndexUpdate(folder string, fs []FileInfo) error {
var myFs = make([]FileInfo, len(fs))
copy(myFs, fs)
@@ -31,7 +31,7 @@ func (c wireFormatConnection) IndexUpdate(folder string, fs []FileInfo, flags ui
myFs[i].Name = norm.NFC.String(filepath.ToSlash(myFs[i].Name))
}
return c.Connection.IndexUpdate(folder, myFs, flags, options)
return c.Connection.IndexUpdate(folder, myFs)
}
func (c wireFormatConnection) Request(folder, name string, offset int64, size int, hash []byte, fromTemporary bool) ([]byte, error) {