提交 86d2d360 编写于 作者: Z zhenshan.cao 提交者: yefu.chen

Add MemoryKV, mockkv and tso test

Signed-off-by: Nzhenshan.cao <zhenshan.cao@zilliz.com>
上级 3ff3a5b6
......@@ -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
......
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()
}
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() {
}
\ No newline at end of file
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()
}
\ No newline at end of file
......@@ -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
}
......
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: &timestampOracle{
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()
}
\ No newline at end of file
......@@ -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 {
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册