cmd/syncthing, lib/db, lib/model, lib/protocol: Implement delta indexes (fixes #438)

GitHub-Pull-Request: https://github.com/syncthing/syncthing/pull/3427
This commit is contained in:
Jakob Borg 2016-07-23 12:46:31 +00:00 committed by Audrius Butkevicius
parent 8ab6b60778
commit 47fa4b0a2c
11 changed files with 428 additions and 222 deletions

View File

@ -682,17 +682,9 @@ func syncthingMain(runtimeOptions RuntimeOptions) {
}
}
// Clear out old indexes for other devices. Otherwise we'll start up and
// start needing a bunch of files which are nowhere to be found. This
// needs to be changed when we correctly do persistent indexes.
// Add and start folders
for _, folderCfg := range cfg.Folders() {
m.AddFolder(folderCfg)
for _, device := range folderCfg.DeviceIDs() {
if device == myID {
continue
}
m.Index(device, folderCfg.ID, nil)
}
m.StartFolder(folderCfg.ID)
}

View File

@ -11,27 +11,10 @@ import (
"fmt"
"github.com/syncthing/syncthing/lib/protocol"
"github.com/syncthing/syncthing/lib/sync"
"github.com/syndtr/goleveldb/leveldb"
"github.com/syndtr/goleveldb/leveldb/opt"
)
var (
clockTick int64
clockMut = sync.NewMutex()
)
func clock(v int64) int64 {
clockMut.Lock()
defer clockMut.Unlock()
if v > clockTick {
clockTick = v + 1
} else {
clockTick++
}
return clockTick
}
const (
KeyTypeDevice = iota
KeyTypeGlobal
@ -41,6 +24,7 @@ const (
KeyTypeVirtualMtime
KeyTypeFolderIdx
KeyTypeDeviceIdx
KeyTypeIndexID
)
func (l VersionList) String() string {

View File

@ -24,7 +24,7 @@ import (
"github.com/syndtr/goleveldb/leveldb/util"
)
type deletionHandler func(t readWriteTransaction, folder, device, name []byte, dbi iterator.Iterator) int64
type deletionHandler func(t readWriteTransaction, folder, device, name []byte, dbi iterator.Iterator)
type Instance struct {
committed int64 // this must be the first attribute in the struct to ensure 64 bit alignment on 32 bit plaforms
@ -86,7 +86,7 @@ func (db *Instance) Committed() int64 {
return atomic.LoadInt64(&db.committed)
}
func (db *Instance) genericReplace(folder, device []byte, fs []protocol.FileInfo, localSize, globalSize *sizeTracker, deleteFn deletionHandler) int64 {
func (db *Instance) genericReplace(folder, device []byte, fs []protocol.FileInfo, localSize, globalSize *sizeTracker, deleteFn deletionHandler) {
sort.Sort(fileList(fs)) // sort list on name, same as in the database
t := db.newReadWriteTransaction()
@ -97,7 +97,6 @@ func (db *Instance) genericReplace(folder, device []byte, fs []protocol.FileInfo
moreDb := dbi.Next()
fsi := 0
var maxLocalVer int64
isLocalDevice := bytes.Equal(device, protocol.LocalDeviceID[:])
for {
@ -124,9 +123,7 @@ func (db *Instance) genericReplace(folder, device []byte, fs []protocol.FileInfo
case moreFs && (!moreDb || cmp == -1):
l.Debugln("generic replace; missing - insert")
// Database is missing this file. Insert it.
if lv := t.insertFile(folder, device, fs[fsi]); lv > maxLocalVer {
maxLocalVer = lv
}
t.insertFile(folder, device, fs[fsi])
if isLocalDevice {
localSize.addFile(fs[fsi])
}
@ -146,9 +143,7 @@ func (db *Instance) genericReplace(folder, device []byte, fs []protocol.FileInfo
ef.Unmarshal(dbi.Value())
if !fs[fsi].Version.Equal(ef.Version) || fs[fsi].Invalid != ef.Invalid {
l.Debugln("generic replace; differs - insert")
if lv := t.insertFile(folder, device, fs[fsi]); lv > maxLocalVer {
maxLocalVer = lv
}
t.insertFile(folder, device, fs[fsi])
if isLocalDevice {
localSize.removeFile(ef)
localSize.addFile(fs[fsi])
@ -167,9 +162,7 @@ func (db *Instance) genericReplace(folder, device []byte, fs []protocol.FileInfo
case moreDb && (!moreFs || cmp == 1):
l.Debugln("generic replace; exists - remove")
if lv := deleteFn(t, folder, device, oldName, dbi); lv > maxLocalVer {
maxLocalVer = lv
}
deleteFn(t, folder, device, oldName, dbi)
moreDb = dbi.Next()
}
@ -177,26 +170,21 @@ func (db *Instance) genericReplace(folder, device []byte, fs []protocol.FileInfo
// growing too large and thus allocating unnecessarily much memory.
t.checkFlush()
}
return maxLocalVer
}
func (db *Instance) replace(folder, device []byte, fs []protocol.FileInfo, localSize, globalSize *sizeTracker) int64 {
// TODO: Return the remaining maxLocalVer?
return db.genericReplace(folder, device, fs, localSize, globalSize, func(t readWriteTransaction, folder, device, name []byte, dbi iterator.Iterator) int64 {
func (db *Instance) replace(folder, device []byte, fs []protocol.FileInfo, localSize, globalSize *sizeTracker) {
db.genericReplace(folder, device, fs, localSize, globalSize, func(t readWriteTransaction, folder, device, name []byte, dbi iterator.Iterator) {
// Database has a file that we are missing. Remove it.
l.Debugf("delete; folder=%q device=%v name=%q", folder, protocol.DeviceIDFromBytes(device), name)
t.removeFromGlobal(folder, device, name, globalSize)
t.Delete(dbi.Key())
return 0
})
}
func (db *Instance) updateFiles(folder, device []byte, fs []protocol.FileInfo, localSize, globalSize *sizeTracker) int64 {
func (db *Instance) updateFiles(folder, device []byte, fs []protocol.FileInfo, localSize, globalSize *sizeTracker) {
t := db.newReadWriteTransaction()
defer t.close()
var maxLocalVer int64
var fk []byte
isLocalDevice := bytes.Equal(device, protocol.LocalDeviceID[:])
for _, f := range fs {
@ -208,9 +196,7 @@ func (db *Instance) updateFiles(folder, device []byte, fs []protocol.FileInfo, l
localSize.addFile(f)
}
if lv := t.insertFile(folder, device, f); lv > maxLocalVer {
maxLocalVer = lv
}
t.insertFile(folder, device, f)
if f.IsInvalid() {
t.removeFromGlobal(folder, device, name, globalSize)
} else {
@ -231,9 +217,7 @@ func (db *Instance) updateFiles(folder, device []byte, fs []protocol.FileInfo, l
localSize.addFile(f)
}
if lv := t.insertFile(folder, device, f); lv > maxLocalVer {
maxLocalVer = lv
}
t.insertFile(folder, device, f)
if f.IsInvalid() {
t.removeFromGlobal(folder, device, name, globalSize)
} else {
@ -245,8 +229,6 @@ func (db *Instance) updateFiles(folder, device []byte, fs []protocol.FileInfo, l
// growing too large and thus allocating unnecessarily much memory.
t.checkFlush()
}
return maxLocalVer
}
func (db *Instance) withHave(folder, device, prefix []byte, truncate bool, fn Iterator) {
@ -699,6 +681,37 @@ func (db *Instance) globalKeyFolder(key []byte) []byte {
return folder
}
func (db *Instance) getIndexID(device, folder []byte) protocol.IndexID {
key := db.indexIDKey(device, folder)
cur, err := db.Get(key, nil)
if err != nil {
return 0
}
var id protocol.IndexID
if err := id.Unmarshal(cur); err != nil {
return 0
}
return id
}
func (db *Instance) setIndexID(device, folder []byte, id protocol.IndexID) {
key := db.indexIDKey(device, folder)
bs, _ := id.Marshal() // marshalling can't fail
if err := db.Put(key, bs, nil); err != nil {
panic("storing index ID: " + err.Error())
}
}
func (db *Instance) indexIDKey(device, folder []byte) []byte {
k := make([]byte, keyPrefixLen+keyDeviceLen+keyFolderLen)
k[0] = KeyTypeIndexID
binary.BigEndian.PutUint32(k[keyPrefixLen:], db.deviceIdx.ID(device))
binary.BigEndian.PutUint32(k[keyPrefixLen+keyDeviceLen:], db.folderIdx.ID(folder))
return k
}
func unmarshalTrunc(bs []byte, truncate bool) (FileIntf, error) {
if truncate {
var tf FileInfoTruncated

View File

@ -74,18 +74,12 @@ func (t readWriteTransaction) flush() {
atomic.AddInt64(&t.db.committed, int64(t.Batch.Len()))
}
func (t readWriteTransaction) insertFile(folder, device []byte, file protocol.FileInfo) int64 {
func (t readWriteTransaction) insertFile(folder, device []byte, file protocol.FileInfo) {
l.Debugf("insert; folder=%q device=%v %v", folder, protocol.DeviceIDFromBytes(device), file)
if file.LocalVersion == 0 {
file.LocalVersion = clock(0)
}
name := []byte(file.Name)
nk := t.db.deviceKey(folder, device, name)
t.Put(nk, mustMarshal(&file))
return file.LocalVersion
}
// updateGlobal adds this device+version to the version list for the given

View File

@ -14,6 +14,7 @@ package db
import (
stdsync "sync"
"sync/atomic"
"github.com/syncthing/syncthing/lib/osutil"
"github.com/syncthing/syncthing/lib/protocol"
@ -21,13 +22,15 @@ import (
)
type FileSet struct {
localVersion map[protocol.DeviceID]int64
mutex sync.Mutex
localVersion int64 // Our local version counter
folder string
db *Instance
blockmap *BlockMap
localSize sizeTracker
globalSize sizeTracker
remoteLocalVersion map[protocol.DeviceID]int64 // Highest seen local versions for other devices
rlvMutex sync.Mutex // protects remoteLocalVersion
}
// FileIntf is the set of methods implemented by both protocol.FileInfo and
@ -95,11 +98,11 @@ func (s *sizeTracker) Size() (files, deleted int, bytes int64) {
func NewFileSet(folder string, db *Instance) *FileSet {
var s = FileSet{
localVersion: make(map[protocol.DeviceID]int64),
folder: folder,
db: db,
blockmap: NewBlockMap(db, db.folderIdx.ID([]byte(folder))),
mutex: sync.NewMutex(),
remoteLocalVersion: make(map[protocol.DeviceID]int64),
folder: folder,
db: db,
blockmap: NewBlockMap(db, db.folderIdx.ID([]byte(folder))),
rlvMutex: sync.NewMutex(),
}
s.db.checkGlobals([]byte(folder), &s.globalSize)
@ -107,16 +110,17 @@ func NewFileSet(folder string, db *Instance) *FileSet {
var deviceID protocol.DeviceID
s.db.withAllFolderTruncated([]byte(folder), func(device []byte, f FileInfoTruncated) bool {
copy(deviceID[:], device)
if f.LocalVersion > s.localVersion[deviceID] {
s.localVersion[deviceID] = f.LocalVersion
}
if deviceID == protocol.LocalDeviceID {
if f.LocalVersion > s.localVersion {
s.localVersion = f.LocalVersion
}
s.localSize.addFile(f)
} else if f.LocalVersion > s.remoteLocalVersion[deviceID] {
s.remoteLocalVersion[deviceID] = f.LocalVersion
}
return true
})
l.Debugf("loaded localVersion for %q: %#v", folder, s.localVersion)
clock(s.localVersion[protocol.LocalDeviceID])
return &s
}
@ -124,13 +128,23 @@ func NewFileSet(folder string, db *Instance) *FileSet {
func (s *FileSet) Replace(device protocol.DeviceID, fs []protocol.FileInfo) {
l.Debugf("%s Replace(%v, [%d])", s.folder, device, len(fs))
normalizeFilenames(fs)
s.mutex.Lock()
defer s.mutex.Unlock()
s.localVersion[device] = s.db.replace([]byte(s.folder), device[:], fs, &s.localSize, &s.globalSize)
if len(fs) == 0 {
// Reset the local version if all files were removed.
s.localVersion[device] = 0
if device == protocol.LocalDeviceID {
if len(fs) == 0 {
s.localVersion = 0
} else {
// Always overwrite LocalVersion on updated files to ensure
// correct ordering. The caller is supposed to leave it set to
// zero anyhow.
for i := range fs {
fs[i].LocalVersion = atomic.AddInt64(&s.localVersion, 1)
}
}
} else {
s.rlvMutex.Lock()
s.remoteLocalVersion[device] = maxLocalVersion(fs)
s.rlvMutex.Unlock()
}
s.db.replace([]byte(s.folder), device[:], fs, &s.localSize, &s.globalSize)
if device == protocol.LocalDeviceID {
s.blockmap.Drop()
s.blockmap.Add(fs)
@ -140,12 +154,11 @@ func (s *FileSet) Replace(device protocol.DeviceID, fs []protocol.FileInfo) {
func (s *FileSet) Update(device protocol.DeviceID, fs []protocol.FileInfo) {
l.Debugf("%s Update(%v, [%d])", s.folder, device, len(fs))
normalizeFilenames(fs)
s.mutex.Lock()
defer s.mutex.Unlock()
if device == protocol.LocalDeviceID {
discards := make([]protocol.FileInfo, 0, len(fs))
updates := make([]protocol.FileInfo, 0, len(fs))
for _, newFile := range fs {
for i, newFile := range fs {
fs[i].LocalVersion = atomic.AddInt64(&s.localVersion, 1)
existingFile, ok := s.db.getFile([]byte(s.folder), device[:], []byte(newFile.Name))
if !ok || !existingFile.Version.Equal(newFile.Version) {
discards = append(discards, existingFile)
@ -154,10 +167,12 @@ func (s *FileSet) Update(device protocol.DeviceID, fs []protocol.FileInfo) {
}
s.blockmap.Discard(discards)
s.blockmap.Update(updates)
} else {
s.rlvMutex.Lock()
s.remoteLocalVersion[device] = maxLocalVersion(fs)
s.rlvMutex.Unlock()
}
if lv := s.db.updateFiles([]byte(s.folder), device[:], fs, &s.localSize, &s.globalSize); lv > s.localVersion[device] {
s.localVersion[device] = lv
}
s.db.updateFiles([]byte(s.folder), device[:], fs, &s.localSize, &s.globalSize)
}
func (s *FileSet) WithNeed(device protocol.DeviceID, fn Iterator) {
@ -230,9 +245,13 @@ func (s *FileSet) Availability(file string) []protocol.DeviceID {
}
func (s *FileSet) LocalVersion(device protocol.DeviceID) int64 {
s.mutex.Lock()
defer s.mutex.Unlock()
return s.localVersion[device]
if device == protocol.LocalDeviceID {
return atomic.LoadInt64(&s.localVersion)
}
s.rlvMutex.Lock()
defer s.rlvMutex.Unlock()
return s.remoteLocalVersion[device]
}
func (s *FileSet) LocalSize() (files, deleted int, bytes int64) {
@ -243,6 +262,37 @@ func (s *FileSet) GlobalSize() (files, deleted int, bytes int64) {
return s.globalSize.Size()
}
func (s *FileSet) IndexID(device protocol.DeviceID) protocol.IndexID {
id := s.db.getIndexID(device[:], []byte(s.folder))
if id == 0 && device == protocol.LocalDeviceID {
// No index ID set yet. We create one now.
id = protocol.NewIndexID()
s.db.setIndexID(device[:], []byte(s.folder), id)
}
return id
}
func (s *FileSet) SetIndexID(device protocol.DeviceID, id protocol.IndexID) {
if device == protocol.LocalDeviceID {
panic("do not explicitly set index ID for local device")
}
s.db.setIndexID(device[:], []byte(s.folder), id)
}
// maxLocalVersion returns the highest of the LocalVersion numbers found in
// the given slice of FileInfos. This should really be the LocalVersion of
// the last item, but Syncthing v0.14.0 and other implementations may not
// implement update sorting....
func maxLocalVersion(fs []protocol.FileInfo) int64 {
var max int64
for _, f := range fs {
if f.LocalVersion > max {
max = f.LocalVersion
}
}
return max
}
// DropFolder clears out all information related to the given folder from the
// database.
func DropFolder(db *Instance, folder string) {

View File

@ -100,11 +100,11 @@ func TestGlobalSet(t *testing.T) {
m := db.NewFileSet("test", ldb)
local0 := fileList{
protocol.FileInfo{Name: "a", Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(1)},
protocol.FileInfo{Name: "b", Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(2)},
protocol.FileInfo{Name: "c", Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(3)},
protocol.FileInfo{Name: "d", Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(4)},
protocol.FileInfo{Name: "z", Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(8)},
protocol.FileInfo{Name: "a", LocalVersion: 1, Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(1)},
protocol.FileInfo{Name: "b", LocalVersion: 2, Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(2)},
protocol.FileInfo{Name: "c", LocalVersion: 3, Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(3)},
protocol.FileInfo{Name: "d", LocalVersion: 4, Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(4)},
protocol.FileInfo{Name: "z", LocalVersion: 5, Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(8)},
}
local1 := fileList{
protocol.FileInfo{Name: "a", Version: protocol.Vector{Counters: []protocol.Counter{{ID: myID, Value: 1000}}}, Blocks: genBlocks(1)},
@ -687,3 +687,35 @@ func BenchmarkUpdateOneFile(b *testing.B) {
b.ReportAllocs()
}
func TestIndexID(t *testing.T) {
ldb := db.OpenMemory()
s := db.NewFileSet("test", ldb)
// The Index ID for some random device is zero by default.
id := s.IndexID(remoteDevice0)
if id != 0 {
t.Errorf("index ID for remote device should default to zero, not %d", id)
}
// The Index ID for someone else should be settable
s.SetIndexID(remoteDevice0, 42)
id = s.IndexID(remoteDevice0)
if id != 42 {
t.Errorf("index ID for remote device should be remembered; got %d, expected %d", id, 42)
}
// Our own index ID should be generated randomly.
id = s.IndexID(protocol.LocalDeviceID)
if id == 0 {
t.Errorf("index ID for local device should be random, not zero")
}
t.Logf("random index ID is 0x%016x", id)
// But of course always the same after that.
again := s.IndexID(protocol.LocalDeviceID)
if again != id {
t.Errorf("index ID changed; %d != %d", again, id)
}
}

View File

@ -8,6 +8,7 @@ package model
import (
"bufio"
"bytes"
"crypto/tls"
"encoding/json"
"errors"
@ -647,6 +648,10 @@ func (m *Model) ClusterConfig(deviceID protocol.DeviceID, cm protocol.ClusterCon
tempIndexFolders := make([]string, 0, len(cm.Folders))
m.pmut.RLock()
conn := m.conn[deviceID]
m.pmut.RUnlock()
m.fmut.Lock()
for _, folder := range cm.Folders {
if !m.folderSharedWithUnlocked(folder.ID, deviceID) {
@ -661,6 +666,71 @@ func (m *Model) ClusterConfig(deviceID protocol.DeviceID, cm protocol.ClusterCon
if !folder.DisableTempIndexes {
tempIndexFolders = append(tempIndexFolders, folder.ID)
}
fs := m.folderFiles[folder.ID]
myIndexID := fs.IndexID(protocol.LocalDeviceID)
myLocalVersion := fs.LocalVersion(protocol.LocalDeviceID)
var startLocalVersion int64
for _, dev := range folder.Devices {
if bytes.Equal(dev.ID, m.id[:]) {
// This is the other side's description of what it knows
// about us. Lets check to see if we can start sending index
// updates directly or need to send the index from start...
if dev.IndexID == myIndexID {
// They say they've seen our index ID before, so we can
// send a delta update only.
if dev.MaxLocalVersion > myLocalVersion {
// Safety check. They claim to have more or newer
// index data than we have - either we have lost
// index data, or reset the index without resetting
// the IndexID, or something else weird has
// happened. We send a full index to reset the
// situation.
l.Infof("Device %v folder %q is delta index compatible, but seems out of sync with reality", deviceID, folder.ID)
startLocalVersion = 0
continue
}
l.Infof("Device %v folder %q is delta index compatible (mlv=%d)", deviceID, folder.ID, dev.MaxLocalVersion)
startLocalVersion = dev.MaxLocalVersion
} else if dev.IndexID != 0 {
// They say they've seen an index ID from us, but it's
// not the right one. Either they are confused or we
// must have reset our database since last talking to
// them. We'll start with a full index transfer.
l.Infof("Device %v folder %q has mismatching index ID for us (%v != %v)", deviceID, folder.ID, dev.IndexID, myIndexID)
startLocalVersion = 0
}
} else if bytes.Equal(dev.ID, deviceID[:]) && dev.IndexID != 0 {
// This is the other side's description of themselves. We
// check to see that it matches the IndexID we have on file,
// otherwise we drop our old index data and expect to get a
// completely new set.
theirIndexID := fs.IndexID(deviceID)
if dev.IndexID == 0 {
// They're not announcing an index ID. This means they
// do not support delta indexes and we should clear any
// information we have from them before accepting their
// index, which will presumably be a full index.
fs.Replace(deviceID, nil)
} else if dev.IndexID != theirIndexID {
// The index ID we have on file is not what they're
// announcing. They must have reset their database and
// will probably send us a full index. We drop any
// information we have and remember this new index ID
// instead.
l.Infof("Device %v folder %q has a new index ID (%v)", deviceID, folder.ID, dev.IndexID)
fs.Replace(deviceID, nil)
fs.SetIndexID(deviceID, dev.IndexID)
}
}
}
go sendIndexes(conn, folder.ID, fs, m.folderIgnores[folder.ID], startLocalVersion)
}
m.fmut.Unlock()
@ -763,12 +833,6 @@ func (m *Model) Close(device protocol.DeviceID, err error) {
})
m.pmut.Lock()
m.fmut.RLock()
for _, folder := range m.deviceFolders[device] {
m.folderFiles[folder].Replace(device, nil)
}
m.fmut.RUnlock()
conn, ok := m.conn[device]
if ok {
m.progressEmitter.temporaryIndexUnsubscribe(conn)
@ -1044,13 +1108,6 @@ func (m *Model) AddConnection(conn connections.Connection, hello protocol.HelloR
cm := m.generateClusterConfig(deviceID)
conn.ClusterConfig(cm)
m.fmut.RLock()
for _, folder := range m.deviceFolders[deviceID] {
fs := m.folderFiles[folder]
go sendIndexes(conn, folder, fs, m.folderIgnores[folder])
}
m.fmut.RUnlock()
m.pmut.Unlock()
device, ok := m.cfg.Devices()[deviceID]
@ -1146,15 +1203,15 @@ func (m *Model) receivedFile(folder string, file protocol.FileInfo) {
m.folderStatRef(folder).ReceivedFile(file.Name, file.IsDeleted())
}
func sendIndexes(conn protocol.Connection, folder string, fs *db.FileSet, ignores *ignore.Matcher) {
func sendIndexes(conn protocol.Connection, folder string, fs *db.FileSet, ignores *ignore.Matcher, startLocalVersion int64) {
deviceID := conn.ID()
name := conn.Name()
var err error
l.Debugf("sendIndexes for %s-%s/%q starting", deviceID, name, folder)
l.Debugf("sendIndexes for %s-%s/%q starting (slv=%d)", deviceID, name, folder, startLocalVersion)
defer l.Debugf("sendIndexes for %s-%s/%q exiting: %v", deviceID, name, folder, err)
minLocalVer, err := sendIndexTo(true, 0, conn, folder, fs, ignores)
minLocalVer, err := sendIndexTo(startLocalVersion, conn, folder, fs, ignores)
// Subscribe to LocalIndexUpdated (we have new information to send) and
// DeviceDisconnected (it might be us who disconnected, so we should
@ -1177,7 +1234,7 @@ func sendIndexes(conn protocol.Connection, folder string, fs *db.FileSet, ignore
continue
}
minLocalVer, err = sendIndexTo(false, minLocalVer, conn, folder, fs, ignores)
minLocalVer, err = sendIndexTo(minLocalVer, conn, folder, fs, ignores)
// Wait a short amount of time before entering the next loop. If there
// are continuous changes happening to the local index, this gives us
@ -1186,12 +1243,13 @@ func sendIndexes(conn protocol.Connection, folder string, fs *db.FileSet, ignore
}
}
func sendIndexTo(initial bool, minLocalVer int64, conn protocol.Connection, folder string, fs *db.FileSet, ignores *ignore.Matcher) (int64, error) {
func sendIndexTo(minLocalVer int64, conn protocol.Connection, folder string, fs *db.FileSet, ignores *ignore.Matcher) (int64, error) {
deviceID := conn.ID()
name := conn.Name()
batch := make([]protocol.FileInfo, 0, indexBatchSize)
currentBatchSize := 0
maxLocalVer := int64(0)
initial := minLocalVer == 0
maxLocalVer := minLocalVer
var err error
sorter := NewIndexSorter()
@ -1340,7 +1398,7 @@ func (m *Model) requestGlobal(deviceID protocol.DeviceID, folder, name string, o
return nil, fmt.Errorf("requestGlobal: no such device: %s", deviceID)
}
l.Debugf("%v REQ(out): %s: %q / %q o=%d s=%d h=%x ft=%t op=%s", m, deviceID, folder, name, offset, size, hash, fromTemporary)
l.Debugf("%v REQ(out): %s: %q / %q o=%d s=%d h=%x ft=%t", m, deviceID, folder, name, offset, size, hash, fromTemporary)
return nc.Request(folder, name, offset, size, hash, fromTemporary)
}
@ -1660,6 +1718,8 @@ func (m *Model) generateClusterConfig(device protocol.DeviceID) protocol.Cluster
m.fmut.RLock()
for _, folder := range m.deviceFolders[device] {
folderCfg := m.cfg.Folders()[folder]
fs := m.folderFiles[folder]
protocolFolder := protocol.Folder{
ID: folder,
Label: folderCfg.Label,
@ -1676,13 +1736,26 @@ func (m *Model) generateClusterConfig(device protocol.DeviceID) protocol.Cluster
// TODO: Set read only bit when relevant, and when we have per device
// access controls.
deviceCfg := m.cfg.Devices()[device]
var indexID protocol.IndexID
var maxLocalVersion int64
if device == m.id {
indexID = fs.IndexID(protocol.LocalDeviceID)
maxLocalVersion = fs.LocalVersion(protocol.LocalDeviceID)
} else {
indexID = fs.IndexID(device)
maxLocalVersion = fs.LocalVersion(device)
}
protocolDevice := protocol.Device{
ID: device[:],
Name: deviceCfg.Name,
Addresses: deviceCfg.Addresses,
Compression: deviceCfg.Compression,
CertName: deviceCfg.CertName,
Introducer: deviceCfg.Introducer,
ID: device[:],
Name: deviceCfg.Name,
Addresses: deviceCfg.Addresses,
Compression: deviceCfg.Compression,
CertName: deviceCfg.CertName,
Introducer: deviceCfg.Introducer,
IndexID: indexID,
MaxLocalVersion: maxLocalVersion,
}
protocolFolder.Devices = append(protocolFolder.Devices, protocolDevice)

View File

@ -433,12 +433,21 @@ func (f *rwFolder) pullerIteration(ignores *ignore.Matcher) int {
l.Debugln(f, "handling", file.Name)
if !handleFile(file) {
// A new or changed file or symlink. This is the only case where we
// do stuff concurrently in the background
f.queue.Push(file.Name, file.Size, file.Modified)
// A new or changed file or symlink. This is the only case where
// we do stuff concurrently in the background. We only queue
// files where we are connected to at least one device that has
// the file.
devices := folderFiles.Availability(file.Name)
for _, dev := range devices {
if f.model.ConnectedTo(dev) {
f.queue.Push(file.Name, file.Size, file.Modified)
changed++
break
}
}
}
changed++
return true
})

View File

@ -262,6 +262,7 @@ type Device struct {
CertName string `protobuf:"bytes,5,opt,name=cert_name,json=certName,proto3" json:"cert_name,omitempty"`
MaxLocalVersion int64 `protobuf:"varint,6,opt,name=max_local_version,json=maxLocalVersion,proto3" json:"max_local_version,omitempty"`
Introducer bool `protobuf:"varint,7,opt,name=introducer,proto3" json:"introducer,omitempty"`
IndexID IndexID `protobuf:"varint,8,opt,name=index_id,json=indexId,proto3,customtype=IndexID" json:"index_id"`
}
func (m *Device) Reset() { *m = Device{} }
@ -672,6 +673,11 @@ func (m *Device) MarshalTo(data []byte) (int, error) {
}
i++
}
if m.IndexID != 0 {
data[i] = 0x40
i++
i = encodeVarintBep(data, i, uint64(m.IndexID))
}
return i, nil
}
@ -1288,6 +1294,9 @@ func (m *Device) ProtoSize() (n int) {
if m.Introducer {
n += 2
}
if m.IndexID != 0 {
n += 1 + sovBep(uint64(m.IndexID))
}
return n
}
@ -2246,6 +2255,25 @@ func (m *Device) Unmarshal(data []byte) error {
}
}
m.Introducer = bool(v != 0)
case 8:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field IndexID", wireType)
}
m.IndexID = 0
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowBep
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := data[iNdEx]
iNdEx++
m.IndexID |= (IndexID(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
default:
iNdEx = preIndex
skippy, err := skipBep(data[iNdEx:])
@ -3898,102 +3926,104 @@ var (
)
var fileDescriptorBep = []byte{
// 1543 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xac, 0x56, 0x41, 0x6f, 0x1a, 0xc7,
0x17, 0x37, 0xb0, 0x2c, 0x30, 0xc6, 0x0e, 0x9e, 0x38, 0x0e, 0xff, 0x8d, 0xff, 0xb6, 0xbb, 0x49,
0x54, 0x17, 0x35, 0x4e, 0x9b, 0x54, 0x8d, 0x54, 0xa9, 0x95, 0x30, 0xac, 0x1d, 0x14, 0xbc, 0x90,
0x05, 0x9c, 0xa6, 0x87, 0xa2, 0x85, 0x1d, 0xf0, 0x2a, 0xcb, 0x0e, 0xdd, 0x5d, 0x92, 0xb8, 0x5f,
0xa0, 0x52, 0xfb, 0x05, 0x7a, 0xa9, 0x94, 0x6b, 0xef, 0xfd, 0x10, 0x39, 0x46, 0x39, 0xf6, 0x10,
0xb5, 0xe9, 0xa5, 0x1f, 0xa0, 0xbd, 0xf7, 0xcd, 0xcc, 0x2e, 0x2c, 0xc6, 0xae, 0x72, 0xe8, 0x01,
0x31, 0xf3, 0xde, 0x6f, 0xde, 0xcc, 0xfc, 0xde, 0xef, 0xbd, 0x59, 0x94, 0xeb, 0x91, 0xf1, 0xde,
0xd8, 0xa3, 0x01, 0xc5, 0x59, 0xfe, 0xd7, 0xa7, 0x8e, 0x72, 0x6b, 0x68, 0x07, 0x27, 0x93, 0xde,
0x5e, 0x9f, 0x8e, 0x6e, 0x0f, 0xe9, 0x90, 0xde, 0xe6, 0x9e, 0xde, 0x64, 0xc0, 0x67, 0x7c, 0xc2,
0x47, 0x62, 0xa1, 0x3a, 0x46, 0xe9, 0xfb, 0xc4, 0x71, 0x28, 0xde, 0x46, 0xcb, 0x16, 0x79, 0x6a,
0xf7, 0x49, 0xd7, 0x35, 0x47, 0xa4, 0x98, 0xd8, 0x49, 0xec, 0xe6, 0x0c, 0x24, 0x4c, 0x3a, 0x58,
0x18, 0xa0, 0xef, 0xd8, 0xc4, 0x0d, 0x04, 0x20, 0x29, 0x00, 0xc2, 0xc4, 0x01, 0x37, 0xd1, 0x6a,
0x08, 0x78, 0x4a, 0x3c, 0xdf, 0xa6, 0x6e, 0x31, 0xc5, 0x31, 0x2b, 0xc2, 0x7a, 0x2c, 0x8c, 0xaa,
0x8f, 0xe4, 0xfb, 0xc4, 0xb4, 0x88, 0x87, 0x3f, 0x40, 0x52, 0x70, 0x3a, 0x16, 0x7b, 0xad, 0xde,
0xb9, 0xb2, 0x17, 0xdd, 0x61, 0xef, 0x88, 0xf8, 0xbe, 0x39, 0x24, 0x6d, 0x70, 0x1a, 0x1c, 0x82,
0xbf, 0x80, 0xcd, 0xe9, 0x68, 0xec, 0x81, 0x83, 0x05, 0x4e, 0xf2, 0x15, 0x9b, 0x0b, 0x2b, 0x2a,
0x33, 0x8c, 0x11, 0x5f, 0xa0, 0x96, 0xd1, 0x4a, 0xc5, 0x99, 0xf8, 0x01, 0xf1, 0x2a, 0xd4, 0x1d,
0xd8, 0x43, 0xfc, 0x11, 0xca, 0x0c, 0xa8, 0x03, 0xa7, 0xf0, 0x61, 0xfb, 0xd4, 0xee, 0xf2, 0x9d,
0xc2, 0x2c, 0xd8, 0x01, 0x77, 0xec, 0x4b, 0x2f, 0xdf, 0x6c, 0x2f, 0x19, 0x11, 0x4c, 0xfd, 0x21,
0x89, 0x64, 0xe1, 0xc1, 0x1b, 0x28, 0x69, 0x5b, 0x82, 0xa2, 0x7d, 0xf9, 0xed, 0x9b, 0xed, 0x64,
0xad, 0x6a, 0x80, 0x05, 0xaf, 0xa3, 0xb4, 0x63, 0xf6, 0x88, 0x13, 0x92, 0x23, 0x26, 0xf8, 0x1a,
0xca, 0x79, 0x70, 0xe1, 0x2e, 0x75, 0x9d, 0x53, 0x4e, 0x49, 0xd6, 0xc8, 0x32, 0x43, 0x03, 0xe6,
0xf8, 0x16, 0xc2, 0xf6, 0xd0, 0xa5, 0x1e, 0xe9, 0x8e, 0x89, 0x37, 0xb2, 0xf9, 0x69, 0xfd, 0xa2,
0xc4, 0x51, 0x6b, 0xc2, 0xd3, 0x9c, 0x39, 0xf0, 0x75, 0xb4, 0x12, 0xc2, 0x2d, 0xe2, 0x90, 0x80,
0x14, 0xd3, 0x1c, 0x99, 0x17, 0xc6, 0x2a, 0xb7, 0xc1, 0xdd, 0xd6, 0x2d, 0xdb, 0x37, 0x7b, 0x0e,
0xe9, 0x06, 0x64, 0x34, 0xee, 0xda, 0xae, 0x45, 0x9e, 0x13, 0xbf, 0x28, 0x73, 0x2c, 0x0e, 0x7d,
0x6d, 0x70, 0xd5, 0x84, 0x87, 0xb1, 0x21, 0x32, 0xed, 0x17, 0x0b, 0x67, 0xd9, 0xa8, 0x72, 0x47,
0xc4, 0x46, 0x08, 0x53, 0xff, 0x4a, 0x20, 0x59, 0x78, 0x62, 0x6c, 0xe4, 0xe7, 0xd8, 0xc0, 0x48,
0x8a, 0x29, 0x85, 0x8f, 0xf1, 0x26, 0xca, 0x99, 0x96, 0xc5, 0xb2, 0x02, 0x5b, 0xa5, 0x60, 0xab,
0x9c, 0x31, 0x33, 0xe0, 0x7b, 0xf3, 0x59, 0x96, 0xce, 0xea, 0xe2, 0xa2, 0xf4, 0x32, 0x8a, 0xfb,
0xc4, 0x0b, 0x95, 0x99, 0xe6, 0xfb, 0x65, 0x99, 0x81, 0xeb, 0xb2, 0x84, 0xd6, 0x46, 0xe6, 0xf3,
0xae, 0x43, 0xfb, 0xa6, 0x33, 0x95, 0x26, 0xe3, 0x22, 0x65, 0x5c, 0x02, 0x47, 0x9d, 0xd9, 0x43,
0x71, 0xe2, 0x2d, 0x84, 0x6c, 0x37, 0xf0, 0xa8, 0x35, 0x81, 0xe5, 0xc5, 0x0c, 0x27, 0x2c, 0x66,
0x51, 0x1b, 0x28, 0xcd, 0x39, 0x83, 0x4b, 0xcb, 0x42, 0x18, 0x61, 0xa5, 0x84, 0x33, 0xbc, 0x87,
0xd2, 0x03, 0xdb, 0x81, 0xcb, 0x25, 0x39, 0x8f, 0x38, 0xa6, 0x2a, 0x30, 0xd7, 0xdc, 0x01, 0x0d,
0x99, 0x14, 0x30, 0xb5, 0x83, 0x96, 0x79, 0xc0, 0xce, 0xd8, 0x32, 0x03, 0xf2, 0x9f, 0x85, 0xfd,
0x2e, 0x85, 0xb2, 0x91, 0x67, 0x9a, 0x88, 0x44, 0x2c, 0x11, 0xa5, 0xb0, 0xf6, 0x44, 0x25, 0x6d,
0x2c, 0xc6, 0x8b, 0x15, 0x1f, 0xac, 0xf7, 0xed, 0x6f, 0x09, 0xd7, 0x6e, 0xca, 0xe0, 0x63, 0xbc,
0x83, 0x96, 0xcf, 0x0a, 0x76, 0xc5, 0x88, 0x9b, 0xb0, 0x82, 0xb2, 0x23, 0x6a, 0xd9, 0x03, 0x9b,
0x58, 0x3c, 0x25, 0x29, 0x63, 0x3a, 0xc7, 0x45, 0xa6, 0x37, 0xa6, 0x55, 0x2b, 0x14, 0x65, 0x34,
0x65, 0x1e, 0xdb, 0x7d, 0x6a, 0x3a, 0xa0, 0x28, 0xc1, 0x7e, 0x34, 0x65, 0xed, 0xc5, 0xa5, 0x73,
0x55, 0x92, 0xe5, 0x80, 0x15, 0x97, 0xc6, 0x2b, 0x04, 0xa4, 0x1c, 0xe5, 0x38, 0x07, 0xfe, 0x39,
0x29, 0x1f, 0x93, 0x7e, 0x40, 0xa7, 0x85, 0x1d, 0xc2, 0x58, 0x4d, 0xcd, 0x6b, 0x03, 0xf1, 0xd3,
0xe6, 0x9d, 0xb8, 0x30, 0x3e, 0x46, 0xf2, 0x3e, 0x18, 0x9e, 0x44, 0x05, 0x72, 0x79, 0x16, 0x95,
0xdb, 0x63, 0x29, 0x90, 0x7b, 0x1c, 0xf8, 0x99, 0xf4, 0xe3, 0x8b, 0xed, 0x25, 0xf5, 0x21, 0xca,
0x4d, 0x01, 0x2c, 0xbd, 0x74, 0x30, 0xf0, 0x49, 0xc0, 0x73, 0x91, 0x32, 0xc2, 0xd9, 0x94, 0x61,
0x96, 0x8d, 0x74, 0xc8, 0x30, 0xd8, 0x4e, 0x4c, 0xff, 0x84, 0xb3, 0x9e, 0x37, 0xf8, 0x38, 0x0c,
0xf9, 0x39, 0x92, 0xc5, 0x4d, 0xf0, 0x5d, 0x94, 0xed, 0xd3, 0x89, 0x1b, 0xcc, 0xda, 0xd8, 0x5a,
0xbc, 0x5a, 0xb8, 0x27, 0x3c, 0xd5, 0x14, 0xa8, 0x1e, 0xa0, 0x4c, 0xe8, 0x02, 0x4e, 0xa3, 0xd2,
0x95, 0xf6, 0xaf, 0x30, 0xd8, 0xaf, 0x6f, 0xb6, 0x33, 0xad, 0x13, 0xea, 0x05, 0xb5, 0xea, 0x7c,
0x5f, 0x83, 0x1c, 0x4c, 0xc4, 0xf9, 0x24, 0x43, 0x4c, 0xd4, 0x5f, 0x12, 0x28, 0x63, 0x90, 0x6f,
0x26, 0xc4, 0x0f, 0x62, 0x3d, 0x20, 0x3d, 0xd7, 0x03, 0x66, 0x7a, 0x4e, 0xce, 0xe9, 0x39, 0x92,
0x64, 0x2a, 0x26, 0xc9, 0x19, 0x39, 0xd2, 0xb9, 0xe4, 0xa4, 0xcf, 0x21, 0x47, 0x9e, 0x91, 0xc3,
0x04, 0x32, 0xf0, 0xe8, 0x88, 0xf7, 0x3c, 0xea, 0x99, 0xde, 0x69, 0xa8, 0xa0, 0x15, 0x66, 0x6d,
0x47, 0x46, 0xb5, 0x8b, 0xb2, 0x06, 0xf1, 0xc7, 0xa0, 0x15, 0x72, 0xe1, 0xb1, 0x21, 0x3c, 0x94,
0xa3, 0xc9, 0x0f, 0x0d, 0xe1, 0xd9, 0x18, 0xbf, 0x8f, 0xa4, 0x3e, 0xb5, 0xc4, 0x91, 0x57, 0xe3,
0xf9, 0xd7, 0x3c, 0x8f, 0xc2, 0xb3, 0x62, 0x41, 0xb9, 0x30, 0x00, 0x3c, 0xa9, 0x85, 0x2a, 0x7d,
0xe6, 0x3a, 0xd4, 0xb4, 0x9a, 0x1e, 0x1d, 0xb2, 0x1e, 0x75, 0x61, 0x5d, 0x57, 0x51, 0x66, 0xc2,
0x2b, 0x3f, 0xaa, 0xec, 0x1b, 0xf3, 0x95, 0x78, 0x36, 0x90, 0x68, 0x13, 0x91, 0x82, 0xc3, 0xa5,
0xea, 0xeb, 0x04, 0x52, 0x2e, 0x46, 0xe3, 0x1a, 0x5a, 0x16, 0xc8, 0x6e, 0xec, 0xb9, 0xdd, 0x7d,
0x97, 0x8d, 0x78, 0x13, 0x40, 0x93, 0xe9, 0xf8, 0xdc, 0x9e, 0x1e, 0xab, 0xb8, 0xd4, 0x3b, 0x57,
0x1c, 0xaf, 0x91, 0xe9, 0xcb, 0x24, 0xc1, 0xdd, 0xd3, 0x46, 0xbe, 0x27, 0x0a, 0x85, 0xdb, 0x54,
0x19, 0x49, 0x4d, 0xdb, 0x1d, 0xaa, 0xdb, 0x28, 0x5d, 0x71, 0x28, 0x4f, 0x96, 0x0c, 0xcf, 0xa6,
0x0f, 0xdb, 0x84, 0x1c, 0x8a, 0x59, 0xe9, 0x75, 0x12, 0x2d, 0xc7, 0xbe, 0x18, 0xe0, 0x3c, 0xab,
0x95, 0x7a, 0xa7, 0xd5, 0xd6, 0x8c, 0x6e, 0xa5, 0xa1, 0x1f, 0xd4, 0x0e, 0x0b, 0x4b, 0xca, 0xe6,
0xf7, 0x3f, 0xed, 0x14, 0x47, 0x33, 0xd0, 0xfc, 0xc7, 0x00, 0x6c, 0x51, 0xd3, 0xab, 0xda, 0x97,
0x85, 0x84, 0xb2, 0x0e, 0xc0, 0x42, 0x0c, 0x28, 0xba, 0xfd, 0x87, 0x28, 0xcf, 0x01, 0xdd, 0x4e,
0xb3, 0x5a, 0x6e, 0x6b, 0x85, 0xa4, 0xa2, 0x00, 0x6e, 0xe3, 0x2c, 0x2e, 0xe4, 0xfb, 0x3a, 0xd4,
0x85, 0xf6, 0xb0, 0xa3, 0xb5, 0xda, 0x85, 0x94, 0xb2, 0x01, 0x40, 0x1c, 0x03, 0x46, 0x15, 0x73,
0x13, 0x64, 0xa8, 0xb5, 0x9a, 0x0d, 0xbd, 0xa5, 0x15, 0x24, 0xe5, 0x2a, 0xa0, 0x2e, 0xcf, 0xa1,
0x42, 0x85, 0x7e, 0x8a, 0xd6, 0xaa, 0x8d, 0x47, 0x7a, 0xbd, 0x51, 0xae, 0x76, 0x9b, 0x46, 0xe3,
0x10, 0xd6, 0xb4, 0x0a, 0x69, 0x65, 0x1b, 0xf0, 0xd7, 0x62, 0xf8, 0x05, 0xc1, 0xfd, 0x1f, 0xd8,
0xab, 0xe9, 0x87, 0x05, 0x59, 0xb9, 0x0c, 0xd0, 0x4b, 0x31, 0x28, 0x23, 0x95, 0xdd, 0xb8, 0x52,
0x6f, 0xc0, 0xd6, 0x99, 0x85, 0x1b, 0x73, 0xb2, 0x4b, 0x5f, 0x23, 0xbc, 0xf8, 0x4d, 0x85, 0x6f,
0x20, 0x49, 0x6f, 0xe8, 0x1a, 0x10, 0xca, 0xef, 0xbf, 0x88, 0xd0, 0xa9, 0x4b, 0xb0, 0x8a, 0x52,
0xf5, 0xaf, 0x3e, 0x01, 0x32, 0xff, 0x07, 0xa0, 0x2b, 0x8b, 0x20, 0x70, 0x96, 0x28, 0x5a, 0x8e,
0x07, 0x56, 0x51, 0xf6, 0x48, 0x6b, 0x97, 0x81, 0xdc, 0x32, 0x04, 0xe7, 0x47, 0x8a, 0xdc, 0x47,
0x24, 0x30, 0x79, 0x01, 0x6e, 0xa2, 0xb4, 0xae, 0x1d, 0x6b, 0x06, 0x04, 0x5e, 0x03, 0xc0, 0x4a,
0x04, 0xd0, 0x09, 0xe8, 0x0a, 0x5e, 0x6e, 0xb9, 0x5c, 0x7f, 0x54, 0x7e, 0xdc, 0x82, 0xe4, 0x60,
0x70, 0xaf, 0x46, 0xee, 0xb2, 0xf3, 0xcc, 0x3c, 0xf5, 0x4b, 0x7f, 0x27, 0x50, 0x3e, 0xfe, 0xb6,
0xc1, 0x02, 0xe9, 0xa0, 0x56, 0xd7, 0xa2, 0xed, 0xe2, 0x3e, 0x36, 0xc6, 0xbb, 0x28, 0x57, 0xad,
0x19, 0x5a, 0xa5, 0xdd, 0x30, 0x1e, 0x47, 0x77, 0x89, 0x83, 0xaa, 0xb6, 0xc7, 0xc5, 0xcd, 0xbe,
0xe1, 0xf2, 0xad, 0xc7, 0x47, 0xf5, 0x9a, 0xfe, 0xa0, 0xcb, 0x23, 0x26, 0x95, 0x6b, 0x00, 0xbe,
0x1a, 0x07, 0xb7, 0x4e, 0x47, 0x8e, 0xed, 0x3e, 0xe1, 0x81, 0xef, 0xa1, 0xb5, 0x08, 0x3e, 0xdb,
0x20, 0xa5, 0xec, 0xc0, 0x9a, 0xcd, 0x73, 0xd6, 0xcc, 0xf6, 0xb9, 0x8b, 0x2e, 0x45, 0x0b, 0x3b,
0xfa, 0x03, 0x1d, 0x64, 0x01, 0xca, 0xd9, 0x82, 0x65, 0xca, 0x39, 0xcb, 0x3a, 0xee, 0x13, 0x17,
0x44, 0x51, 0xfa, 0x39, 0x81, 0x72, 0xd3, 0x0e, 0xc5, 0x78, 0xd6, 0x1b, 0x5d, 0xcd, 0x30, 0x1a,
0x46, 0x74, 0xf1, 0xa9, 0x53, 0xa7, 0x7c, 0x88, 0xdf, 0x43, 0x99, 0x43, 0x4d, 0xd7, 0x8c, 0x5a,
0x25, 0xaa, 0x87, 0x29, 0xe4, 0x90, 0xb8, 0xc4, 0xb3, 0xfb, 0xf0, 0xe5, 0x9e, 0x87, 0x30, 0xad,
0x4e, 0xe5, 0x7e, 0x74, 0x63, 0x2e, 0xe0, 0x58, 0xa8, 0xd6, 0xa4, 0x7f, 0xc2, 0x6f, 0x5b, 0x62,
0xa5, 0x73, 0x5c, 0xae, 0xd7, 0xaa, 0x02, 0x9a, 0x52, 0x8a, 0x00, 0x5d, 0x9f, 0x42, 0x6b, 0xe2,
0x79, 0x67, 0xd8, 0x92, 0x85, 0xb6, 0xfe, 0xbd, 0x17, 0xc1, 0x67, 0x87, 0x5c, 0x6e, 0x36, 0x35,
0xbd, 0x1a, 0x9d, 0x7e, 0xe6, 0x2b, 0x8f, 0xc7, 0xc4, 0xb5, 0x18, 0xe2, 0xa0, 0x61, 0x1c, 0x6a,
0xed, 0xe8, 0xf0, 0x33, 0xc4, 0x01, 0xf5, 0x86, 0x24, 0xd8, 0xdf, 0x7c, 0xf9, 0xfb, 0xd6, 0xd2,
0x2b, 0xf8, 0xbd, 0x7c, 0xbb, 0x95, 0x78, 0x05, 0xbf, 0xdf, 0xde, 0x6e, 0x2d, 0xfd, 0x09, 0xff,
0x2f, 0xfe, 0xd8, 0x4a, 0xf4, 0x64, 0xde, 0xbb, 0xee, 0xfe, 0x13, 0x00, 0x00, 0xff, 0xff, 0x88,
0x06, 0x3b, 0x0f, 0x5d, 0x0d, 0x00, 0x00,
// 1569 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xac, 0x56, 0x4f, 0x6f, 0x1b, 0x45,
0x14, 0x8f, 0xed, 0xf5, 0xbf, 0x89, 0x93, 0x3a, 0xd3, 0x34, 0x35, 0xdb, 0x90, 0x84, 0x6d, 0x2b,
0x82, 0x45, 0x53, 0x68, 0x81, 0x4a, 0x48, 0x20, 0x39, 0xf6, 0x26, 0xb5, 0xea, 0xac, 0xdd, 0xb5,
0x9d, 0x52, 0x0e, 0x58, 0x6b, 0xef, 0xd8, 0x59, 0x75, 0xbd, 0x63, 0x76, 0xd7, 0x6d, 0xc3, 0x17,
0x40, 0x82, 0x2f, 0xc0, 0x05, 0xa9, 0x57, 0xc4, 0x95, 0x0f, 0xd1, 0x63, 0xd5, 0x23, 0x87, 0x0a,
0xc2, 0x85, 0x2f, 0xc0, 0x9d, 0x37, 0x33, 0xbb, 0xde, 0x75, 0xfe, 0xa0, 0x1e, 0x38, 0x58, 0x9e,
0x79, 0xef, 0x37, 0x6f, 0x66, 0x7e, 0xef, 0xf7, 0xde, 0x2c, 0xca, 0xf7, 0xc9, 0x64, 0x67, 0xe2,
0x52, 0x9f, 0xe2, 0x1c, 0xff, 0x1b, 0x50, 0x5b, 0xbe, 0x35, 0xb2, 0xfc, 0xa3, 0x69, 0x7f, 0x67,
0x40, 0xc7, 0xb7, 0x47, 0x74, 0x44, 0x6f, 0x73, 0x4f, 0x7f, 0x3a, 0xe4, 0x33, 0x3e, 0xe1, 0x23,
0xb1, 0x50, 0x99, 0xa0, 0xf4, 0x7d, 0x62, 0xdb, 0x14, 0x6f, 0xa2, 0x45, 0x93, 0x3c, 0xb5, 0x06,
0xa4, 0xe7, 0x18, 0x63, 0x52, 0x4a, 0x6c, 0x25, 0xb6, 0xf3, 0x3a, 0x12, 0x26, 0x0d, 0x2c, 0x0c,
0x30, 0xb0, 0x2d, 0xe2, 0xf8, 0x02, 0x90, 0x14, 0x00, 0x61, 0xe2, 0x80, 0x9b, 0x68, 0x39, 0x00,
0x3c, 0x25, 0xae, 0x67, 0x51, 0xa7, 0x94, 0xe2, 0x98, 0x25, 0x61, 0x3d, 0x14, 0x46, 0xc5, 0x43,
0x99, 0xfb, 0xc4, 0x30, 0x89, 0x8b, 0x3f, 0x40, 0x92, 0x7f, 0x3c, 0x11, 0x7b, 0x2d, 0xdf, 0xb9,
0xb2, 0x13, 0xde, 0x61, 0xe7, 0x80, 0x78, 0x9e, 0x31, 0x22, 0x1d, 0x70, 0xea, 0x1c, 0x82, 0xbf,
0x84, 0xcd, 0xe9, 0x78, 0xe2, 0x82, 0x83, 0x05, 0x4e, 0xf2, 0x15, 0xeb, 0x67, 0x56, 0x54, 0x23,
0x8c, 0x1e, 0x5f, 0xa0, 0x54, 0xd0, 0x52, 0xd5, 0x9e, 0x7a, 0x3e, 0x71, 0xab, 0xd4, 0x19, 0x5a,
0x23, 0xfc, 0x11, 0xca, 0x0e, 0xa9, 0x0d, 0xa7, 0xf0, 0x60, 0xfb, 0xd4, 0xf6, 0xe2, 0x9d, 0x62,
0x14, 0x6c, 0x8f, 0x3b, 0x76, 0xa5, 0x97, 0x6f, 0x36, 0x17, 0xf4, 0x10, 0xa6, 0xfc, 0x98, 0x44,
0x19, 0xe1, 0xc1, 0x6b, 0x28, 0x69, 0x99, 0x82, 0xa2, 0xdd, 0xcc, 0xc9, 0x9b, 0xcd, 0x64, 0xbd,
0xa6, 0x83, 0x05, 0xaf, 0xa2, 0xb4, 0x6d, 0xf4, 0x89, 0x1d, 0x90, 0x23, 0x26, 0xf8, 0x1a, 0xca,
0xbb, 0x70, 0xe1, 0x1e, 0x75, 0xec, 0x63, 0x4e, 0x49, 0x4e, 0xcf, 0x31, 0x43, 0x13, 0xe6, 0xf8,
0x16, 0xc2, 0xd6, 0xc8, 0xa1, 0x2e, 0xe9, 0x4d, 0x88, 0x3b, 0xb6, 0xf8, 0x69, 0xbd, 0x92, 0xc4,
0x51, 0x2b, 0xc2, 0xd3, 0x8a, 0x1c, 0xf8, 0x3a, 0x5a, 0x0a, 0xe0, 0x26, 0xb1, 0x89, 0x4f, 0x4a,
0x69, 0x8e, 0x2c, 0x08, 0x63, 0x8d, 0xdb, 0xe0, 0x6e, 0xab, 0xa6, 0xe5, 0x19, 0x7d, 0x9b, 0xf4,
0x7c, 0x32, 0x9e, 0xf4, 0x2c, 0xc7, 0x24, 0xcf, 0x89, 0x57, 0xca, 0x70, 0x2c, 0x0e, 0x7c, 0x1d,
0x70, 0xd5, 0x85, 0x87, 0xb1, 0x21, 0x32, 0xed, 0x95, 0x8a, 0xa7, 0xd9, 0xa8, 0x71, 0x47, 0xc8,
0x46, 0x00, 0x53, 0x7e, 0x05, 0x36, 0x84, 0x27, 0xc6, 0x46, 0x61, 0x8e, 0x0d, 0x8c, 0xa4, 0x98,
0x52, 0xf8, 0x18, 0xaf, 0xa3, 0xbc, 0x61, 0x9a, 0x2c, 0x2b, 0xb0, 0x55, 0x0a, 0xb6, 0xca, 0xeb,
0x91, 0x01, 0xdf, 0x9b, 0xcf, 0xb2, 0x74, 0x5a, 0x17, 0x17, 0xa5, 0x97, 0x51, 0x3c, 0x20, 0x6e,
0xa0, 0xcc, 0x34, 0xdf, 0x2f, 0xc7, 0x0c, 0x5c, 0x97, 0x65, 0xb4, 0x32, 0x36, 0x9e, 0xf7, 0x6c,
0x3a, 0x30, 0xec, 0x99, 0x34, 0x19, 0x17, 0x29, 0xfd, 0x12, 0x38, 0x1a, 0xcc, 0x1e, 0x88, 0x13,
0x6f, 0x20, 0x64, 0x39, 0xbe, 0x4b, 0xcd, 0x29, 0x2c, 0x2f, 0x65, 0x39, 0x61, 0x31, 0x0b, 0xfe,
0x14, 0xe5, 0x38, 0x9b, 0x3d, 0xb8, 0x71, 0x0e, 0xbc, 0xd2, 0xae, 0xcc, 0x78, 0xf9, 0xfd, 0xcd,
0x66, 0x96, 0x73, 0x59, 0xaf, 0x9d, 0x44, 0x43, 0x3d, 0xcb, 0xb1, 0x75, 0x53, 0x69, 0xa2, 0x34,
0xb7, 0x01, 0x57, 0x19, 0xa1, 0xa7, 0xa0, 0xc0, 0x82, 0x19, 0xde, 0x41, 0xe9, 0xa1, 0x65, 0x03,
0x27, 0x49, 0x4e, 0x3f, 0x8e, 0x89, 0x11, 0xcc, 0x75, 0x67, 0x48, 0x83, 0x04, 0x08, 0x98, 0xd2,
0x45, 0x8b, 0x3c, 0x60, 0x77, 0x62, 0x1a, 0x3e, 0xf9, 0xdf, 0xc2, 0x7e, 0x9f, 0x42, 0xb9, 0xd0,
0x33, 0xcb, 0x5f, 0x22, 0x96, 0xbf, 0x72, 0x50, 0xb2, 0xa2, 0x00, 0xd7, 0xce, 0xc6, 0x8b, 0xd5,
0x2c, 0xac, 0xf7, 0xac, 0xef, 0x08, 0x97, 0x7c, 0x4a, 0xe7, 0x63, 0xbc, 0x85, 0x16, 0x4f, 0xeb,
0x7c, 0x49, 0x8f, 0x9b, 0xb0, 0x8c, 0x72, 0x63, 0x6a, 0x5a, 0x43, 0x8b, 0x98, 0x3c, 0x93, 0x29,
0x7d, 0x36, 0xc7, 0x25, 0x26, 0x53, 0x26, 0x71, 0x33, 0xd0, 0x72, 0x38, 0x65, 0x1e, 0xcb, 0x79,
0x6a, 0xd8, 0x90, 0x16, 0x91, 0xb4, 0x70, 0xca, 0xba, 0x92, 0x43, 0xe7, 0x8a, 0x2b, 0xc7, 0x01,
0x4b, 0x0e, 0x8d, 0x17, 0x16, 0x54, 0x40, 0x28, 0x8d, 0x3c, 0xf8, 0xe7, 0x2a, 0xe0, 0x90, 0x0c,
0x7c, 0x3a, 0xeb, 0x07, 0x01, 0x8c, 0x95, 0xe2, 0xbc, 0xa4, 0x10, 0x3f, 0x6d, 0xc1, 0x8e, 0xeb,
0xe9, 0x63, 0x94, 0xd9, 0x05, 0xc3, 0x93, 0xb0, 0xae, 0x2e, 0x47, 0x51, 0xb9, 0x3d, 0x96, 0x82,
0x4c, 0x9f, 0x03, 0x3f, 0x97, 0x7e, 0x7a, 0xb1, 0xb9, 0xa0, 0x3c, 0x44, 0xf9, 0x19, 0x80, 0xa5,
0x97, 0x0e, 0x87, 0x1e, 0xf1, 0x79, 0x2e, 0x52, 0x7a, 0x30, 0x9b, 0x31, 0xcc, 0xb2, 0x91, 0x0e,
0x18, 0x06, 0xdb, 0x91, 0xe1, 0x1d, 0x71, 0xd6, 0x0b, 0x3a, 0x1f, 0x07, 0x21, 0xbf, 0x40, 0x19,
0x71, 0x13, 0x7c, 0x17, 0xe5, 0x06, 0x74, 0xea, 0xf8, 0x51, 0xf7, 0x5b, 0x89, 0x17, 0x19, 0xf7,
0x04, 0xa7, 0x9a, 0x01, 0x95, 0x3d, 0x94, 0x0d, 0x5c, 0xc0, 0x69, 0x58, 0xf1, 0xd2, 0xee, 0x95,
0x50, 0xff, 0xed, 0x23, 0xea, 0xfa, 0x5c, 0xff, 0xb1, 0x76, 0x08, 0x39, 0x98, 0x8a, 0xf3, 0x49,
0xba, 0x98, 0x28, 0xbf, 0x25, 0x50, 0x56, 0x27, 0xdf, 0x4e, 0x89, 0xe7, 0xc7, 0x5a, 0x47, 0x7a,
0xae, 0x75, 0x44, 0x7a, 0x4e, 0xce, 0xe9, 0x39, 0x94, 0x64, 0x2a, 0x26, 0xc9, 0x88, 0x1c, 0xe9,
0x5c, 0x72, 0xd2, 0xe7, 0x90, 0x93, 0x89, 0xc8, 0x61, 0x02, 0x19, 0xba, 0x74, 0xcc, 0x5b, 0x25,
0x75, 0x0d, 0xf7, 0x38, 0x50, 0xd0, 0x12, 0xb3, 0x76, 0x42, 0xa3, 0xd2, 0x43, 0x39, 0x9d, 0x78,
0x13, 0xd0, 0x0a, 0xb9, 0xf0, 0xd8, 0x10, 0x1e, 0xca, 0xd1, 0xe0, 0x87, 0x86, 0xf0, 0x6c, 0x8c,
0xdf, 0x47, 0xd2, 0x80, 0x9a, 0xe2, 0xc8, 0xcb, 0xf1, 0xfc, 0xab, 0xae, 0x4b, 0xe1, 0x35, 0x32,
0xa1, 0x5c, 0x18, 0x00, 0x5e, 0xe2, 0x62, 0x8d, 0x3e, 0x73, 0x6c, 0x6a, 0x98, 0x2d, 0x97, 0x8e,
0x58, 0x6b, 0xbb, 0xb0, 0xae, 0x6b, 0x28, 0x3b, 0xe5, 0x95, 0x1f, 0x56, 0xf6, 0x8d, 0xf9, 0x4a,
0x3c, 0x1d, 0x48, 0xb4, 0x89, 0x50, 0xc1, 0xc1, 0x52, 0xe5, 0x75, 0x02, 0xc9, 0x17, 0xa3, 0x71,
0x1d, 0x2d, 0x0a, 0x64, 0x2f, 0xf6, 0x4a, 0x6f, 0xbf, 0xcd, 0x46, 0xbc, 0x09, 0xa0, 0xe9, 0x6c,
0x7c, 0xee, 0x53, 0x10, 0xab, 0xb8, 0xd4, 0x5b, 0x57, 0x1c, 0xaf, 0x91, 0xd9, 0x83, 0x26, 0xc1,
0xdd, 0xd3, 0x7a, 0xa1, 0x2f, 0x0a, 0x85, 0xdb, 0x94, 0x0c, 0x92, 0x5a, 0x96, 0x33, 0x52, 0x36,
0x51, 0xba, 0x6a, 0x53, 0x9e, 0xac, 0x0c, 0xbc, 0xb6, 0x1e, 0x6c, 0x13, 0x70, 0x28, 0x66, 0xe5,
0xd7, 0x49, 0xb4, 0x18, 0xfb, 0xd0, 0x80, 0xf3, 0x2c, 0x57, 0x1b, 0xdd, 0x76, 0x47, 0xd5, 0x7b,
0xd5, 0xa6, 0xb6, 0x57, 0xdf, 0x2f, 0x2e, 0xc8, 0xeb, 0x3f, 0xfc, 0xbc, 0x55, 0x1a, 0x47, 0xa0,
0xf9, 0x6f, 0x08, 0xd8, 0xa2, 0xae, 0xd5, 0xd4, 0xaf, 0x8a, 0x09, 0x79, 0x15, 0x80, 0xc5, 0x18,
0x50, 0x74, 0xfb, 0x0f, 0x51, 0x81, 0x03, 0x7a, 0xdd, 0x56, 0xad, 0xd2, 0x51, 0x8b, 0x49, 0x59,
0x06, 0xdc, 0xda, 0x69, 0x5c, 0xc0, 0xf7, 0x75, 0xa8, 0x0b, 0xf5, 0x61, 0x57, 0x6d, 0x77, 0x8a,
0x29, 0x79, 0x0d, 0x80, 0x38, 0x06, 0x0c, 0x2b, 0xe6, 0x26, 0xc8, 0x50, 0x6d, 0xb7, 0x9a, 0x5a,
0x5b, 0x2d, 0x4a, 0xf2, 0x55, 0x40, 0x5d, 0x9e, 0x43, 0x05, 0x0a, 0xfd, 0x0c, 0xad, 0xd4, 0x9a,
0x8f, 0xb4, 0x46, 0xb3, 0x52, 0xeb, 0xb5, 0xf4, 0xe6, 0x3e, 0xac, 0x69, 0x17, 0xd3, 0xf2, 0x26,
0xe0, 0xaf, 0xc5, 0xf0, 0x67, 0x04, 0xf7, 0x2e, 0xb0, 0x57, 0xd7, 0xf6, 0x8b, 0x19, 0xf9, 0x32,
0x40, 0x2f, 0xc5, 0xa0, 0x8c, 0x54, 0x76, 0xe3, 0x6a, 0xa3, 0x09, 0x5b, 0x67, 0xcf, 0xdc, 0x98,
0x93, 0x5d, 0xfe, 0x06, 0xe1, 0xb3, 0x9f, 0x62, 0xf8, 0x06, 0x92, 0xb4, 0xa6, 0xa6, 0x02, 0xa1,
0xfc, 0xfe, 0x67, 0x11, 0x1a, 0x75, 0x08, 0x56, 0x50, 0xaa, 0xf1, 0xf5, 0x27, 0x40, 0xe6, 0x3b,
0x00, 0xba, 0x72, 0x16, 0x04, 0xce, 0x32, 0x45, 0x8b, 0xf1, 0xc0, 0x0a, 0xca, 0x1d, 0xa8, 0x9d,
0x0a, 0x90, 0x5b, 0x81, 0xe0, 0xfc, 0x48, 0xa1, 0xfb, 0x80, 0xf8, 0x06, 0x2f, 0xc0, 0x75, 0x94,
0xd6, 0xd4, 0x43, 0x55, 0x87, 0xc0, 0x2b, 0x00, 0x58, 0x0a, 0x01, 0x1a, 0x01, 0x5d, 0xc1, 0x83,
0x9f, 0xa9, 0x34, 0x1e, 0x55, 0x1e, 0xb7, 0x21, 0x39, 0x18, 0xdc, 0xcb, 0xa1, 0xbb, 0x62, 0x3f,
0x33, 0x8e, 0xbd, 0xf2, 0x3f, 0x09, 0x54, 0x88, 0xbf, 0x6d, 0xb0, 0x40, 0xda, 0xab, 0x37, 0xd4,
0x70, 0xbb, 0xb8, 0x8f, 0x8d, 0xf1, 0x36, 0xca, 0xd7, 0xea, 0xba, 0x5a, 0xed, 0x34, 0xf5, 0xc7,
0xe1, 0x5d, 0xe2, 0xa0, 0x9a, 0xe5, 0x72, 0x71, 0xb3, 0x4f, 0xbf, 0x42, 0xfb, 0xf1, 0x41, 0xa3,
0xae, 0x3d, 0xe8, 0xf1, 0x88, 0x49, 0xf9, 0x1a, 0x80, 0xaf, 0xc6, 0xc1, 0xed, 0xe3, 0xb1, 0x6d,
0x39, 0x4f, 0x78, 0xe0, 0x7b, 0x68, 0x25, 0x84, 0x47, 0x1b, 0xa4, 0xe4, 0x2d, 0x58, 0xb3, 0x7e,
0xce, 0x9a, 0x68, 0x9f, 0xbb, 0xe8, 0x52, 0xb8, 0xb0, 0xab, 0x3d, 0xd0, 0x40, 0x16, 0xa0, 0x9c,
0x0d, 0x58, 0x26, 0x9f, 0xb3, 0xac, 0xeb, 0x3c, 0x71, 0x40, 0x14, 0xe5, 0x5f, 0x12, 0x28, 0x3f,
0xeb, 0x50, 0x8c, 0x67, 0xad, 0xd9, 0x53, 0x75, 0xbd, 0xa9, 0x87, 0x17, 0x9f, 0x39, 0x35, 0xca,
0x87, 0xf8, 0x3d, 0x94, 0xdd, 0x57, 0x35, 0x55, 0xaf, 0x57, 0xc3, 0x7a, 0x98, 0x41, 0xf6, 0x89,
0x43, 0x5c, 0x6b, 0x00, 0x1f, 0xfc, 0x05, 0x08, 0xd3, 0xee, 0x56, 0xef, 0x87, 0x37, 0xe6, 0x02,
0x8e, 0x85, 0x6a, 0x4f, 0x07, 0x47, 0xfc, 0xb6, 0x65, 0x56, 0x3a, 0x87, 0x95, 0x46, 0xbd, 0x26,
0xa0, 0x29, 0xb9, 0x04, 0xd0, 0xd5, 0x19, 0xb4, 0x2e, 0x9e, 0x77, 0x86, 0x2d, 0x9b, 0x68, 0xe3,
0xbf, 0x7b, 0x11, 0x7c, 0x76, 0x64, 0x2a, 0xad, 0x96, 0xaa, 0xd5, 0xc2, 0xd3, 0x47, 0xbe, 0xca,
0x64, 0x42, 0x1c, 0x93, 0x21, 0xf6, 0x9a, 0xfa, 0xbe, 0xda, 0x09, 0x0f, 0x1f, 0x21, 0xf6, 0xa8,
0x3b, 0x22, 0xfe, 0xee, 0xfa, 0xcb, 0x3f, 0x37, 0x16, 0x5e, 0xc1, 0xef, 0xe5, 0xc9, 0x46, 0xe2,
0x15, 0xfc, 0xfe, 0x38, 0xd9, 0x58, 0xf8, 0x1b, 0xfe, 0x5f, 0xfc, 0xb5, 0x91, 0xe8, 0x67, 0x78,
0xef, 0xba, 0xfb, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x5f, 0x7a, 0x46, 0xf6, 0x94, 0x0d, 0x00,
0x00,
}

View File

@ -70,6 +70,7 @@ message Device {
string cert_name = 5;
int64 max_local_version = 6;
bool introducer = 7;
uint64 index_id = 8 [(gogoproto.customname) = "IndexID", (gogoproto.customtype) = "IndexID", (gogoproto.nullable) = false];
}
enum Compression {

View File

@ -8,7 +8,11 @@ package protocol
import (
"bytes"
"crypto/sha256"
"encoding/binary"
"errors"
"fmt"
"github.com/syncthing/syncthing/lib/rand"
)
var (
@ -21,8 +25,8 @@ func (m Hello) Magic() uint32 {
}
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)
return fmt.Sprintf("File{Name:%q, Type:%v, LocalVersion:%d, Permissions:0%o, Modified:%d, Version:%v, Length:%d, Deleted:%v, Invalid:%v, NoPermissions:%v, Blocks:%v}",
f.Name, f.Type, f.LocalVersion, f.Permissions, f.Modified, f.Version, f.Size, f.Deleted, f.Invalid, f.NoPermissions, f.Blocks)
}
func (f FileInfo) IsDeleted() bool {
@ -94,3 +98,27 @@ func (b BlockInfo) String() string {
func (b BlockInfo) IsEmpty() bool {
return b.Size == BlockSize && bytes.Equal(b.Hash, sha256OfEmptyBlock[:])
}
type IndexID uint64
func (i IndexID) String() string {
return fmt.Sprintf("0x%16X", uint64(i))
}
func (i IndexID) Marshal() ([]byte, error) {
bs := make([]byte, 8)
binary.BigEndian.PutUint64(bs, uint64(i))
return bs, nil
}
func (i *IndexID) Unmarshal(bs []byte) error {
if len(bs) != 8 {
return errors.New("incorrect IndexID length")
}
*i = IndexID(binary.BigEndian.Uint64(bs))
return nil
}
func NewIndexID() IndexID {
return IndexID(rand.Int64())
}