diff --git a/cmd/storage/benchmark.go b/cmd/storage/benchmark.go deleted file mode 100644 index 6a7c06f9a2..0000000000 --- a/cmd/storage/benchmark.go +++ /dev/null @@ -1,315 +0,0 @@ -package main - -import ( - "context" - "crypto/md5" - "flag" - "fmt" - "log" - "math/rand" - "os" - "sync" - "sync/atomic" - "time" - - "github.com/pivotal-golang/bytefmt" - "github.com/zilliztech/milvus-distributed/internal/storage" - storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -// Global variables -var durationSecs, threads, loops, numVersion, batchOpSize int -var valueSize uint64 -var valueData []byte -var batchValueData [][]byte -var counter, totalKeyCount, keyNum int32 -var endTime, setFinish, getFinish, deleteFinish time.Time -var totalKeys [][]byte - -var logFileName = "benchmark.log" -var logFile *os.File - -var store storagetype.Store -var wg sync.WaitGroup - -func runSet() { - for time.Now().Before(endTime) { - num := atomic.AddInt32(&keyNum, 1) - key := []byte(fmt.Sprint("key", num)) - for ver := 1; ver <= numVersion; ver++ { - atomic.AddInt32(&counter, 1) - err := store.PutRow(context.Background(), key, valueData, "empty", uint64(ver)) - if err != nil { - log.Fatalf("Error setting key %s, %s", key, err.Error()) - //atomic.AddInt32(&setCount, -1) - } - } - } - // Remember last done time - setFinish = time.Now() - wg.Done() -} - -func runBatchSet() { - for time.Now().Before(endTime) { - num := atomic.AddInt32(&keyNum, int32(batchOpSize)) - keys := make([][]byte, batchOpSize) - versions := make([]uint64, batchOpSize) - batchSuffix := make([]string, batchOpSize) - for n := batchOpSize; n > 0; n-- { - keys[n-1] = []byte(fmt.Sprint("key", num-int32(n))) - } - for ver := 1; ver <= numVersion; ver++ { - atomic.AddInt32(&counter, 1) - err := store.PutRows(context.Background(), keys, batchValueData, batchSuffix, versions) - if err != nil { - log.Fatalf("Error setting batch keys %s %s", keys, err.Error()) - //atomic.AddInt32(&batchSetCount, -1) - } - } - } - setFinish = time.Now() - wg.Done() -} - -func runGet() { - for time.Now().Before(endTime) { - num := atomic.AddInt32(&counter, 1) - //num := atomic.AddInt32(&keyNum, 1) - //key := []byte(fmt.Sprint("key", num)) - num = num % totalKeyCount - key := totalKeys[num] - _, err := store.GetRow(context.Background(), key, uint64(numVersion)) - if err != nil { - log.Fatalf("Error getting key %s, %s", key, err.Error()) - //atomic.AddInt32(&getCount, -1) - } - } - // Remember last done time - getFinish = time.Now() - wg.Done() -} - -func runBatchGet() { - for time.Now().Before(endTime) { - num := atomic.AddInt32(&keyNum, int32(batchOpSize)) - //keys := make([][]byte, batchOpSize) - //for n := batchOpSize; n > 0; n-- { - // keys[n-1] = []byte(fmt.Sprint("key", num-int32(n))) - //} - end := num % totalKeyCount - if end < int32(batchOpSize) { - end = int32(batchOpSize) - } - start := end - int32(batchOpSize) - keys := totalKeys[start:end] - versions := make([]uint64, batchOpSize) - for i := range versions { - versions[i] = uint64(numVersion) - } - atomic.AddInt32(&counter, 1) - _, err := store.GetRows(context.Background(), keys, versions) - if err != nil { - log.Fatalf("Error getting key %s, %s", keys, err.Error()) - //atomic.AddInt32(&batchGetCount, -1) - } - } - // Remember last done time - getFinish = time.Now() - wg.Done() -} - -func runDelete() { - for time.Now().Before(endTime) { - num := atomic.AddInt32(&counter, 1) - //num := atomic.AddInt32(&keyNum, 1) - //key := []byte(fmt.Sprint("key", num)) - num = num % totalKeyCount - key := totalKeys[num] - err := store.DeleteRow(context.Background(), key, uint64(numVersion)) - if err != nil { - log.Fatalf("Error getting key %s, %s", key, err.Error()) - //atomic.AddInt32(&deleteCount, -1) - } - } - // Remember last done time - deleteFinish = time.Now() - wg.Done() -} - -func runBatchDelete() { - for time.Now().Before(endTime) { - num := atomic.AddInt32(&keyNum, int32(batchOpSize)) - //keys := make([][]byte, batchOpSize) - //for n := batchOpSize; n > 0; n-- { - // keys[n-1] = []byte(fmt.Sprint("key", num-int32(n))) - //} - end := num % totalKeyCount - if end < int32(batchOpSize) { - end = int32(batchOpSize) - } - start := end - int32(batchOpSize) - keys := totalKeys[start:end] - atomic.AddInt32(&counter, 1) - versions := make([]uint64, batchOpSize) - for i := range versions { - versions[i] = uint64(numVersion) - } - err := store.DeleteRows(context.Background(), keys, versions) - if err != nil { - log.Fatalf("Error getting key %s, %s", keys, err.Error()) - //atomic.AddInt32(&batchDeleteCount, -1) - } - } - // Remember last done time - getFinish = time.Now() - wg.Done() -} - -func main() { - // Parse command line - myflag := flag.NewFlagSet("myflag", flag.ExitOnError) - myflag.IntVar(&durationSecs, "d", 5, "Duration of each test in seconds") - myflag.IntVar(&threads, "t", 1, "Number of threads to run") - myflag.IntVar(&loops, "l", 1, "Number of times to repeat test") - var sizeArg string - var storeType string - myflag.StringVar(&sizeArg, "z", "1k", "Size of objects in bytes with postfix K, M, and G") - myflag.StringVar(&storeType, "s", "s3", "Storage type, tikv or minio or s3") - myflag.IntVar(&numVersion, "v", 1, "Max versions for each key") - myflag.IntVar(&batchOpSize, "b", 100, "Batch operation kv pair number") - - if err := myflag.Parse(os.Args[1:]); err != nil { - os.Exit(1) - } - - // Check the arguments - var err error - if valueSize, err = bytefmt.ToBytes(sizeArg); err != nil { - log.Fatalf("Invalid -z argument for object size: %v", err) - } - var option = storagetype.Option{TikvAddress: "localhost:2379", Type: storeType, BucketName: "zilliz-hz"} - - store, err = storage.NewStore(context.Background(), option) - if err != nil { - log.Fatalf("Error when creating storage " + err.Error()) - } - logFile, err = os.OpenFile(logFileName, os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0777) - if err != nil { - log.Fatalf("Prepare log file error, " + err.Error()) - } - - // Echo the parameters - log.Printf("Benchmark log will write to file %s\n", logFile.Name()) - fmt.Fprintf(logFile, "Parameters: duration=%d, threads=%d, loops=%d, valueSize=%s, batchSize=%d, versions=%d\n", durationSecs, threads, loops, sizeArg, batchOpSize, numVersion) - // Init test data - valueData = make([]byte, valueSize) - rand.Read(valueData) - hasher := md5.New() - hasher.Write(valueData) - - batchValueData = make([][]byte, batchOpSize) - for i := range batchValueData { - batchValueData[i] = make([]byte, valueSize) - rand.Read(batchValueData[i]) - hasher := md5.New() - hasher.Write(batchValueData[i]) - } - - // Loop running the tests - for loop := 1; loop <= loops; loop++ { - - // reset counters - counter = 0 - keyNum = 0 - totalKeyCount = 0 - totalKeys = nil - - // Run the batchSet case - // key seq start from setCount - counter = 0 - startTime := time.Now() - endTime = startTime.Add(time.Second * time.Duration(durationSecs)) - for n := 1; n <= threads; n++ { - wg.Add(1) - go runBatchSet() - } - wg.Wait() - - setTime := setFinish.Sub(startTime).Seconds() - bps := float64(uint64(counter)*valueSize*uint64(batchOpSize)) / setTime - fmt.Fprintf(logFile, "Loop %d: BATCH PUT time %.1f secs, batchs = %d, kv pairs = %d, speed = %sB/sec, %.1f operations/sec, %.1f kv/sec.\n", - loop, setTime, counter, counter*int32(batchOpSize), bytefmt.ByteSize(uint64(bps)), float64(counter)/setTime, float64(counter*int32(batchOpSize))/setTime) - // Record all test keys - //totalKeyCount = keyNum - //totalKeys = make([][]byte, totalKeyCount) - //for i := int32(0); i < totalKeyCount; i++ { - // totalKeys[i] = []byte(fmt.Sprint("key", i)) - //} - // - //// Run the get case - //counter = 0 - //startTime = time.Now() - //endTime = startTime.Add(time.Second * time.Duration(durationSecs)) - //for n := 1; n <= threads; n++ { - // wg.Add(1) - // go runGet() - //} - //wg.Wait() - // - //getTime := getFinish.Sub(startTime).Seconds() - //bps = float64(uint64(counter)*valueSize) / getTime - //fmt.Fprint(logFile, fmt.Sprintf("Loop %d: GET time %.1f secs, kv pairs = %d, speed = %sB/sec, %.1f operations/sec, %.1f kv/sec.\n", - // loop, getTime, counter, bytefmt.ByteSize(uint64(bps)), float64(counter)/getTime, float64(counter)/getTime)) - - // Run the batchGet case - //counter = 0 - //startTime = time.Now() - //endTime = startTime.Add(time.Second * time.Duration(durationSecs)) - //for n := 1; n <= threads; n++ { - // wg.Add(1) - // go runBatchGet() - //} - //wg.Wait() - // - //getTime = getFinish.Sub(startTime).Seconds() - //bps = float64(uint64(counter)*valueSize*uint64(batchOpSize)) / getTime - //fmt.Fprint(logFile, fmt.Sprintf("Loop %d: BATCH GET time %.1f secs, batchs = %d, kv pairs = %d, speed = %sB/sec, %.1f operations/sec, %.1f kv/sec.\n", - // loop, getTime, counter, counter*int32(batchOpSize), bytefmt.ByteSize(uint64(bps)), float64(counter)/getTime, float64(counter * int32(batchOpSize))/getTime)) - // - //// Run the delete case - //counter = 0 - //startTime = time.Now() - //endTime = startTime.Add(time.Second * time.Duration(durationSecs)) - //for n := 1; n <= threads; n++ { - // wg.Add(1) - // go runDelete() - //} - //wg.Wait() - // - //deleteTime := deleteFinish.Sub(startTime).Seconds() - //bps = float64(uint64(counter)*valueSize) / deleteTime - //fmt.Fprint(logFile, fmt.Sprintf("Loop %d: Delete time %.1f secs, kv pairs = %d, %.1f operations/sec, %.1f kv/sec.\n", - // loop, deleteTime, counter, float64(counter)/deleteTime, float64(counter)/deleteTime)) - // - //// Run the batchDelete case - //counter = 0 - //startTime = time.Now() - //endTime = startTime.Add(time.Second * time.Duration(durationSecs)) - //for n := 1; n <= threads; n++ { - // wg.Add(1) - // go runBatchDelete() - //} - //wg.Wait() - // - //deleteTime = setFinish.Sub(startTime).Seconds() - //bps = float64(uint64(counter)*valueSize*uint64(batchOpSize)) / setTime - //fmt.Fprint(logFile, fmt.Sprintf("Loop %d: BATCH DELETE time %.1f secs, batchs = %d, kv pairs = %d, %.1f operations/sec, %.1f kv/sec.\n", - // loop, setTime, counter, counter*int32(batchOpSize), float64(counter)/setTime, float64(counter * int32(batchOpSize))/setTime)) - - // Print line mark - lineMark := "\n" - fmt.Fprint(logFile, lineMark) - } - log.Print("Benchmark test done.") -} diff --git a/deployments/docker/docker-compose.yml b/deployments/docker/docker-compose.yml index 0ae708a19e..60bf5d9fff 100644 --- a/deployments/docker/docker-compose.yml +++ b/deployments/docker/docker-compose.yml @@ -36,14 +36,6 @@ services: networks: - milvus - jaeger: - image: jaegertracing/all-in-one:latest - ports: - - "6831:6831/udp" - - "16686:16686" - networks: - - milvus - networks: milvus: diff --git a/docker-compose.yml b/docker-compose.yml index 9f3599abb9..cba23befab 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -83,10 +83,5 @@ services: networks: - milvus - jaeger: - image: jaegertracing/all-in-one:latest - networks: - - milvus - networks: milvus: diff --git a/internal/indexbuilder/indexbuilder.go b/internal/indexbuilder/indexbuilder.go index 5b21e68dd4..4acfffc3d2 100644 --- a/internal/indexbuilder/indexbuilder.go +++ b/internal/indexbuilder/indexbuilder.go @@ -11,9 +11,6 @@ import ( miniokv "github.com/zilliztech/milvus-distributed/internal/kv/minio" - "github.com/minio/minio-go/v7" - "github.com/minio/minio-go/v7/pkg/credentials" - "go.etcd.io/etcd/clientv3" "github.com/zilliztech/milvus-distributed/internal/allocator" @@ -71,19 +68,16 @@ func CreateBuilder(ctx context.Context) (*Builder, error) { idAllocator, err := allocator.NewIDAllocator(b.loopCtx, Params.MasterAddress) - minIOEndPoint := Params.MinIOAddress - minIOAccessKeyID := Params.MinIOAccessKeyID - minIOSecretAccessKey := Params.MinIOSecretAccessKey - minIOUseSSL := Params.MinIOUseSSL - minIOClient, err := minio.New(minIOEndPoint, &minio.Options{ - Creds: credentials.NewStaticV4(minIOAccessKeyID, minIOSecretAccessKey, ""), - Secure: minIOUseSSL, - }) - if err != nil { - return nil, err + option := &miniokv.Option{ + Address: Params.MinIOAddress, + AccessKeyID: Params.MinIOAccessKeyID, + SecretAccessKeyID: Params.MinIOSecretAccessKey, + UseSSL: Params.MinIOUseSSL, + BucketName: Params.MinioBucketName, + CreateBucket: true, } - b.kv, err = miniokv.NewMinIOKV(b.loopCtx, minIOClient, Params.MinioBucketName) + b.kv, err = miniokv.NewMinIOKV(b.loopCtx, option) if err != nil { return nil, err } diff --git a/internal/kv/minio/minio_kv.go b/internal/kv/minio/minio_kv.go index 68bb3a3438..6b3522fb45 100644 --- a/internal/kv/minio/minio_kv.go +++ b/internal/kv/minio/minio_kv.go @@ -2,11 +2,15 @@ package miniokv import ( "context" + "fmt" + "io" "log" "strings" "github.com/minio/minio-go/v7" + "github.com/minio/minio-go/v7/pkg/credentials" + "github.com/zilliztech/milvus-distributed/internal/errors" ) type MinIOKV struct { @@ -15,24 +19,46 @@ type MinIOKV struct { bucketName string } -// NewMinIOKV creates a new MinIO kv. -func NewMinIOKV(ctx context.Context, client *minio.Client, bucketName string) (*MinIOKV, error) { +type Option struct { + Address string + AccessKeyID string + BucketName string + SecretAccessKeyID string + UseSSL bool + CreateBucket bool // when bucket not existed, create it +} - bucketExists, err := client.BucketExists(ctx, bucketName) +func NewMinIOKV(ctx context.Context, option *Option) (*MinIOKV, error) { + minIOClient, err := minio.New(option.Address, &minio.Options{ + Creds: credentials.NewStaticV4(option.AccessKeyID, option.SecretAccessKeyID, ""), + Secure: option.UseSSL, + }) if err != nil { return nil, err } - if !bucketExists { - err = client.MakeBucket(ctx, bucketName, minio.MakeBucketOptions{}) - if err != nil { - return nil, err + bucketExists, err := minIOClient.BucketExists(ctx, option.BucketName) + if err != nil { + return nil, err + } + + if option.CreateBucket { + if !bucketExists { + err = minIOClient.MakeBucket(ctx, option.BucketName, minio.MakeBucketOptions{}) + if err != nil { + return nil, err + } + } + } else { + if !bucketExists { + return nil, errors.New(fmt.Sprintf("Bucket %s not Existed.", option.BucketName)) } } + return &MinIOKV{ ctx: ctx, - minioClient: client, - bucketName: bucketName, + minioClient: minIOClient, + bucketName: option.BucketName, }, nil } diff --git a/internal/kv/minio/minio_kv_test.go b/internal/kv/minio/minio_kv_test.go index ac2a3180b2..2e50545b40 100644 --- a/internal/kv/minio/minio_kv_test.go +++ b/internal/kv/minio/minio_kv_test.go @@ -5,8 +5,6 @@ import ( "strconv" "testing" - "github.com/minio/minio-go/v7" - "github.com/minio/minio-go/v7/pkg/credentials" miniokv "github.com/zilliztech/milvus-distributed/internal/kv/minio" "github.com/zilliztech/milvus-distributed/internal/util/paramtable" @@ -15,24 +13,31 @@ import ( var Params paramtable.BaseTable -func TestMinIOKV_Load(t *testing.T) { - Params.Init() +func newMinIOKVClient(ctx context.Context, bucketName string) (*miniokv.MinIOKV, error) { endPoint, _ := Params.Load("_MinioAddress") accessKeyID, _ := Params.Load("minio.accessKeyID") secretAccessKey, _ := Params.Load("minio.secretAccessKey") useSSLStr, _ := Params.Load("minio.useSSL") + useSSL, _ := strconv.ParseBool(useSSLStr) + option := &miniokv.Option{ + Address: endPoint, + AccessKeyID: accessKeyID, + SecretAccessKeyID: secretAccessKey, + UseSSL: useSSL, + BucketName: bucketName, + CreateBucket: true, + } + client, err := miniokv.NewMinIOKV(ctx, option) + return client, err +} + +func TestMinIOKV_Load(t *testing.T) { + Params.Init() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - useSSL, _ := strconv.ParseBool(useSSLStr) - - minioClient, err := minio.New(endPoint, &minio.Options{ - Creds: credentials.NewStaticV4(accessKeyID, secretAccessKey, ""), - Secure: useSSL, - }) - assert.Nil(t, err) bucketName := "fantastic-tech-test" - MinIOKV, err := miniokv.NewMinIOKV(ctx, minioClient, bucketName) + MinIOKV, err := newMinIOKVClient(ctx, bucketName) assert.Nil(t, err) defer MinIOKV.RemoveWithPrefix("") @@ -79,25 +84,14 @@ func TestMinIOKV_Load(t *testing.T) { } func TestMinIOKV_MultiSave(t *testing.T) { + Params.Init() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - Params.Init() - endPoint, _ := Params.Load("_MinioAddress") - accessKeyID, _ := Params.Load("minio.accessKeyID") - secretAccessKey, _ := Params.Load("minio.secretAccessKey") - useSSLStr, _ := Params.Load("minio.useSSL") - useSSL, _ := strconv.ParseBool(useSSLStr) - - minioClient, err := minio.New(endPoint, &minio.Options{ - Creds: credentials.NewStaticV4(accessKeyID, secretAccessKey, ""), - Secure: useSSL, - }) - assert.Nil(t, err) - bucketName := "fantastic-tech-test" - MinIOKV, err := miniokv.NewMinIOKV(ctx, minioClient, bucketName) + MinIOKV, err := newMinIOKVClient(ctx, bucketName) assert.Nil(t, err) + defer MinIOKV.RemoveWithPrefix("") err = MinIOKV.Save("key_1", "111") @@ -117,25 +111,13 @@ func TestMinIOKV_MultiSave(t *testing.T) { } func TestMinIOKV_Remove(t *testing.T) { + Params.Init() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - Params.Init() - endPoint, _ := Params.Load("_MinioAddress") - accessKeyID, _ := Params.Load("minio.accessKeyID") - secretAccessKey, _ := Params.Load("minio.secretAccessKey") - useSSLStr, _ := Params.Load("minio.useSSL") - useSSL, _ := strconv.ParseBool(useSSLStr) - - minioClient, err := minio.New(endPoint, &minio.Options{ - Creds: credentials.NewStaticV4(accessKeyID, secretAccessKey, ""), - Secure: useSSL, - }) - assert.Nil(t, err) - bucketName := "fantastic-tech-test" - MinIOKV, err := miniokv.NewMinIOKV(ctx, minioClient, bucketName) + MinIOKV, err := newMinIOKVClient(ctx, bucketName) assert.Nil(t, err) defer MinIOKV.RemoveWithPrefix("") diff --git a/internal/master/client.go b/internal/master/client.go index 7c644e02ac..88e44d8f70 100644 --- a/internal/master/client.go +++ b/internal/master/client.go @@ -9,19 +9,25 @@ import ( ) type WriteNodeClient interface { - FlushSegment(segmentID UniqueID) error + FlushSegment(segmentID UniqueID, collectionID UniqueID, partitionTag string, timestamp Timestamp) error DescribeSegment(segmentID UniqueID) (*writerclient.SegmentDescription, error) GetInsertBinlogPaths(segmentID UniqueID) (map[UniqueID][]string, error) } type MockWriteNodeClient struct { - segmentID UniqueID - flushTime time.Time + segmentID UniqueID + flushTime time.Time + partitionTag string + timestamp Timestamp + collectionID UniqueID } -func (m *MockWriteNodeClient) FlushSegment(segmentID UniqueID) error { +func (m *MockWriteNodeClient) FlushSegment(segmentID UniqueID, collectionID UniqueID, partitionTag string, timestamp Timestamp) error { m.flushTime = time.Now() m.segmentID = segmentID + m.collectionID = collectionID + m.partitionTag = partitionTag + m.timestamp = timestamp return nil } diff --git a/internal/master/flush_scheduler.go b/internal/master/flush_scheduler.go index d27511c2c8..4ceb1d8269 100644 --- a/internal/master/flush_scheduler.go +++ b/internal/master/flush_scheduler.go @@ -37,7 +37,13 @@ func NewFlushScheduler(ctx context.Context, client WriteNodeClient, metaTable *m func (scheduler *FlushScheduler) schedule(id interface{}) error { segmentID := id.(UniqueID) - err := scheduler.client.FlushSegment(segmentID) + segmentMeta, err := scheduler.metaTable.GetSegmentByID(segmentID) + if err != nil { + return err + } + + // todo set corrent timestamp + err = scheduler.client.FlushSegment(segmentID, segmentMeta.CollectionID, segmentMeta.PartitionTag, Timestamp(0)) log.Printf("flush segment %d", segmentID) if err != nil { return err diff --git a/internal/master/master.go b/internal/master/master.go index e8fb413461..5476a7c8dc 100644 --- a/internal/master/master.go +++ b/internal/master/master.go @@ -218,7 +218,6 @@ func CreateServer(ctx context.Context) (*Master, error) { m.grpcServer = grpc.NewServer() masterpb.RegisterMasterServer(m.grpcServer, m) - return m, nil } diff --git a/internal/master/master_test.go b/internal/master/master_test.go index a605e73aa7..0a44ed90e8 100644 --- a/internal/master/master_test.go +++ b/internal/master/master_test.go @@ -110,7 +110,6 @@ func TestMaster(t *testing.T) { conn, err := grpc.DialContext(ctx, Params.Address, grpc.WithInsecure(), grpc.WithBlock()) require.Nil(t, err) - cli := masterpb.NewMasterClient(conn) t.Run("TestConfigTask", func(t *testing.T) { @@ -887,6 +886,12 @@ func TestMaster(t *testing.T) { var k2sMsgstream ms.MsgStream = k2sMs assert.True(t, receiveTimeTickMsg(&k2sMsgstream)) + conn, err := grpc.DialContext(ctx, Params.Address, grpc.WithInsecure(), grpc.WithBlock()) + assert.Nil(t, err) + defer conn.Close() + + cli := masterpb.NewMasterClient(conn) + sch := schemapb.CollectionSchema{ Name: "name" + strconv.FormatUint(rand.Uint64(), 10), Description: "test collection", diff --git a/internal/msgstream/msg.go b/internal/msgstream/msg.go index a71d1cabfe..518bcfa7af 100644 --- a/internal/msgstream/msg.go +++ b/internal/msgstream/msg.go @@ -1,8 +1,6 @@ package msgstream import ( - "context" - "github.com/golang/protobuf/proto" internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" ) @@ -10,8 +8,6 @@ import ( type MsgType = internalPb.MsgType type TsMsg interface { - GetContext() context.Context - SetContext(context.Context) BeginTs() Timestamp EndTs() Timestamp Type() MsgType @@ -21,7 +17,6 @@ type TsMsg interface { } type BaseMsg struct { - ctx context.Context BeginTimestamp Timestamp EndTimestamp Timestamp HashValues []uint32 @@ -49,14 +44,6 @@ func (it *InsertMsg) Type() MsgType { return it.MsgType } -func (it *InsertMsg) GetContext() context.Context { - return it.ctx -} - -func (it *InsertMsg) SetContext(ctx context.Context) { - it.ctx = ctx -} - func (it *InsertMsg) Marshal(input TsMsg) ([]byte, error) { insertMsg := input.(*InsertMsg) insertRequest := &insertMsg.InsertRequest @@ -101,13 +88,6 @@ func (fl *FlushMsg) Type() MsgType { return fl.GetMsgType() } -func (fl *FlushMsg) GetContext() context.Context { - return fl.ctx -} -func (fl *FlushMsg) SetContext(ctx context.Context) { - fl.ctx = ctx -} - func (fl *FlushMsg) Marshal(input TsMsg) ([]byte, error) { flushMsgTask := input.(*FlushMsg) flushMsg := &flushMsgTask.FlushMsg @@ -141,14 +121,6 @@ func (dt *DeleteMsg) Type() MsgType { return dt.MsgType } -func (dt *DeleteMsg) GetContext() context.Context { - return dt.ctx -} - -func (dt *DeleteMsg) SetContext(ctx context.Context) { - dt.ctx = ctx -} - func (dt *DeleteMsg) Marshal(input TsMsg) ([]byte, error) { deleteTask := input.(*DeleteMsg) deleteRequest := &deleteTask.DeleteRequest @@ -193,14 +165,6 @@ func (st *SearchMsg) Type() MsgType { return st.MsgType } -func (st *SearchMsg) GetContext() context.Context { - return st.ctx -} - -func (st *SearchMsg) SetContext(ctx context.Context) { - st.ctx = ctx -} - func (st *SearchMsg) Marshal(input TsMsg) ([]byte, error) { searchTask := input.(*SearchMsg) searchRequest := &searchTask.SearchRequest @@ -234,14 +198,6 @@ func (srt *SearchResultMsg) Type() MsgType { return srt.MsgType } -func (srt *SearchResultMsg) GetContext() context.Context { - return srt.ctx -} - -func (srt *SearchResultMsg) SetContext(ctx context.Context) { - srt.ctx = ctx -} - func (srt *SearchResultMsg) Marshal(input TsMsg) ([]byte, error) { searchResultTask := input.(*SearchResultMsg) searchResultRequest := &searchResultTask.SearchResult @@ -275,14 +231,6 @@ func (tst *TimeTickMsg) Type() MsgType { return tst.MsgType } -func (tst *TimeTickMsg) GetContext() context.Context { - return tst.ctx -} - -func (tst *TimeTickMsg) SetContext(ctx context.Context) { - tst.ctx = ctx -} - func (tst *TimeTickMsg) Marshal(input TsMsg) ([]byte, error) { timeTickTask := input.(*TimeTickMsg) timeTick := &timeTickTask.TimeTickMsg @@ -316,14 +264,6 @@ func (qs *QueryNodeStatsMsg) Type() MsgType { return qs.MsgType } -func (qs *QueryNodeStatsMsg) GetContext() context.Context { - return qs.ctx -} - -func (qs *QueryNodeStatsMsg) SetContext(ctx context.Context) { - qs.ctx = ctx -} - func (qs *QueryNodeStatsMsg) Marshal(input TsMsg) ([]byte, error) { queryNodeSegStatsTask := input.(*QueryNodeStatsMsg) queryNodeSegStats := &queryNodeSegStatsTask.QueryNodeStats @@ -365,14 +305,6 @@ func (cc *CreateCollectionMsg) Type() MsgType { return cc.MsgType } -func (cc *CreateCollectionMsg) GetContext() context.Context { - return cc.ctx -} - -func (cc *CreateCollectionMsg) SetContext(ctx context.Context) { - cc.ctx = ctx -} - func (cc *CreateCollectionMsg) Marshal(input TsMsg) ([]byte, error) { createCollectionMsg := input.(*CreateCollectionMsg) createCollectionRequest := &createCollectionMsg.CreateCollectionRequest @@ -405,13 +337,6 @@ type DropCollectionMsg struct { func (dc *DropCollectionMsg) Type() MsgType { return dc.MsgType } -func (dc *DropCollectionMsg) GetContext() context.Context { - return dc.ctx -} - -func (dc *DropCollectionMsg) SetContext(ctx context.Context) { - dc.ctx = ctx -} func (dc *DropCollectionMsg) Marshal(input TsMsg) ([]byte, error) { dropCollectionMsg := input.(*DropCollectionMsg) @@ -436,20 +361,111 @@ func (dc *DropCollectionMsg) Unmarshal(input []byte) (TsMsg, error) { return dropCollectionMsg, nil } +/////////////////////////////////////////HasCollection////////////////////////////////////////// +type HasCollectionMsg struct { + BaseMsg + internalPb.HasCollectionRequest +} + +func (hc *HasCollectionMsg) Type() MsgType { + return hc.MsgType +} + +func (hc *HasCollectionMsg) Marshal(input TsMsg) ([]byte, error) { + hasCollectionMsg := input.(*HasCollectionMsg) + hasCollectionRequest := &hasCollectionMsg.HasCollectionRequest + mb, err := proto.Marshal(hasCollectionRequest) + if err != nil { + return nil, err + } + return mb, nil +} + +func (hc *HasCollectionMsg) Unmarshal(input []byte) (TsMsg, error) { + hasCollectionRequest := internalPb.HasCollectionRequest{} + err := proto.Unmarshal(input, &hasCollectionRequest) + if err != nil { + return nil, err + } + hasCollectionMsg := &HasCollectionMsg{HasCollectionRequest: hasCollectionRequest} + hasCollectionMsg.BeginTimestamp = hasCollectionMsg.Timestamp + hasCollectionMsg.EndTimestamp = hasCollectionMsg.Timestamp + + return hasCollectionMsg, nil +} + +/////////////////////////////////////////DescribeCollection////////////////////////////////////////// +type DescribeCollectionMsg struct { + BaseMsg + internalPb.DescribeCollectionRequest +} + +func (dc *DescribeCollectionMsg) Type() MsgType { + return dc.MsgType +} + +func (dc *DescribeCollectionMsg) Marshal(input TsMsg) ([]byte, error) { + describeCollectionMsg := input.(*DescribeCollectionMsg) + describeCollectionRequest := &describeCollectionMsg.DescribeCollectionRequest + mb, err := proto.Marshal(describeCollectionRequest) + if err != nil { + return nil, err + } + return mb, nil +} + +func (dc *DescribeCollectionMsg) Unmarshal(input []byte) (TsMsg, error) { + describeCollectionRequest := internalPb.DescribeCollectionRequest{} + err := proto.Unmarshal(input, &describeCollectionRequest) + if err != nil { + return nil, err + } + describeCollectionMsg := &DescribeCollectionMsg{DescribeCollectionRequest: describeCollectionRequest} + describeCollectionMsg.BeginTimestamp = describeCollectionMsg.Timestamp + describeCollectionMsg.EndTimestamp = describeCollectionMsg.Timestamp + + return describeCollectionMsg, nil +} + +/////////////////////////////////////////ShowCollection////////////////////////////////////////// +type ShowCollectionMsg struct { + BaseMsg + internalPb.ShowCollectionRequest +} + +func (sc *ShowCollectionMsg) Type() MsgType { + return sc.MsgType +} + +func (sc *ShowCollectionMsg) Marshal(input TsMsg) ([]byte, error) { + showCollectionMsg := input.(*ShowCollectionMsg) + showCollectionRequest := &showCollectionMsg.ShowCollectionRequest + mb, err := proto.Marshal(showCollectionRequest) + if err != nil { + return nil, err + } + return mb, nil +} + +func (sc *ShowCollectionMsg) Unmarshal(input []byte) (TsMsg, error) { + showCollectionRequest := internalPb.ShowCollectionRequest{} + err := proto.Unmarshal(input, &showCollectionRequest) + if err != nil { + return nil, err + } + showCollectionMsg := &ShowCollectionMsg{ShowCollectionRequest: showCollectionRequest} + showCollectionMsg.BeginTimestamp = showCollectionMsg.Timestamp + showCollectionMsg.EndTimestamp = showCollectionMsg.Timestamp + + return showCollectionMsg, nil +} + /////////////////////////////////////////CreatePartition////////////////////////////////////////// type CreatePartitionMsg struct { BaseMsg internalPb.CreatePartitionRequest } -func (cc *CreatePartitionMsg) GetContext() context.Context { - return cc.ctx -} - -func (cc *CreatePartitionMsg) SetContext(ctx context.Context) { - cc.ctx = ctx -} - func (cc *CreatePartitionMsg) Type() MsgType { return cc.MsgType } @@ -483,14 +499,6 @@ type DropPartitionMsg struct { internalPb.DropPartitionRequest } -func (dc *DropPartitionMsg) GetContext() context.Context { - return dc.ctx -} - -func (dc *DropPartitionMsg) SetContext(ctx context.Context) { - dc.ctx = ctx -} - func (dc *DropPartitionMsg) Type() MsgType { return dc.MsgType } @@ -518,6 +526,105 @@ func (dc *DropPartitionMsg) Unmarshal(input []byte) (TsMsg, error) { return dropPartitionMsg, nil } +/////////////////////////////////////////HasPartition////////////////////////////////////////// +type HasPartitionMsg struct { + BaseMsg + internalPb.HasPartitionRequest +} + +func (hc *HasPartitionMsg) Type() MsgType { + return hc.MsgType +} + +func (hc *HasPartitionMsg) Marshal(input TsMsg) ([]byte, error) { + hasPartitionMsg := input.(*HasPartitionMsg) + hasPartitionRequest := &hasPartitionMsg.HasPartitionRequest + mb, err := proto.Marshal(hasPartitionRequest) + if err != nil { + return nil, err + } + return mb, nil +} + +func (hc *HasPartitionMsg) Unmarshal(input []byte) (TsMsg, error) { + hasPartitionRequest := internalPb.HasPartitionRequest{} + err := proto.Unmarshal(input, &hasPartitionRequest) + if err != nil { + return nil, err + } + hasPartitionMsg := &HasPartitionMsg{HasPartitionRequest: hasPartitionRequest} + hasPartitionMsg.BeginTimestamp = hasPartitionMsg.Timestamp + hasPartitionMsg.EndTimestamp = hasPartitionMsg.Timestamp + + return hasPartitionMsg, nil +} + +/////////////////////////////////////////DescribePartition////////////////////////////////////////// +type DescribePartitionMsg struct { + BaseMsg + internalPb.DescribePartitionRequest +} + +func (dc *DescribePartitionMsg) Type() MsgType { + return dc.MsgType +} + +func (dc *DescribePartitionMsg) Marshal(input TsMsg) ([]byte, error) { + describePartitionMsg := input.(*DescribePartitionMsg) + describePartitionRequest := &describePartitionMsg.DescribePartitionRequest + mb, err := proto.Marshal(describePartitionRequest) + if err != nil { + return nil, err + } + return mb, nil +} + +func (dc *DescribePartitionMsg) Unmarshal(input []byte) (TsMsg, error) { + describePartitionRequest := internalPb.DescribePartitionRequest{} + err := proto.Unmarshal(input, &describePartitionRequest) + if err != nil { + return nil, err + } + describePartitionMsg := &DescribePartitionMsg{DescribePartitionRequest: describePartitionRequest} + describePartitionMsg.BeginTimestamp = describePartitionMsg.Timestamp + describePartitionMsg.EndTimestamp = describePartitionMsg.Timestamp + + return describePartitionMsg, nil +} + +/////////////////////////////////////////ShowPartition////////////////////////////////////////// +type ShowPartitionMsg struct { + BaseMsg + internalPb.ShowPartitionRequest +} + +func (sc *ShowPartitionMsg) Type() MsgType { + return sc.MsgType +} + +func (sc *ShowPartitionMsg) Marshal(input TsMsg) ([]byte, error) { + showPartitionMsg := input.(*ShowPartitionMsg) + showPartitionRequest := &showPartitionMsg.ShowPartitionRequest + mb, err := proto.Marshal(showPartitionRequest) + if err != nil { + return nil, err + } + return mb, nil +} + +func (sc *ShowPartitionMsg) Unmarshal(input []byte) (TsMsg, error) { + showPartitionRequest := internalPb.ShowPartitionRequest{} + err := proto.Unmarshal(input, &showPartitionRequest) + if err != nil { + return nil, err + } + showPartitionMsg := &ShowPartitionMsg{ShowPartitionRequest: showPartitionRequest} + showPartitionMsg.BeginTimestamp = showPartitionMsg.Timestamp + showPartitionMsg.EndTimestamp = showPartitionMsg.Timestamp + + return showPartitionMsg, nil +} + /////////////////////////////////////////LoadIndex////////////////////////////////////////// type LoadIndexMsg struct { BaseMsg @@ -528,14 +635,6 @@ func (lim *LoadIndexMsg) Type() MsgType { return lim.MsgType } -func (lim *LoadIndexMsg) GetContext() context.Context { - return lim.ctx -} - -func (lim *LoadIndexMsg) SetContext(ctx context.Context) { - lim.ctx = ctx -} - func (lim *LoadIndexMsg) Marshal(input TsMsg) ([]byte, error) { loadIndexMsg := input.(*LoadIndexMsg) loadIndexRequest := &loadIndexMsg.LoadIndex diff --git a/internal/msgstream/msgstream.go b/internal/msgstream/msgstream.go index 969755feb3..37dd71c053 100644 --- a/internal/msgstream/msgstream.go +++ b/internal/msgstream/msgstream.go @@ -4,13 +4,9 @@ import ( "context" "log" "reflect" - "strings" "sync" "time" - "github.com/opentracing/opentracing-go" - "github.com/opentracing/opentracing-go/ext" - "github.com/apache/pulsar-client-go/pulsar" "github.com/golang/protobuf/proto" @@ -155,29 +151,6 @@ func (ms *PulsarMsgStream) Close() { } } -type propertiesReaderWriter struct { - ppMap map[string]string -} - -func (ppRW *propertiesReaderWriter) Set(key, val string) { - // The GRPC HPACK implementation rejects any uppercase keys here. - // - // As such, since the HTTP_HEADERS format is case-insensitive anyway, we - // blindly lowercase the key (which is guaranteed to work in the - // Inject/Extract sense per the OpenTracing spec). - key = strings.ToLower(key) - ppRW.ppMap[key] = val -} - -func (ppRW *propertiesReaderWriter) ForeachKey(handler func(key, val string) error) error { - for k, val := range ppRW.ppMap { - if err := handler(k, val); err != nil { - return err - } - } - return nil -} - func (ms *PulsarMsgStream) Produce(msgPack *MsgPack) error { tsMsgs := msgPack.Msgs if len(tsMsgs) <= 0 { @@ -227,41 +200,12 @@ func (ms *PulsarMsgStream) Produce(msgPack *MsgPack) error { if err != nil { return err } - - msg := &pulsar.ProducerMessage{Payload: mb} - var child opentracing.Span - if v.Msgs[i].Type() == internalPb.MsgType_kInsert || v.Msgs[i].Type() == internalPb.MsgType_kSearch { - tracer := opentracing.GlobalTracer() - ctx := v.Msgs[i].GetContext() - if ctx == nil { - ctx = context.Background() - } - - if parent := opentracing.SpanFromContext(ctx); parent != nil { - child = tracer.StartSpan("start send pulsar msg", - opentracing.FollowsFrom(parent.Context())) - } else { - child = tracer.StartSpan("start send pulsar msg") - } - child.SetTag("hash keys", v.Msgs[i].HashKeys()) - child.SetTag("start time", v.Msgs[i].BeginTs()) - child.SetTag("end time", v.Msgs[i].EndTs()) - msg.Properties = make(map[string]string) - err = tracer.Inject(child.Context(), opentracing.TextMap, &propertiesReaderWriter{msg.Properties}) - if err != nil { - return err - } - } - if _, err := (*ms.producers[k]).Send( context.Background(), - msg, + &pulsar.ProducerMessage{Payload: mb}, ); err != nil { return err } - if child != nil { - child.Finish() - } } } return nil @@ -274,34 +218,10 @@ func (ms *PulsarMsgStream) Broadcast(msgPack *MsgPack) error { if err != nil { return err } - msg := &pulsar.ProducerMessage{Payload: mb} - if v.Type() == internalPb.MsgType_kInsert || v.Type() == internalPb.MsgType_kSearch { - tracer := opentracing.GlobalTracer() - ctx := v.GetContext() - if ctx == nil { - ctx = context.Background() - } - var child opentracing.Span - if parent := opentracing.SpanFromContext(ctx); parent != nil { - child = tracer.StartSpan("start send pulsar msg", - opentracing.FollowsFrom(parent.Context())) - } else { - child = tracer.StartSpan("start send pulsar msg, start time: %d") - } - child.SetTag("hash keys", v.HashKeys()) - child.SetTag("start time", v.BeginTs()) - child.SetTag("end time", v.EndTs()) - msg.Properties = make(map[string]string) - err = tracer.Inject(child.Context(), opentracing.TextMap, &propertiesReaderWriter{msg.Properties}) - if err != nil { - return err - } - child.Finish() - } for i := 0; i < producerLen; i++ { if _, err := (*ms.producers[i]).Send( context.Background(), - msg, + &pulsar.ProducerMessage{Payload: mb}, ); err != nil { return err } @@ -338,7 +258,6 @@ func (ms *PulsarMsgStream) bufMsgPackToChannel() { for { select { case <-ms.ctx.Done(): - log.Println("done") return default: tsMsgList := make([]TsMsg, 0) @@ -351,7 +270,6 @@ func (ms *PulsarMsgStream) bufMsgPackToChannel() { } pulsarMsg, ok := value.Interface().(pulsar.ConsumerMessage) - if !ok { log.Printf("type assertion failed, not consumer message type") continue @@ -365,21 +283,6 @@ func (ms *PulsarMsgStream) bufMsgPackToChannel() { continue } tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.Payload(), headerMsg.MsgType) - if tsMsg.Type() == internalPb.MsgType_kInsert || tsMsg.Type() == internalPb.MsgType_kSearch { - tracer := opentracing.GlobalTracer() - spanContext, err := tracer.Extract(opentracing.HTTPHeaders, &propertiesReaderWriter{pulsarMsg.Properties()}) - if err != nil { - log.Println("extract message err") - log.Println(err.Error()) - } - span := opentracing.StartSpan("pulsar msg received", - ext.RPCServerOption(spanContext)) - span.SetTag("hash keys", tsMsg.HashKeys()) - span.SetTag("start time", tsMsg.BeginTs()) - span.SetTag("end time", tsMsg.EndTs()) - tsMsg.SetContext(opentracing.ContextWithSpan(context.Background(), span)) - span.Finish() - } if err != nil { log.Printf("Failed to unmarshal tsMsg, error = %v", err) continue @@ -517,23 +420,6 @@ func (ms *PulsarTtMsgStream) findTimeTick(channelIndex int, if err != nil { log.Printf("Failed to unmarshal, error = %v", err) } - - if tsMsg.Type() == internalPb.MsgType_kInsert || tsMsg.Type() == internalPb.MsgType_kSearch { - tracer := opentracing.GlobalTracer() - spanContext, err := tracer.Extract(opentracing.HTTPHeaders, &propertiesReaderWriter{pulsarMsg.Properties()}) - if err != nil { - log.Println("extract message err") - log.Println(err.Error()) - } - span := opentracing.StartSpan("pulsar msg received", - ext.RPCServerOption(spanContext)) - span.SetTag("hash keys", tsMsg.HashKeys()) - span.SetTag("start time", tsMsg.BeginTs()) - span.SetTag("end time", tsMsg.EndTs()) - tsMsg.SetContext(opentracing.ContextWithSpan(context.Background(), span)) - span.Finish() - } - if headerMsg.MsgType == internalPb.MsgType_kTimeTick { eofMsgMap[channelIndex] = tsMsg.(*TimeTickMsg).Timestamp return @@ -614,7 +500,7 @@ func insertRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e result := make(map[int32]*MsgPack) for i, request := range tsMsgs { if request.Type() != internalPb.MsgType_kInsert { - return nil, errors.New("msg's must be Insert") + return nil, errors.New(string("msg's must be Insert")) } insertRequest := request.(*InsertMsg) keys := hashKeys[i] @@ -625,7 +511,7 @@ func insertRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e keysLen := len(keys) if keysLen != timestampLen || keysLen != rowIDLen || keysLen != rowDataLen { - return nil, errors.New("the length of hashValue, timestamps, rowIDs, RowData are not equal") + return nil, errors.New(string("the length of hashValue, timestamps, rowIDs, RowData are not equal")) } for index, key := range keys { _, ok := result[key] @@ -648,9 +534,6 @@ func insertRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e } insertMsg := &InsertMsg{ - BaseMsg: BaseMsg{ - ctx: request.GetContext(), - }, InsertRequest: sliceRequest, } result[key].Msgs = append(result[key].Msgs, insertMsg) @@ -663,7 +546,7 @@ func deleteRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e result := make(map[int32]*MsgPack) for i, request := range tsMsgs { if request.Type() != internalPb.MsgType_kDelete { - return nil, errors.New("msg's must be Delete") + return nil, errors.New(string("msg's must be Delete")) } deleteRequest := request.(*DeleteMsg) keys := hashKeys[i] @@ -673,7 +556,7 @@ func deleteRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e keysLen := len(keys) if keysLen != timestampLen || keysLen != primaryKeysLen { - return nil, errors.New("the length of hashValue, timestamps, primaryKeys are not equal") + return nil, errors.New(string("the length of hashValue, timestamps, primaryKeys are not equal")) } for index, key := range keys { @@ -707,7 +590,7 @@ func defaultRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, for i, request := range tsMsgs { keys := hashKeys[i] if len(keys) != 1 { - return nil, errors.New("len(msg.hashValue) must equal 1") + return nil, errors.New(string("len(msg.hashValue) must equal 1")) } key := keys[0] _, ok := result[key] diff --git a/internal/proto/internal_msg.proto b/internal/proto/internal_msg.proto index cb934231a2..6dbfb48353 100644 --- a/internal/proto/internal_msg.proto +++ b/internal/proto/internal_msg.proto @@ -272,6 +272,8 @@ message FlushMsg { MsgType msg_type = 1; int64 segmentID = 2; uint64 timestamp = 3; + int64 collectionID = 4; + string partitionTag = 5; } diff --git a/internal/proto/internalpb/internal_msg.pb.go b/internal/proto/internalpb/internal_msg.pb.go index d159b04dde..d290d5e274 100644 --- a/internal/proto/internalpb/internal_msg.pb.go +++ b/internal/proto/internalpb/internal_msg.pb.go @@ -1881,6 +1881,8 @@ type FlushMsg struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionTag string `protobuf:"bytes,5,opt,name=partitionTag,proto3" json:"partitionTag,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1932,6 +1934,20 @@ func (m *FlushMsg) GetTimestamp() uint64 { return 0 } +func (m *FlushMsg) GetCollectionID() int64 { + if m != nil { + return m.CollectionID + } + return 0 +} + +func (m *FlushMsg) GetPartitionTag() string { + if m != nil { + return m.PartitionTag + } + return "" +} + type Key2Seg struct { RowID int64 `protobuf:"varint,1,opt,name=rowID,proto3" json:"rowID,omitempty"` PrimaryKey int64 `protobuf:"varint,2,opt,name=primary_key,json=primaryKey,proto3" json:"primary_key,omitempty"` @@ -2661,121 +2677,122 @@ func init() { func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) } var fileDescriptor_7eb37f6b80b23116 = []byte{ - // 1852 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x99, 0xcd, 0x6f, 0x23, 0x49, - 0x15, 0xc0, 0xb7, 0xbb, 0xfd, 0x11, 0x3f, 0x3b, 0x4e, 0x4f, 0x25, 0x99, 0xf1, 0xec, 0x2c, 0x3b, - 0x99, 0x1e, 0xc4, 0x86, 0x45, 0x24, 0x90, 0xe1, 0xc0, 0xde, 0x20, 0xb1, 0x96, 0x35, 0x43, 0x46, - 0xa1, 0x13, 0x2d, 0x12, 0x5a, 0xa9, 0xd5, 0xb1, 0x5f, 0xec, 0x52, 0x7f, 0x39, 0x55, 0xed, 0x49, - 0x3c, 0x07, 0x24, 0xc4, 0x9c, 0x11, 0x1f, 0xe2, 0xc0, 0x8d, 0x3b, 0xec, 0x8a, 0x05, 0xf1, 0x3f, - 0xf0, 0x29, 0x24, 0xfe, 0x0b, 0x38, 0x80, 0xc4, 0x2e, 0x07, 0x6e, 0xa8, 0xaa, 0xfa, 0xc3, 0x9d, - 0xd8, 0x4e, 0xb4, 0xc9, 0x2c, 0xb3, 0xda, 0xb9, 0x75, 0x3d, 0x57, 0x57, 0xbf, 0xf7, 0x7b, 0x1f, - 0x55, 0xf5, 0x0c, 0x84, 0x86, 0x31, 0xb2, 0xd0, 0xf5, 0x9d, 0x80, 0xf7, 0x37, 0x86, 0x2c, 0x8a, - 0x23, 0xb2, 0x1a, 0x50, 0xff, 0xf1, 0x88, 0xab, 0xd1, 0x46, 0x3a, 0xe1, 0xe5, 0x46, 0x37, 0x0a, - 0x82, 0x28, 0x54, 0xe2, 0x97, 0x6f, 0x70, 0x64, 0x8f, 0x69, 0x17, 0xf3, 0xf7, 0xac, 0x10, 0x6a, - 0x9d, 0xb6, 0x8d, 0xc7, 0x23, 0xe4, 0x31, 0xb9, 0x09, 0x95, 0x21, 0x22, 0xeb, 0xb4, 0x5b, 0xda, - 0x9a, 0xb6, 0x6e, 0xd8, 0xc9, 0x88, 0x3c, 0x80, 0x12, 0x8b, 0x7c, 0x6c, 0xe9, 0x6b, 0xda, 0x7a, - 0x73, 0xeb, 0xee, 0xc6, 0xd4, 0x6f, 0x6d, 0xec, 0x21, 0x32, 0x3b, 0xf2, 0xd1, 0x96, 0x93, 0xc9, - 0x0a, 0x94, 0xbb, 0xd1, 0x28, 0x8c, 0x5b, 0xc6, 0x9a, 0xb6, 0xbe, 0x68, 0xab, 0x81, 0xd5, 0x07, - 0x10, 0xdf, 0xe3, 0xc3, 0x28, 0xe4, 0x48, 0x1e, 0x40, 0x85, 0xc7, 0x6e, 0x3c, 0xe2, 0xf2, 0x83, - 0xf5, 0xad, 0x3b, 0xc5, 0xa5, 0x13, 0xe5, 0xf7, 0xe5, 0x14, 0x3b, 0x99, 0x4a, 0x9a, 0xa0, 0x77, - 0xda, 0x52, 0x17, 0xc3, 0xd6, 0x3b, 0xed, 0x19, 0x1f, 0x8a, 0x00, 0x0e, 0x78, 0xf4, 0x31, 0x5a, - 0xf6, 0x18, 0xea, 0xf2, 0x83, 0x57, 0x31, 0xed, 0x15, 0xa8, 0xc5, 0x34, 0x40, 0x1e, 0xbb, 0xc1, - 0x50, 0xea, 0x54, 0xb2, 0x73, 0xc1, 0x8c, 0xef, 0x3e, 0xd5, 0xa0, 0xb1, 0x8f, 0xfd, 0xdc, 0x8b, - 0xd9, 0x34, 0x6d, 0x62, 0x9a, 0x58, 0xba, 0x3b, 0x70, 0xc3, 0x10, 0xfd, 0x04, 0x5e, 0xd9, 0xce, - 0x05, 0xe4, 0x0e, 0xd4, 0xba, 0x91, 0xef, 0x3b, 0xa1, 0x1b, 0xa0, 0x5c, 0xbe, 0x66, 0x2f, 0x08, - 0xc1, 0x23, 0x37, 0x40, 0x72, 0x1f, 0x16, 0x87, 0x2e, 0x8b, 0x69, 0x4c, 0xa3, 0xd0, 0x89, 0xdd, - 0x7e, 0xab, 0x24, 0x27, 0x34, 0x32, 0xe1, 0x81, 0xdb, 0xb7, 0xde, 0xd3, 0x80, 0x7c, 0x9d, 0x73, - 0xda, 0x0f, 0x0b, 0xca, 0x5c, 0x2b, 0xf8, 0x87, 0xb0, 0x34, 0x44, 0xe6, 0x24, 0x6a, 0x3b, 0x0c, - 0x8f, 0x5b, 0xc6, 0x9a, 0xb1, 0x5e, 0xdf, 0xba, 0x3f, 0xe3, 0xfd, 0x49, 0x55, 0xec, 0xc5, 0x21, - 0xb2, 0x1d, 0xf5, 0xaa, 0x8d, 0xc7, 0xd6, 0x07, 0x1a, 0x2c, 0xc9, 0xdf, 0x95, 0xd6, 0x01, 0x86, - 0x12, 0x1d, 0x17, 0xa2, 0x44, 0x59, 0x35, 0xb8, 0x00, 0xdd, 0x54, 0xaf, 0x14, 0x81, 0x96, 0x2e, - 0x02, 0x5a, 0x3e, 0x0f, 0x94, 0xdc, 0x85, 0x3a, 0x9e, 0x0e, 0x29, 0x43, 0x47, 0x44, 0x40, 0xab, - 0x22, 0xa3, 0x01, 0x94, 0xe8, 0x80, 0x06, 0x93, 0x11, 0x56, 0xbd, 0x74, 0x84, 0x59, 0x1c, 0x96, - 0x0b, 0x5e, 0x4a, 0xa2, 0xf5, 0x1d, 0xb8, 0x39, 0x49, 0xd6, 0xcd, 0x90, 0xb4, 0x34, 0x09, 0xf8, - 0x73, 0xf3, 0x00, 0xe7, 0x00, 0xed, 0x95, 0x9c, 0x71, 0x2e, 0xb5, 0xfe, 0xab, 0xc1, 0xad, 0x1d, - 0x86, 0x6e, 0x8c, 0x3b, 0x91, 0xef, 0x63, 0x57, 0x98, 0x98, 0x06, 0xc8, 0x1b, 0xb0, 0x10, 0xf0, - 0xbe, 0x13, 0x8f, 0x87, 0x28, 0xa9, 0x37, 0xb7, 0x5e, 0x9d, 0xf1, 0xad, 0x5d, 0xde, 0x3f, 0x18, - 0x0f, 0xd1, 0xae, 0x06, 0xea, 0x81, 0x58, 0xd0, 0xe8, 0x66, 0xeb, 0x65, 0x25, 0xa1, 0x20, 0x13, - 0xde, 0x61, 0x78, 0xdc, 0x69, 0x4b, 0xef, 0x18, 0xb6, 0x1a, 0x14, 0xf3, 0xac, 0x74, 0x36, 0xcf, - 0x5a, 0x50, 0x1d, 0xb2, 0xe8, 0x74, 0xdc, 0x69, 0x4b, 0xc7, 0x18, 0x76, 0x3a, 0x24, 0x5f, 0x86, - 0x0a, 0xef, 0x0e, 0x30, 0x70, 0xa5, 0x3b, 0xea, 0x5b, 0xb7, 0xa7, 0x22, 0xdf, 0xf6, 0xa3, 0x43, - 0x3b, 0x99, 0x68, 0xfd, 0x54, 0x87, 0xd5, 0x36, 0x8b, 0x86, 0x9f, 0x70, 0xcb, 0x77, 0x61, 0x29, - 0x5f, 0x5d, 0x45, 0xb5, 0x42, 0xf0, 0xd9, 0xa2, 0xce, 0xc9, 0x0e, 0xb3, 0x91, 0x9b, 0x2b, 0x22, - 0xde, 0x6e, 0x76, 0x0b, 0x63, 0xeb, 0x9f, 0x1a, 0xac, 0xbc, 0xe5, 0xf2, 0x6b, 0x85, 0x92, 0x19, - 0xac, 0xcf, 0x34, 0xd8, 0x98, 0x63, 0x70, 0xe9, 0x42, 0x83, 0xcb, 0x57, 0x30, 0xf8, 0x03, 0x0d, - 0x6e, 0xb7, 0x91, 0x77, 0x19, 0x3d, 0xc4, 0x4f, 0x8f, 0xd5, 0xbf, 0xd0, 0x60, 0x75, 0x7f, 0x10, - 0x9d, 0x3c, 0xbf, 0x16, 0x5b, 0xbf, 0xd5, 0xe1, 0xa6, 0xaa, 0x4d, 0x7b, 0x69, 0xf5, 0xfd, 0x98, - 0x12, 0x74, 0x0d, 0xea, 0x59, 0xc1, 0xcf, 0xd2, 0x74, 0x52, 0x94, 0x5b, 0x5a, 0x9a, 0x69, 0x69, - 0x79, 0x8e, 0xa5, 0x95, 0xa2, 0x6f, 0xbf, 0x09, 0xcd, 0x7c, 0xd7, 0x91, 0xae, 0x55, 0xfb, 0xc6, - 0xfd, 0xe9, 0xae, 0xcd, 0x70, 0x48, 0xcf, 0xe6, 0x1b, 0x96, 0x74, 0xec, 0xfb, 0x3a, 0xac, 0x88, - 0xaa, 0xf6, 0x82, 0xd9, 0xe5, 0x99, 0xfd, 0x43, 0x83, 0xe5, 0xb7, 0x5c, 0x7e, 0x9d, 0xc8, 0xae, - 0x37, 0xf9, 0xcf, 0x1b, 0x5b, 0xfe, 0xc8, 0xc6, 0xfe, 0x4b, 0x83, 0x56, 0x5a, 0xef, 0x3e, 0x1d, - 0x16, 0x8b, 0x2d, 0x4d, 0xd4, 0xba, 0xe7, 0xd7, 0xda, 0x6b, 0x2e, 0xee, 0xff, 0xd6, 0x61, 0xb1, - 0x13, 0x72, 0x64, 0xf1, 0x33, 0xb3, 0xf4, 0xb5, 0xf3, 0x1a, 0xab, 0xcb, 0xc9, 0x19, 0x5d, 0x2e, - 0x75, 0x45, 0x11, 0xdc, 0x38, 0xf6, 0xc5, 0x89, 0x34, 0x3b, 0xdf, 0xe4, 0x82, 0xe2, 0x29, 0x5f, - 0x95, 0x81, 0x89, 0x53, 0xfe, 0x04, 0xd5, 0x6a, 0x91, 0xea, 0xab, 0x00, 0x19, 0x7c, 0xde, 0x5a, - 0x58, 0x33, 0xc4, 0x31, 0x3d, 0x97, 0x88, 0x1b, 0x10, 0x8b, 0x4e, 0x3a, 0x6d, 0xde, 0xaa, 0xad, - 0x19, 0xe2, 0x06, 0xa4, 0x46, 0xe4, 0x2b, 0xb0, 0xc0, 0xa2, 0x13, 0xa7, 0xe7, 0xc6, 0x6e, 0x0b, - 0xe4, 0x21, 0x7b, 0xce, 0x69, 0xb2, 0xca, 0xa2, 0x93, 0xb6, 0x1b, 0xbb, 0xd6, 0x53, 0x1d, 0x16, - 0xdb, 0xe8, 0x63, 0x8c, 0xff, 0x7f, 0xe8, 0x05, 0x62, 0xa5, 0x39, 0xc4, 0xca, 0xf3, 0x88, 0x55, - 0xce, 0x11, 0xbb, 0x07, 0x8d, 0x21, 0xa3, 0x81, 0xcb, 0xc6, 0x8e, 0x87, 0x63, 0x71, 0xbd, 0x31, - 0x64, 0x95, 0x57, 0xb2, 0x87, 0x38, 0xe6, 0xd6, 0x87, 0x1a, 0x2c, 0xee, 0xa3, 0xcb, 0xba, 0x83, - 0x67, 0x86, 0x61, 0x42, 0x7f, 0xa3, 0xa8, 0xff, 0xfc, 0x33, 0xf4, 0xe7, 0xc1, 0x64, 0xc8, 0x47, - 0x7e, 0xec, 0xe4, 0x70, 0x14, 0x80, 0x25, 0x25, 0xdf, 0xc9, 0x10, 0x6d, 0x42, 0xf9, 0x78, 0x84, - 0x6c, 0x7c, 0xf1, 0x6d, 0x42, 0xcd, 0xb3, 0xfe, 0xa6, 0x81, 0xb9, 0x3f, 0xe6, 0x3b, 0x51, 0x78, - 0x44, 0xfb, 0xcf, 0x9d, 0xe5, 0x04, 0x4a, 0xd2, 0x5f, 0xe5, 0x35, 0x63, 0xbd, 0x66, 0xcb, 0x67, - 0xe1, 0x4b, 0x0f, 0xc7, 0xce, 0x90, 0xe1, 0x11, 0x3d, 0x45, 0xe5, 0xed, 0x9a, 0x5d, 0xf7, 0x70, - 0xbc, 0x97, 0x88, 0xac, 0xbf, 0xea, 0xd0, 0x48, 0x7d, 0x29, 0xf8, 0x5c, 0xc5, 0xa0, 0xfc, 0x4e, - 0xac, 0x5f, 0xbe, 0xeb, 0x32, 0xfd, 0xa6, 0x34, 0xbb, 0x8e, 0xde, 0x83, 0x86, 0x74, 0x87, 0x13, - 0x46, 0x3d, 0xcc, 0xbc, 0x5b, 0x97, 0xb2, 0x47, 0x52, 0x54, 0x04, 0x55, 0xb9, 0x4c, 0x88, 0x54, - 0xa7, 0x87, 0x08, 0x81, 0xd2, 0x80, 0xc6, 0xaa, 0xae, 0x34, 0x6c, 0xf9, 0x4c, 0xee, 0x42, 0x3d, - 0xc0, 0x98, 0xd1, 0xae, 0x42, 0x54, 0x93, 0xc9, 0x09, 0x4a, 0x24, 0x28, 0x58, 0xdf, 0x83, 0xfa, - 0x01, 0x0d, 0xf0, 0x80, 0x76, 0xbd, 0x5d, 0xde, 0xbf, 0x0a, 0xcf, 0xbc, 0x7d, 0xa3, 0x17, 0xda, - 0x37, 0x73, 0xb7, 0x20, 0xeb, 0xfb, 0x1a, 0x2c, 0xbc, 0xe9, 0x8f, 0xf8, 0xe0, 0x8a, 0x5f, 0x2f, - 0x14, 0x6c, 0x7d, 0x4a, 0xc1, 0x9e, 0xa3, 0xc3, 0xcf, 0x35, 0xa8, 0x3e, 0xc4, 0xf1, 0xd6, 0x3e, - 0xf6, 0xa5, 0x83, 0x45, 0xd1, 0x4d, 0xdb, 0x3a, 0x72, 0x20, 0x30, 0x4e, 0x94, 0x99, 0x64, 0x7d, - 0xc8, 0xab, 0xcc, 0x05, 0xfb, 0xec, 0x6d, 0x58, 0xa0, 0xdc, 0x79, 0xec, 0xfa, 0xb4, 0x27, 0x03, - 0x64, 0xc1, 0xae, 0x52, 0xfe, 0xb6, 0x18, 0x8a, 0x02, 0x97, 0xa9, 0xa9, 0xd2, 0xc1, 0xb0, 0x27, - 0x24, 0xd6, 0x3b, 0x00, 0x89, 0x6a, 0x02, 0x50, 0x16, 0x7e, 0xda, 0x64, 0xf8, 0x7d, 0x15, 0xaa, - 0x1e, 0x8e, 0xb7, 0x38, 0xf6, 0x5b, 0xba, 0xdc, 0x1d, 0x66, 0x51, 0x4b, 0x56, 0xb2, 0xd3, 0xe9, - 0xd6, 0x0f, 0x74, 0xa8, 0x7d, 0x2b, 0x72, 0x7b, 0x9d, 0xb0, 0x87, 0xa7, 0xcf, 0x14, 0xff, 0x11, - 0x45, 0xbf, 0xf7, 0x28, 0xdf, 0x20, 0x72, 0x81, 0xc8, 0x1e, 0x39, 0xc8, 0xb3, 0x27, 0x19, 0x0a, - 0xec, 0x54, 0x68, 0xe6, 0x0c, 0xdd, 0x78, 0x90, 0x16, 0x0b, 0x90, 0xa2, 0x3d, 0x21, 0x21, 0x6d, - 0x68, 0xa4, 0x13, 0x98, 0x1b, 0xa8, 0x92, 0x51, 0xdf, 0xba, 0x37, 0x35, 0x93, 0x1f, 0xe2, 0xf8, - 0x6d, 0xd7, 0x1f, 0xe1, 0x9e, 0x4b, 0x99, 0x5d, 0x4f, 0x16, 0x11, 0x6f, 0x59, 0x4f, 0x35, 0x00, - 0x49, 0x40, 0x24, 0xfb, 0xf9, 0x45, 0xb5, 0x8f, 0xb2, 0x28, 0xf9, 0x12, 0xac, 0x84, 0xa3, 0xc0, - 0x61, 0xe8, 0xbb, 0x31, 0xf6, 0x9c, 0x04, 0x06, 0x4f, 0xe0, 0x90, 0x70, 0x14, 0xd8, 0xea, 0xa7, - 0xfd, 0xe4, 0x17, 0xeb, 0x87, 0x1a, 0xc0, 0x9b, 0xc2, 0x72, 0xa5, 0xc6, 0xd9, 0x3b, 0x8e, 0x36, - 0xe5, 0x8e, 0x33, 0x81, 0x4e, 0x2f, 0xa2, 0xdb, 0x4e, 0xd1, 0x89, 0xc2, 0xc5, 0x93, 0xde, 0xe7, - 0xbd, 0x19, 0xee, 0xcc, 0x8d, 0x4f, 0xe8, 0xca, 0x67, 0xeb, 0x67, 0xaa, 0x5d, 0x2c, 0xb4, 0x53, - 0x2a, 0x15, 0xbc, 0xac, 0x9d, 0xf5, 0xb2, 0xac, 0x35, 0x41, 0xc4, 0xc6, 0x0e, 0xa7, 0x4f, 0x30, - 0x4d, 0x12, 0x25, 0xda, 0xa7, 0x4f, 0x50, 0xa4, 0x81, 0x44, 0x12, 0x9d, 0xf0, 0x74, 0xb7, 0x10, - 0x18, 0xa2, 0x13, 0x4e, 0xbe, 0x00, 0x37, 0x18, 0x76, 0x31, 0x8c, 0xfd, 0xb1, 0x13, 0x44, 0x3d, - 0x7a, 0x44, 0x31, 0x4d, 0x15, 0x33, 0xfd, 0x61, 0x37, 0x91, 0x5b, 0x7f, 0xd7, 0xa0, 0xf9, 0xed, - 0xb4, 0x82, 0x2a, 0xcd, 0x9e, 0x41, 0xdd, 0xfa, 0x9a, 0x34, 0xb6, 0xc0, 0x6f, 0x4e, 0xef, 0x38, - 0x83, 0x64, 0x2f, 0x70, 0xec, 0x2b, 0xa5, 0xb6, 0xa1, 0x2e, 0xdd, 0x91, 0xac, 0x51, 0x9a, 0xeb, - 0x83, 0xdc, 0xf3, 0x36, 0x1c, 0x65, 0xcf, 0xd6, 0x2f, 0x75, 0x20, 0xaa, 0xe7, 0x20, 0x9d, 0xf4, - 0xdc, 0x5d, 0x14, 0x5e, 0x9b, 0x7e, 0x51, 0x38, 0x7f, 0x02, 0xfc, 0x0c, 0x28, 0xb3, 0xf2, 0x86, - 0x60, 0xa1, 0x08, 0xb4, 0xa1, 0x81, 0xa7, 0x31, 0x73, 0xd3, 0xa4, 0xab, 0x5e, 0x3a, 0xe9, 0xe4, - 0x6b, 0x49, 0x26, 0xbf, 0xab, 0xc3, 0x4a, 0x7a, 0x95, 0x7c, 0xc1, 0xeb, 0x62, 0x5e, 0xbf, 0xd3, - 0xe1, 0x95, 0x02, 0xaf, 0x3d, 0x16, 0xf5, 0x19, 0x72, 0xfe, 0x82, 0xdb, 0x3c, 0x6e, 0xaf, 0xff, + // 1867 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcd, 0x6f, 0x24, 0x47, + 0x15, 0x4f, 0x77, 0xcf, 0x87, 0xe7, 0xcd, 0x78, 0xdc, 0x5b, 0xb6, 0x77, 0x67, 0xb3, 0x21, 0xeb, + 0xed, 0x45, 0xc4, 0x04, 0x61, 0x83, 0x97, 0x03, 0xb9, 0x81, 0x3d, 0x0a, 0x19, 0x16, 0xaf, 0x4c, + 0xdb, 0x0a, 0x12, 0x8a, 0xd4, 0x6a, 0xcf, 0x3c, 0xcf, 0x94, 0xfa, 0x6b, 0x5c, 0xd5, 0xb3, 0xf6, + 0xec, 0x81, 0x0b, 0x7b, 0x46, 0x7c, 0x88, 0x03, 0x37, 0xee, 0x90, 0x88, 0x80, 0xf8, 0x1f, 0x08, + 0x1f, 0x42, 0xe2, 0xbf, 0x80, 0x03, 0x48, 0x24, 0x1c, 0xb8, 0xa1, 0xaa, 0xea, 0x8f, 0x69, 0x7b, + 0x66, 0x6c, 0xc5, 0xde, 0xb0, 0x51, 0xf6, 0xd6, 0xf5, 0xba, 0xba, 0xea, 0xbd, 0xdf, 0x7b, 0xef, + 0x57, 0xf5, 0x5e, 0x03, 0xa1, 0x61, 0x8c, 0x2c, 0x74, 0x7d, 0x27, 0xe0, 0xfd, 0x8d, 0x21, 0x8b, + 0xe2, 0x88, 0xac, 0x06, 0xd4, 0x7f, 0x3c, 0xe2, 0x6a, 0xb4, 0x91, 0x4e, 0x78, 0xb9, 0xd1, 0x8d, + 0x82, 0x20, 0x0a, 0x95, 0xf8, 0xe5, 0x1b, 0x1c, 0xd9, 0x63, 0xda, 0xc5, 0xfc, 0x3b, 0x2b, 0x84, + 0x5a, 0xa7, 0x6d, 0xe3, 0xf1, 0x08, 0x79, 0x4c, 0x6e, 0x42, 0x65, 0x88, 0xc8, 0x3a, 0xed, 0x96, + 0xb6, 0xa6, 0xad, 0x1b, 0x76, 0x32, 0x22, 0x0f, 0xa0, 0xc4, 0x22, 0x1f, 0x5b, 0xfa, 0x9a, 0xb6, + 0xde, 0xdc, 0xba, 0xbb, 0x31, 0x75, 0xaf, 0x8d, 0x3d, 0x44, 0x66, 0x47, 0x3e, 0xda, 0x72, 0x32, + 0x59, 0x81, 0x72, 0x37, 0x1a, 0x85, 0x71, 0xcb, 0x58, 0xd3, 0xd6, 0x17, 0x6d, 0x35, 0xb0, 0xfa, + 0x00, 0x62, 0x3f, 0x3e, 0x8c, 0x42, 0x8e, 0xe4, 0x01, 0x54, 0x78, 0xec, 0xc6, 0x23, 0x2e, 0x37, + 0xac, 0x6f, 0xdd, 0x29, 0x2e, 0x9d, 0x28, 0xbf, 0x2f, 0xa7, 0xd8, 0xc9, 0x54, 0xd2, 0x04, 0xbd, + 0xd3, 0x96, 0xba, 0x18, 0xb6, 0xde, 0x69, 0xcf, 0xd8, 0x28, 0x02, 0x38, 0xe0, 0xd1, 0x27, 0x68, + 0xd9, 0x63, 0xa8, 0xcb, 0x0d, 0xaf, 0x62, 0xda, 0x2b, 0x50, 0x8b, 0x69, 0x80, 0x3c, 0x76, 0x83, + 0xa1, 0xd4, 0xa9, 0x64, 0xe7, 0x82, 0x19, 0xfb, 0x3e, 0xd5, 0xa0, 0xb1, 0x8f, 0xfd, 0xdc, 0x8b, + 0xd9, 0x34, 0x6d, 0x62, 0x9a, 0x58, 0xba, 0x3b, 0x70, 0xc3, 0x10, 0xfd, 0x04, 0xbc, 0xb2, 0x9d, + 0x0b, 0xc8, 0x1d, 0xa8, 0x75, 0x23, 0xdf, 0x77, 0x42, 0x37, 0x40, 0xb9, 0x7c, 0xcd, 0x5e, 0x10, + 0x82, 0x47, 0x6e, 0x80, 0xe4, 0x3e, 0x2c, 0x0e, 0x5d, 0x16, 0xd3, 0x98, 0x46, 0xa1, 0x13, 0xbb, + 0xfd, 0x56, 0x49, 0x4e, 0x68, 0x64, 0xc2, 0x03, 0xb7, 0x6f, 0xbd, 0xa7, 0x01, 0xf9, 0x26, 0xe7, + 0xb4, 0x1f, 0x16, 0x94, 0xb9, 0x56, 0xe0, 0x1f, 0xc2, 0xd2, 0x10, 0x99, 0x93, 0xa8, 0xed, 0x30, + 0x3c, 0x6e, 0x19, 0x6b, 0xc6, 0x7a, 0x7d, 0xeb, 0xfe, 0x8c, 0xef, 0x27, 0x55, 0xb1, 0x17, 0x87, + 0xc8, 0x76, 0xd4, 0xa7, 0x36, 0x1e, 0x5b, 0x1f, 0x6a, 0xb0, 0x24, 0xdf, 0x2b, 0xad, 0x03, 0x0c, + 0x25, 0x74, 0x5c, 0x88, 0x12, 0x65, 0xd5, 0xe0, 0x02, 0xe8, 0xa6, 0x7a, 0xa5, 0x08, 0x68, 0xe9, + 0x22, 0x40, 0xcb, 0xe7, 0x01, 0x25, 0x77, 0xa1, 0x8e, 0xa7, 0x43, 0xca, 0xd0, 0x11, 0x11, 0xd0, + 0xaa, 0xc8, 0x68, 0x00, 0x25, 0x3a, 0xa0, 0xc1, 0x64, 0x84, 0x55, 0x2f, 0x1d, 0x61, 0x16, 0x87, + 0xe5, 0x82, 0x97, 0x92, 0x68, 0x7d, 0x07, 0x6e, 0x4e, 0x22, 0xeb, 0x66, 0x90, 0xb4, 0x34, 0x09, + 0xf0, 0x17, 0xe6, 0x01, 0x9c, 0x03, 0x68, 0xaf, 0xe4, 0x18, 0xe7, 0x52, 0xeb, 0xbf, 0x1a, 0xdc, + 0xda, 0x61, 0xe8, 0xc6, 0xb8, 0x13, 0xf9, 0x3e, 0x76, 0x85, 0x89, 0x69, 0x80, 0xbc, 0x01, 0x0b, + 0x01, 0xef, 0x3b, 0xf1, 0x78, 0x88, 0x12, 0xf5, 0xe6, 0xd6, 0xab, 0x33, 0xf6, 0xda, 0xe5, 0xfd, + 0x83, 0xf1, 0x10, 0xed, 0x6a, 0xa0, 0x1e, 0x88, 0x05, 0x8d, 0x6e, 0xb6, 0x5e, 0x46, 0x09, 0x05, + 0x99, 0xf0, 0x0e, 0xc3, 0xe3, 0x4e, 0x5b, 0x7a, 0xc7, 0xb0, 0xd5, 0xa0, 0x98, 0x67, 0xa5, 0xb3, + 0x79, 0xd6, 0x82, 0xea, 0x90, 0x45, 0xa7, 0xe3, 0x4e, 0x5b, 0x3a, 0xc6, 0xb0, 0xd3, 0x21, 0xf9, + 0x2a, 0x54, 0x78, 0x77, 0x80, 0x81, 0x2b, 0xdd, 0x51, 0xdf, 0xba, 0x3d, 0x15, 0xf2, 0x6d, 0x3f, + 0x3a, 0xb4, 0x93, 0x89, 0xd6, 0xcf, 0x74, 0x58, 0x6d, 0xb3, 0x68, 0xf8, 0x29, 0xb7, 0x7c, 0x17, + 0x96, 0xf2, 0xd5, 0x55, 0x54, 0x2b, 0x08, 0x3e, 0x5f, 0xd4, 0x39, 0x39, 0x61, 0x36, 0x72, 0x73, + 0x45, 0xc4, 0xdb, 0xcd, 0x6e, 0x61, 0x6c, 0xfd, 0x53, 0x83, 0x95, 0xb7, 0x5c, 0x7e, 0xad, 0xa0, + 0x64, 0x06, 0xeb, 0x33, 0x0d, 0x36, 0xe6, 0x18, 0x5c, 0xba, 0xd0, 0xe0, 0xf2, 0x15, 0x0c, 0xfe, + 0x50, 0x83, 0xdb, 0x6d, 0xe4, 0x5d, 0x46, 0x0f, 0xf1, 0xb3, 0x63, 0xf5, 0x2f, 0x35, 0x58, 0xdd, + 0x1f, 0x44, 0x27, 0xcf, 0xaf, 0xc5, 0xd6, 0xef, 0x74, 0xb8, 0xa9, 0xb8, 0x69, 0x2f, 0x65, 0xdf, + 0x4f, 0x28, 0x41, 0xd7, 0xa0, 0x9e, 0x11, 0x7e, 0x96, 0xa6, 0x93, 0xa2, 0xdc, 0xd2, 0xd2, 0x4c, + 0x4b, 0xcb, 0x73, 0x2c, 0xad, 0x14, 0x7d, 0xfb, 0x6d, 0x68, 0xe6, 0xa7, 0x8e, 0x74, 0xad, 0x3a, + 0x37, 0xee, 0x4f, 0x77, 0x6d, 0x06, 0x87, 0xf4, 0x6c, 0x7e, 0x60, 0x49, 0xc7, 0xbe, 0xaf, 0xc3, + 0x8a, 0x60, 0xb5, 0x17, 0x98, 0x5d, 0x1e, 0xb3, 0x7f, 0x68, 0xb0, 0xfc, 0x96, 0xcb, 0xaf, 0x13, + 0xb2, 0xeb, 0x4d, 0xfe, 0xf3, 0xc6, 0x96, 0x3f, 0xb6, 0xb1, 0xff, 0xd2, 0xa0, 0x95, 0xf2, 0xdd, + 0x67, 0xc3, 0x62, 0x71, 0xa4, 0x09, 0xae, 0x7b, 0x7e, 0xad, 0xbd, 0x66, 0x72, 0xff, 0xb7, 0x0e, + 0x8b, 0x9d, 0x90, 0x23, 0x8b, 0x9f, 0x99, 0xa5, 0xaf, 0x9d, 0xd7, 0x58, 0x15, 0x27, 0x67, 0x74, + 0xb9, 0x54, 0x89, 0x22, 0x70, 0xe3, 0xd8, 0x17, 0x37, 0xd2, 0xec, 0x7e, 0x93, 0x0b, 0x8a, 0xb7, + 0x7c, 0x45, 0x03, 0x13, 0xb7, 0xfc, 0x09, 0x54, 0xab, 0x45, 0x54, 0x5f, 0x05, 0xc8, 0xc0, 0xe7, + 0xad, 0x85, 0x35, 0x43, 0x5c, 0xd3, 0x73, 0x89, 0xa8, 0x80, 0x58, 0x74, 0xd2, 0x69, 0xf3, 0x56, + 0x6d, 0xcd, 0x10, 0x15, 0x90, 0x1a, 0x91, 0xaf, 0xc1, 0x02, 0x8b, 0x4e, 0x9c, 0x9e, 0x1b, 0xbb, + 0x2d, 0x90, 0x97, 0xec, 0x39, 0xb7, 0xc9, 0x2a, 0x8b, 0x4e, 0xda, 0x6e, 0xec, 0x5a, 0x4f, 0x75, + 0x58, 0x6c, 0xa3, 0x8f, 0x31, 0xfe, 0xff, 0x41, 0x2f, 0x20, 0x56, 0x9a, 0x83, 0x58, 0x79, 0x1e, + 0x62, 0x95, 0x73, 0x88, 0xdd, 0x83, 0xc6, 0x90, 0xd1, 0xc0, 0x65, 0x63, 0xc7, 0xc3, 0xb1, 0x28, + 0x6f, 0x0c, 0xc9, 0xf2, 0x4a, 0xf6, 0x10, 0xc7, 0xdc, 0xfa, 0x48, 0x83, 0xc5, 0x7d, 0x74, 0x59, + 0x77, 0xf0, 0xcc, 0x60, 0x98, 0xd0, 0xdf, 0x28, 0xea, 0x3f, 0xff, 0x0e, 0xfd, 0x45, 0x30, 0x19, + 0xf2, 0x91, 0x1f, 0x3b, 0x39, 0x38, 0x0a, 0x80, 0x25, 0x25, 0xdf, 0xc9, 0x20, 0xda, 0x84, 0xf2, + 0xf1, 0x08, 0xd9, 0xf8, 0xe2, 0x6a, 0x42, 0xcd, 0xb3, 0xfe, 0xa6, 0x81, 0xb9, 0x3f, 0xe6, 0x3b, + 0x51, 0x78, 0x44, 0xfb, 0xcf, 0x9d, 0xe5, 0x04, 0x4a, 0xd2, 0x5f, 0xe5, 0x35, 0x63, 0xbd, 0x66, + 0xcb, 0x67, 0xe1, 0x4b, 0x0f, 0xc7, 0xce, 0x90, 0xe1, 0x11, 0x3d, 0x45, 0xe5, 0xed, 0x9a, 0x5d, + 0xf7, 0x70, 0xbc, 0x97, 0x88, 0xac, 0xbf, 0xea, 0xd0, 0x48, 0x7d, 0x29, 0xf0, 0xb9, 0x8a, 0x41, + 0x79, 0x4d, 0xac, 0x5f, 0xbe, 0xeb, 0x32, 0xbd, 0x52, 0x9a, 0xcd, 0xa3, 0xf7, 0xa0, 0x21, 0xdd, + 0xe1, 0x84, 0x51, 0x0f, 0x33, 0xef, 0xd6, 0xa5, 0xec, 0x91, 0x14, 0x15, 0x81, 0xaa, 0x5c, 0x26, + 0x44, 0xaa, 0xd3, 0x43, 0x84, 0x40, 0x69, 0x40, 0x63, 0xc5, 0x2b, 0x0d, 0x5b, 0x3e, 0x93, 0xbb, + 0x50, 0x0f, 0x30, 0x66, 0xb4, 0xab, 0x20, 0xaa, 0xc9, 0xe4, 0x04, 0x25, 0x12, 0x28, 0x58, 0x3f, + 0x80, 0xfa, 0x01, 0x0d, 0xf0, 0x80, 0x76, 0xbd, 0x5d, 0xde, 0xbf, 0x0a, 0x9e, 0x79, 0xfb, 0x46, + 0x2f, 0xb4, 0x6f, 0xe6, 0x1e, 0x41, 0xd6, 0x07, 0x1a, 0x2c, 0xbc, 0xe9, 0x8f, 0xf8, 0xe0, 0x8a, + 0xbb, 0x17, 0x08, 0x5b, 0x9f, 0x42, 0xd8, 0x73, 0x8e, 0xc1, 0xb3, 0xb7, 0xc9, 0xd2, 0x94, 0xdb, + 0xa4, 0x05, 0x85, 0x03, 0x62, 0x5a, 0x1b, 0xc6, 0xfa, 0x85, 0x06, 0xd5, 0x87, 0x38, 0xde, 0xda, + 0xc7, 0xbe, 0x0c, 0x14, 0x41, 0xde, 0x69, 0x7b, 0x48, 0x0e, 0x84, 0x3b, 0x26, 0xe8, 0x2a, 0xd1, + 0x13, 0x72, 0xb6, 0xba, 0x40, 0xd1, 0xdb, 0xb0, 0x40, 0xb9, 0xf3, 0xd8, 0xf5, 0x69, 0x4f, 0x2a, + 0xb9, 0x60, 0x57, 0x29, 0x7f, 0x5b, 0x0c, 0x05, 0x51, 0x66, 0xe6, 0xaa, 0xb4, 0x32, 0xec, 0x09, + 0x89, 0xf5, 0x0e, 0x40, 0xa2, 0x9a, 0x00, 0x3a, 0x0b, 0x63, 0x6d, 0x32, 0x8c, 0xbf, 0x0e, 0x55, + 0x0f, 0xc7, 0x5b, 0x1c, 0xfb, 0x2d, 0x5d, 0x9e, 0x32, 0xb3, 0xd0, 0x4f, 0x56, 0xb2, 0xd3, 0xe9, + 0xd6, 0x0f, 0x75, 0xa8, 0x7d, 0x27, 0x72, 0x7b, 0x9d, 0xb0, 0x87, 0xa7, 0xcf, 0xd4, 0x8d, 0x47, + 0x14, 0xfd, 0xde, 0xa3, 0xfc, 0xa0, 0xc9, 0x05, 0x22, 0x0b, 0xe5, 0x20, 0xcf, 0xc2, 0x64, 0x28, + 0x60, 0xa7, 0x42, 0x33, 0x67, 0xe8, 0xc6, 0x83, 0x94, 0x74, 0x40, 0x8a, 0xf6, 0x84, 0x84, 0xb4, + 0xa1, 0x91, 0x4e, 0x60, 0x6e, 0xa0, 0xa8, 0xa7, 0xbe, 0x75, 0x6f, 0x2a, 0x23, 0x3c, 0xc4, 0xf1, + 0xdb, 0xae, 0x3f, 0xc2, 0x3d, 0x97, 0x32, 0xbb, 0x9e, 0x2c, 0x22, 0xbe, 0xb2, 0x9e, 0x6a, 0x00, + 0x12, 0x01, 0x41, 0x1a, 0xe7, 0x17, 0xd5, 0x3e, 0xce, 0xa2, 0xe4, 0x2b, 0xb0, 0x12, 0x8e, 0x02, + 0x87, 0xa1, 0xef, 0xc6, 0xd8, 0x73, 0x12, 0x30, 0x78, 0x02, 0x0e, 0x09, 0x47, 0x81, 0xad, 0x5e, + 0xed, 0x27, 0x6f, 0xac, 0x1f, 0x69, 0x00, 0x6f, 0x0a, 0xcb, 0x95, 0x1a, 0x67, 0xa3, 0x5b, 0x9b, + 0x12, 0xdd, 0x13, 0xd0, 0xe9, 0x45, 0xe8, 0xb6, 0x53, 0xe8, 0x04, 0x01, 0xf2, 0xa4, 0x87, 0x7a, + 0x6f, 0x86, 0x3b, 0x73, 0xe3, 0x13, 0x74, 0xe5, 0xb3, 0xf5, 0x73, 0xd5, 0x76, 0x16, 0xda, 0x29, + 0x95, 0x0a, 0x5e, 0xd6, 0xce, 0x7a, 0x59, 0x72, 0x56, 0x10, 0xb1, 0xb1, 0xc3, 0xe9, 0x13, 0x4c, + 0x93, 0x44, 0x89, 0xf6, 0xe9, 0x13, 0x14, 0x69, 0x20, 0x21, 0x89, 0x4e, 0x78, 0x7a, 0xea, 0x08, + 0x18, 0xa2, 0x13, 0x4e, 0xbe, 0x04, 0x37, 0x18, 0x76, 0x31, 0x8c, 0xfd, 0xb1, 0x13, 0x44, 0x3d, + 0x7a, 0x44, 0x31, 0x4d, 0x15, 0x33, 0x7d, 0xb1, 0x9b, 0xc8, 0xad, 0xbf, 0x6b, 0xd0, 0xfc, 0x6e, + 0xca, 0xc4, 0x4a, 0xb3, 0x67, 0xc0, 0x7f, 0xdf, 0x90, 0xc6, 0x16, 0xf0, 0x9b, 0xd3, 0x83, 0xce, + 0x40, 0xb2, 0x17, 0x38, 0xf6, 0x95, 0x52, 0xdb, 0x50, 0x97, 0xee, 0x48, 0xd6, 0x28, 0xcd, 0xf5, + 0x41, 0xee, 0x79, 0x1b, 0x8e, 0xb2, 0x67, 0xeb, 0x57, 0x3a, 0x10, 0xd5, 0xbb, 0x90, 0x4e, 0x7a, + 0xee, 0x0a, 0x8e, 0xd7, 0xa6, 0x17, 0x1c, 0xe7, 0x6f, 0x92, 0x9f, 0x03, 0x65, 0x56, 0xde, 0x58, + 0x2c, 0x90, 0x40, 0x1b, 0x1a, 0x78, 0x1a, 0x33, 0x37, 0x4d, 0xba, 0xea, 0xa5, 0x93, 0x4e, 0x7e, + 0x96, 0x64, 0xf2, 0xbb, 0x3a, 0xac, 0xa4, 0x25, 0xe9, 0x0b, 0xbc, 0x2e, 0xc6, 0xeb, 0xf7, 0x3a, + 0xbc, 0x52, 0xc0, 0x6b, 0x8f, 0x45, 0x7d, 0x86, 0x9c, 0xbf, 0xc0, 0x6d, 0x1e, 0x6e, 0xaf, 0xff, 0xc5, 0x80, 0x6a, 0x62, 0x30, 0xa9, 0x41, 0xd9, 0x7b, 0x14, 0x85, 0x68, 0xbe, 0x44, 0x56, 0xe1, - 0x86, 0x77, 0xf6, 0xcf, 0x0b, 0xb3, 0x47, 0x96, 0x61, 0xc9, 0x2b, 0xf6, 0xf5, 0x4d, 0x24, 0x04, - 0x9a, 0x5e, 0xa1, 0xad, 0x6d, 0x1e, 0x91, 0x5b, 0xb0, 0xec, 0x9d, 0xef, 0xfc, 0x9a, 0x62, 0xdf, - 0x37, 0xbd, 0x62, 0x73, 0x94, 0x9b, 0x03, 0xb9, 0xc4, 0x37, 0x30, 0xce, 0x4e, 0xf9, 0xdc, 0xa4, - 0x64, 0x15, 0x4c, 0xef, 0x4c, 0x8f, 0xd2, 0xfc, 0xbd, 0x46, 0x96, 0xa1, 0xe9, 0x15, 0x9a, 0x70, - 0xe6, 0x1f, 0x34, 0x42, 0x60, 0xd1, 0x9b, 0xec, 0x32, 0x99, 0x7f, 0xd4, 0xc8, 0x2d, 0x20, 0xde, - 0xb9, 0x66, 0x8c, 0xf9, 0x27, 0x8d, 0xac, 0xc0, 0x92, 0x57, 0xe8, 0x59, 0x70, 0xf3, 0xcf, 0x1a, - 0xb9, 0x01, 0x0d, 0x6f, 0xa2, 0x3c, 0x99, 0xbf, 0xd2, 0xd5, 0xa7, 0x26, 0x63, 0xca, 0x7c, 0x57, - 0x27, 0x77, 0xe0, 0xa6, 0x37, 0x35, 0xd0, 0xcc, 0xf7, 0x74, 0xd2, 0x80, 0xaa, 0xa7, 0xba, 0x03, - 0xe6, 0x8f, 0x0c, 0x39, 0x52, 0xd7, 0x56, 0xf3, 0xc7, 0x06, 0xa9, 0x43, 0xc5, 0x93, 0xe7, 0x43, - 0xf3, 0x27, 0xea, 0x27, 0x75, 0xfc, 0x37, 0x3f, 0x34, 0xa4, 0xfa, 0x93, 0x97, 0x01, 0xf3, 0x3f, - 0x06, 0x69, 0x42, 0xcd, 0x4b, 0xcf, 0xb3, 0xe6, 0xaf, 0x6b, 0x52, 0xeb, 0xe2, 0x56, 0x61, 0xbe, - 0x5f, 0x23, 0x4b, 0x00, 0x5e, 0x76, 0xec, 0x31, 0x7f, 0x53, 0x7b, 0xfd, 0x0d, 0x58, 0x48, 0xff, - 0x3f, 0x24, 0x00, 0x95, 0x5d, 0x97, 0xc7, 0xc8, 0xcc, 0x97, 0xc4, 0xb3, 0x8d, 0x6e, 0x0f, 0x99, - 0xa9, 0x89, 0xe7, 0xef, 0x30, 0x2a, 0xe4, 0xba, 0xf0, 0xf9, 0x9e, 0x08, 0x4c, 0xd3, 0xd8, 0x6e, - 0x7f, 0x77, 0xbb, 0x4f, 0xe3, 0xc1, 0xe8, 0x50, 0x44, 0xcd, 0xe6, 0x13, 0xea, 0xfb, 0xf4, 0x49, - 0x8c, 0xdd, 0xc1, 0xa6, 0x8a, 0xa8, 0x2f, 0xf6, 0x28, 0x8f, 0x19, 0x3d, 0x1c, 0xc5, 0xd8, 0xdb, - 0x4c, 0x93, 0x65, 0x53, 0x86, 0x59, 0x36, 0x1c, 0x1e, 0x1e, 0x56, 0xa4, 0xe4, 0xc1, 0xff, 0x02, - 0x00, 0x00, 0xff, 0xff, 0xc1, 0x89, 0xef, 0x28, 0xb0, 0x1f, 0x00, 0x00, + 0x86, 0x77, 0xf6, 0x27, 0x88, 0xd9, 0x23, 0xcb, 0xb0, 0xe4, 0x15, 0xff, 0x0f, 0x98, 0x48, 0x08, + 0x34, 0xbd, 0x42, 0x7b, 0xdc, 0x3c, 0x22, 0xb7, 0x60, 0xd9, 0x3b, 0xdf, 0x41, 0x36, 0xc5, 0xb9, + 0x6f, 0x7a, 0xc5, 0x26, 0x2b, 0x37, 0x07, 0x72, 0x89, 0x6f, 0x61, 0x9c, 0x55, 0x0b, 0xdc, 0xa4, + 0x64, 0x15, 0x4c, 0xef, 0x4c, 0xaf, 0xd3, 0xfc, 0x83, 0x46, 0x96, 0xa1, 0xe9, 0x15, 0x9a, 0x79, + 0xe6, 0x07, 0x1a, 0x21, 0xb0, 0xe8, 0x4d, 0x76, 0xab, 0xcc, 0x3f, 0x6a, 0xe4, 0x16, 0x10, 0xef, + 0x5c, 0x53, 0xc7, 0xfc, 0x93, 0x46, 0x56, 0x60, 0xc9, 0x2b, 0xf4, 0x3e, 0xb8, 0xf9, 0x67, 0x8d, + 0xdc, 0x80, 0x86, 0x37, 0x41, 0x4f, 0xe6, 0xaf, 0x75, 0xb5, 0xd5, 0x64, 0x4c, 0x99, 0xef, 0xea, + 0xe4, 0x0e, 0xdc, 0xf4, 0xa6, 0x06, 0x9a, 0xf9, 0x9e, 0x4e, 0x1a, 0x50, 0xf5, 0x54, 0x97, 0xc1, + 0xfc, 0xb1, 0x21, 0x47, 0xaa, 0xfc, 0x35, 0x7f, 0x62, 0x90, 0x3a, 0x54, 0x3c, 0x79, 0xcf, 0x34, + 0x7f, 0xaa, 0x5e, 0xa9, 0x32, 0xc2, 0xfc, 0xc8, 0x90, 0xea, 0x4f, 0x16, 0x15, 0xe6, 0x7f, 0x0c, + 0xd2, 0x84, 0x9a, 0x97, 0xde, 0x8b, 0xcd, 0xdf, 0xd4, 0xa4, 0xd6, 0xc5, 0xa3, 0xc2, 0x7c, 0xbf, + 0x46, 0x96, 0x00, 0xbc, 0xec, 0xda, 0x63, 0xfe, 0xb6, 0xf6, 0xfa, 0x1b, 0xb0, 0x90, 0xfe, 0x87, + 0x24, 0x00, 0x95, 0x5d, 0x97, 0xc7, 0xc8, 0xcc, 0x97, 0xc4, 0xb3, 0x8d, 0x6e, 0x0f, 0x99, 0xa9, + 0x89, 0xe7, 0xef, 0x31, 0x2a, 0xe4, 0xba, 0xf0, 0xf9, 0x9e, 0x08, 0x4c, 0xd3, 0xd8, 0x6e, 0x7f, + 0x7f, 0xbb, 0x4f, 0xe3, 0xc1, 0xe8, 0x50, 0x44, 0xcd, 0xe6, 0x13, 0xea, 0xfb, 0xf4, 0x49, 0x8c, + 0xdd, 0xc1, 0xa6, 0x8a, 0xa8, 0x2f, 0xf7, 0x28, 0x8f, 0x19, 0x3d, 0x1c, 0xc5, 0xd8, 0xdb, 0x4c, + 0x93, 0x65, 0x53, 0x86, 0x59, 0x36, 0x1c, 0x1e, 0x1e, 0x56, 0xa4, 0xe4, 0xc1, 0xff, 0x02, 0x00, + 0x00, 0xff, 0xff, 0xc4, 0x5d, 0xb1, 0xba, 0xf8, 0x1f, 0x00, 0x00, } diff --git a/internal/proxy/proxy.go b/internal/proxy/proxy.go index 6a312c2bc0..f4232bc822 100644 --- a/internal/proxy/proxy.go +++ b/internal/proxy/proxy.go @@ -2,8 +2,6 @@ package proxy import ( "context" - "fmt" - "io" "log" "math/rand" "net" @@ -11,10 +9,6 @@ import ( "sync" "time" - "github.com/opentracing/opentracing-go" - "github.com/uber/jaeger-client-go" - "github.com/uber/jaeger-client-go/config" - "google.golang.org/grpc" "github.com/zilliztech/milvus-distributed/internal/allocator" @@ -45,9 +39,6 @@ type Proxy struct { manipulationMsgStream *msgstream.PulsarMsgStream queryMsgStream *msgstream.PulsarMsgStream - tracer opentracing.Tracer - closer io.Closer - // Add callback functions at different stages startCallbacks []func() closeCallbacks []func() @@ -60,28 +51,11 @@ func Init() { func CreateProxy(ctx context.Context) (*Proxy, error) { rand.Seed(time.Now().UnixNano()) ctx1, cancel := context.WithCancel(ctx) - var err error p := &Proxy{ proxyLoopCtx: ctx1, proxyLoopCancel: cancel, } - cfg := &config.Configuration{ - ServiceName: "tracing", - Sampler: &config.SamplerConfig{ - Type: "const", - Param: 1, - }, - Reporter: &config.ReporterConfig{ - LogSpans: true, - }, - } - p.tracer, p.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger)) - if err != nil { - panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err)) - } - opentracing.SetGlobalTracer(p.tracer) - pulsarAddress := Params.PulsarAddress() p.queryMsgStream = msgstream.NewPulsarMsgStream(p.proxyLoopCtx, Params.MsgStreamSearchBufSize()) @@ -224,8 +198,6 @@ func (p *Proxy) stopProxyLoop() { p.tick.Close() p.proxyLoopWg.Wait() - - p.closer.Close() } // Close closes the server. diff --git a/internal/proxy/repack_func.go b/internal/proxy/repack_func.go index f8873fe12f..44139999e0 100644 --- a/internal/proxy/repack_func.go +++ b/internal/proxy/repack_func.go @@ -182,7 +182,6 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg, insertMsg := &msgstream.InsertMsg{ InsertRequest: sliceRequest, } - insertMsg.SetContext(request.GetContext()) if together { // all rows with same hash value are accumulated to only one message if len(result[key].Msgs) <= 0 { result[key].Msgs = append(result[key].Msgs, insertMsg) diff --git a/internal/proxy/task.go b/internal/proxy/task.go index d01c45f063..425cae75cf 100644 --- a/internal/proxy/task.go +++ b/internal/proxy/task.go @@ -7,9 +7,6 @@ import ( "math" "strconv" - "github.com/opentracing/opentracing-go" - oplog "github.com/opentracing/opentracing-go/log" - "github.com/golang/protobuf/proto" "github.com/zilliztech/milvus-distributed/internal/allocator" "github.com/zilliztech/milvus-distributed/internal/msgstream" @@ -77,21 +74,12 @@ func (it *InsertTask) Type() internalpb.MsgType { } func (it *InsertTask) PreExecute() error { - span := opentracing.StartSpan("InsertTask preExecute") - defer span.Finish() - it.ctx = opentracing.ContextWithSpan(it.ctx, span) - span.SetTag("hash keys", it.ReqID) - span.SetTag("start time", it.BeginTs()) collectionName := it.BaseInsertTask.CollectionName if err := ValidateCollectionName(collectionName); err != nil { - span.LogFields(oplog.Error(err)) - span.Finish() return err } partitionTag := it.BaseInsertTask.PartitionTag if err := ValidatePartitionTag(partitionTag, true); err != nil { - span.LogFields(oplog.Error(err)) - span.Finish() return err } @@ -99,36 +87,22 @@ func (it *InsertTask) PreExecute() error { } func (it *InsertTask) Execute() error { - span, ctx := opentracing.StartSpanFromContext(it.ctx, "InsertTask Execute") - defer span.Finish() - it.ctx = ctx - span.SetTag("hash keys", it.ReqID) - span.SetTag("start time", it.BeginTs()) collectionName := it.BaseInsertTask.CollectionName - span.LogFields(oplog.String("collection_name", collectionName)) if !globalMetaCache.Hit(collectionName) { err := globalMetaCache.Sync(collectionName) if err != nil { - span.LogFields(oplog.Error(err)) - span.Finish() return err } } description, err := globalMetaCache.Get(collectionName) if err != nil || description == nil { - span.LogFields(oplog.Error(err)) - span.Finish() return err } autoID := description.Schema.AutoID - span.LogFields(oplog.Bool("auto_id", autoID)) var rowIDBegin UniqueID var rowIDEnd UniqueID rowNums := len(it.BaseInsertTask.RowData) rowIDBegin, rowIDEnd, _ = it.rowIDAllocator.Alloc(uint32(rowNums)) - span.LogFields(oplog.Int("rowNums", rowNums), - oplog.Int("rowIDBegin", int(rowIDBegin)), - oplog.Int("rowIDEnd", int(rowIDEnd))) it.BaseInsertTask.RowIDs = make([]UniqueID, rowNums) for i := rowIDBegin; i < rowIDEnd; i++ { offset := i - rowIDBegin @@ -151,8 +125,6 @@ func (it *InsertTask) Execute() error { EndTs: it.EndTs(), Msgs: make([]msgstream.TsMsg, 1), } - tsMsg.SetContext(it.ctx) - span.LogFields(oplog.String("send msg", "send msg")) msgPack.Msgs[0] = tsMsg err = it.manipulationMsgStream.Produce(msgPack) @@ -166,14 +138,11 @@ func (it *InsertTask) Execute() error { if err != nil { it.result.Status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR it.result.Status.Reason = err.Error() - span.LogFields(oplog.Error(err)) } return nil } func (it *InsertTask) PostExecute() error { - span, _ := opentracing.StartSpanFromContext(it.ctx, "InsertTask postExecute") - defer span.Finish() return nil } @@ -383,38 +352,24 @@ func (qt *QueryTask) SetTs(ts Timestamp) { } func (qt *QueryTask) PreExecute() error { - span := opentracing.StartSpan("InsertTask preExecute") - defer span.Finish() - qt.ctx = opentracing.ContextWithSpan(qt.ctx, span) - span.SetTag("hash keys", qt.ReqID) - span.SetTag("start time", qt.BeginTs()) - collectionName := qt.query.CollectionName if !globalMetaCache.Hit(collectionName) { err := globalMetaCache.Sync(collectionName) if err != nil { - span.LogFields(oplog.Error(err)) - span.Finish() return err } } _, err := globalMetaCache.Get(collectionName) if err != nil { // err is not nil if collection not exists - span.LogFields(oplog.Error(err)) - span.Finish() return err } if err := ValidateCollectionName(qt.query.CollectionName); err != nil { - span.LogFields(oplog.Error(err)) - span.Finish() return err } for _, tag := range qt.query.PartitionTags { if err := ValidatePartitionTag(tag, false); err != nil { - span.LogFields(oplog.Error(err)) - span.Finish() return err } } @@ -424,8 +379,6 @@ func (qt *QueryTask) PreExecute() error { } queryBytes, err := proto.Marshal(qt.query) if err != nil { - span.LogFields(oplog.Error(err)) - span.Finish() return err } qt.Query = &commonpb.Blob{ @@ -435,10 +388,6 @@ func (qt *QueryTask) PreExecute() error { } func (qt *QueryTask) Execute() error { - span, ctx := opentracing.StartSpanFromContext(qt.ctx, "InsertTask Execute") - defer span.Finish() - span.SetTag("hash keys", qt.ReqID) - span.SetTag("start time", qt.BeginTs()) var tsMsg msgstream.TsMsg = &msgstream.SearchMsg{ SearchRequest: qt.SearchRequest, BaseMsg: msgstream.BaseMsg{ @@ -452,28 +401,20 @@ func (qt *QueryTask) Execute() error { EndTs: qt.Timestamp, Msgs: make([]msgstream.TsMsg, 1), } - tsMsg.SetContext(ctx) msgPack.Msgs[0] = tsMsg err := qt.queryMsgStream.Produce(msgPack) log.Printf("[Proxy] length of searchMsg: %v", len(msgPack.Msgs)) if err != nil { - span.LogFields(oplog.Error(err)) - span.Finish() log.Printf("[Proxy] send search request failed: %v", err) } return err } func (qt *QueryTask) PostExecute() error { - span, _ := opentracing.StartSpanFromContext(qt.ctx, "InsertTask postExecute") - span.SetTag("hash keys", qt.ReqID) - span.SetTag("start time", qt.BeginTs()) for { select { case <-qt.ctx.Done(): log.Print("wait to finish failed, timeout!") - span.LogFields(oplog.String("wait to finish failed, timeout", "wait to finish failed, timeout")) - span.Finish() return errors.New("wait to finish failed, timeout") case searchResults := <-qt.resultBuf: filterSearchResult := make([]*internalpb.SearchResult, 0) @@ -494,8 +435,6 @@ func (qt *QueryTask) PostExecute() error { Reason: filterReason, }, } - span.LogFields(oplog.Error(errors.New(filterReason))) - span.Finish() return errors.New(filterReason) } @@ -526,7 +465,6 @@ func (qt *QueryTask) PostExecute() error { Reason: filterReason, }, } - span.Finish() return nil } @@ -538,7 +476,6 @@ func (qt *QueryTask) PostExecute() error { Reason: filterReason, }, } - span.Finish() return nil } @@ -589,13 +526,10 @@ func (qt *QueryTask) PostExecute() error { reducedHitsBs, err := proto.Marshal(reducedHits) if err != nil { log.Println("marshal error") - span.LogFields(oplog.Error(err)) - span.Finish() return err } qt.result.Hits = append(qt.result.Hits, reducedHitsBs) } - span.Finish() return nil } } @@ -703,10 +637,7 @@ func (dct *DescribeCollectionTask) PreExecute() error { func (dct *DescribeCollectionTask) Execute() error { var err error dct.result, err = dct.masterClient.DescribeCollection(dct.ctx, &dct.DescribeCollectionRequest) - if err != nil { - return err - } - err = globalMetaCache.Update(dct.CollectionName.CollectionName, dct.result) + globalMetaCache.Update(dct.CollectionName.CollectionName, dct.result) return err } diff --git a/internal/querynode/load_index_service.go b/internal/querynode/load_index_service.go index cedbd50bb0..d8ae759b67 100644 --- a/internal/querynode/load_index_service.go +++ b/internal/querynode/load_index_service.go @@ -11,9 +11,6 @@ import ( "strings" "time" - "github.com/minio/minio-go/v7" - "github.com/minio/minio-go/v7/pkg/credentials" - minioKV "github.com/zilliztech/milvus-distributed/internal/kv/minio" "github.com/zilliztech/milvus-distributed/internal/msgstream" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" @@ -38,16 +35,17 @@ type loadIndexService struct { func newLoadIndexService(ctx context.Context, replica collectionReplica) *loadIndexService { ctx1, cancel := context.WithCancel(ctx) - // init minio - minioClient, err := minio.New(Params.MinioEndPoint, &minio.Options{ - Creds: credentials.NewStaticV4(Params.MinioAccessKeyID, Params.MinioSecretAccessKey, ""), - Secure: Params.MinioUseSSLStr, - }) - if err != nil { - panic(err) + option := &minioKV.Option{ + Address: Params.MinioEndPoint, + AccessKeyID: Params.MinioAccessKeyID, + SecretAccessKeyID: Params.MinioSecretAccessKey, + UseSSL: Params.MinioUseSSLStr, + CreateBucket: true, + BucketName: Params.MinioBucketName, } - MinioKV, err := minioKV.NewMinIOKV(ctx1, minioClient, Params.MinioBucketName) + // TODO: load bucketName from config + MinioKV, err := minioKV.NewMinIOKV(ctx1, option) if err != nil { panic(err) } diff --git a/internal/querynode/load_index_service_test.go b/internal/querynode/load_index_service_test.go index cb2fb8504e..000edb49df 100644 --- a/internal/querynode/load_index_service_test.go +++ b/internal/querynode/load_index_service_test.go @@ -5,8 +5,6 @@ import ( "sort" "testing" - "github.com/minio/minio-go/v7" - "github.com/minio/minio-go/v7/pkg/credentials" "github.com/stretchr/testify/assert" "github.com/zilliztech/milvus-distributed/internal/indexbuilder" @@ -68,13 +66,16 @@ func TestLoadIndexService(t *testing.T) { binarySet, err := index.Serialize() assert.Equal(t, err, nil) - //save index to minio - minioClient, err := minio.New(Params.MinioEndPoint, &minio.Options{ - Creds: credentials.NewStaticV4(Params.MinioAccessKeyID, Params.MinioSecretAccessKey, ""), - Secure: Params.MinioUseSSLStr, - }) - assert.Equal(t, err, nil) - minioKV, err := minioKV.NewMinIOKV(node.queryNodeLoopCtx, minioClient, Params.MinioBucketName) + option := &minioKV.Option{ + Address: Params.MinioEndPoint, + AccessKeyID: Params.MinioAccessKeyID, + SecretAccessKeyID: Params.MinioSecretAccessKey, + UseSSL: Params.MinioUseSSLStr, + BucketName: Params.MinioBucketName, + CreateBucket: true, + } + + minioKV, err := minioKV.NewMinIOKV(node.queryNodeLoopCtx, option) assert.Equal(t, err, nil) indexPaths := make([]string, 0) for _, index := range binarySet { diff --git a/internal/storage/internal/S3/S3_test.go b/internal/storage/internal/S3/S3_test.go deleted file mode 100644 index c565f4a5a1..0000000000 --- a/internal/storage/internal/S3/S3_test.go +++ /dev/null @@ -1,134 +0,0 @@ -package s3driver - -import ( - "context" - "testing" - - "github.com/stretchr/testify/assert" - storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -var option = storagetype.Option{BucketName: "zilliz-hz"} -var ctx = context.Background() -var client, err = NewS3Driver(ctx, option) - -func TestS3Driver_PutRowAndGetRow(t *testing.T) { - err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("bar"), []byte("djhfkjsbdfbsdughorsgsdjhgoisdgh"), "SegmentA", 2) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("bar"), []byte("123854676ershdgfsgdfk,sdhfg;sdi8"), "SegmentB", 3) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("bar1"), []byte("testkeybarorbar_1"), "SegmentC", 3) - assert.Nil(t, err) - object, _ := client.GetRow(ctx, []byte("bar"), 1) - assert.Equal(t, "abcdefghijklmnoopqrstuvwxyz", string(object)) - object, _ = client.GetRow(ctx, []byte("bar"), 2) - assert.Equal(t, "djhfkjsbdfbsdughorsgsdjhgoisdgh", string(object)) - object, _ = client.GetRow(ctx, []byte("bar"), 5) - assert.Equal(t, "123854676ershdgfsgdfk,sdhfg;sdi8", string(object)) - object, _ = client.GetRow(ctx, []byte("bar1"), 5) - assert.Equal(t, "testkeybarorbar_1", string(object)) -} - -func TestS3Driver_DeleteRow(t *testing.T) { - err = client.DeleteRow(ctx, []byte("bar"), 5) - assert.Nil(t, err) - object, _ := client.GetRow(ctx, []byte("bar"), 6) - assert.Nil(t, object) - err = client.DeleteRow(ctx, []byte("bar1"), 5) - assert.Nil(t, err) - object2, _ := client.GetRow(ctx, []byte("bar1"), 6) - assert.Nil(t, object2) -} - -func TestS3Driver_GetSegments(t *testing.T) { - err = client.PutRow(ctx, []byte("seg"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("seg"), []byte("djhfkjsbdfbsdughorsgsdjhgoisdgh"), "SegmentA", 2) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("seg"), []byte("123854676ershdgfsgdfk,sdhfg;sdi8"), "SegmentB", 3) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("seg2"), []byte("testkeybarorbar_1"), "SegmentC", 1) - assert.Nil(t, err) - - segements, err := client.GetSegments(ctx, []byte("seg"), 4) - assert.Nil(t, err) - assert.Equal(t, 2, len(segements)) - if segements[0] == "SegmentA" { - assert.Equal(t, "SegmentA", segements[0]) - assert.Equal(t, "SegmentB", segements[1]) - } else { - assert.Equal(t, "SegmentB", segements[0]) - assert.Equal(t, "SegmentA", segements[1]) - } -} - -func TestS3Driver_PutRowsAndGetRows(t *testing.T) { - keys := [][]byte{[]byte("foo"), []byte("bar")} - values := [][]byte{[]byte("The key is foo!"), []byte("The key is bar!")} - segments := []string{"segmentA", "segmentB"} - timestamps := []uint64{1, 2} - err = client.PutRows(ctx, keys, values, segments, timestamps) - assert.Nil(t, err) - - objects, err := client.GetRows(ctx, keys, timestamps) - assert.Nil(t, err) - assert.Equal(t, "The key is foo!", string(objects[0])) - assert.Equal(t, "The key is bar!", string(objects[1])) -} - -func TestS3Driver_DeleteRows(t *testing.T) { - keys := [][]byte{[]byte("foo"), []byte("bar")} - timestamps := []uint64{3, 3} - err := client.DeleteRows(ctx, keys, timestamps) - assert.Nil(t, err) - - objects, err := client.GetRows(ctx, keys, timestamps) - assert.Nil(t, err) - assert.Nil(t, objects[0]) - assert.Nil(t, objects[1]) -} - -func TestS3Driver_PutLogAndGetLog(t *testing.T) { - err = client.PutLog(ctx, []byte("insert"), []byte("This is insert log!"), 1, 11) - assert.Nil(t, err) - err = client.PutLog(ctx, []byte("delete"), []byte("This is delete log!"), 2, 10) - assert.Nil(t, err) - err = client.PutLog(ctx, []byte("update"), []byte("This is update log!"), 3, 9) - assert.Nil(t, err) - err = client.PutLog(ctx, []byte("select"), []byte("This is select log!"), 4, 8) - assert.Nil(t, err) - - channels := []int{5, 8, 9, 10, 11, 12, 13} - logValues, err := client.GetLog(ctx, 0, 5, channels) - assert.Nil(t, err) - assert.Equal(t, "This is select log!", string(logValues[0])) - assert.Equal(t, "This is update log!", string(logValues[1])) - assert.Equal(t, "This is delete log!", string(logValues[2])) - assert.Equal(t, "This is insert log!", string(logValues[3])) -} - -func TestS3Driver_Segment(t *testing.T) { - err := client.PutSegmentIndex(ctx, "segmentA", []byte("This is segmentA's index!")) - assert.Nil(t, err) - - segmentIndex, err := client.GetSegmentIndex(ctx, "segmentA") - assert.Equal(t, "This is segmentA's index!", string(segmentIndex)) - assert.Nil(t, err) - - err = client.DeleteSegmentIndex(ctx, "segmentA") - assert.Nil(t, err) -} - -func TestS3Driver_SegmentDL(t *testing.T) { - err := client.PutSegmentDL(ctx, "segmentB", []byte("This is segmentB's delete log!")) - assert.Nil(t, err) - - segmentDL, err := client.GetSegmentDL(ctx, "segmentB") - assert.Nil(t, err) - assert.Equal(t, "This is segmentB's delete log!", string(segmentDL)) - - err = client.DeleteSegmentDL(ctx, "segmentB") - assert.Nil(t, err) -} diff --git a/internal/storage/internal/S3/s3_engine.go b/internal/storage/internal/S3/s3_engine.go deleted file mode 100644 index 8034d679e7..0000000000 --- a/internal/storage/internal/S3/s3_engine.go +++ /dev/null @@ -1,173 +0,0 @@ -package s3driver - -import ( - "bytes" - "context" - "io" - - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/session" - "github.com/aws/aws-sdk-go/service/s3" - . "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -type S3Store struct { - client *s3.S3 -} - -func NewS3Store(config aws.Config) (*S3Store, error) { - sess := session.Must(session.NewSession(&config)) - service := s3.New(sess) - - return &S3Store{ - client: service, - }, nil -} - -func (s *S3Store) Put(ctx context.Context, key Key, value Value) error { - _, err := s.client.PutObjectWithContext(ctx, &s3.PutObjectInput{ - Bucket: aws.String(bucketName), - Key: aws.String(string(key)), - Body: bytes.NewReader(value), - }) - - //sess := session.Must(session.NewSessionWithOptions(session.Options{ - // SharedConfigState: session.SharedConfigEnable, - //})) - //uploader := s3manager.NewUploader(sess) - // - //_, err := uploader.Upload(&s3manager.UploadInput{ - // Bucket: aws.String(bucketName), - // Key: aws.String(string(key)), - // Body: bytes.NewReader(value), - //}) - - return err -} - -func (s *S3Store) Get(ctx context.Context, key Key) (Value, error) { - object, err := s.client.GetObjectWithContext(ctx, &s3.GetObjectInput{ - Bucket: aws.String(bucketName), - Key: aws.String(string(key)), - }) - if err != nil { - return nil, err - } - - //TODO: get size - size := 256 * 1024 - buf := make([]byte, size) - n, err := object.Body.Read(buf) - if err != nil && err != io.EOF { - return nil, err - } - return buf[:n], nil -} - -func (s *S3Store) GetByPrefix(ctx context.Context, prefix Key, keyOnly bool) ([]Key, []Value, error) { - objectsOutput, err := s.client.ListObjectsWithContext(ctx, &s3.ListObjectsInput{ - Bucket: aws.String(bucketName), - Prefix: aws.String(string(prefix)), - }) - - var objectsKeys []Key - var objectsValues []Value - - if objectsOutput != nil && err == nil { - for _, object := range objectsOutput.Contents { - objectsKeys = append(objectsKeys, []byte(*object.Key)) - if !keyOnly { - value, err := s.Get(ctx, []byte(*object.Key)) - if err != nil { - return nil, nil, err - } - objectsValues = append(objectsValues, value) - } - } - } else { - return nil, nil, err - } - - return objectsKeys, objectsValues, nil - -} - -func (s *S3Store) Scan(ctx context.Context, keyStart Key, keyEnd Key, limit int, keyOnly bool) ([]Key, []Value, error) { - var keys []Key - var values []Value - limitCount := uint(limit) - objects, err := s.client.ListObjectsWithContext(ctx, &s3.ListObjectsInput{ - Bucket: aws.String(bucketName), - Prefix: aws.String(string(keyStart)), - }) - if err == nil && objects != nil { - for _, object := range objects.Contents { - if *object.Key >= string(keyEnd) { - keys = append(keys, []byte(*object.Key)) - if !keyOnly { - value, err := s.Get(ctx, []byte(*object.Key)) - if err != nil { - return nil, nil, err - } - values = append(values, value) - } - limitCount-- - if limitCount <= 0 { - break - } - } - } - } - - return keys, values, err -} - -func (s *S3Store) Delete(ctx context.Context, key Key) error { - _, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{ - Bucket: aws.String(bucketName), - Key: aws.String(string(key)), - }) - return err -} - -func (s *S3Store) DeleteByPrefix(ctx context.Context, prefix Key) error { - - objects, err := s.client.ListObjectsWithContext(ctx, &s3.ListObjectsInput{ - Bucket: aws.String(bucketName), - Prefix: aws.String(string(prefix)), - }) - - if objects != nil && err == nil { - for _, object := range objects.Contents { - _, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{ - Bucket: aws.String(bucketName), - Key: object.Key, - }) - return err - } - } - - return nil -} - -func (s *S3Store) DeleteRange(ctx context.Context, keyStart Key, keyEnd Key) error { - - objects, err := s.client.ListObjectsWithContext(ctx, &s3.ListObjectsInput{ - Bucket: aws.String(bucketName), - Prefix: aws.String(string(keyStart)), - }) - - if objects != nil && err == nil { - for _, object := range objects.Contents { - if *object.Key > string(keyEnd) { - _, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{ - Bucket: aws.String(bucketName), - Key: object.Key, - }) - return err - } - } - } - - return nil -} diff --git a/internal/storage/internal/S3/s3_store.go b/internal/storage/internal/S3/s3_store.go deleted file mode 100644 index 19199baa54..0000000000 --- a/internal/storage/internal/S3/s3_store.go +++ /dev/null @@ -1,339 +0,0 @@ -package s3driver - -import ( - "context" - - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/endpoints" - "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec" - . "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -type S3Driver struct { - driver *S3Store -} - -var bucketName string - -func NewS3Driver(ctx context.Context, option Option) (*S3Driver, error) { - // to-do read conf - - bucketName = option.BucketName - - S3Client, err := NewS3Store(aws.Config{ - Region: aws.String(endpoints.CnNorthwest1RegionID)}) - - if err != nil { - return nil, err - } - - return &S3Driver{ - S3Client, - }, nil -} - -func (s *S3Driver) put(ctx context.Context, key Key, value Value, timestamp Timestamp, suffix string) error { - minioKey, err := codec.MvccEncode(key, timestamp, suffix) - if err != nil { - return err - } - - err = s.driver.Put(ctx, minioKey, value) - return err -} - -func (s *S3Driver) scanLE(ctx context.Context, key Key, timestamp Timestamp, keyOnly bool) ([]Timestamp, []Key, []Value, error) { - keyEnd, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - return nil, nil, nil, err - } - - keys, values, err := s.driver.Scan(ctx, key, []byte(keyEnd), -1, keyOnly) - if err != nil { - return nil, nil, nil, err - } - - var timestamps []Timestamp - for _, key := range keys { - _, timestamp, _, _ := codec.MvccDecode(key) - timestamps = append(timestamps, timestamp) - } - - return timestamps, keys, values, nil -} - -func (s *S3Driver) scanGE(ctx context.Context, key Key, timestamp Timestamp, keyOnly bool) ([]Timestamp, []Key, []Value, error) { - keyStart, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - return nil, nil, nil, err - } - - keys, values, err := s.driver.Scan(ctx, key, keyStart, -1, keyOnly) - if err != nil { - return nil, nil, nil, err - } - - var timestamps []Timestamp - for _, key := range keys { - _, timestamp, _, _ := codec.MvccDecode(key) - timestamps = append(timestamps, timestamp) - } - - return timestamps, keys, values, nil -} - -//scan(ctx context.Context, key Key, start Timestamp, end Timestamp, withValue bool) ([]Timestamp, []Key, []Value, error) -func (s *S3Driver) deleteLE(ctx context.Context, key Key, timestamp Timestamp) error { - keyEnd, err := codec.MvccEncode(key, timestamp, "delete") - if err != nil { - return err - } - err = s.driver.DeleteRange(ctx, key, keyEnd) - return err -} -func (s *S3Driver) deleteGE(ctx context.Context, key Key, timestamp Timestamp) error { - keys, _, err := s.driver.GetByPrefix(ctx, key, true) - if err != nil { - return err - } - keyStart, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - panic(err) - } - err = s.driver.DeleteRange(ctx, []byte(keyStart), keys[len(keys)-1]) - return err -} -func (s *S3Driver) deleteRange(ctx context.Context, key Key, start Timestamp, end Timestamp) error { - keyStart, err := codec.MvccEncode(key, start, "") - if err != nil { - return err - } - keyEnd, err := codec.MvccEncode(key, end, "") - if err != nil { - return err - } - err = s.driver.DeleteRange(ctx, keyStart, keyEnd) - return err -} - -func (s *S3Driver) GetRow(ctx context.Context, key Key, timestamp Timestamp) (Value, error) { - minioKey, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - return nil, err - } - - keys, values, err := s.driver.Scan(ctx, append(key, byte('_')), minioKey, 1, false) - if values == nil || keys == nil { - return nil, err - } - - _, _, suffix, err := codec.MvccDecode(keys[0]) - if err != nil { - return nil, err - } - if suffix == "delete" { - return nil, nil - } - - return values[0], err -} -func (s *S3Driver) GetRows(ctx context.Context, keys []Key, timestamps []Timestamp) ([]Value, error) { - var values []Value - for i, key := range keys { - value, err := s.GetRow(ctx, key, timestamps[i]) - if err != nil { - return nil, err - } - values = append(values, value) - } - return values, nil -} - -func (s *S3Driver) PutRow(ctx context.Context, key Key, value Value, segment string, timestamp Timestamp) error { - minioKey, err := codec.MvccEncode(key, timestamp, segment) - if err != nil { - return err - } - err = s.driver.Put(ctx, minioKey, value) - return err -} -func (s *S3Driver) PutRows(ctx context.Context, keys []Key, values []Value, segments []string, timestamps []Timestamp) error { - maxThread := 100 - batchSize := 1 - keysLength := len(keys) - - if keysLength/batchSize > maxThread { - batchSize = keysLength / maxThread - } - - batchNums := keysLength / batchSize - - if keysLength%batchSize != 0 { - batchNums = keysLength/batchSize + 1 - } - - errCh := make(chan error) - f := func(ctx2 context.Context, keys2 []Key, values2 []Value, segments2 []string, timestamps2 []Timestamp) { - for i := 0; i < len(keys2); i++ { - err := s.PutRow(ctx2, keys2[i], values2[i], segments2[i], timestamps2[i]) - errCh <- err - } - } - for i := 0; i < batchNums; i++ { - j := i - go func() { - start, end := j*batchSize, (j+1)*batchSize - if len(keys) < end { - end = len(keys) - } - f(ctx, keys[start:end], values[start:end], segments[start:end], timestamps[start:end]) - }() - } - - for i := 0; i < len(keys); i++ { - if err := <-errCh; err != nil { - return err - } - } - return nil -} - -func (s *S3Driver) GetSegments(ctx context.Context, key Key, timestamp Timestamp) ([]string, error) { - keyEnd, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - return nil, err - } - keys, _, err := s.driver.Scan(ctx, append(key, byte('_')), keyEnd, -1, true) - if err != nil { - return nil, err - } - segmentsSet := map[string]bool{} - for _, key := range keys { - _, _, segment, err := codec.MvccDecode(key) - if err != nil { - panic("must no error") - } - if segment != "delete" { - segmentsSet[segment] = true - } - } - - var segments []string - for k, v := range segmentsSet { - if v { - segments = append(segments, k) - } - } - return segments, err -} - -func (s *S3Driver) DeleteRow(ctx context.Context, key Key, timestamp Timestamp) error { - minioKey, err := codec.MvccEncode(key, timestamp, "delete") - if err != nil { - return err - } - value := []byte("0") - err = s.driver.Put(ctx, minioKey, value) - return err -} - -func (s *S3Driver) DeleteRows(ctx context.Context, keys []Key, timestamps []Timestamp) error { - maxThread := 100 - batchSize := 1 - keysLength := len(keys) - - if keysLength/batchSize > maxThread { - batchSize = keysLength / maxThread - } - - batchNums := keysLength / batchSize - - if keysLength%batchSize != 0 { - batchNums = keysLength/batchSize + 1 - } - - errCh := make(chan error) - f := func(ctx2 context.Context, keys2 []Key, timestamps2 []Timestamp) { - for i := 0; i < len(keys2); i++ { - err := s.DeleteRow(ctx2, keys2[i], timestamps2[i]) - errCh <- err - } - } - for i := 0; i < batchNums; i++ { - j := i - go func() { - start, end := j*batchSize, (j+1)*batchSize - if len(keys) < end { - end = len(keys) - } - f(ctx, keys[start:end], timestamps[start:end]) - }() - } - - for i := 0; i < len(keys); i++ { - if err := <-errCh; err != nil { - return err - } - } - return nil -} - -func (s *S3Driver) PutLog(ctx context.Context, key Key, value Value, timestamp Timestamp, channel int) error { - logKey := codec.LogEncode(key, timestamp, channel) - err := s.driver.Put(ctx, logKey, value) - return err -} - -func (s *S3Driver) GetLog(ctx context.Context, start Timestamp, end Timestamp, channels []int) ([]Value, error) { - keys, values, err := s.driver.GetByPrefix(ctx, []byte("log_"), false) - if err != nil { - return nil, err - } - - var resultValues []Value - for i, key := range keys { - _, ts, channel, err := codec.LogDecode(string(key)) - if err != nil { - return nil, err - } - if ts >= start && ts <= end { - for j := 0; j < len(channels); j++ { - if channel == channels[j] { - resultValues = append(resultValues, values[i]) - } - } - } - } - - return resultValues, nil -} - -func (s *S3Driver) GetSegmentIndex(ctx context.Context, segment string) (SegmentIndex, error) { - - return s.driver.Get(ctx, codec.SegmentEncode(segment, "index")) -} - -func (s *S3Driver) PutSegmentIndex(ctx context.Context, segment string, index SegmentIndex) error { - - return s.driver.Put(ctx, codec.SegmentEncode(segment, "index"), index) -} - -func (s *S3Driver) DeleteSegmentIndex(ctx context.Context, segment string) error { - - return s.driver.Delete(ctx, codec.SegmentEncode(segment, "index")) -} - -func (s *S3Driver) GetSegmentDL(ctx context.Context, segment string) (SegmentDL, error) { - - return s.driver.Get(ctx, codec.SegmentEncode(segment, "DL")) -} - -func (s *S3Driver) PutSegmentDL(ctx context.Context, segment string, log SegmentDL) error { - - return s.driver.Put(ctx, codec.SegmentEncode(segment, "DL"), log) -} - -func (s *S3Driver) DeleteSegmentDL(ctx context.Context, segment string) error { - - return s.driver.Delete(ctx, codec.SegmentEncode(segment, "DL")) -} diff --git a/internal/storage/internal/minio/codec/codec.go b/internal/storage/internal/minio/codec/codec.go deleted file mode 100644 index 4d2b76ee2f..0000000000 --- a/internal/storage/internal/minio/codec/codec.go +++ /dev/null @@ -1,101 +0,0 @@ -package codec - -import ( - "errors" - "fmt" -) - -func MvccEncode(key []byte, ts uint64, suffix string) ([]byte, error) { - return []byte(string(key) + "_" + fmt.Sprintf("%016x", ^ts) + "_" + suffix), nil -} - -func MvccDecode(key []byte) (string, uint64, string, error) { - if len(key) < 16 { - return "", 0, "", errors.New("insufficient bytes to decode value") - } - - suffixIndex := 0 - TSIndex := 0 - undersCount := 0 - for i := len(key) - 1; i > 0; i-- { - if key[i] == byte('_') { - undersCount++ - if undersCount == 1 { - suffixIndex = i + 1 - } - if undersCount == 2 { - TSIndex = i + 1 - break - } - } - } - if suffixIndex == 0 || TSIndex == 0 { - return "", 0, "", errors.New("key is wrong formatted") - } - - var TS uint64 - _, err := fmt.Sscanf(string(key[TSIndex:suffixIndex-1]), "%x", &TS) - TS = ^TS - if err != nil { - return "", 0, "", err - } - - return string(key[0 : TSIndex-1]), TS, string(key[suffixIndex:]), nil -} - -func LogEncode(key []byte, ts uint64, channel int) []byte { - suffix := string(key) + "_" + fmt.Sprintf("%d", channel) - logKey, err := MvccEncode([]byte("log"), ts, suffix) - if err != nil { - return nil - } - return logKey -} - -func LogDecode(logKey string) (string, uint64, int, error) { - if len(logKey) < 16 { - return "", 0, 0, errors.New("insufficient bytes to decode value") - } - - channelIndex := 0 - keyIndex := 0 - TSIndex := 0 - undersCount := 0 - - for i := len(logKey) - 1; i > 0; i-- { - if logKey[i] == '_' { - undersCount++ - if undersCount == 1 { - channelIndex = i + 1 - } - if undersCount == 2 { - keyIndex = i + 1 - } - if undersCount == 3 { - TSIndex = i + 1 - break - } - } - } - if channelIndex == 0 || TSIndex == 0 || keyIndex == 0 || logKey[:TSIndex-1] != "log" { - return "", 0, 0, errors.New("key is wrong formatted") - } - - var TS uint64 - var channel int - _, err := fmt.Sscanf(logKey[TSIndex:keyIndex-1], "%x", &TS) - if err != nil { - return "", 0, 0, err - } - TS = ^TS - - _, err = fmt.Sscanf(logKey[channelIndex:], "%d", &channel) - if err != nil { - return "", 0, 0, err - } - return logKey[keyIndex : channelIndex-1], TS, channel, nil -} - -func SegmentEncode(segment string, suffix string) []byte { - return []byte(segment + "_" + suffix) -} diff --git a/internal/storage/internal/minio/minio_store.go b/internal/storage/internal/minio/minio_store.go deleted file mode 100644 index 18e2512401..0000000000 --- a/internal/storage/internal/minio/minio_store.go +++ /dev/null @@ -1,361 +0,0 @@ -package miniodriver - -import ( - "context" - - "github.com/minio/minio-go/v7" - "github.com/minio/minio-go/v7/pkg/credentials" - "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec" - storageType "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -type MinioDriver struct { - driver *minioStore -} - -var bucketName string - -func NewMinioDriver(ctx context.Context, option storageType.Option) (*MinioDriver, error) { - // to-do read conf - var endPoint = "localhost:9000" - var accessKeyID = "testminio" - var secretAccessKey = "testminio" - var useSSL = false - - bucketName := option.BucketName - - minioClient, err := minio.New(endPoint, &minio.Options{ - Creds: credentials.NewStaticV4(accessKeyID, secretAccessKey, ""), - Secure: useSSL, - }) - - if err != nil { - return nil, err - } - - bucketExists, err := minioClient.BucketExists(ctx, bucketName) - if err != nil { - return nil, err - } - - if !bucketExists { - err = minioClient.MakeBucket(ctx, bucketName, minio.MakeBucketOptions{}) - if err != nil { - return nil, err - } - } - return &MinioDriver{ - &minioStore{ - client: minioClient, - }, - }, nil -} - -func (s *MinioDriver) put(ctx context.Context, key storageType.Key, value storageType.Value, timestamp storageType.Timestamp, suffix string) error { - minioKey, err := codec.MvccEncode(key, timestamp, suffix) - if err != nil { - return err - } - - err = s.driver.Put(ctx, minioKey, value) - return err -} - -func (s *MinioDriver) scanLE(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp, keyOnly bool) ([]storageType.Timestamp, []storageType.Key, []storageType.Value, error) { - keyEnd, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - return nil, nil, nil, err - } - - keys, values, err := s.driver.Scan(ctx, key, []byte(keyEnd), -1, keyOnly) - if err != nil { - return nil, nil, nil, err - } - - var timestamps []storageType.Timestamp - for _, key := range keys { - _, timestamp, _, _ := codec.MvccDecode(key) - timestamps = append(timestamps, timestamp) - } - - return timestamps, keys, values, nil -} - -func (s *MinioDriver) scanGE(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp, keyOnly bool) ([]storageType.Timestamp, []storageType.Key, []storageType.Value, error) { - keyStart, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - return nil, nil, nil, err - } - - keys, values, err := s.driver.Scan(ctx, key, keyStart, -1, keyOnly) - if err != nil { - return nil, nil, nil, err - } - - var timestamps []storageType.Timestamp - for _, key := range keys { - _, timestamp, _, _ := codec.MvccDecode(key) - timestamps = append(timestamps, timestamp) - } - - return timestamps, keys, values, nil -} - -//scan(ctx context.Context, key storageType.Key, start storageType.Timestamp, end storageType.Timestamp, withValue bool) ([]storageType.Timestamp, []storageType.Key, []storageType.Value, error) -func (s *MinioDriver) deleteLE(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) error { - keyEnd, err := codec.MvccEncode(key, timestamp, "delete") - if err != nil { - return err - } - err = s.driver.DeleteRange(ctx, key, keyEnd) - return err -} -func (s *MinioDriver) deleteGE(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) error { - keys, _, err := s.driver.GetByPrefix(ctx, key, true) - if err != nil { - return err - } - keyStart, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - panic(err) - } - err = s.driver.DeleteRange(ctx, keyStart, keys[len(keys)-1]) - if err != nil { - panic(err) - } - return nil -} -func (s *MinioDriver) deleteRange(ctx context.Context, key storageType.Key, start storageType.Timestamp, end storageType.Timestamp) error { - keyStart, err := codec.MvccEncode(key, start, "") - if err != nil { - return err - } - keyEnd, err := codec.MvccEncode(key, end, "") - if err != nil { - return err - } - err = s.driver.DeleteRange(ctx, keyStart, keyEnd) - return err -} - -func (s *MinioDriver) GetRow(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) (storageType.Value, error) { - minioKey, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - return nil, err - } - - keys, values, err := s.driver.Scan(ctx, append(key, byte('_')), minioKey, 1, false) - if values == nil || keys == nil { - return nil, err - } - - _, _, suffix, err := codec.MvccDecode(keys[0]) - if err != nil { - return nil, err - } - if suffix == "delete" { - return nil, nil - } - - return values[0], err -} -func (s *MinioDriver) GetRows(ctx context.Context, keys []storageType.Key, timestamps []storageType.Timestamp) ([]storageType.Value, error) { - var values []storageType.Value - for i, key := range keys { - value, err := s.GetRow(ctx, key, timestamps[i]) - if err != nil { - return nil, err - } - values = append(values, value) - } - return values, nil -} - -func (s *MinioDriver) PutRow(ctx context.Context, key storageType.Key, value storageType.Value, segment string, timestamp storageType.Timestamp) error { - minioKey, err := codec.MvccEncode(key, timestamp, segment) - if err != nil { - return err - } - err = s.driver.Put(ctx, minioKey, value) - return err -} -func (s *MinioDriver) PutRows(ctx context.Context, keys []storageType.Key, values []storageType.Value, segments []string, timestamps []storageType.Timestamp) error { - maxThread := 100 - batchSize := 1 - keysLength := len(keys) - - if keysLength/batchSize > maxThread { - batchSize = keysLength / maxThread - } - - batchNums := keysLength / batchSize - - if keysLength%batchSize != 0 { - batchNums = keysLength/batchSize + 1 - } - - errCh := make(chan error) - f := func(ctx2 context.Context, keys2 []storageType.Key, values2 []storageType.Value, segments2 []string, timestamps2 []storageType.Timestamp) { - for i := 0; i < len(keys2); i++ { - err := s.PutRow(ctx2, keys2[i], values2[i], segments2[i], timestamps2[i]) - errCh <- err - } - } - for i := 0; i < batchNums; i++ { - j := i - go func() { - start, end := j*batchSize, (j+1)*batchSize - if len(keys) < end { - end = len(keys) - } - f(ctx, keys[start:end], values[start:end], segments[start:end], timestamps[start:end]) - }() - } - - for i := 0; i < len(keys); i++ { - if err := <-errCh; err != nil { - return err - } - } - return nil -} - -func (s *MinioDriver) GetSegments(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) ([]string, error) { - keyEnd, err := codec.MvccEncode(key, timestamp, "") - if err != nil { - return nil, err - } - keys, _, err := s.driver.Scan(ctx, append(key, byte('_')), keyEnd, -1, true) - if err != nil { - return nil, err - } - segmentsSet := map[string]bool{} - for _, key := range keys { - _, _, segment, err := codec.MvccDecode(key) - if err != nil { - panic("must no error") - } - if segment != "delete" { - segmentsSet[segment] = true - } - } - - var segments []string - for k, v := range segmentsSet { - if v { - segments = append(segments, k) - } - } - return segments, err -} - -func (s *MinioDriver) DeleteRow(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) error { - minioKey, err := codec.MvccEncode(key, timestamp, "delete") - if err != nil { - return err - } - value := []byte("0") - err = s.driver.Put(ctx, minioKey, value) - return err -} - -func (s *MinioDriver) DeleteRows(ctx context.Context, keys []storageType.Key, timestamps []storageType.Timestamp) error { - maxThread := 100 - batchSize := 1 - keysLength := len(keys) - - if keysLength/batchSize > maxThread { - batchSize = keysLength / maxThread - } - - batchNums := keysLength / batchSize - - if keysLength%batchSize != 0 { - batchNums = keysLength/batchSize + 1 - } - - errCh := make(chan error) - f := func(ctx2 context.Context, keys2 []storageType.Key, timestamps2 []storageType.Timestamp) { - for i := 0; i < len(keys2); i++ { - err := s.DeleteRow(ctx2, keys2[i], timestamps2[i]) - errCh <- err - } - } - for i := 0; i < batchNums; i++ { - j := i - go func() { - start, end := j*batchSize, (j+1)*batchSize - if len(keys) < end { - end = len(keys) - } - f(ctx, keys[start:end], timestamps[start:end]) - }() - } - - for i := 0; i < len(keys); i++ { - if err := <-errCh; err != nil { - return err - } - } - return nil -} - -func (s *MinioDriver) PutLog(ctx context.Context, key storageType.Key, value storageType.Value, timestamp storageType.Timestamp, channel int) error { - logKey := codec.LogEncode(key, timestamp, channel) - err := s.driver.Put(ctx, logKey, value) - return err -} - -func (s *MinioDriver) GetLog(ctx context.Context, start storageType.Timestamp, end storageType.Timestamp, channels []int) ([]storageType.Value, error) { - keys, values, err := s.driver.GetByPrefix(ctx, []byte("log_"), false) - if err != nil { - return nil, err - } - - var resultValues []storageType.Value - for i, key := range keys { - _, ts, channel, err := codec.LogDecode(string(key)) - if err != nil { - return nil, err - } - if ts >= start && ts <= end { - for j := 0; j < len(channels); j++ { - if channel == channels[j] { - resultValues = append(resultValues, values[i]) - } - } - } - } - - return resultValues, nil -} - -func (s *MinioDriver) GetSegmentIndex(ctx context.Context, segment string) (storageType.SegmentIndex, error) { - - return s.driver.Get(ctx, codec.SegmentEncode(segment, "index")) -} - -func (s *MinioDriver) PutSegmentIndex(ctx context.Context, segment string, index storageType.SegmentIndex) error { - - return s.driver.Put(ctx, codec.SegmentEncode(segment, "index"), index) -} - -func (s *MinioDriver) DeleteSegmentIndex(ctx context.Context, segment string) error { - - return s.driver.Delete(ctx, codec.SegmentEncode(segment, "index")) -} - -func (s *MinioDriver) GetSegmentDL(ctx context.Context, segment string) (storageType.SegmentDL, error) { - - return s.driver.Get(ctx, codec.SegmentEncode(segment, "DL")) -} - -func (s *MinioDriver) PutSegmentDL(ctx context.Context, segment string, log storageType.SegmentDL) error { - - return s.driver.Put(ctx, codec.SegmentEncode(segment, "DL"), log) -} - -func (s *MinioDriver) DeleteSegmentDL(ctx context.Context, segment string) error { - - return s.driver.Delete(ctx, codec.SegmentEncode(segment, "DL")) -} diff --git a/internal/storage/internal/minio/minio_storeEngine.go b/internal/storage/internal/minio/minio_storeEngine.go deleted file mode 100644 index 64d74e8590..0000000000 --- a/internal/storage/internal/minio/minio_storeEngine.go +++ /dev/null @@ -1,130 +0,0 @@ -package miniodriver - -import ( - "bytes" - "context" - "io" - - "github.com/minio/minio-go/v7" - . "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -type minioStore struct { - client *minio.Client -} - -func (s *minioStore) Put(ctx context.Context, key Key, value Value) error { - reader := bytes.NewReader(value) - _, err := s.client.PutObject(ctx, bucketName, string(key), reader, int64(len(value)), minio.PutObjectOptions{}) - - if err != nil { - return err - } - - return err -} - -func (s *minioStore) Get(ctx context.Context, key Key) (Value, error) { - object, err := s.client.GetObject(ctx, bucketName, string(key), minio.GetObjectOptions{}) - if err != nil { - return nil, err - } - - size := 256 * 1024 - buf := make([]byte, size) - n, err := object.Read(buf) - if err != nil && err != io.EOF { - return nil, err - } - return buf[:n], nil -} - -func (s *minioStore) GetByPrefix(ctx context.Context, prefix Key, keyOnly bool) ([]Key, []Value, error) { - objects := s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(prefix)}) - - var objectsKeys []Key - var objectsValues []Value - - for object := range objects { - objectsKeys = append(objectsKeys, []byte(object.Key)) - if !keyOnly { - value, err := s.Get(ctx, []byte(object.Key)) - if err != nil { - return nil, nil, err - } - objectsValues = append(objectsValues, value) - } - } - - return objectsKeys, objectsValues, nil - -} - -func (s *minioStore) Scan(ctx context.Context, keyStart Key, keyEnd Key, limit int, keyOnly bool) ([]Key, []Value, error) { - var keys []Key - var values []Value - limitCount := uint(limit) - for object := range s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(keyStart)}) { - if object.Key >= string(keyEnd) { - keys = append(keys, []byte(object.Key)) - if !keyOnly { - value, err := s.Get(ctx, []byte(object.Key)) - if err != nil { - return nil, nil, err - } - values = append(values, value) - } - limitCount-- - if limitCount <= 0 { - break - } - } - } - - return keys, values, nil -} - -func (s *minioStore) Delete(ctx context.Context, key Key) error { - err := s.client.RemoveObject(ctx, bucketName, string(key), minio.RemoveObjectOptions{}) - return err -} - -func (s *minioStore) DeleteByPrefix(ctx context.Context, prefix Key) error { - objectsCh := make(chan minio.ObjectInfo) - - go func() { - defer close(objectsCh) - - for object := range s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(prefix)}) { - objectsCh <- object - } - }() - - for rErr := range s.client.RemoveObjects(ctx, bucketName, objectsCh, minio.RemoveObjectsOptions{GovernanceBypass: true}) { - if rErr.Err != nil { - return rErr.Err - } - } - return nil -} - -func (s *minioStore) DeleteRange(ctx context.Context, keyStart Key, keyEnd Key) error { - objectsCh := make(chan minio.ObjectInfo) - - go func() { - defer close(objectsCh) - - for object := range s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(keyStart)}) { - if object.Key <= string(keyEnd) { - objectsCh <- object - } - } - }() - - for rErr := range s.client.RemoveObjects(ctx, bucketName, objectsCh, minio.RemoveObjectsOptions{GovernanceBypass: true}) { - if rErr.Err != nil { - return rErr.Err - } - } - return nil -} diff --git a/internal/storage/internal/minio/minio_test.go b/internal/storage/internal/minio/minio_test.go deleted file mode 100644 index d98a98cfea..0000000000 --- a/internal/storage/internal/minio/minio_test.go +++ /dev/null @@ -1,134 +0,0 @@ -package miniodriver - -import ( - "context" - "testing" - - "github.com/stretchr/testify/assert" - storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -var option = storagetype.Option{BucketName: "zilliz-hz"} -var ctx = context.Background() -var client, err = NewMinioDriver(ctx, option) - -func TestMinioDriver_PutRowAndGetRow(t *testing.T) { - err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("bar"), []byte("djhfkjsbdfbsdughorsgsdjhgoisdgh"), "SegmentA", 2) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("bar"), []byte("123854676ershdgfsgdfk,sdhfg;sdi8"), "SegmentB", 3) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("bar1"), []byte("testkeybarorbar_1"), "SegmentC", 3) - assert.Nil(t, err) - object, _ := client.GetRow(ctx, []byte("bar"), 5) - assert.Equal(t, "abcdefghijklmnoopqrstuvwxyz", string(object)) - object, _ = client.GetRow(ctx, []byte("bar"), 2) - assert.Equal(t, "djhfkjsbdfbsdughorsgsdjhgoisdgh", string(object)) - object, _ = client.GetRow(ctx, []byte("bar"), 5) - assert.Equal(t, "123854676ershdgfsgdfk,sdhfg;sdi8", string(object)) - object, _ = client.GetRow(ctx, []byte("bar1"), 5) - assert.Equal(t, "testkeybarorbar_1", string(object)) -} - -func TestMinioDriver_DeleteRow(t *testing.T) { - err = client.DeleteRow(ctx, []byte("bar"), 5) - assert.Nil(t, err) - object, _ := client.GetRow(ctx, []byte("bar"), 6) - assert.Nil(t, object) - err = client.DeleteRow(ctx, []byte("bar1"), 5) - assert.Nil(t, err) - object2, _ := client.GetRow(ctx, []byte("bar1"), 6) - assert.Nil(t, object2) -} - -func TestMinioDriver_GetSegments(t *testing.T) { - err = client.PutRow(ctx, []byte("seg"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("seg"), []byte("djhfkjsbdfbsdughorsgsdjhgoisdgh"), "SegmentA", 2) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("seg"), []byte("123854676ershdgfsgdfk,sdhfg;sdi8"), "SegmentB", 3) - assert.Nil(t, err) - err = client.PutRow(ctx, []byte("seg2"), []byte("testkeybarorbar_1"), "SegmentC", 1) - assert.Nil(t, err) - - segements, err := client.GetSegments(ctx, []byte("seg"), 4) - assert.Nil(t, err) - assert.Equal(t, 2, len(segements)) - if segements[0] == "SegmentA" { - assert.Equal(t, "SegmentA", segements[0]) - assert.Equal(t, "SegmentB", segements[1]) - } else { - assert.Equal(t, "SegmentB", segements[0]) - assert.Equal(t, "SegmentA", segements[1]) - } -} - -func TestMinioDriver_PutRowsAndGetRows(t *testing.T) { - keys := [][]byte{[]byte("foo"), []byte("bar")} - values := [][]byte{[]byte("The key is foo!"), []byte("The key is bar!")} - segments := []string{"segmentA", "segmentB"} - timestamps := []uint64{1, 2} - err = client.PutRows(ctx, keys, values, segments, timestamps) - assert.Nil(t, err) - - objects, err := client.GetRows(ctx, keys, timestamps) - assert.Nil(t, err) - assert.Equal(t, "The key is foo!", string(objects[0])) - assert.Equal(t, "The key is bar!", string(objects[1])) -} - -func TestMinioDriver_DeleteRows(t *testing.T) { - keys := [][]byte{[]byte("foo"), []byte("bar")} - timestamps := []uint64{3, 3} - err := client.DeleteRows(ctx, keys, timestamps) - assert.Nil(t, err) - - objects, err := client.GetRows(ctx, keys, timestamps) - assert.Nil(t, err) - assert.Nil(t, objects[0]) - assert.Nil(t, objects[1]) -} - -func TestMinioDriver_PutLogAndGetLog(t *testing.T) { - err = client.PutLog(ctx, []byte("insert"), []byte("This is insert log!"), 1, 11) - assert.Nil(t, err) - err = client.PutLog(ctx, []byte("delete"), []byte("This is delete log!"), 2, 10) - assert.Nil(t, err) - err = client.PutLog(ctx, []byte("update"), []byte("This is update log!"), 3, 9) - assert.Nil(t, err) - err = client.PutLog(ctx, []byte("select"), []byte("This is select log!"), 4, 8) - assert.Nil(t, err) - - channels := []int{5, 8, 9, 10, 11, 12, 13} - logValues, err := client.GetLog(ctx, 0, 5, channels) - assert.Nil(t, err) - assert.Equal(t, "This is select log!", string(logValues[0])) - assert.Equal(t, "This is update log!", string(logValues[1])) - assert.Equal(t, "This is delete log!", string(logValues[2])) - assert.Equal(t, "This is insert log!", string(logValues[3])) -} - -func TestMinioDriver_Segment(t *testing.T) { - err := client.PutSegmentIndex(ctx, "segmentA", []byte("This is segmentA's index!")) - assert.Nil(t, err) - - segmentIndex, err := client.GetSegmentIndex(ctx, "segmentA") - assert.Equal(t, "This is segmentA's index!", string(segmentIndex)) - assert.Nil(t, err) - - err = client.DeleteSegmentIndex(ctx, "segmentA") - assert.Nil(t, err) -} - -func TestMinioDriver_SegmentDL(t *testing.T) { - err := client.PutSegmentDL(ctx, "segmentB", []byte("This is segmentB's delete log!")) - assert.Nil(t, err) - - segmentDL, err := client.GetSegmentDL(ctx, "segmentB") - assert.Nil(t, err) - assert.Equal(t, "This is segmentB's delete log!", string(segmentDL)) - - err = client.DeleteSegmentDL(ctx, "segmentB") - assert.Nil(t, err) -} diff --git a/internal/storage/internal/tikv/codec/codec.go b/internal/storage/internal/tikv/codec/codec.go deleted file mode 100644 index ca09296097..0000000000 --- a/internal/storage/internal/tikv/codec/codec.go +++ /dev/null @@ -1,62 +0,0 @@ -package codec - -import ( - "encoding/binary" - "errors" - - "github.com/tikv/client-go/codec" -) - -var ( - Delimiter = byte('_') - DelimiterPlusOne = Delimiter + 0x01 - DeleteMark = byte('d') - SegmentIndexMark = byte('i') - SegmentDLMark = byte('d') -) - -// EncodeKey append timestamp, delimiter, and suffix string -// to one slice key. -// Note: suffix string should not contains Delimiter -func EncodeKey(key []byte, timestamp uint64, suffix string) []byte { - //TODO: should we encode key to memory comparable - ret := EncodeDelimiter(key, Delimiter) - ret = codec.EncodeUintDesc(ret, timestamp) - return append(ret, suffix...) -} - -func DecodeKey(key []byte) ([]byte, uint64, string, error) { - if len(key) < 8 { - return nil, 0, "", errors.New("insufficient bytes to decode value") - } - - lenDeKey := 0 - for i := len(key) - 1; i > 0; i-- { - if key[i] == Delimiter { - lenDeKey = i - break - } - } - - if lenDeKey == 0 || lenDeKey+8 > len(key) { - return nil, 0, "", errors.New("insufficient bytes to decode value") - } - - tsBytes := key[lenDeKey+1 : lenDeKey+9] - ts := binary.BigEndian.Uint64(tsBytes) - suffix := string(key[lenDeKey+9:]) - key = key[:lenDeKey-1] - return key, ^ts, suffix, nil -} - -// EncodeDelimiter append a delimiter byte to slice b, and return the appended slice. -func EncodeDelimiter(b []byte, delimiter byte) []byte { - return append(b, delimiter) -} - -func EncodeSegment(segName []byte, segType byte) []byte { - segmentKey := []byte("segment") - segmentKey = append(segmentKey, Delimiter) - segmentKey = append(segmentKey, segName...) - return append(segmentKey, Delimiter, segType) -} diff --git a/internal/storage/internal/tikv/tikv_store.go b/internal/storage/internal/tikv/tikv_store.go deleted file mode 100644 index 5ecf8936f6..0000000000 --- a/internal/storage/internal/tikv/tikv_store.go +++ /dev/null @@ -1,389 +0,0 @@ -package tikvdriver - -import ( - "context" - "errors" - "strconv" - "strings" - - "github.com/tikv/client-go/config" - "github.com/tikv/client-go/rawkv" - . "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec" - . "github.com/zilliztech/milvus-distributed/internal/storage/type" - storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -func keyAddOne(key Key) Key { - if key == nil { - return nil - } - lenKey := len(key) - ret := make(Key, lenKey) - copy(ret, key) - ret[lenKey-1] += 0x01 - return ret -} - -type tikvEngine struct { - client *rawkv.Client - conf config.Config -} - -func (e tikvEngine) Put(ctx context.Context, key Key, value Value) error { - return e.client.Put(ctx, key, value) -} - -func (e tikvEngine) BatchPut(ctx context.Context, keys []Key, values []Value) error { - return e.client.BatchPut(ctx, keys, values) -} - -func (e tikvEngine) Get(ctx context.Context, key Key) (Value, error) { - return e.client.Get(ctx, key) -} - -func (e tikvEngine) GetByPrefix(ctx context.Context, prefix Key, keyOnly bool) (keys []Key, values []Value, err error) { - startKey := prefix - endKey := keyAddOne(prefix) - limit := e.conf.Raw.MaxScanLimit - for { - ks, vs, err := e.Scan(ctx, startKey, endKey, limit, keyOnly) - if err != nil { - return keys, values, err - } - keys = append(keys, ks...) - values = append(values, vs...) - if len(ks) < limit { - break - } - // update the start key, and exclude the start key - startKey = append(ks[len(ks)-1], '\000') - } - return -} - -func (e tikvEngine) Scan(ctx context.Context, startKey Key, endKey Key, limit int, keyOnly bool) ([]Key, []Value, error) { - return e.client.Scan(ctx, startKey, endKey, limit, rawkv.ScanOption{KeyOnly: keyOnly}) -} - -func (e tikvEngine) Delete(ctx context.Context, key Key) error { - return e.client.Delete(ctx, key) -} - -func (e tikvEngine) DeleteByPrefix(ctx context.Context, prefix Key) error { - startKey := prefix - endKey := keyAddOne(prefix) - return e.client.DeleteRange(ctx, startKey, endKey) -} - -func (e tikvEngine) DeleteRange(ctx context.Context, startKey Key, endKey Key) error { - return e.client.DeleteRange(ctx, startKey, endKey) -} - -func (e tikvEngine) Close() error { - return e.client.Close() -} - -type TikvStore struct { - engine *tikvEngine -} - -func NewTikvStore(ctx context.Context, option storagetype.Option) (*TikvStore, error) { - - conf := config.Default() - client, err := rawkv.NewClient(ctx, []string{option.TikvAddress}, conf) - if err != nil { - return nil, err - } - return &TikvStore{ - &tikvEngine{ - client: client, - conf: conf, - }, - }, nil -} - -func (s *TikvStore) Name() string { - return "TiKV storage" -} - -func (s *TikvStore) put(ctx context.Context, key Key, value Value, timestamp Timestamp, suffix string) error { - return s.engine.Put(ctx, EncodeKey(key, timestamp, suffix), value) -} - -func (s *TikvStore) scanLE(ctx context.Context, key Key, timestamp Timestamp, keyOnly bool) ([]Timestamp, []Key, []Value, error) { - panic("implement me") -} - -func (s *TikvStore) scanGE(ctx context.Context, key Key, timestamp Timestamp, keyOnly bool) ([]Timestamp, []Key, []Value, error) { - panic("implement me") -} - -func (s *TikvStore) scan(ctx context.Context, key Key, start Timestamp, end Timestamp, keyOnly bool) ([]Timestamp, []Key, []Value, error) { - //startKey := EncodeKey(key, start, "") - //endKey := EncodeKey(EncodeDelimiter(key, DelimiterPlusOne), end, "") - //return s.engine.Scan(ctx, startKey, endKey, -1, keyOnly) - panic("implement me") -} - -func (s *TikvStore) deleteLE(ctx context.Context, key Key, timestamp Timestamp) error { - panic("implement me") -} - -func (s *TikvStore) deleteGE(ctx context.Context, key Key, timestamp Timestamp) error { - panic("implement me") -} - -func (s *TikvStore) deleteRange(ctx context.Context, key Key, start Timestamp, end Timestamp) error { - panic("implement me") -} - -func (s *TikvStore) GetRow(ctx context.Context, key Key, timestamp Timestamp) (Value, error) { - startKey := EncodeKey(key, timestamp, "") - endKey := EncodeDelimiter(key, DelimiterPlusOne) - keys, values, err := s.engine.Scan(ctx, startKey, endKey, 1, false) - if err != nil || keys == nil { - return nil, err - } - _, _, suffix, err := DecodeKey(keys[0]) - if err != nil { - return nil, err - } - // key is marked deleted - if suffix == string(DeleteMark) { - return nil, nil - } - return values[0], nil -} - -// TODO: how to spilt keys to some batches -var batchSize = 100 - -type kvPair struct { - key Key - value Value - err error -} - -func batchKeys(keys []Key) [][]Key { - keysLen := len(keys) - numBatch := (keysLen-1)/batchSize + 1 - batches := make([][]Key, numBatch) - - for i := 0; i < numBatch; i++ { - batchStart := i * batchSize - batchEnd := batchStart + batchSize - // the last batch - if i == numBatch-1 { - batchEnd = keysLen - } - batches[i] = keys[batchStart:batchEnd] - } - return batches -} - -func (s *TikvStore) GetRows(ctx context.Context, keys []Key, timestamps []Timestamp) ([]Value, error) { - if len(keys) != len(timestamps) { - return nil, errors.New("the len of keys is not equal to the len of timestamps") - } - - batches := batchKeys(keys) - ch := make(chan kvPair, len(keys)) - ctx, cancel := context.WithCancel(ctx) - defer cancel() - - for n, b := range batches { - batch := b - numBatch := n - go func() { - for i, key := range batch { - select { - case <-ctx.Done(): - return - default: - v, err := s.GetRow(ctx, key, timestamps[numBatch*batchSize+i]) - ch <- kvPair{ - key: key, - value: v, - err: err, - } - } - } - }() - } - - var err error - var values []Value - kvMap := make(map[string]Value) - for i := 0; i < len(keys); i++ { - kv := <-ch - if kv.err != nil { - cancel() - if err == nil { - err = kv.err - } - } - kvMap[string(kv.key)] = kv.value - } - for _, key := range keys { - values = append(values, kvMap[string(key)]) - } - return values, err -} - -func (s *TikvStore) PutRow(ctx context.Context, key Key, value Value, segment string, timestamp Timestamp) error { - return s.put(ctx, key, value, timestamp, segment) -} - -func (s *TikvStore) PutRows(ctx context.Context, keys []Key, values []Value, segments []string, timestamps []Timestamp) error { - if len(keys) != len(values) { - return errors.New("the len of keys is not equal to the len of values") - } - if len(keys) != len(timestamps) { - return errors.New("the len of keys is not equal to the len of timestamps") - } - - encodedKeys := make([]Key, len(keys)) - for i, key := range keys { - encodedKeys[i] = EncodeKey(key, timestamps[i], segments[i]) - } - return s.engine.BatchPut(ctx, encodedKeys, values) -} - -func (s *TikvStore) DeleteRow(ctx context.Context, key Key, timestamp Timestamp) error { - return s.put(ctx, key, Value{0x00}, timestamp, string(DeleteMark)) -} - -func (s *TikvStore) DeleteRows(ctx context.Context, keys []Key, timestamps []Timestamp) error { - encodeKeys := make([]Key, len(keys)) - values := make([]Value, len(keys)) - for i, key := range keys { - encodeKeys[i] = EncodeKey(key, timestamps[i], string(DeleteMark)) - values[i] = Value{0x00} - } - return s.engine.BatchPut(ctx, encodeKeys, values) -} - -//func (s *TikvStore) DeleteRows(ctx context.Context, keys []Key, timestamp Timestamp) error { -// batches := batchKeys(keys) -// ch := make(chan error, len(batches)) -// ctx, cancel := context.WithCancel(ctx) -// -// for _, b := range batches { -// batch := b -// go func() { -// for _, key := range batch { -// select { -// case <-ctx.Done(): -// return -// default: -// ch <- s.DeleteRow(ctx, key, timestamp) -// } -// } -// }() -// } -// -// var err error -// for i := 0; i < len(keys); i++ { -// if e := <-ch; e != nil { -// cancel() -// if err == nil { -// err = e -// } -// } -// } -// return err -//} - -func (s *TikvStore) PutLog(ctx context.Context, key Key, value Value, timestamp Timestamp, channel int) error { - suffix := string(EncodeDelimiter(key, DelimiterPlusOne)) + strconv.Itoa(channel) - return s.put(ctx, Key("log"), value, timestamp, suffix) -} - -func (s *TikvStore) GetLog(ctx context.Context, start Timestamp, end Timestamp, channels []int) (logs []Value, err error) { - key := Key("log") - startKey := EncodeKey(key, end, "") - endKey := EncodeKey(key, start, "") - // TODO: use for loop to ensure get all keys - keys, values, err := s.engine.Scan(ctx, startKey, endKey, s.engine.conf.Raw.MaxScanLimit, false) - if err != nil || keys == nil { - return nil, err - } - - for i, key := range keys { - _, _, suffix, err := DecodeKey(key) - log := values[i] - if err != nil { - return logs, err - } - - // no channels filter - if len(channels) == 0 { - logs = append(logs, log) - } - slice := strings.Split(suffix, string(DelimiterPlusOne)) - channel, err := strconv.Atoi(slice[len(slice)-1]) - if err != nil { - panic(err) - } - for _, item := range channels { - if item == channel { - logs = append(logs, log) - break - } - } - } - return -} - -func (s *TikvStore) GetSegmentIndex(ctx context.Context, segment string) (SegmentIndex, error) { - return s.engine.Get(ctx, EncodeSegment([]byte(segment), SegmentIndexMark)) -} - -func (s *TikvStore) PutSegmentIndex(ctx context.Context, segment string, index SegmentIndex) error { - return s.engine.Put(ctx, EncodeSegment([]byte(segment), SegmentIndexMark), index) -} - -func (s *TikvStore) DeleteSegmentIndex(ctx context.Context, segment string) error { - return s.engine.Delete(ctx, EncodeSegment([]byte(segment), SegmentIndexMark)) -} - -func (s *TikvStore) GetSegmentDL(ctx context.Context, segment string) (SegmentDL, error) { - return s.engine.Get(ctx, EncodeSegment([]byte(segment), SegmentDLMark)) -} - -func (s *TikvStore) PutSegmentDL(ctx context.Context, segment string, log SegmentDL) error { - return s.engine.Put(ctx, EncodeSegment([]byte(segment), SegmentDLMark), log) -} - -func (s *TikvStore) DeleteSegmentDL(ctx context.Context, segment string) error { - return s.engine.Delete(ctx, EncodeSegment([]byte(segment), SegmentDLMark)) -} - -func (s *TikvStore) GetSegments(ctx context.Context, key Key, timestamp Timestamp) ([]string, error) { - keys, _, err := s.engine.GetByPrefix(ctx, EncodeDelimiter(key, Delimiter), true) - if err != nil { - return nil, err - } - segmentsSet := map[string]bool{} - for _, key := range keys { - _, ts, segment, err := DecodeKey(key) - if err != nil { - panic("must no error") - } - if ts <= timestamp && segment != string(DeleteMark) { - segmentsSet[segment] = true - } - } - - var segments []string - for k, v := range segmentsSet { - if v { - segments = append(segments, k) - } - } - return segments, err -} - -func (s *TikvStore) Close() error { - return s.engine.Close() -} diff --git a/internal/storage/internal/tikv/tikv_test.go b/internal/storage/internal/tikv/tikv_test.go deleted file mode 100644 index 4e69d14d2c..0000000000 --- a/internal/storage/internal/tikv/tikv_test.go +++ /dev/null @@ -1,293 +0,0 @@ -package tikvdriver - -import ( - "bytes" - "context" - "fmt" - "math" - "os" - "sort" - "testing" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - . "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec" - . "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -//var store TikvStore -var store *TikvStore -var option = Option{TikvAddress: "localhost:2379"} - -func TestMain(m *testing.M) { - store, _ = NewTikvStore(context.Background(), option) - exitCode := m.Run() - _ = store.Close() - os.Exit(exitCode) -} - -func TestTikvEngine_Prefix(t *testing.T) { - ctx := context.Background() - prefix := Key("key") - engine := store.engine - value := Value("value") - - // Put some key with same prefix - key := prefix - err := engine.Put(ctx, key, value) - require.Nil(t, err) - key = EncodeKey(prefix, 0, "") - err = engine.Put(ctx, key, value) - assert.Nil(t, err) - - // Get by prefix - ks, _, err := engine.GetByPrefix(ctx, prefix, true) - assert.Equal(t, 2, len(ks)) - assert.Nil(t, err) - - // Delete by prefix - err = engine.DeleteByPrefix(ctx, prefix) - assert.Nil(t, err) - ks, _, err = engine.GetByPrefix(ctx, prefix, true) - assert.Equal(t, 0, len(ks)) - assert.Nil(t, err) - - //Test large amount keys - num := engine.conf.Raw.MaxScanLimit + 1 - keys := make([]Key, num) - values := make([]Value, num) - for i := 0; i < num; i++ { - key = EncodeKey(prefix, uint64(i), "") - keys[i] = key - values[i] = value - } - err = engine.BatchPut(ctx, keys, values) - assert.Nil(t, err) - - ks, _, err = engine.GetByPrefix(ctx, prefix, true) - assert.Nil(t, err) - assert.Equal(t, num, len(ks)) - err = engine.DeleteByPrefix(ctx, prefix) - assert.Nil(t, err) -} - -func TestTikvStore_Row(t *testing.T) { - ctx := context.Background() - key := Key("key") - - // Add same row with different timestamp - err := store.PutRow(ctx, key, Value("value0"), "segment0", 0) - assert.Nil(t, err) - err = store.PutRow(ctx, key, Value("value1"), "segment0", 2) - assert.Nil(t, err) - - // Get most recent row using key and timestamp - v, err := store.GetRow(ctx, key, 3) - assert.Nil(t, err) - assert.Equal(t, Value("value1"), v) - v, err = store.GetRow(ctx, key, 2) - assert.Nil(t, err) - assert.Equal(t, Value("value1"), v) - v, err = store.GetRow(ctx, key, 1) - assert.Nil(t, err) - assert.Equal(t, Value("value0"), v) - - // Add a different row, but with same prefix - key1 := Key("key_y") - err = store.PutRow(ctx, key1, Value("valuey"), "segment0", 2) - assert.Nil(t, err) - - // Get most recent row using key and timestamp - v, err = store.GetRow(ctx, key, 3) - assert.Nil(t, err) - assert.Equal(t, Value("value1"), v) - v, err = store.GetRow(ctx, key1, 3) - assert.Nil(t, err) - assert.Equal(t, Value("valuey"), v) - - // Delete a row - err = store.DeleteRow(ctx, key, 4) - assert.Nil(t, err) - v, err = store.GetRow(ctx, key, 5) - assert.Nil(t, err) - assert.Nil(t, v) - - // Clear test data - err = store.engine.DeleteByPrefix(ctx, key) - assert.Nil(t, err) - k, va, err := store.engine.GetByPrefix(ctx, key, false) - assert.Nil(t, err) - assert.Nil(t, k) - assert.Nil(t, va) -} - -func TestTikvStore_BatchRow(t *testing.T) { - ctx := context.Background() - - // Prepare test data - size := 0 - var testKeys []Key - var testValues []Value - var segments []string - var timestamps []Timestamp - for i := 0; size/store.engine.conf.Raw.MaxBatchPutSize < 1; i++ { - key := fmt.Sprint("key", i) - size += len(key) - testKeys = append(testKeys, []byte(key)) - value := fmt.Sprint("value", i) - size += len(value) - testValues = append(testValues, []byte(value)) - segments = append(segments, "test") - v, err := store.GetRow(ctx, Key(key), math.MaxUint64) - assert.Nil(t, v) - assert.Nil(t, err) - } - - // Batch put rows - for range testKeys { - timestamps = append(timestamps, 1) - } - err := store.PutRows(ctx, testKeys, testValues, segments, timestamps) - assert.Nil(t, err) - - // Batch get rows - for i := range timestamps { - timestamps[i] = 2 - } - checkValues, err := store.GetRows(ctx, testKeys, timestamps) - assert.NotNil(t, checkValues) - assert.Nil(t, err) - assert.Equal(t, len(checkValues), len(testValues)) - for i := range testKeys { - assert.Equal(t, testValues[i], checkValues[i]) - } - - // Delete all test rows - for i := range timestamps { - timestamps[i] = math.MaxUint64 - } - err = store.DeleteRows(ctx, testKeys, timestamps) - assert.Nil(t, err) - // Ensure all test row is deleted - for i := range timestamps { - timestamps[i] = math.MaxUint64 - } - checkValues, err = store.GetRows(ctx, testKeys, timestamps) - assert.Nil(t, err) - for _, value := range checkValues { - assert.Nil(t, value) - } - - // Clean test data - err = store.engine.DeleteByPrefix(ctx, Key("key")) - assert.Nil(t, err) -} - -func TestTikvStore_GetSegments(t *testing.T) { - ctx := context.Background() - key := Key("key") - - // Put rows - err := store.PutRow(ctx, key, Value{0}, "a", 1) - assert.Nil(t, err) - err = store.PutRow(ctx, key, Value{0}, "a", 2) - assert.Nil(t, err) - err = store.PutRow(ctx, key, Value{0}, "c", 3) - assert.Nil(t, err) - - // Get segments - segs, err := store.GetSegments(ctx, key, 2) - assert.Nil(t, err) - assert.Equal(t, 1, len(segs)) - assert.Equal(t, "a", segs[0]) - - segs, err = store.GetSegments(ctx, key, 3) - assert.Nil(t, err) - assert.Equal(t, 2, len(segs)) - - // Clean test data - err = store.engine.DeleteByPrefix(ctx, key) - assert.Nil(t, err) -} - -func TestTikvStore_Log(t *testing.T) { - ctx := context.Background() - - // Put some log - err := store.PutLog(ctx, Key("key1"), Value("value1"), 1, 1) - assert.Nil(t, err) - err = store.PutLog(ctx, Key("key1"), Value("value1_1"), 1, 2) - assert.Nil(t, err) - err = store.PutLog(ctx, Key("key2"), Value("value2"), 2, 1) - assert.Nil(t, err) - - // Check log - log, err := store.GetLog(ctx, 0, 2, []int{1, 2}) - if err != nil { - panic(err) - } - sort.Slice(log, func(i, j int) bool { - return bytes.Compare(log[i], log[j]) == -1 - }) - assert.Equal(t, log[0], Value("value1")) - assert.Equal(t, log[1], Value("value1_1")) - assert.Equal(t, log[2], Value("value2")) - - // Delete test data - err = store.engine.DeleteByPrefix(ctx, Key("log")) - assert.Nil(t, err) -} - -func TestTikvStore_SegmentIndex(t *testing.T) { - ctx := context.Background() - - // Put segment index - err := store.PutSegmentIndex(ctx, "segment0", []byte("index0")) - assert.Nil(t, err) - err = store.PutSegmentIndex(ctx, "segment1", []byte("index1")) - assert.Nil(t, err) - - // Get segment index - index, err := store.GetSegmentIndex(ctx, "segment0") - assert.Nil(t, err) - assert.Equal(t, []byte("index0"), index) - index, err = store.GetSegmentIndex(ctx, "segment1") - assert.Nil(t, err) - assert.Equal(t, []byte("index1"), index) - - // Delete segment index - err = store.DeleteSegmentIndex(ctx, "segment0") - assert.Nil(t, err) - err = store.DeleteSegmentIndex(ctx, "segment1") - assert.Nil(t, err) - index, err = store.GetSegmentIndex(ctx, "segment0") - assert.Nil(t, err) - assert.Nil(t, index) -} - -func TestTikvStore_DeleteSegmentDL(t *testing.T) { - ctx := context.Background() - - // Put segment delete log - err := store.PutSegmentDL(ctx, "segment0", []byte("index0")) - assert.Nil(t, err) - err = store.PutSegmentDL(ctx, "segment1", []byte("index1")) - assert.Nil(t, err) - - // Get segment delete log - index, err := store.GetSegmentDL(ctx, "segment0") - assert.Nil(t, err) - assert.Equal(t, []byte("index0"), index) - index, err = store.GetSegmentDL(ctx, "segment1") - assert.Nil(t, err) - assert.Equal(t, []byte("index1"), index) - - // Delete segment delete log - err = store.DeleteSegmentDL(ctx, "segment0") - assert.Nil(t, err) - err = store.DeleteSegmentDL(ctx, "segment1") - assert.Nil(t, err) - index, err = store.GetSegmentDL(ctx, "segment0") - assert.Nil(t, err) - assert.Nil(t, index) -} diff --git a/internal/storage/storage.go b/internal/storage/storage.go deleted file mode 100644 index 67e9e44e89..0000000000 --- a/internal/storage/storage.go +++ /dev/null @@ -1,39 +0,0 @@ -package storage - -import ( - "context" - "errors" - - S3Driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3" - minIODriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio" - tikvDriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv" - storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type" -) - -func NewStore(ctx context.Context, option storagetype.Option) (storagetype.Store, error) { - var err error - var store storagetype.Store - switch option.Type { - case storagetype.TIKVDriver: - store, err = tikvDriver.NewTikvStore(ctx, option) - if err != nil { - panic(err.Error()) - } - return store, nil - case storagetype.MinIODriver: - store, err = minIODriver.NewMinioDriver(ctx, option) - if err != nil { - //panic(err.Error()) - return nil, err - } - return store, nil - case storagetype.S3DRIVER: - store, err = S3Driver.NewS3Driver(ctx, option) - if err != nil { - //panic(err.Error()) - return nil, err - } - return store, nil - } - return nil, errors.New("unsupported driver") -} diff --git a/internal/storage/type/storagetype.go b/internal/storage/type/storagetype.go deleted file mode 100644 index 9549a106e5..0000000000 --- a/internal/storage/type/storagetype.go +++ /dev/null @@ -1,79 +0,0 @@ -package storagetype - -import ( - "context" - - "github.com/zilliztech/milvus-distributed/internal/util/typeutil" -) - -type Key = []byte -type Value = []byte -type Timestamp = typeutil.Timestamp -type DriverType = string -type SegmentIndex = []byte -type SegmentDL = []byte - -type Option struct { - Type DriverType - TikvAddress string - BucketName string -} - -const ( - MinIODriver DriverType = "MinIO" - TIKVDriver DriverType = "TIKV" - S3DRIVER DriverType = "S3" -) - -/* -type Store interface { - Get(ctx context.Context, key Key, timestamp Timestamp) (Value, error) - BatchGet(ctx context.Context, keys [] Key, timestamp Timestamp) ([]Value, error) - Set(ctx context.Context, key Key, v Value, timestamp Timestamp) error - BatchSet(ctx context.Context, keys []Key, v []Value, timestamp Timestamp) error - Delete(ctx context.Context, key Key, timestamp Timestamp) error - BatchDelete(ctx context.Context, keys []Key, timestamp Timestamp) error - Close() error -} -*/ - -type storeEngine interface { - Put(ctx context.Context, key Key, value Value) error - Get(ctx context.Context, key Key) (Value, error) - GetByPrefix(ctx context.Context, prefix Key, keyOnly bool) ([]Key, []Value, error) - Scan(ctx context.Context, startKey Key, endKey Key, limit int, keyOnly bool) ([]Key, []Value, error) - Delete(ctx context.Context, key Key) error - DeleteByPrefix(ctx context.Context, prefix Key) error - DeleteRange(ctx context.Context, keyStart Key, keyEnd Key) error -} - -type Store interface { - //put(ctx context.Context, key Key, value Value, timestamp Timestamp, suffix string) error - //scanLE(ctx context.Context, key Key, timestamp Timestamp, keyOnly bool) ([]Timestamp, []Key, []Value, error) - //scanGE(ctx context.Context, key Key, timestamp Timestamp, keyOnly bool) ([]Timestamp, []Key, []Value, error) - //deleteLE(ctx context.Context, key Key, timestamp Timestamp) error - //deleteGE(ctx context.Context, key Key, timestamp Timestamp) error - //deleteRange(ctx context.Context, key Key, start Timestamp, end Timestamp) error - - GetRow(ctx context.Context, key Key, timestamp Timestamp) (Value, error) - GetRows(ctx context.Context, keys []Key, timestamps []Timestamp) ([]Value, error) - - PutRow(ctx context.Context, key Key, value Value, segment string, timestamp Timestamp) error - PutRows(ctx context.Context, keys []Key, values []Value, segments []string, timestamps []Timestamp) error - - GetSegments(ctx context.Context, key Key, timestamp Timestamp) ([]string, error) - - DeleteRow(ctx context.Context, key Key, timestamp Timestamp) error - DeleteRows(ctx context.Context, keys []Key, timestamps []Timestamp) error - - PutLog(ctx context.Context, key Key, value Value, timestamp Timestamp, channel int) error - GetLog(ctx context.Context, start Timestamp, end Timestamp, channels []int) ([]Value, error) - - GetSegmentIndex(ctx context.Context, segment string) (SegmentIndex, error) - PutSegmentIndex(ctx context.Context, segment string, index SegmentIndex) error - DeleteSegmentIndex(ctx context.Context, segment string) error - - GetSegmentDL(ctx context.Context, segment string) (SegmentDL, error) - PutSegmentDL(ctx context.Context, segment string, log SegmentDL) error - DeleteSegmentDL(ctx context.Context, segment string) error -} diff --git a/internal/util/flowgraph/input_node.go b/internal/util/flowgraph/input_node.go index b7891040e8..7c4271b23b 100644 --- a/internal/util/flowgraph/input_node.go +++ b/internal/util/flowgraph/input_node.go @@ -1,12 +1,8 @@ package flowgraph import ( - "fmt" "log" - "github.com/opentracing/opentracing-go" - "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" - "github.com/zilliztech/milvus-distributed/internal/msgstream" ) @@ -29,31 +25,10 @@ func (inNode *InputNode) InStream() *msgstream.MsgStream { } // empty input and return one *Msg -func (inNode *InputNode) Operate([]*Msg) []*Msg { +func (inNode *InputNode) Operate(in []*Msg) []*Msg { //fmt.Println("Do InputNode operation") - msgPack := (*inNode.inStream).Consume() - var childs []opentracing.Span - tracer := opentracing.GlobalTracer() - if tracer != nil && msgPack != nil { - for _, msg := range msgPack.Msgs { - if msg.Type() == internalpb.MsgType_kInsert || msg.Type() == internalpb.MsgType_kSearch { - var child opentracing.Span - ctx := msg.GetContext() - if parent := opentracing.SpanFromContext(ctx); parent != nil { - child = tracer.StartSpan(fmt.Sprintf("through msg input node, start time = %d", msg.BeginTs()), - opentracing.FollowsFrom(parent.Context())) - } else { - child = tracer.StartSpan(fmt.Sprintf("through msg input node, start time = %d", msg.BeginTs())) - } - child.SetTag("hash keys", msg.HashKeys()) - child.SetTag("start time", msg.BeginTs()) - child.SetTag("end time", msg.EndTs()) - msg.SetContext(opentracing.ContextWithSpan(ctx, child)) - childs = append(childs, child) - } - } - } + msgPack := (*inNode.inStream).Consume() // TODO: add status if msgPack == nil { @@ -67,10 +42,6 @@ func (inNode *InputNode) Operate([]*Msg) []*Msg { timestampMax: msgPack.EndTs, } - for _, child := range childs { - child.Finish() - } - return []*Msg{&msgStreamMsg} } diff --git a/internal/writenode/client/client.go b/internal/writenode/client/client.go index a966ff8862..6c922e2e1c 100644 --- a/internal/writenode/client/client.go +++ b/internal/writenode/client/client.go @@ -48,7 +48,7 @@ type SegmentDescription struct { CloseTime Timestamp } -func (c *Client) FlushSegment(segmentID UniqueID) error { +func (c *Client) FlushSegment(segmentID UniqueID, collectionID UniqueID, partitionTag string, timestamp Timestamp) error { baseMsg := msgstream.BaseMsg{ BeginTimestamp: 0, EndTimestamp: 0, @@ -56,9 +56,11 @@ func (c *Client) FlushSegment(segmentID UniqueID) error { } flushMsg := internalPb.FlushMsg{ - MsgType: internalPb.MsgType_kFlush, - SegmentID: segmentID, - Timestamp: Timestamp(0), + MsgType: internalPb.MsgType_kFlush, + SegmentID: segmentID, + CollectionID: collectionID, + PartitionTag: partitionTag, + Timestamp: timestamp, } fMsg := &msgstream.FlushMsg{ diff --git a/internal/writenode/collection.go b/internal/writenode/collection.go new file mode 100644 index 0000000000..21d411d110 --- /dev/null +++ b/internal/writenode/collection.go @@ -0,0 +1,37 @@ +package writenode + +import ( + "log" + + "github.com/golang/protobuf/proto" + "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" +) + +type Collection struct { + schema *schemapb.CollectionSchema + id UniqueID +} + +func (c *Collection) Name() string { + return c.schema.Name +} + +func (c *Collection) ID() UniqueID { + return c.id +} + +func newCollection(collectionID UniqueID, schemaStr string) *Collection { + + var schema schemapb.CollectionSchema + err := proto.UnmarshalText(schemaStr, &schema) + if err != nil { + log.Println(err) + return nil + } + + var newCollection = &Collection{ + schema: &schema, + id: collectionID, + } + return newCollection +} diff --git a/internal/writenode/collection_replica.go b/internal/writenode/collection_replica.go new file mode 100644 index 0000000000..2310802c8f --- /dev/null +++ b/internal/writenode/collection_replica.go @@ -0,0 +1,94 @@ +package writenode + +import ( + "strconv" + "sync" + + "github.com/zilliztech/milvus-distributed/internal/errors" +) + +type collectionReplica interface { + + // collection + getCollectionNum() int + addCollection(collectionID UniqueID, schemaBlob string) error + removeCollection(collectionID UniqueID) error + getCollectionByID(collectionID UniqueID) (*Collection, error) + getCollectionByName(collectionName string) (*Collection, error) + hasCollection(collectionID UniqueID) bool +} + +type collectionReplicaImpl struct { + mu sync.RWMutex + collections []*Collection +} + +//----------------------------------------------------------------------------------------------------- collection +func (colReplica *collectionReplicaImpl) getCollectionNum() int { + colReplica.mu.RLock() + defer colReplica.mu.RUnlock() + + return len(colReplica.collections) +} + +func (colReplica *collectionReplicaImpl) addCollection(collectionID UniqueID, schemaBlob string) error { + colReplica.mu.Lock() + defer colReplica.mu.Unlock() + + var newCollection = newCollection(collectionID, schemaBlob) + colReplica.collections = append(colReplica.collections, newCollection) + + return nil +} + +func (colReplica *collectionReplicaImpl) removeCollection(collectionID UniqueID) error { + colReplica.mu.Lock() + defer colReplica.mu.Unlock() + + tmpCollections := make([]*Collection, 0) + for _, col := range colReplica.collections { + if col.ID() != collectionID { + tmpCollections = append(tmpCollections, col) + } + } + colReplica.collections = tmpCollections + return nil +} + +func (colReplica *collectionReplicaImpl) getCollectionByID(collectionID UniqueID) (*Collection, error) { + colReplica.mu.RLock() + defer colReplica.mu.RUnlock() + + for _, collection := range colReplica.collections { + if collection.ID() == collectionID { + return collection, nil + } + } + + return nil, errors.New("cannot find collection, id = " + strconv.FormatInt(collectionID, 10)) +} + +func (colReplica *collectionReplicaImpl) getCollectionByName(collectionName string) (*Collection, error) { + colReplica.mu.RLock() + defer colReplica.mu.RUnlock() + + for _, collection := range colReplica.collections { + if collection.Name() == collectionName { + return collection, nil + } + } + + return nil, errors.New("Cannot found collection: " + collectionName) +} + +func (colReplica *collectionReplicaImpl) hasCollection(collectionID UniqueID) bool { + colReplica.mu.RLock() + defer colReplica.mu.RUnlock() + + for _, col := range colReplica.collections { + if col.ID() == collectionID { + return true + } + } + return false +} diff --git a/internal/writenode/collection_replica_test.go b/internal/writenode/collection_replica_test.go new file mode 100644 index 0000000000..be17d2b880 --- /dev/null +++ b/internal/writenode/collection_replica_test.go @@ -0,0 +1,153 @@ +package writenode + +import ( + "testing" + + "github.com/golang/protobuf/proto" + "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" + "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" + "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" + + "github.com/stretchr/testify/assert" +) + +func newReplica() collectionReplica { + collections := make([]*Collection, 0) + + var replica collectionReplica = &collectionReplicaImpl{ + collections: collections, + } + return replica +} + +func genTestCollectionMeta(collectionName string, collectionID UniqueID) *etcdpb.CollectionMeta { + fieldVec := schemapb.FieldSchema{ + FieldID: UniqueID(100), + Name: "vec", + IsPrimaryKey: false, + DataType: schemapb.DataType_VECTOR_FLOAT, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: "dim", + Value: "16", + }, + }, + IndexParams: []*commonpb.KeyValuePair{ + { + Key: "metric_type", + Value: "L2", + }, + }, + } + + fieldInt := schemapb.FieldSchema{ + FieldID: UniqueID(101), + Name: "age", + IsPrimaryKey: false, + DataType: schemapb.DataType_INT32, + } + + schema := schemapb.CollectionSchema{ + Name: collectionName, + AutoID: true, + Fields: []*schemapb.FieldSchema{ + &fieldVec, &fieldInt, + }, + } + + collectionMeta := etcdpb.CollectionMeta{ + ID: collectionID, + Schema: &schema, + CreateTime: Timestamp(0), + SegmentIDs: []UniqueID{0}, + PartitionTags: []string{"default"}, + } + + return &collectionMeta +} + +func initTestMeta(t *testing.T, replica collectionReplica, collectionName string, collectionID UniqueID, segmentID UniqueID) { + collectionMeta := genTestCollectionMeta(collectionName, collectionID) + + schemaBlob := proto.MarshalTextString(collectionMeta.Schema) + assert.NotEqual(t, "", schemaBlob) + + var err = replica.addCollection(collectionMeta.ID, schemaBlob) + assert.NoError(t, err) + + collection, err := replica.getCollectionByName(collectionName) + assert.NoError(t, err) + assert.Equal(t, collection.Name(), collectionName) + assert.Equal(t, collection.ID(), collectionID) + assert.Equal(t, replica.getCollectionNum(), 1) + +} + +//----------------------------------------------------------------------------------------------------- collection +func TestCollectionReplica_getCollectionNum(t *testing.T) { + replica := newReplica() + initTestMeta(t, replica, "collection0", 0, 0) + assert.Equal(t, replica.getCollectionNum(), 1) +} + +func TestCollectionReplica_addCollection(t *testing.T) { + replica := newReplica() + initTestMeta(t, replica, "collection0", 0, 0) +} + +func TestCollectionReplica_removeCollection(t *testing.T) { + replica := newReplica() + initTestMeta(t, replica, "collection0", 0, 0) + assert.Equal(t, replica.getCollectionNum(), 1) + + err := replica.removeCollection(0) + assert.NoError(t, err) + assert.Equal(t, replica.getCollectionNum(), 0) +} + +func TestCollectionReplica_getCollectionByID(t *testing.T) { + replica := newReplica() + collectionName := "collection0" + collectionID := UniqueID(0) + initTestMeta(t, replica, collectionName, collectionID, 0) + targetCollection, err := replica.getCollectionByID(collectionID) + assert.NoError(t, err) + assert.NotNil(t, targetCollection) + assert.Equal(t, targetCollection.Name(), collectionName) + assert.Equal(t, targetCollection.ID(), collectionID) +} + +func TestCollectionReplica_getCollectionByName(t *testing.T) { + replica := newReplica() + collectionName := "collection0" + collectionID := UniqueID(0) + initTestMeta(t, replica, collectionName, collectionID, 0) + + targetCollection, err := replica.getCollectionByName(collectionName) + assert.NoError(t, err) + assert.NotNil(t, targetCollection) + assert.Equal(t, targetCollection.Name(), collectionName) + assert.Equal(t, targetCollection.ID(), collectionID) + +} + +func TestCollectionReplica_hasCollection(t *testing.T) { + replica := newReplica() + collectionName := "collection0" + collectionID := UniqueID(0) + initTestMeta(t, replica, collectionName, collectionID, 0) + + hasCollection := replica.hasCollection(collectionID) + assert.Equal(t, hasCollection, true) + hasCollection = replica.hasCollection(UniqueID(1)) + assert.Equal(t, hasCollection, false) + +} + +func TestCollectionReplica_freeAll(t *testing.T) { + replica := newReplica() + collectionName := "collection0" + collectionID := UniqueID(0) + initTestMeta(t, replica, collectionName, collectionID, 0) + +} diff --git a/internal/writenode/collection_test.go b/internal/writenode/collection_test.go new file mode 100644 index 0000000000..6df2ad5d49 --- /dev/null +++ b/internal/writenode/collection_test.go @@ -0,0 +1,34 @@ +package writenode + +import ( + "testing" + + "github.com/golang/protobuf/proto" + "github.com/stretchr/testify/assert" +) + +func TestCollection_newCollection(t *testing.T) { + collectionName := "collection0" + collectionID := UniqueID(0) + collectionMeta := genTestCollectionMeta(collectionName, collectionID) + + schemaBlob := proto.MarshalTextString(collectionMeta.Schema) + assert.NotEqual(t, "", schemaBlob) + + collection := newCollection(collectionMeta.ID, schemaBlob) + assert.Equal(t, collection.Name(), collectionName) + assert.Equal(t, collection.ID(), collectionID) +} + +func TestCollection_deleteCollection(t *testing.T) { + collectionName := "collection0" + collectionID := UniqueID(0) + collectionMeta := genTestCollectionMeta(collectionName, collectionID) + + schemaBlob := proto.MarshalTextString(collectionMeta.Schema) + assert.NotEqual(t, "", schemaBlob) + + collection := newCollection(collectionMeta.ID, schemaBlob) + assert.Equal(t, collection.Name(), collectionName) + assert.Equal(t, collection.ID(), collectionID) +} diff --git a/internal/writenode/data_sync_service.go b/internal/writenode/data_sync_service.go index da91d06a76..efcd62b036 100644 --- a/internal/writenode/data_sync_service.go +++ b/internal/writenode/data_sync_service.go @@ -12,16 +12,18 @@ type dataSyncService struct { fg *flowgraph.TimeTickedFlowGraph ddChan chan *ddlFlushSyncMsg insertChan chan *insertFlushSyncMsg + replica collectionReplica } func newDataSyncService(ctx context.Context, - ddChan chan *ddlFlushSyncMsg, insertChan chan *insertFlushSyncMsg) *dataSyncService { + ddChan chan *ddlFlushSyncMsg, insertChan chan *insertFlushSyncMsg, replica collectionReplica) *dataSyncService { return &dataSyncService{ ctx: ctx, fg: nil, ddChan: ddChan, insertChan: insertChan, + replica: replica, } } @@ -46,8 +48,8 @@ func (dsService *dataSyncService) initNodes() { var filterDmNode Node = newFilteredDmNode() - var ddNode Node = newDDNode(dsService.ctx, dsService.ddChan) - var insertBufferNode Node = newInsertBufferNode(dsService.ctx, dsService.insertChan) + var ddNode Node = newDDNode(dsService.ctx, dsService.ddChan, dsService.replica) + var insertBufferNode Node = newInsertBufferNode(dsService.ctx, dsService.insertChan, dsService.replica) dsService.fg.AddNode(&dmStreamNode) dsService.fg.AddNode(&ddStreamNode) diff --git a/internal/writenode/data_sync_service_test.go b/internal/writenode/data_sync_service_test.go index a37425ec3e..df82cec4d9 100644 --- a/internal/writenode/data_sync_service_test.go +++ b/internal/writenode/data_sync_service_test.go @@ -197,7 +197,8 @@ func TestDataSyncService_Start(t *testing.T) { assert.NoError(t, err) // dataSync - node.dataSyncService = newDataSyncService(node.ctx, nil, nil) + replica := newReplica() + node.dataSyncService = newDataSyncService(node.ctx, nil, nil, replica) go node.dataSyncService.start() node.Close() diff --git a/internal/writenode/flow_graph_dd_node.go b/internal/writenode/flow_graph_dd_node.go index 7dd8e1fd43..9d3b4db158 100644 --- a/internal/writenode/flow_graph_dd_node.go +++ b/internal/writenode/flow_graph_dd_node.go @@ -9,9 +9,6 @@ import ( "strconv" "github.com/golang/protobuf/proto" - "github.com/minio/minio-go/v7" - "github.com/minio/minio-go/v7/pkg/credentials" - "github.com/zilliztech/milvus-distributed/internal/allocator" "github.com/zilliztech/milvus-distributed/internal/kv" miniokv "github.com/zilliztech/milvus-distributed/internal/kv/minio" @@ -30,6 +27,7 @@ type ddNode struct { idAllocator *allocator.IDAllocator kv kv.Base + replica collectionReplica } type ddData struct { @@ -228,6 +226,15 @@ func (ddNode *ddNode) createCollection(msg *msgstream.CreateCollectionMsg) { log.Println(err) return } + + schemaStr := proto.MarshalTextString(&schema) + // add collection + err = ddNode.replica.addCollection(collectionID, schemaStr) + if err != nil { + log.Println(err) + return + } + collectionName := schema.Name ddNode.ddMsg.collectionRecords[collectionName] = append(ddNode.ddMsg.collectionRecords[collectionName], metaOperateRecord{ @@ -252,6 +259,11 @@ func (ddNode *ddNode) createCollection(msg *msgstream.CreateCollectionMsg) { func (ddNode *ddNode) dropCollection(msg *msgstream.DropCollectionMsg) { collectionID := msg.CollectionID + err := ddNode.replica.removeCollection(collectionID) + if err != nil { + log.Println(err) + } + // remove collection if _, ok := ddNode.ddRecords.collectionRecords[collectionID]; !ok { err := errors.New("cannot found collection " + strconv.FormatInt(collectionID, 10)) @@ -347,7 +359,7 @@ func (ddNode *ddNode) dropPartition(msg *msgstream.DropPartitionMsg) { ddNode.ddBuffer.ddData[collectionID].eventTypes = append(ddNode.ddBuffer.ddData[collectionID].eventTypes, storage.DropPartitionEventType) } -func newDDNode(ctx context.Context, outCh chan *ddlFlushSyncMsg) *ddNode { +func newDDNode(ctx context.Context, outCh chan *ddlFlushSyncMsg, replica collectionReplica) *ddNode { maxQueueLength := Params.FlowGraphMaxQueueLength maxParallelism := Params.FlowGraphMaxParallelism @@ -360,19 +372,16 @@ func newDDNode(ctx context.Context, outCh chan *ddlFlushSyncMsg) *ddNode { partitionRecords: make(map[UniqueID]interface{}), } - minIOEndPoint := Params.MinioAddress - minIOAccessKeyID := Params.MinioAccessKeyID - minIOSecretAccessKey := Params.MinioSecretAccessKey - minIOUseSSL := Params.MinioUseSSL - minIOClient, err := minio.New(minIOEndPoint, &minio.Options{ - Creds: credentials.NewStaticV4(minIOAccessKeyID, minIOSecretAccessKey, ""), - Secure: minIOUseSSL, - }) - if err != nil { - panic(err) - } bucketName := Params.MinioBucketName - minioKV, err := miniokv.NewMinIOKV(ctx, minIOClient, bucketName) + option := &miniokv.Option{ + Address: Params.MinioAddress, + AccessKeyID: Params.MinioAccessKeyID, + SecretAccessKeyID: Params.MinioSecretAccessKey, + UseSSL: Params.MinioUseSSL, + BucketName: bucketName, + CreateBucket: true, + } + minioKV, err := miniokv.NewMinIOKV(ctx, option) if err != nil { panic(err) } @@ -397,5 +406,6 @@ func newDDNode(ctx context.Context, outCh chan *ddlFlushSyncMsg) *ddNode { idAllocator: idAllocator, kv: minioKV, + replica: replica, } } diff --git a/internal/writenode/flow_graph_dd_node_test.go b/internal/writenode/flow_graph_dd_node_test.go index 25de3697ea..9b5d71ffb9 100644 --- a/internal/writenode/flow_graph_dd_node_test.go +++ b/internal/writenode/flow_graph_dd_node_test.go @@ -45,8 +45,8 @@ func TestFlowGraphDDNode_Operate(t *testing.T) { go fService.start() Params.FlushDdBufSize = 4 - - ddNode := newDDNode(ctx, ddChan) + replica := newReplica() + ddNode := newDDNode(ctx, ddChan, replica) colID := UniqueID(0) colName := "col-test-0" diff --git a/internal/writenode/flow_graph_filter_dm_node.go b/internal/writenode/flow_graph_filter_dm_node.go index 0bca67ebcb..98e9ec0bc4 100644 --- a/internal/writenode/flow_graph_filter_dm_node.go +++ b/internal/writenode/flow_graph_filter_dm_node.go @@ -1,11 +1,8 @@ package writenode import ( - "context" "log" - "github.com/opentracing/opentracing-go" - "github.com/zilliztech/milvus-distributed/internal/msgstream" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" @@ -34,34 +31,11 @@ func (fdmNode *filterDmNode) Operate(in []*Msg) []*Msg { // TODO: add error handling } - var childs []opentracing.Span - tracer := opentracing.GlobalTracer() - if tracer != nil { - for _, msg := range msgStreamMsg.TsMessages() { - if msg.Type() == internalPb.MsgType_kInsert { - var child opentracing.Span - ctx := msg.GetContext() - if parent := opentracing.SpanFromContext(ctx); parent != nil { - child = tracer.StartSpan("pass filter node", - opentracing.FollowsFrom(parent.Context())) - } else { - child = tracer.StartSpan("pass filter node") - } - child.SetTag("hash keys", msg.HashKeys()) - child.SetTag("start time", msg.BeginTs()) - child.SetTag("end time", msg.EndTs()) - msg.SetContext(opentracing.ContextWithSpan(ctx, child)) - childs = append(childs, child) - } - } - } - ddMsg, ok := (*in[1]).(*ddMsg) if !ok { log.Println("type assertion failed for ddMsg") // TODO: add error handling } - fdmNode.ddMsg = ddMsg var iMsg = insertMsg{ @@ -82,20 +56,11 @@ func (fdmNode *filterDmNode) Operate(in []*Msg) []*Msg { } } - for key, msg := range msgStreamMsg.TsMessages() { + for _, msg := range msgStreamMsg.TsMessages() { switch msg.Type() { case internalPb.MsgType_kInsert: - var ctx2 context.Context - if childs != nil { - if childs[key] != nil { - ctx2 = opentracing.ContextWithSpan(msg.GetContext(), childs[key]) - } else { - ctx2 = context.Background() - } - } resMsg := fdmNode.filterInvalidInsertMessage(msg.(*msgstream.InsertMsg)) if resMsg != nil { - resMsg.SetContext(ctx2) iMsg.insertMessages = append(iMsg.insertMessages, resMsg) } // case internalPb.MsgType_kDelete: @@ -104,11 +69,8 @@ func (fdmNode *filterDmNode) Operate(in []*Msg) []*Msg { log.Println("Non supporting message type:", msg.Type()) } } - var res Msg = &iMsg - for _, child := range childs { - child.Finish() - } + var res Msg = &iMsg return []*Msg{&res} } diff --git a/internal/writenode/flow_graph_insert_buffer_node.go b/internal/writenode/flow_graph_insert_buffer_node.go index 927e751a62..e1b667946f 100644 --- a/internal/writenode/flow_graph_insert_buffer_node.go +++ b/internal/writenode/flow_graph_insert_buffer_node.go @@ -4,22 +4,13 @@ import ( "bytes" "context" "encoding/binary" - "fmt" "log" "path" "strconv" - "time" "unsafe" - "github.com/opentracing/opentracing-go" - oplog "github.com/opentracing/opentracing-go/log" - - "github.com/golang/protobuf/proto" - "github.com/minio/minio-go/v7" - "github.com/minio/minio-go/v7/pkg/credentials" "github.com/zilliztech/milvus-distributed/internal/allocator" "github.com/zilliztech/milvus-distributed/internal/kv" - etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd" miniokv "github.com/zilliztech/milvus-distributed/internal/kv/minio" "github.com/zilliztech/milvus-distributed/internal/msgstream" "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" @@ -27,7 +18,6 @@ import ( "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "github.com/zilliztech/milvus-distributed/internal/storage" "github.com/zilliztech/milvus-distributed/internal/util/typeutil" - "go.etcd.io/etcd/clientv3" ) const ( @@ -41,13 +31,13 @@ type ( insertBufferNode struct { BaseNode - kvClient *etcdkv.EtcdKV insertBuffer *insertBuffer minIOKV kv.Base minioPrifex string idAllocator *allocator.IDAllocator outCh chan *insertFlushSyncMsg pulsarWriteNodeTimeTickStream *msgstream.PulsarMsgStream + replica collectionReplica } insertBuffer struct { @@ -106,22 +96,12 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg { // iMsg is insertMsg // 1. iMsg -> buffer for _, msg := range iMsg.insertMessages { - ctx := msg.GetContext() - var span opentracing.Span - if ctx != nil { - span, _ = opentracing.StartSpanFromContext(ctx, fmt.Sprintf("insert buffer node, start time = %d", msg.BeginTs())) - } else { - span = opentracing.StartSpan(fmt.Sprintf("insert buffer node, start time = %d", msg.BeginTs())) - } - span.SetTag("hash keys", msg.HashKeys()) - span.SetTag("start time", msg.BeginTs()) - span.SetTag("end time", msg.EndTs()) if len(msg.RowIDs) != len(msg.Timestamps) || len(msg.RowIDs) != len(msg.RowData) { log.Println("Error: misaligned messages detected") continue } currentSegID := msg.GetSegmentID() - span.LogFields(oplog.Int("segment id", int(currentSegID))) + collectionName := msg.GetCollectionName() idata, ok := ibNode.insertBuffer.insertData[currentSegID] if !ok { @@ -130,32 +110,20 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg { } } - // Timestamps - _, ok = idata.Data[1].(*storage.Int64FieldData) - if !ok { - idata.Data[1] = &storage.Int64FieldData{ - Data: []int64{}, - NumRows: 0, - } - } - tsData := idata.Data[1].(*storage.Int64FieldData) - for _, ts := range msg.Timestamps { - tsData.Data = append(tsData.Data, int64(ts)) - } - tsData.NumRows += len(msg.Timestamps) - span.LogFields(oplog.Int("tsData numRows", tsData.NumRows)) - // 1.1 Get CollectionMeta from etcd - segMeta, collMeta, err := ibNode.getMeta(currentSegID) + collection, err := ibNode.replica.getCollectionByName(collectionName) + //collSchema, err := ibNode.getCollectionSchemaByName(collectionName) if err != nil { // GOOSE TODO add error handler log.Println("Get meta wrong:", err) continue } + collectionID := collection.ID() + collSchema := collection.schema // 1.2 Get Fields var pos int = 0 // Record position of blob - for _, field := range collMeta.Schema.Fields { + for _, field := range collSchema.Fields { switch field.DataType { case schemapb.DataType_VECTOR_FLOAT: var dim int @@ -390,11 +358,9 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg { // 1.3 store in buffer ibNode.insertBuffer.insertData[currentSegID] = idata - span.LogFields(oplog.String("store in buffer", "store in buffer")) // 1.4 if full // 1.4.1 generate binlogs - span.LogFields(oplog.String("generate binlogs", "generate binlogs")) if ibNode.insertBuffer.full(currentSegID) { log.Printf(". Insert Buffer full, auto flushing (%v) rows of data...", ibNode.insertBuffer.size(currentSegID)) // partitionTag -> partitionID @@ -404,7 +370,10 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg { log.Println("partitionTag to partitionID wrong") // TODO GOOSE add error handler } - + collMeta := &etcdpb.CollectionMeta{ + Schema: collSchema, + ID: collectionID, + } inCodec := storage.NewInsertCodec(collMeta) // buffer data to binlogs @@ -420,7 +389,7 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg { log.Println(".. Clearing buffer") // 1.5.2 binLogs -> minIO/S3 - collIDStr := strconv.FormatInt(segMeta.GetCollectionID(), 10) + collIDStr := strconv.FormatInt(collectionID, 10) partitionIDStr := strconv.FormatInt(partitionID, 10) segIDStr := strconv.FormatInt(currentSegID, 10) keyPrefix := path.Join(ibNode.minioPrifex, collIDStr, partitionIDStr, segIDStr) @@ -460,7 +429,6 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg { ibNode.outCh <- inBinlogMsg } } - span.Finish() } if len(iMsg.insertMessages) > 0 { @@ -480,20 +448,24 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg { for _, msg := range iMsg.flushMessages { currentSegID := msg.GetSegmentID() flushTs := msg.GetTimestamp() - + partitionTag := msg.GetPartitionTag() + collectionID := msg.GetCollectionID() log.Printf(". Receiving flush message segID(%v)...", currentSegID) if ibNode.insertBuffer.size(currentSegID) > 0 { log.Println(".. Buffer not empty, flushing ...") - segMeta, collMeta, err := ibNode.getMeta(currentSegID) + collSchema, err := ibNode.getCollectionSchemaByID(collectionID) if err != nil { // GOOSE TODO add error handler log.Println("Get meta wrong: ", err) } + collMeta := &etcdpb.CollectionMeta{ + Schema: collSchema, + ID: collectionID, + } inCodec := storage.NewInsertCodec(collMeta) // partitionTag -> partitionID - partitionTag := segMeta.GetPartitionTag() partitionID, err := typeutil.Hash32String(partitionTag) if err != nil { // GOOSE TODO add error handler @@ -511,7 +483,7 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg { delete(ibNode.insertBuffer.insertData, currentSegID) // binLogs -> minIO/S3 - collIDStr := strconv.FormatInt(segMeta.GetCollectionID(), 10) + collIDStr := strconv.FormatInt(collectionID, 10) partitionIDStr := strconv.FormatInt(partitionID, 10) segIDStr := strconv.FormatInt(currentSegID, 10) keyPrefix := path.Join(ibNode.minioPrifex, collIDStr, partitionIDStr, segIDStr) @@ -570,31 +542,20 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg { return nil } -func (ibNode *insertBufferNode) getMeta(segID UniqueID) (*etcdpb.SegmentMeta, *etcdpb.CollectionMeta, error) { +func (ibNode *insertBufferNode) getCollectionSchemaByID(collectionID UniqueID) (*schemapb.CollectionSchema, error) { + ret, err := ibNode.replica.getCollectionByID(collectionID) + if err != nil { + return nil, err + } + return ret.schema, nil +} - segMeta := &etcdpb.SegmentMeta{} - - key := path.Join(SegmentPrefix, strconv.FormatInt(segID, 10)) - value, err := ibNode.kvClient.Load(key) +func (ibNode *insertBufferNode) getCollectionSchemaByName(collectionName string) (*schemapb.CollectionSchema, error) { + ret, err := ibNode.replica.getCollectionByName(collectionName) if err != nil { - return nil, nil, err + return nil, err } - err = proto.UnmarshalText(value, segMeta) - if err != nil { - return nil, nil, err - } - - collMeta := &etcdpb.CollectionMeta{} - key = path.Join(CollectionPrefix, strconv.FormatInt(segMeta.GetCollectionID(), 10)) - value, err = ibNode.kvClient.Load(key) - if err != nil { - return nil, nil, err - } - err = proto.UnmarshalText(value, collMeta) - if err != nil { - return nil, nil, err - } - return segMeta, collMeta, nil + return ret.schema, nil } func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp) error { @@ -615,7 +576,7 @@ func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp) error { return ibNode.pulsarWriteNodeTimeTickStream.Produce(&msgPack) } -func newInsertBufferNode(ctx context.Context, outCh chan *insertFlushSyncMsg) *insertBufferNode { +func newInsertBufferNode(ctx context.Context, outCh chan *insertFlushSyncMsg, replica collectionReplica) *insertBufferNode { maxQueueLength := Params.FlowGraphMaxQueueLength maxParallelism := Params.FlowGraphMaxParallelism @@ -629,34 +590,18 @@ func newInsertBufferNode(ctx context.Context, outCh chan *insertFlushSyncMsg) *i maxSize: maxSize, } - // EtcdKV - ETCDAddr := Params.EtcdAddress - MetaRootPath := Params.MetaRootPath - log.Println("metaRootPath: ", MetaRootPath) - cli, err := clientv3.New(clientv3.Config{ - Endpoints: []string{ETCDAddr}, - DialTimeout: 5 * time.Second, - }) - if err != nil { - panic(err) - } - kvClient := etcdkv.NewEtcdKV(cli, MetaRootPath) - // MinIO - minioendPoint := Params.MinioAddress - miniioAccessKeyID := Params.MinioAccessKeyID - miniioSecretAccessKey := Params.MinioSecretAccessKey - minioUseSSL := Params.MinioUseSSL - minioBucketName := Params.MinioBucketName - minioClient, err := minio.New(minioendPoint, &minio.Options{ - Creds: credentials.NewStaticV4(miniioAccessKeyID, miniioSecretAccessKey, ""), - Secure: minioUseSSL, - }) - if err != nil { - panic(err) + option := &miniokv.Option{ + Address: Params.MinioAddress, + AccessKeyID: Params.MinioAccessKeyID, + SecretAccessKeyID: Params.MinioSecretAccessKey, + UseSSL: Params.MinioUseSSL, + CreateBucket: true, + BucketName: Params.MinioBucketName, } - minIOKV, err := miniokv.NewMinIOKV(ctx, minioClient, minioBucketName) + + minIOKV, err := miniokv.NewMinIOKV(ctx, option) if err != nil { panic(err) } @@ -677,12 +622,12 @@ func newInsertBufferNode(ctx context.Context, outCh chan *insertFlushSyncMsg) *i return &insertBufferNode{ BaseNode: baseNode, - kvClient: kvClient, insertBuffer: iBuffer, minIOKV: minIOKV, minioPrifex: minioPrefix, idAllocator: idAllocator, outCh: outCh, pulsarWriteNodeTimeTickStream: wTt, + replica: replica, } } diff --git a/internal/writenode/flow_graph_insert_buffer_node_test.go b/internal/writenode/flow_graph_insert_buffer_node_test.go index ef268dd3ba..567b90c34f 100644 --- a/internal/writenode/flow_graph_insert_buffer_node_test.go +++ b/internal/writenode/flow_graph_insert_buffer_node_test.go @@ -47,7 +47,8 @@ func TestFlowGraphInputBufferNode_Operate(t *testing.T) { go fService.start() // Params.FlushInsertBufSize = 2 - iBNode := newInsertBufferNode(ctx, insertChan) + replica := newReplica() + iBNode := newInsertBufferNode(ctx, insertChan, replica) newMeta() inMsg := genInsertMsg() diff --git a/internal/writenode/meta_service.go b/internal/writenode/meta_service.go new file mode 100644 index 0000000000..75dfb4a402 --- /dev/null +++ b/internal/writenode/meta_service.go @@ -0,0 +1,135 @@ +package writenode + +import ( + "context" + "fmt" + "log" + "path" + "reflect" + "strings" + "time" + + "github.com/golang/protobuf/proto" + "go.etcd.io/etcd/clientv3" + + etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd" + "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" +) + +type metaService struct { + ctx context.Context + kvBase *etcdkv.EtcdKV + replica collectionReplica +} + +func newMetaService(ctx context.Context, replica collectionReplica) *metaService { + ETCDAddr := Params.EtcdAddress + MetaRootPath := Params.MetaRootPath + + cli, _ := clientv3.New(clientv3.Config{ + Endpoints: []string{ETCDAddr}, + DialTimeout: 5 * time.Second, + }) + + return &metaService{ + ctx: ctx, + kvBase: etcdkv.NewEtcdKV(cli, MetaRootPath), + replica: replica, + } +} + +func (mService *metaService) start() { + // init from meta + err := mService.loadCollections() + if err != nil { + log.Fatal("metaService loadCollections failed") + } +} + +func GetCollectionObjID(key string) string { + ETCDRootPath := Params.MetaRootPath + + prefix := path.Join(ETCDRootPath, CollectionPrefix) + "/" + return strings.TrimPrefix(key, prefix) +} + +func isCollectionObj(key string) bool { + ETCDRootPath := Params.MetaRootPath + + prefix := path.Join(ETCDRootPath, CollectionPrefix) + "/" + prefix = strings.TrimSpace(prefix) + index := strings.Index(key, prefix) + + return index == 0 +} + +func isSegmentObj(key string) bool { + ETCDRootPath := Params.MetaRootPath + + prefix := path.Join(ETCDRootPath, SegmentPrefix) + "/" + prefix = strings.TrimSpace(prefix) + index := strings.Index(key, prefix) + + return index == 0 +} + +func printCollectionStruct(obj *etcdpb.CollectionMeta) { + v := reflect.ValueOf(obj) + v = reflect.Indirect(v) + typeOfS := v.Type() + + for i := 0; i < v.NumField(); i++ { + if typeOfS.Field(i).Name == "GrpcMarshalString" { + continue + } + fmt.Printf("Field: %s\tValue: %v\n", typeOfS.Field(i).Name, v.Field(i).Interface()) + } +} + +func (mService *metaService) processCollectionCreate(id string, value string) { + //println(fmt.Sprintf("Create Collection:$%s$", id)) + + col := mService.collectionUnmarshal(value) + if col != nil { + schema := col.Schema + schemaBlob := proto.MarshalTextString(schema) + err := mService.replica.addCollection(col.ID, schemaBlob) + if err != nil { + log.Println(err) + } + } +} + +func (mService *metaService) loadCollections() error { + keys, values, err := mService.kvBase.LoadWithPrefix(CollectionPrefix) + if err != nil { + return err + } + + for i := range keys { + objID := GetCollectionObjID(keys[i]) + mService.processCollectionCreate(objID, values[i]) + } + + return nil +} + +//----------------------------------------------------------------------- Unmarshal and Marshal +func (mService *metaService) collectionUnmarshal(value string) *etcdpb.CollectionMeta { + col := etcdpb.CollectionMeta{} + err := proto.UnmarshalText(value, &col) + if err != nil { + log.Println(err) + return nil + } + return &col +} + +func (mService *metaService) collectionMarshal(col *etcdpb.CollectionMeta) string { + value := proto.MarshalTextString(col) + if value == "" { + log.Println("marshal collection failed") + return "" + } + return value +} diff --git a/internal/writenode/meta_service_test.go b/internal/writenode/meta_service_test.go new file mode 100644 index 0000000000..b1e8a9e038 --- /dev/null +++ b/internal/writenode/meta_service_test.go @@ -0,0 +1,100 @@ +package writenode + +import ( + "context" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestMetaService_start(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + replica := newReplica() + + metaService := newMetaService(ctx, replica) + + metaService.start() +} + +func TestMetaService_getCollectionObjId(t *testing.T) { + var key = "/collection/collection0" + var collectionObjID1 = GetCollectionObjID(key) + + assert.Equal(t, collectionObjID1, "/collection/collection0") + + key = "fakeKey" + var collectionObjID2 = GetCollectionObjID(key) + + assert.Equal(t, collectionObjID2, "fakeKey") +} + +func TestMetaService_isCollectionObj(t *testing.T) { + var key = Params.MetaRootPath + "/collection/collection0" + var b1 = isCollectionObj(key) + + assert.Equal(t, b1, true) + + key = Params.MetaRootPath + "/segment/segment0" + var b2 = isCollectionObj(key) + + assert.Equal(t, b2, false) +} + +func TestMetaService_processCollectionCreate(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + replica := newReplica() + metaService := newMetaService(ctx, replica) + defer cancel() + id := "0" + value := `schema: < + name: "test" + fields: < + fieldID:100 + name: "vec" + data_type: VECTOR_FLOAT + type_params: < + key: "dim" + value: "16" + > + index_params: < + key: "metric_type" + value: "L2" + > + > + fields: < + fieldID:101 + name: "age" + data_type: INT32 + type_params: < + key: "dim" + value: "1" + > + > + > + segmentIDs: 0 + partition_tags: "default" + ` + + metaService.processCollectionCreate(id, value) + + collectionNum := replica.getCollectionNum() + assert.Equal(t, collectionNum, 1) + + collection, err := replica.getCollectionByName("test") + assert.NoError(t, err) + assert.Equal(t, collection.ID(), UniqueID(0)) +} + +func TestMetaService_loadCollections(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + replica := newReplica() + + metaService := newMetaService(ctx, replica) + + err2 := (*metaService).loadCollections() + assert.Nil(t, err2) +} diff --git a/internal/writenode/write_node.go b/internal/writenode/write_node.go index 77aeea5668..d3ce6f84c7 100644 --- a/internal/writenode/write_node.go +++ b/internal/writenode/write_node.go @@ -2,12 +2,6 @@ package writenode import ( "context" - "fmt" - "io" - - "github.com/opentracing/opentracing-go" - "github.com/uber/jaeger-client-go" - "github.com/uber/jaeger-client-go/config" ) type WriteNode struct { @@ -15,18 +9,25 @@ type WriteNode struct { WriteNodeID uint64 dataSyncService *dataSyncService flushSyncService *flushSyncService - - tracer opentracing.Tracer - closer io.Closer + metaService *metaService + replica collectionReplica } func NewWriteNode(ctx context.Context, writeNodeID uint64) *WriteNode { + collections := make([]*Collection, 0) + + var replica collectionReplica = &collectionReplicaImpl{ + collections: collections, + } + node := &WriteNode{ ctx: ctx, WriteNodeID: writeNodeID, dataSyncService: nil, flushSyncService: nil, + metaService: nil, + replica: replica, } return node @@ -37,22 +38,6 @@ func Init() { } func (node *WriteNode) Start() error { - cfg := &config.Configuration{ - ServiceName: "tracing", - Sampler: &config.SamplerConfig{ - Type: "const", - Param: 1, - }, - Reporter: &config.ReporterConfig{ - LogSpans: true, - }, - } - var err error - node.tracer, node.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger)) - if err != nil { - panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err)) - } - opentracing.SetGlobalTracer(node.tracer) // TODO GOOSE Init Size?? chanSize := 100 @@ -60,11 +45,12 @@ func (node *WriteNode) Start() error { insertChan := make(chan *insertFlushSyncMsg, chanSize) node.flushSyncService = newFlushSyncService(node.ctx, ddChan, insertChan) - node.dataSyncService = newDataSyncService(node.ctx, ddChan, insertChan) + node.dataSyncService = newDataSyncService(node.ctx, ddChan, insertChan, node.replica) + node.metaService = newMetaService(node.ctx, node.replica) go node.dataSyncService.start() go node.flushSyncService.start() - + node.metaService.start() return nil }