mirror of https://github.com/milvus-io/milvus.git
Add MemoryKV, mockkv and tso test
Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>pull/4973/head^2
parent
3ff3a5b659
commit
86d2d36053
1
go.mod
1
go.mod
|
@ -12,6 +12,7 @@ require (
|
|||
github.com/gogo/protobuf v1.3.1
|
||||
github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e // indirect
|
||||
github.com/golang/protobuf v1.3.2
|
||||
github.com/google/btree v1.0.0
|
||||
github.com/json-iterator/go v1.1.10
|
||||
github.com/klauspost/compress v1.10.11 // indirect
|
||||
github.com/kr/text v0.2.0 // indirect
|
||||
|
|
|
@ -1,17 +1,13 @@
|
|||
package kv
|
||||
|
||||
import "go.etcd.io/etcd/clientv3"
|
||||
|
||||
type Base interface {
|
||||
type KVBase interface {
|
||||
Load(key string) (string, error)
|
||||
MultiLoad(keys []string) ([]string, error)
|
||||
LoadWithPrefix(key string) ([]string, []string, error)
|
||||
Save(key, value string) error
|
||||
MultiSave(kvs map[string]string) error
|
||||
Remove(key string) error
|
||||
MultiRemove(keys []string) error
|
||||
Watch(key string) clientv3.WatchChan
|
||||
MultiSaveAndRemove(saves map[string]string, removals []string) error
|
||||
WatchWithPrefix(key string) clientv3.WatchChan
|
||||
LoadWithPrefix(key string) ([]string, []string, error)
|
||||
Close()
|
||||
}
|
||||
|
|
|
@ -0,0 +1,124 @@
|
|||
|
||||
|
||||
package kv
|
||||
|
||||
import (
|
||||
"github.com/google/btree"
|
||||
"sync"
|
||||
)
|
||||
|
||||
type MemoryKV struct {
|
||||
sync.RWMutex
|
||||
tree *btree.BTree
|
||||
}
|
||||
|
||||
|
||||
// NewMemoryKV returns an in-memory kvBase for testing.
|
||||
func NewMemoryKV() *MemoryKV {
|
||||
return &MemoryKV{
|
||||
tree: btree.New(2),
|
||||
}
|
||||
}
|
||||
|
||||
type memoryKVItem struct {
|
||||
key, value string
|
||||
}
|
||||
|
||||
|
||||
func (s memoryKVItem) Less(than btree.Item) bool {
|
||||
return s.key < than.(memoryKVItem).key
|
||||
}
|
||||
|
||||
|
||||
|
||||
func (kv *MemoryKV) Load(key string) (string, error) {
|
||||
kv.RLock()
|
||||
defer kv.RUnlock()
|
||||
item := kv.tree.Get(memoryKVItem{key, ""})
|
||||
if item == nil {
|
||||
return "", nil
|
||||
}
|
||||
return item.(memoryKVItem).value, nil
|
||||
}
|
||||
|
||||
func (kv *MemoryKV) LoadRange(key, endKey string, limit int) ([]string, []string, error) {
|
||||
kv.RLock()
|
||||
defer kv.RUnlock()
|
||||
keys := make([]string, 0, limit)
|
||||
values := make([]string, 0, limit)
|
||||
kv.tree.AscendRange(memoryKVItem{key, ""}, memoryKVItem{endKey, ""}, func(item btree.Item) bool {
|
||||
keys = append(keys, item.(memoryKVItem).key)
|
||||
values = append(values, item.(memoryKVItem).value)
|
||||
if limit > 0 {
|
||||
return len(keys) < limit
|
||||
}
|
||||
return true
|
||||
})
|
||||
return keys, values, nil
|
||||
}
|
||||
|
||||
func (kv *MemoryKV) Save(key, value string) error {
|
||||
kv.Lock()
|
||||
defer kv.Unlock()
|
||||
kv.tree.ReplaceOrInsert(memoryKVItem{key, value})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (kv *MemoryKV) Remove(key string) error {
|
||||
kv.Lock()
|
||||
defer kv.Unlock()
|
||||
|
||||
kv.tree.Delete(memoryKVItem{key, ""})
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
func (kv *MemoryKV) MultiLoad(keys []string) ([]string, error) {
|
||||
kv.RLock()
|
||||
defer kv.RUnlock()
|
||||
result := make([]string, 0, len(keys))
|
||||
for _,key := range keys {
|
||||
item := kv.tree.Get(memoryKVItem{key, ""})
|
||||
result = append(result, item.(memoryKVItem).value)
|
||||
}
|
||||
return result, nil
|
||||
}
|
||||
|
||||
func (kv *MemoryKV) MultiSave(kvs map[string]string) error {
|
||||
kv.Lock()
|
||||
defer kv.Unlock()
|
||||
for key, value := range kvs {
|
||||
kv.tree.ReplaceOrInsert(memoryKVItem{key, value})
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (kv *MemoryKV) MultiRemove(keys []string) error {
|
||||
kv.Lock()
|
||||
defer kv.Unlock()
|
||||
for _, key := range keys {
|
||||
kv.tree.Delete(memoryKVItem{key, ""})
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
func (kv *MemoryKV) MultiSaveAndRemove(saves map[string]string, removals []string) error {
|
||||
kv.Lock()
|
||||
defer kv.Unlock()
|
||||
for key, value := range saves {
|
||||
kv.tree.ReplaceOrInsert(memoryKVItem{key, value})
|
||||
}
|
||||
for _, key := range removals {
|
||||
kv.tree.Delete(memoryKVItem{key, ""})
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// todo
|
||||
func (kv *MemoryKV) LoadWithPrefix(key string) ([]string, []string, error) {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (kv *MemoryKV) Close() {
|
||||
}
|
|
@ -0,0 +1,15 @@
|
|||
package mockkv
|
||||
|
||||
import (
|
||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||
)
|
||||
|
||||
// use MemoryKV to mock EtcdKV
|
||||
func NewEtcdKV() *kv.MemoryKV {
|
||||
return kv.NewMemoryKV()
|
||||
}
|
||||
|
||||
// use MemoryKV to mock EtcdKV
|
||||
func NewMemoryKV() *kv.MemoryKV {
|
||||
return kv.NewMemoryKV()
|
||||
}
|
|
@ -18,7 +18,7 @@ import (
|
|||
|
||||
type UniqueID = typeutil.UniqueID
|
||||
|
||||
func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, errch chan error) {
|
||||
func CollectionController(ch chan *schemapb.CollectionSchema, kvbase *kv.EtcdKV, errch chan error) {
|
||||
for collectionMeta := range ch {
|
||||
sID, _ := id.AllocOne()
|
||||
cID, _ := id.AllocOne()
|
||||
|
@ -57,7 +57,7 @@ func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, er
|
|||
}
|
||||
}
|
||||
|
||||
func WriteCollection2Datastore(collectionMeta *schemapb.CollectionSchema, kvbase kv.Base) error {
|
||||
func WriteCollection2Datastore(collectionMeta *schemapb.CollectionSchema, kvbase *kv.EtcdKV) error {
|
||||
sID, _ := id.AllocOne()
|
||||
cID, _ := id.AllocOne()
|
||||
fieldMetas := []*schemapb.FieldSchema{}
|
||||
|
|
|
@ -14,7 +14,7 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
|
||||
func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase *kv.EtcdKV) error {
|
||||
if int(ss.MemorySize) > int(conf.Config.Master.SegmentThreshole*0.8) {
|
||||
currentTime := time.Now()
|
||||
//memRate := int(ss.MemoryRate)
|
||||
|
|
|
@ -11,7 +11,7 @@ import (
|
|||
"go.etcd.io/etcd/clientv3"
|
||||
)
|
||||
|
||||
func newKvBase() kv.Base {
|
||||
func newKvBase() *kv.EtcdKV {
|
||||
etcdAddr := conf.Config.Etcd.Address
|
||||
etcdAddr += ":"
|
||||
etcdAddr += strconv.FormatInt(int64(conf.Config.Etcd.Port), 10)
|
||||
|
|
|
@ -17,7 +17,7 @@ func (s *Master) CreateCollection(ctx context.Context, in *internalpb.CreateColl
|
|||
var t task = &createCollectionTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
@ -50,7 +50,7 @@ func (s *Master) DropCollection(ctx context.Context, in *internalpb.DropCollecti
|
|||
var t task = &dropCollectionTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
@ -83,7 +83,7 @@ func (s *Master) HasCollection(ctx context.Context, in *internalpb.HasCollection
|
|||
var t task = &hasCollectionTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
@ -126,7 +126,7 @@ func (s *Master) DescribeCollection(ctx context.Context, in *internalpb.Describe
|
|||
var t task = &describeCollectionTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
@ -152,7 +152,7 @@ func (s *Master) ShowCollections(ctx context.Context, in *internalpb.ShowCollect
|
|||
var t task = &showCollectionsTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
@ -179,7 +179,7 @@ func (s *Master) CreatePartition(ctx context.Context, in *internalpb.CreateParti
|
|||
var t task = &createPartitionTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
@ -212,7 +212,7 @@ func (s *Master) DropPartition(ctx context.Context, in *internalpb.DropPartition
|
|||
var t task = &dropPartitionTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
@ -245,7 +245,7 @@ func (s *Master) HasPartition(ctx context.Context, in *internalpb.HasPartitionRe
|
|||
var t task = &hasPartitionTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
@ -288,7 +288,7 @@ func (s *Master) DescribePartition(ctx context.Context, in *internalpb.DescribeP
|
|||
var t task = &describePartitionTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
@ -314,7 +314,7 @@ func (s *Master) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitio
|
|||
var t task = &showPartitionTask{
|
||||
req: in,
|
||||
baseTask: baseTask{
|
||||
kvBase: &s.kvBase,
|
||||
kvBase: s.kvBase,
|
||||
mt: &s.mt,
|
||||
cv: make(chan int),
|
||||
},
|
||||
|
|
|
@ -50,7 +50,7 @@ type Master struct {
|
|||
// chans
|
||||
ssChan chan internalpb.SegmentStatistics
|
||||
|
||||
kvBase kv.Base
|
||||
kvBase *kv.EtcdKV
|
||||
scheduler *ddRequestScheduler
|
||||
mt metaTable
|
||||
// Add callback functions at different stages
|
||||
|
@ -58,7 +58,7 @@ type Master struct {
|
|||
closeCallbacks []func()
|
||||
}
|
||||
|
||||
func newKvBase() kv.Base {
|
||||
func newKvBase() *kv.EtcdKV {
|
||||
etcdAddr := conf.Config.Etcd.Address
|
||||
etcdAddr += ":"
|
||||
etcdAddr += strconv.FormatInt(int64(conf.Config.Etcd.Port), 10)
|
||||
|
|
|
@ -15,7 +15,7 @@ import (
|
|||
type UniqueID = typeutil.UniqueID
|
||||
|
||||
type metaTable struct {
|
||||
client kv.Base // client of a reliable kv service, i.e. etcd client
|
||||
client *kv.EtcdKV // client of a reliable kv service, i.e. etcd client
|
||||
tenantId2Meta map[UniqueID]pb.TenantMeta // tenant id to tenant meta
|
||||
proxyId2Meta map[UniqueID]pb.ProxyMeta // proxy id to proxy meta
|
||||
collId2Meta map[UniqueID]pb.CollectionMeta // collection id to collection meta
|
||||
|
@ -27,7 +27,7 @@ type metaTable struct {
|
|||
ddLock sync.RWMutex
|
||||
}
|
||||
|
||||
func NewMetaTable(kv kv.Base) (*metaTable, error) {
|
||||
func NewMetaTable(kv *kv.EtcdKV) (*metaTable, error) {
|
||||
mt := &metaTable{
|
||||
client: kv,
|
||||
tenantLock: sync.RWMutex{},
|
||||
|
|
|
@ -11,7 +11,7 @@ import (
|
|||
// TODO: get timestamp from timestampOracle
|
||||
|
||||
type baseTask struct {
|
||||
kvBase *kv.Base
|
||||
kvBase *kv.EtcdKV
|
||||
mt *metaTable
|
||||
cv chan int
|
||||
}
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
package tso
|
||||
|
||||
import (
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||
"github.com/zilliztech/milvus-distributed/internal/kv/mockkv"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
var GTsoAllocator *GlobalTSOAllocator
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
GTsoAllocator = &GlobalTSOAllocator{
|
||||
timestampOracle: ×tampOracle{
|
||||
kvBase: mockkv.NewEtcdKV(),
|
||||
rootPath: conf.Config.Etcd.Rootpath,
|
||||
saveInterval: 3 * time.Second,
|
||||
maxResetTSGap: func() time.Duration { return 3 * time.Second },
|
||||
key: "tso",
|
||||
},
|
||||
}
|
||||
exitCode := m.Run()
|
||||
os.Exit(exitCode)
|
||||
}
|
||||
|
||||
func TestGlobalTSOAllocator_Initialize(t *testing.T) {
|
||||
err := GTsoAllocator.Initialize()
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
func TestGlobalTSOAllocator_GenerateTSO(t *testing.T) {
|
||||
count := 1000
|
||||
perCount := uint32(100)
|
||||
startTs, err := GTsoAllocator.GenerateTSO(perCount)
|
||||
assert.Nil(t, err)
|
||||
lastPhysical, lastLogical := tsoutil.ParseTS(startTs)
|
||||
for i:=0;i < count; i++{
|
||||
ts, _ := GTsoAllocator.GenerateTSO(perCount)
|
||||
physical, logical := tsoutil.ParseTS(ts)
|
||||
if lastPhysical == physical {
|
||||
diff := logical - lastLogical
|
||||
assert.Equal(t, uint64(perCount), diff)
|
||||
}
|
||||
lastPhysical, lastLogical = physical, logical
|
||||
}
|
||||
}
|
||||
|
||||
func TestGlobalTSOAllocator_SetTSO(t *testing.T) {
|
||||
curTime := time.Now()
|
||||
nextTime := curTime.Add(2 * time.Second )
|
||||
physical := nextTime.UnixNano() / int64(time.Millisecond)
|
||||
logical := int64(0)
|
||||
err := GTsoAllocator.SetTSO(tsoutil.ComposeTS(physical, logical))
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
func TestGlobalTSOAllocator_UpdateTSO(t *testing.T) {
|
||||
err := GTsoAllocator.UpdateTSO()
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
func TestGlobalTSOAllocator_Reset(t *testing.T) {
|
||||
GTsoAllocator.Reset()
|
||||
}
|
|
@ -48,7 +48,7 @@ type atomicObject struct {
|
|||
type timestampOracle struct {
|
||||
rootPath string
|
||||
key string
|
||||
kvBase *kv.EtcdKV
|
||||
kvBase kv.KVBase
|
||||
|
||||
// TODO: remove saveInterval
|
||||
saveInterval time.Duration
|
||||
|
@ -117,9 +117,6 @@ func (t *timestampOracle) SyncTimestamp() error {
|
|||
|
||||
// ResetUserTimestamp update the physical part with specified tso.
|
||||
func (t *timestampOracle) ResetUserTimestamp(tso uint64) error {
|
||||
//if !leadership.Check() {
|
||||
// return errors.New("Setup timestamp failed, lease expired")
|
||||
//}
|
||||
physical, _ := tsoutil.ParseTS(tso)
|
||||
next := physical.Add(time.Millisecond)
|
||||
prev := (*atomicObject)(atomic.LoadPointer(&t.TSO))
|
||||
|
@ -160,7 +157,7 @@ func (t *timestampOracle) UpdateTimestamp() error {
|
|||
now := time.Now()
|
||||
|
||||
jetLag := typeutil.SubTimeByWallClock(now, prev.physical)
|
||||
if jetLag > 3*UpdateTimestampStep {
|
||||
if jetLag > 3 * UpdateTimestampStep {
|
||||
log.Print("clock offset", zap.Duration("jet-lag", jetLag), zap.Time("prev-physical", prev.physical), zap.Time("now", now))
|
||||
}
|
||||
|
||||
|
|
|
@ -27,8 +27,6 @@ func repackFunc(msgs []*TsMsg, hashKeys [][]int32) map[int32]*MsgPack {
|
|||
func getTsMsg(msgType MsgType, reqId UniqueID, hashValue int32) *TsMsg {
|
||||
var tsMsg TsMsg
|
||||
baseMsg := BaseMsg{
|
||||
beginTs: 0,
|
||||
endTs: 0,
|
||||
HashValues: []int32{hashValue},
|
||||
}
|
||||
switch msgType {
|
||||
|
|
Loading…
Reference in New Issue