Add Etcd support for Import Tasks. (#16164)

issue: #15604
Signed-off-by: Yuchen Gao <yuchen.gao@zilliz.com>
This commit is contained in:
Ten Thousand Leaves 2022-03-25 11:03:25 +08:00 committed by GitHub
parent ed9d3c32ff
commit a23ca745bb
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 1033 additions and 320 deletions

View File

@ -333,6 +333,7 @@ func (kv *EmbedEtcdKV) SaveBytes(key string, value []byte) error {
// SaveWithLease is a function to put value in etcd with etcd lease options.
func (kv *EmbedEtcdKV) SaveWithLease(key, value string, id clientv3.LeaseID) error {
log.Debug("Embedded Etcd saving with lease", zap.String("etcd_key", key))
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
@ -340,6 +341,16 @@ func (kv *EmbedEtcdKV) SaveWithLease(key, value string, id clientv3.LeaseID) err
return err
}
// SaveWithIgnoreLease updates the key without changing its current lease.
func (kv *EmbedEtcdKV) SaveWithIgnoreLease(key, value string) error {
log.Debug("Embedded Etcd saving with ignore lease", zap.String("etcd_key", key))
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Put(ctx, key, value, clientv3.WithIgnoreLease())
return err
}
// SaveBytesWithLease is a function to put value in etcd with etcd lease options.
func (kv *EmbedEtcdKV) SaveBytesWithLease(key string, value []byte, id clientv3.LeaseID) error {
key = path.Join(kv.rootPath, key)

View File

@ -20,6 +20,7 @@ import (
"errors"
"os"
"testing"
"time"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
@ -898,6 +899,10 @@ func TestEmbedEtcd(te *testing.T) {
}
for k, v := range tests {
// SaveWithIgnoreLease must be used when the key already exists.
err = metaKv.SaveWithIgnoreLease(k, v)
assert.Error(t, err)
err = metaKv.SaveWithLease(k, v, leaseID)
assert.NoError(t, err)
@ -907,33 +912,41 @@ func TestEmbedEtcd(te *testing.T) {
})
te.Run("Etcd Lease Bytes", func(t *testing.T) {
rootPath := "/etcd/test/root/lease_bytes"
_metaKv, err := embed_etcd_kv.NewMetaKvFactory(rootPath, &param.EtcdCfg)
metaKv := _metaKv.(*embed_etcd_kv.EmbedEtcdKV)
te.Run("Etcd Lease Ignore", func(t *testing.T) {
rootPath := "/etcd/test/root/lease_ignore"
metaKv, err := embed_etcd_kv.NewMetaKvFactory(rootPath, &param.EtcdCfg)
assert.Nil(t, err)
defer metaKv.Close()
defer metaKv.RemoveWithPrefix("")
leaseID, err := metaKv.Grant(10)
assert.NoError(t, err)
metaKv.KeepAlive(leaseID)
tests := map[string][]byte{
"a/b": []byte("v1"),
"a/b/c": []byte("v2"),
"x": []byte("v3"),
tests := map[string]string{
"a/b": "v1",
"a/b/c": "v2",
"x": "v3",
}
for k, v := range tests {
err = metaKv.SaveBytesWithLease(k, v, leaseID)
leaseID, err := metaKv.Grant(1)
assert.NoError(t, err)
err = metaKv.SaveBytesWithLease(k, v, clientv3.LeaseID(999))
err = metaKv.SaveWithLease(k, v, leaseID)
assert.NoError(t, err)
err = metaKv.SaveWithIgnoreLease(k, "updated_"+v)
assert.NoError(t, err)
// Record should be updated correctly.
value, err := metaKv.Load(k)
assert.NoError(t, err)
assert.Equal(t, "updated_"+v, value)
// Let the lease expire. 3 seconds should be pretty safe.
time.Sleep(3 * time.Second)
// Updated record should still expire with lease.
_, err = metaKv.Load(k)
assert.Error(t, err)
}
})
}

View File

@ -317,6 +317,7 @@ func (kv *EtcdKV) SaveBytes(key string, value []byte) error {
// SaveWithLease is a function to put value in etcd with etcd lease options.
func (kv *EtcdKV) SaveWithLease(key, value string, id clientv3.LeaseID) error {
log.Debug("Etcd saving with lease", zap.String("etcd_key", key))
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
@ -326,6 +327,18 @@ func (kv *EtcdKV) SaveWithLease(key, value string, id clientv3.LeaseID) error {
return err
}
// SaveWithIgnoreLease updates the key without changing its current lease. Must be used when key already exists.
func (kv *EtcdKV) SaveWithIgnoreLease(key, value string) error {
log.Debug("Etcd saving with ignore lease", zap.String("etcd_key", key))
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Put(ctx, key, value, clientv3.WithIgnoreLease())
CheckElapseAndWarn(start, "Slow etcd operation save with lease")
return err
}
// SaveBytesWithLease is a function to put value in etcd with etcd lease options.
func (kv *EtcdKV) SaveBytesWithLease(key string, value []byte, id clientv3.LeaseID) error {
start := time.Now()

View File

@ -816,13 +816,53 @@ func TestEtcdKV_Load(te *testing.T) {
}
for k, v := range tests {
// SaveWithIgnoreLease must be used when the key already exists.
err = etcdKV.SaveWithIgnoreLease(k, v)
assert.Error(t, err)
err = etcdKV.SaveWithLease(k, v, leaseID)
assert.NoError(t, err)
err = etcdKV.SaveWithLease(k, v, clientv3.LeaseID(999))
assert.Error(t, err)
}
})
te.Run("Etcd Lease Ignore", func(t *testing.T) {
rootPath := "/etcd/test/root/lease_ignore"
etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("")
tests := map[string]string{
"a/b": "v1",
"a/b/c": "v2",
"x": "v3",
}
for k, v := range tests {
leaseID, err := etcdKV.Grant(1)
assert.NoError(t, err)
err = etcdKV.SaveWithLease(k, v, leaseID)
assert.NoError(t, err)
err = etcdKV.SaveWithIgnoreLease(k, "updated_"+v)
assert.NoError(t, err)
// Record should be updated correctly.
value, err := etcdKV.Load(k)
assert.NoError(t, err)
assert.Equal(t, "updated_"+v, value)
// Let the lease expire. 3 seconds should be pretty safe.
time.Sleep(3 * time.Second)
// Updated record should still expire with lease.
_, err = etcdKV.Load(k)
assert.Error(t, err)
}
})
te.Run("Etcd Lease Bytes", func(t *testing.T) {
@ -850,7 +890,6 @@ func TestEtcdKV_Load(te *testing.T) {
err = etcdKV.SaveBytesWithLease(k, v, clientv3.LeaseID(999))
assert.Error(t, err)
}
})
}

View File

@ -76,6 +76,7 @@ type MetaKv interface {
WatchWithPrefix(key string) clientv3.WatchChan
WatchWithRevision(key string, revision int64) clientv3.WatchChan
SaveWithLease(key, value string, id clientv3.LeaseID) error
SaveWithIgnoreLease(key, value string) error
Grant(ttl int64) (id clientv3.LeaseID, err error)
KeepAlive(id clientv3.LeaseID) (<-chan *clientv3.LeaseKeepAliveResponse, error)
CompareValueAndSwap(key, value, target string, opts ...clientv3.OpOption) error

152
internal/kv/mock_kv.go Normal file
View File

@ -0,0 +1,152 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package kv
import (
"strings"
"github.com/milvus-io/milvus/internal/log"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
)
type mockBaseKV struct {
InMemKv map[string]string
}
func (m *mockBaseKV) Load(key string) (string, error) {
if val, ok := m.InMemKv[key]; ok {
return val, nil
}
return "", nil
}
func (m *mockBaseKV) MultiLoad(keys []string) ([]string, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockBaseKV) LoadWithPrefix(key string) ([]string, []string, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockBaseKV) Save(key string, value string) error {
panic("not implemented") // TODO: Implement
}
func (m *mockBaseKV) MultiSave(kvs map[string]string) error {
panic("not implemented") // TODO: Implement
}
func (m *mockBaseKV) Remove(key string) error {
panic("not implemented") // TODO: Implement
}
func (m *mockBaseKV) MultiRemove(keys []string) error {
panic("not implemented") // TODO: Implement
}
func (m *mockBaseKV) RemoveWithPrefix(key string) error {
panic("not implemented") // TODO: Implement
}
func (m *mockBaseKV) Close() {
panic("not implemented") // TODO: Implement
}
type mockTxnKV struct {
mockBaseKV
}
func (m *mockTxnKV) MultiSaveAndRemove(saves map[string]string, removals []string) error {
panic("not implemented") // TODO: Implement
}
func (m *mockTxnKV) MultiRemoveWithPrefix(keys []string) error {
panic("not implemented") // TODO: Implement
}
func (m *mockTxnKV) MultiSaveAndRemoveWithPrefix(saves map[string]string, removals []string) error {
panic("not implemented") // TODO: Implement
}
type MockMetaKV struct {
mockTxnKV
}
func (m *MockMetaKV) GetPath(key string) string {
panic("not implemented") // TODO: Implement
}
func (m *MockMetaKV) LoadWithPrefix(prefix string) ([]string, []string, error) {
keys := make([]string, 0, len(m.InMemKv))
values := make([]string, 0, len(m.InMemKv))
for k, v := range m.InMemKv {
if strings.HasPrefix(k, prefix) {
keys = append(keys, k)
values = append(values, v)
}
}
return keys, values, nil
}
func (m *MockMetaKV) LoadWithPrefix2(key string) ([]string, []string, []int64, error) {
panic("not implemented") // TODO: Implement
}
func (m *MockMetaKV) LoadWithRevision(key string) ([]string, []string, int64, error) {
panic("not implemented") // TODO: Implement
}
func (m *MockMetaKV) Watch(key string) clientv3.WatchChan {
panic("not implemented") // TODO: Implement
}
func (m *MockMetaKV) WatchWithPrefix(key string) clientv3.WatchChan {
panic("not implemented") // TODO: Implement
}
func (m *MockMetaKV) WatchWithRevision(key string, revision int64) clientv3.WatchChan {
panic("not implemented") // TODO: Implement
}
func (m *MockMetaKV) SaveWithLease(key, value string, id clientv3.LeaseID) error {
m.InMemKv[key] = value
log.Debug("Doing SaveWithLease", zap.String("key", key))
return nil
}
func (m *MockMetaKV) SaveWithIgnoreLease(key, value string) error {
m.InMemKv[key] = value
log.Debug("Doing SaveWithIgnoreLease", zap.String("key", key))
return nil
}
func (m *MockMetaKV) Grant(ttl int64) (id clientv3.LeaseID, err error) {
return 1, nil
}
func (m *MockMetaKV) KeepAlive(id clientv3.LeaseID) (<-chan *clientv3.LeaseKeepAliveResponse, error) {
panic("not implemented") // TODO: Implement
}
func (m *MockMetaKV) CompareValueAndSwap(key, value, target string, opts ...clientv3.OpOption) error {
panic("not implemented") // TODO: Implement
}
func (m *MockMetaKV) CompareVersionAndSwap(key string, version int64, target string, opts ...clientv3.OpOption) error {
panic("not implemented") // TODO: Implement
}

128
internal/kv/mock_kv_test.go Normal file
View File

@ -0,0 +1,128 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package kv
import (
"testing"
"github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
)
const testKey = "key"
const testValue = "value"
func TestMockKV_MetaKV(t *testing.T) {
mockKv := &MockMetaKV{}
mockKv.InMemKv = make(map[string]string)
var err error
value, err := mockKv.Load(testKey)
assert.Equal(t, "", value)
assert.NoError(t, err)
assert.Panics(t, func() {
mockKv.MultiLoad([]string{testKey})
})
_, _, err = mockKv.LoadWithPrefix(testKey)
assert.NoError(t, err)
assert.Panics(t, func() {
mockKv.Save(testKey, testValue)
})
assert.Panics(t, func() {
mockKv.MultiSave(map[string]string{testKey: testValue})
})
assert.Panics(t, func() {
mockKv.Remove(testKey)
})
assert.Panics(t, func() {
mockKv.MultiRemove([]string{testKey})
})
assert.Panics(t, func() {
mockKv.RemoveWithPrefix(testKey)
})
assert.Panics(t, func() {
mockKv.Close()
})
assert.Panics(t, func() {
mockKv.MultiSaveAndRemove(map[string]string{testKey: testValue}, []string{testKey})
})
assert.Panics(t, func() {
mockKv.MultiRemoveWithPrefix([]string{testKey})
})
assert.Panics(t, func() {
mockKv.MultiSaveAndRemoveWithPrefix(map[string]string{testKey: testValue}, []string{testKey})
})
assert.Panics(t, func() {
mockKv.GetPath(testKey)
})
assert.Panics(t, func() {
mockKv.LoadWithPrefix2(testKey)
})
assert.Panics(t, func() {
mockKv.LoadWithPrefix2(testKey)
})
assert.Panics(t, func() {
mockKv.LoadWithRevision(testKey)
})
assert.Panics(t, func() {
mockKv.Watch(testKey)
})
assert.Panics(t, func() {
mockKv.WatchWithPrefix(testKey)
})
assert.Panics(t, func() {
mockKv.WatchWithRevision(testKey, 100)
})
err = mockKv.SaveWithLease(testKey, testValue, 100)
assert.NoError(t, err)
err = mockKv.SaveWithIgnoreLease(testKey, testValue)
assert.NoError(t, err)
leaseID, err := mockKv.Grant(100)
assert.Equal(t, clientv3.LeaseID(1), leaseID)
assert.NoError(t, err)
assert.Panics(t, func() {
mockKv.KeepAlive(100)
})
assert.Panics(t, func() {
mockKv.CompareValueAndSwap(testKey, testValue, testValue)
})
assert.Panics(t, func() {
mockKv.CompareVersionAndSwap(testKey, 100, testKey)
})
}

View File

@ -428,6 +428,27 @@ message ImportTask {
repeated common.KeyValuePair infos = 7; // more informations about the task, bucket, etc.
}
message ImportTaskState {
common.ImportState stateCode = 1; // Import state code.
repeated int64 segments = 2; // Ids of segments created in import task.
repeated int64 row_ids = 3; // Row IDs for the newly inserted rows.
int64 row_count = 4; // # of rows added in the import task.
string error_message = 5; // Error message for the failed task.
}
message ImportTaskInfo {
int64 id = 1; // Task ID.
int64 request_id = 2; // Request ID of the import task.
int64 datanode_id = 3; // ID of DataNode that processes the task.
string collection_id = 4; // Collection ID for the import task.
string partition_id = 5; // Partition ID for the import task.
string bucket = 6; // Bucket for the import task.
bool row_based = 7; // Boolean indicating whether import files are row-based or column-based.
repeated string files = 8; // A list of files to import.
int64 create_ts = 9; // Timestamp when the import task is created.
ImportTaskState state = 10; // State of the import task.
}
message ImportTaskResponse {
common.Status status = 1;
int64 datanode_id = 2; // which datanode takes this task

View File

@ -3200,6 +3200,188 @@ func (m *ImportTask) GetInfos() []*commonpb.KeyValuePair {
return nil
}
type ImportTaskState struct {
StateCode commonpb.ImportState `protobuf:"varint,1,opt,name=stateCode,proto3,enum=milvus.proto.common.ImportState" json:"stateCode,omitempty"`
Segments []int64 `protobuf:"varint,2,rep,packed,name=segments,proto3" json:"segments,omitempty"`
RowIds []int64 `protobuf:"varint,3,rep,packed,name=row_ids,json=rowIds,proto3" json:"row_ids,omitempty"`
RowCount int64 `protobuf:"varint,4,opt,name=row_count,json=rowCount,proto3" json:"row_count,omitempty"`
ErrorMessage string `protobuf:"bytes,5,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ImportTaskState) Reset() { *m = ImportTaskState{} }
func (m *ImportTaskState) String() string { return proto.CompactTextString(m) }
func (*ImportTaskState) ProtoMessage() {}
func (*ImportTaskState) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{51}
}
func (m *ImportTaskState) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ImportTaskState.Unmarshal(m, b)
}
func (m *ImportTaskState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ImportTaskState.Marshal(b, m, deterministic)
}
func (m *ImportTaskState) XXX_Merge(src proto.Message) {
xxx_messageInfo_ImportTaskState.Merge(m, src)
}
func (m *ImportTaskState) XXX_Size() int {
return xxx_messageInfo_ImportTaskState.Size(m)
}
func (m *ImportTaskState) XXX_DiscardUnknown() {
xxx_messageInfo_ImportTaskState.DiscardUnknown(m)
}
var xxx_messageInfo_ImportTaskState proto.InternalMessageInfo
func (m *ImportTaskState) GetStateCode() commonpb.ImportState {
if m != nil {
return m.StateCode
}
return commonpb.ImportState_ImportPending
}
func (m *ImportTaskState) GetSegments() []int64 {
if m != nil {
return m.Segments
}
return nil
}
func (m *ImportTaskState) GetRowIds() []int64 {
if m != nil {
return m.RowIds
}
return nil
}
func (m *ImportTaskState) GetRowCount() int64 {
if m != nil {
return m.RowCount
}
return 0
}
func (m *ImportTaskState) GetErrorMessage() string {
if m != nil {
return m.ErrorMessage
}
return ""
}
type ImportTaskInfo struct {
Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
RequestId int64 `protobuf:"varint,2,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"`
DatanodeId int64 `protobuf:"varint,3,opt,name=datanode_id,json=datanodeId,proto3" json:"datanode_id,omitempty"`
CollectionId string `protobuf:"bytes,4,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
PartitionId string `protobuf:"bytes,5,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"`
Bucket string `protobuf:"bytes,6,opt,name=bucket,proto3" json:"bucket,omitempty"`
RowBased bool `protobuf:"varint,7,opt,name=row_based,json=rowBased,proto3" json:"row_based,omitempty"`
Files []string `protobuf:"bytes,8,rep,name=files,proto3" json:"files,omitempty"`
CreateTs int64 `protobuf:"varint,9,opt,name=create_ts,json=createTs,proto3" json:"create_ts,omitempty"`
State *ImportTaskState `protobuf:"bytes,10,opt,name=state,proto3" json:"state,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ImportTaskInfo) Reset() { *m = ImportTaskInfo{} }
func (m *ImportTaskInfo) String() string { return proto.CompactTextString(m) }
func (*ImportTaskInfo) ProtoMessage() {}
func (*ImportTaskInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{52}
}
func (m *ImportTaskInfo) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ImportTaskInfo.Unmarshal(m, b)
}
func (m *ImportTaskInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ImportTaskInfo.Marshal(b, m, deterministic)
}
func (m *ImportTaskInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_ImportTaskInfo.Merge(m, src)
}
func (m *ImportTaskInfo) XXX_Size() int {
return xxx_messageInfo_ImportTaskInfo.Size(m)
}
func (m *ImportTaskInfo) XXX_DiscardUnknown() {
xxx_messageInfo_ImportTaskInfo.DiscardUnknown(m)
}
var xxx_messageInfo_ImportTaskInfo proto.InternalMessageInfo
func (m *ImportTaskInfo) GetId() int64 {
if m != nil {
return m.Id
}
return 0
}
func (m *ImportTaskInfo) GetRequestId() int64 {
if m != nil {
return m.RequestId
}
return 0
}
func (m *ImportTaskInfo) GetDatanodeId() int64 {
if m != nil {
return m.DatanodeId
}
return 0
}
func (m *ImportTaskInfo) GetCollectionId() string {
if m != nil {
return m.CollectionId
}
return ""
}
func (m *ImportTaskInfo) GetPartitionId() string {
if m != nil {
return m.PartitionId
}
return ""
}
func (m *ImportTaskInfo) GetBucket() string {
if m != nil {
return m.Bucket
}
return ""
}
func (m *ImportTaskInfo) GetRowBased() bool {
if m != nil {
return m.RowBased
}
return false
}
func (m *ImportTaskInfo) GetFiles() []string {
if m != nil {
return m.Files
}
return nil
}
func (m *ImportTaskInfo) GetCreateTs() int64 {
if m != nil {
return m.CreateTs
}
return 0
}
func (m *ImportTaskInfo) GetState() *ImportTaskState {
if m != nil {
return m.State
}
return nil
}
type ImportTaskResponse struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
DatanodeId int64 `protobuf:"varint,2,opt,name=datanode_id,json=datanodeId,proto3" json:"datanode_id,omitempty"`
@ -3212,7 +3394,7 @@ func (m *ImportTaskResponse) Reset() { *m = ImportTaskResponse{} }
func (m *ImportTaskResponse) String() string { return proto.CompactTextString(m) }
func (*ImportTaskResponse) ProtoMessage() {}
func (*ImportTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{51}
return fileDescriptor_82cd95f524594f49, []int{53}
}
func (m *ImportTaskResponse) XXX_Unmarshal(b []byte) error {
@ -3301,205 +3483,218 @@ func init() {
proto.RegisterType((*DropVirtualChannelSegment)(nil), "milvus.proto.data.DropVirtualChannelSegment")
proto.RegisterType((*DropVirtualChannelResponse)(nil), "milvus.proto.data.DropVirtualChannelResponse")
proto.RegisterType((*ImportTask)(nil), "milvus.proto.data.ImportTask")
proto.RegisterType((*ImportTaskState)(nil), "milvus.proto.data.ImportTaskState")
proto.RegisterType((*ImportTaskInfo)(nil), "milvus.proto.data.ImportTaskInfo")
proto.RegisterType((*ImportTaskResponse)(nil), "milvus.proto.data.ImportTaskResponse")
}
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
var fileDescriptor_82cd95f524594f49 = []byte{
// 3057 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x1b, 0x4d, 0x6f, 0x1b, 0xc7,
0xd5, 0xcb, 0x25, 0x29, 0xf2, 0xf1, 0x43, 0xf4, 0xd8, 0x91, 0x69, 0xda, 0x91, 0xe5, 0x4d, 0x6c,
0x2b, 0x8e, 0x63, 0x27, 0x72, 0x83, 0x04, 0x4d, 0xd2, 0x20, 0xb2, 0x22, 0x85, 0xa8, 0xe4, 0x2a,
0x2b, 0x25, 0x29, 0x9a, 0xa2, 0xc4, 0x8a, 0x3b, 0xa2, 0x36, 0xda, 0x0f, 0x66, 0x67, 0xe9, 0x8f,
0x5c, 0x62, 0x34, 0x40, 0x81, 0x16, 0x41, 0xda, 0xa2, 0x40, 0x81, 0x02, 0x45, 0x51, 0xf4, 0xd4,
0xa2, 0x97, 0xf6, 0xd8, 0x16, 0xbd, 0x07, 0xed, 0xbf, 0xe8, 0xa5, 0xb7, 0xfe, 0x86, 0x62, 0x3e,
0x76, 0xf6, 0x83, 0x4b, 0x72, 0x25, 0xd9, 0xf1, 0x4d, 0xf3, 0xf6, 0xbd, 0x37, 0x6f, 0xde, 0xbc,
0x6f, 0x8e, 0xa0, 0x65, 0x1a, 0x81, 0xd1, 0xeb, 0x7b, 0x9e, 0x6f, 0xde, 0x1c, 0xfa, 0x5e, 0xe0,
0xa1, 0xd3, 0x8e, 0x65, 0xdf, 0x1b, 0x11, 0xbe, 0xba, 0x49, 0x3f, 0x77, 0xea, 0x7d, 0xcf, 0x71,
0x3c, 0x97, 0x83, 0x3a, 0x4d, 0xcb, 0x0d, 0xb0, 0xef, 0x1a, 0xb6, 0x58, 0xd7, 0xe3, 0x04, 0x9d,
0x3a, 0xe9, 0x1f, 0x60, 0xc7, 0xe0, 0x2b, 0xed, 0x01, 0xd4, 0xd7, 0xed, 0x11, 0x39, 0xd0, 0xf1,
0xa7, 0x23, 0x4c, 0x02, 0xf4, 0x32, 0x14, 0xf7, 0x0c, 0x82, 0xdb, 0xca, 0x92, 0xb2, 0x5c, 0x5b,
0xb9, 0x78, 0x33, 0xb1, 0x97, 0xd8, 0x65, 0x8b, 0x0c, 0x56, 0x0d, 0x82, 0x75, 0x86, 0x89, 0x10,
0x14, 0xcd, 0xbd, 0xee, 0x5a, 0xbb, 0xb0, 0xa4, 0x2c, 0xab, 0x3a, 0xfb, 0x1b, 0x69, 0x50, 0xef,
0x7b, 0xb6, 0x8d, 0xfb, 0x81, 0xe5, 0xb9, 0xdd, 0xb5, 0x76, 0x91, 0x7d, 0x4b, 0xc0, 0xb4, 0xdf,
0x2a, 0xd0, 0x10, 0x5b, 0x93, 0xa1, 0xe7, 0x12, 0x8c, 0x6e, 0x43, 0x99, 0x04, 0x46, 0x30, 0x22,
0x62, 0xf7, 0x0b, 0x99, 0xbb, 0xef, 0x30, 0x14, 0x5d, 0xa0, 0xe6, 0xda, 0x5e, 0x1d, 0xdf, 0x1e,
0x2d, 0x02, 0x10, 0x3c, 0x70, 0xb0, 0x1b, 0x74, 0xd7, 0x48, 0xbb, 0xb8, 0xa4, 0x2e, 0xab, 0x7a,
0x0c, 0xa2, 0xfd, 0x52, 0x81, 0xd6, 0x4e, 0xb8, 0x0c, 0xb5, 0x73, 0x16, 0x4a, 0x7d, 0x6f, 0xe4,
0x06, 0x4c, 0xc0, 0x86, 0xce, 0x17, 0xe8, 0x32, 0xd4, 0xfb, 0x07, 0x86, 0xeb, 0x62, 0xbb, 0xe7,
0x1a, 0x0e, 0x66, 0xa2, 0x54, 0xf5, 0x9a, 0x80, 0xdd, 0x35, 0x1c, 0x9c, 0x4b, 0xa2, 0x25, 0xa8,
0x0d, 0x0d, 0x3f, 0xb0, 0x12, 0x3a, 0x8b, 0x83, 0xb4, 0xdf, 0x2b, 0xb0, 0xf0, 0x0e, 0x21, 0xd6,
0xc0, 0x1d, 0x93, 0x6c, 0x01, 0xca, 0xae, 0x67, 0xe2, 0xee, 0x1a, 0x13, 0x4d, 0xd5, 0xc5, 0x0a,
0x5d, 0x80, 0xea, 0x10, 0x63, 0xbf, 0xe7, 0x7b, 0x76, 0x28, 0x58, 0x85, 0x02, 0x74, 0xcf, 0xc6,
0xe8, 0x7d, 0x38, 0x4d, 0x52, 0x8c, 0x48, 0x5b, 0x5d, 0x52, 0x97, 0x6b, 0x2b, 0xcf, 0xdd, 0x1c,
0xb3, 0xb2, 0x9b, 0xe9, 0x4d, 0xf5, 0x71, 0x6a, 0xed, 0x51, 0x01, 0xce, 0x48, 0x3c, 0x2e, 0x2b,
0xfd, 0x9b, 0x6a, 0x8e, 0xe0, 0x81, 0x14, 0x8f, 0x2f, 0xf2, 0x68, 0x4e, 0xaa, 0x5c, 0x8d, 0xab,
0x3c, 0x87, 0x81, 0xa5, 0xf5, 0x59, 0x1a, 0xd3, 0x27, 0xba, 0x04, 0x35, 0xfc, 0x60, 0x68, 0xf9,
0xb8, 0x17, 0x58, 0x0e, 0x6e, 0x97, 0x97, 0x94, 0xe5, 0xa2, 0x0e, 0x1c, 0xb4, 0x6b, 0x39, 0x71,
0x8b, 0x9c, 0xcb, 0x6d, 0x91, 0xda, 0x1f, 0x14, 0x38, 0x37, 0x76, 0x4b, 0xc2, 0xc4, 0x75, 0x68,
0xb1, 0x93, 0x47, 0x9a, 0xa1, 0xc6, 0x4e, 0x15, 0x7e, 0x75, 0x9a, 0xc2, 0x23, 0x74, 0x7d, 0x8c,
0x3e, 0x26, 0x64, 0x21, 0xbf, 0x90, 0x87, 0x70, 0x6e, 0x03, 0x07, 0x62, 0x03, 0xfa, 0x0d, 0x93,
0xe3, 0x87, 0x80, 0xa4, 0x2f, 0x15, 0xc6, 0x7c, 0xe9, 0x2f, 0x05, 0xe9, 0x4b, 0x6c, 0xab, 0xae,
0xbb, 0xef, 0xa1, 0x8b, 0x50, 0x95, 0x28, 0xc2, 0x2a, 0x22, 0x00, 0x7a, 0x0d, 0x4a, 0x54, 0x52,
0x6e, 0x12, 0xcd, 0x95, 0xcb, 0xd9, 0x67, 0x8a, 0xf1, 0xd4, 0x39, 0x3e, 0xea, 0x42, 0x93, 0x04,
0x86, 0x1f, 0xf4, 0x86, 0x1e, 0x61, 0xf7, 0xcc, 0x0c, 0xa7, 0xb6, 0xa2, 0x25, 0x39, 0xc8, 0x10,
0xb9, 0x45, 0x06, 0xdb, 0x02, 0x53, 0x6f, 0x30, 0xca, 0x70, 0x89, 0xde, 0x85, 0x3a, 0x76, 0xcd,
0x88, 0x51, 0x31, 0x37, 0xa3, 0x1a, 0x76, 0x4d, 0xc9, 0x26, 0xba, 0x9f, 0x52, 0xfe, 0xfb, 0xf9,
0x52, 0x81, 0xf6, 0xf8, 0x05, 0x9d, 0x24, 0x50, 0xbe, 0xc1, 0x89, 0x30, 0xbf, 0xa0, 0xa9, 0x1e,
0x2e, 0x2f, 0x49, 0x17, 0x24, 0x9a, 0x05, 0xcf, 0x44, 0xd2, 0xb0, 0x2f, 0x4f, 0xcc, 0x58, 0xbe,
0x50, 0x60, 0x21, 0xbd, 0xd7, 0x49, 0xce, 0xfd, 0x2d, 0x28, 0x59, 0xee, 0xbe, 0x17, 0x1e, 0x7b,
0x71, 0x8a, 0x9f, 0xd1, 0xbd, 0x38, 0xb2, 0xe6, 0xc0, 0x85, 0x0d, 0x1c, 0x74, 0x5d, 0x82, 0xfd,
0x60, 0xd5, 0x72, 0x6d, 0x6f, 0xb0, 0x6d, 0x04, 0x07, 0x27, 0xf0, 0x91, 0x84, 0xb9, 0x17, 0x52,
0xe6, 0xae, 0xfd, 0x51, 0x81, 0x8b, 0xd9, 0xfb, 0x89, 0xa3, 0x77, 0xa0, 0xb2, 0x6f, 0x61, 0xdb,
0xa4, 0x3a, 0x53, 0x98, 0xce, 0xe4, 0x9a, 0xfa, 0xca, 0x90, 0x22, 0x8b, 0x13, 0x5e, 0x9e, 0x60,
0xa0, 0x3b, 0x81, 0x6f, 0xb9, 0x83, 0x4d, 0x8b, 0x04, 0x3a, 0xc7, 0x8f, 0xe9, 0x53, 0xcd, 0x6f,
0x99, 0x3f, 0x53, 0x60, 0x71, 0x03, 0x07, 0x77, 0x64, 0xa8, 0xa5, 0xdf, 0x2d, 0x12, 0x58, 0x7d,
0xf2, 0x64, 0x8b, 0x88, 0x8c, 0x9c, 0xa9, 0xfd, 0x5c, 0x81, 0x4b, 0x13, 0x85, 0x11, 0xaa, 0x13,
0xa1, 0x24, 0x0c, 0xb4, 0xd9, 0xa1, 0xe4, 0xbb, 0xf8, 0xe1, 0x87, 0x86, 0x3d, 0xc2, 0xdb, 0x86,
0xe5, 0xf3, 0x50, 0x72, 0xcc, 0xc0, 0xfa, 0x67, 0x05, 0x9e, 0xdd, 0xc0, 0xc1, 0x76, 0x98, 0x66,
0x9e, 0xa2, 0x76, 0x72, 0x54, 0x14, 0x5f, 0xf1, 0xcb, 0xcc, 0x94, 0xf6, 0xa9, 0xa8, 0x6f, 0x91,
0xf9, 0x41, 0xcc, 0x21, 0xef, 0xf0, 0x5a, 0x40, 0x28, 0x4f, 0x7b, 0xa4, 0x42, 0xfd, 0x43, 0x51,
0x1f, 0xb0, 0x34, 0x92, 0xd6, 0x83, 0x92, 0xad, 0x87, 0x58, 0x49, 0x91, 0x55, 0x65, 0x6c, 0x40,
0x83, 0x60, 0x7c, 0x78, 0x9c, 0xa4, 0x51, 0xa7, 0x84, 0x32, 0xd8, 0x6f, 0xc2, 0xe9, 0x91, 0xbb,
0x4f, 0xcb, 0x5a, 0x6c, 0x8a, 0x53, 0xf0, 0xea, 0x72, 0x76, 0xe4, 0x19, 0x27, 0x44, 0xef, 0xc1,
0x7c, 0x9a, 0x57, 0x29, 0x17, 0xaf, 0x34, 0x19, 0xea, 0x42, 0xcb, 0xf4, 0xbd, 0xe1, 0x10, 0x9b,
0x3d, 0x12, 0xb2, 0x2a, 0xe7, 0x63, 0x25, 0xe8, 0x42, 0x56, 0xda, 0x4f, 0x15, 0x58, 0xf8, 0xc8,
0x08, 0xfa, 0x07, 0x6b, 0x8e, 0xb8, 0x9c, 0x13, 0x98, 0xf6, 0x5b, 0x50, 0xbd, 0x27, 0x2e, 0x22,
0x8c, 0x5f, 0x97, 0x32, 0x04, 0x8a, 0x5f, 0xb9, 0x1e, 0x51, 0x68, 0x5f, 0x2b, 0x70, 0x96, 0x35,
0x11, 0xa1, 0x74, 0xdf, 0xbc, 0x93, 0xcd, 0x68, 0x24, 0xd0, 0x55, 0x68, 0x3a, 0x86, 0x7f, 0xb8,
0x13, 0xe1, 0x94, 0x18, 0x4e, 0x0a, 0xaa, 0x3d, 0x00, 0x10, 0xab, 0x2d, 0x32, 0x38, 0x86, 0xfc,
0xaf, 0xc3, 0x9c, 0xd8, 0x55, 0xf8, 0xdb, 0xac, 0x8b, 0x0d, 0xd1, 0xb5, 0x7f, 0x29, 0xd0, 0x8c,
0x22, 0x28, 0xf3, 0xaa, 0x26, 0x14, 0xa4, 0x2f, 0x15, 0xba, 0x6b, 0xe8, 0x2d, 0x28, 0xf3, 0xb6,
0x51, 0xf0, 0xbe, 0x92, 0xe4, 0x2d, 0x5a, 0xca, 0x58, 0x18, 0x66, 0x00, 0x5d, 0x10, 0x51, 0x1d,
0xc9, 0xa8, 0xc3, 0x3b, 0x0c, 0x55, 0x8f, 0x41, 0x50, 0x17, 0xe6, 0x93, 0x45, 0x5b, 0xe8, 0x33,
0x4b, 0x93, 0xa2, 0xcd, 0x9a, 0x11, 0x18, 0x2c, 0xd8, 0x34, 0x13, 0x35, 0x1b, 0xd1, 0xfe, 0x57,
0x82, 0x5a, 0xec, 0x94, 0x63, 0x27, 0x49, 0x5f, 0x69, 0x61, 0x76, 0xdc, 0x54, 0xc7, 0x3b, 0x87,
0x2b, 0xd0, 0xb4, 0x58, 0xae, 0xee, 0x09, 0x53, 0x64, 0xc1, 0xb5, 0xaa, 0x37, 0x38, 0x54, 0xf8,
0x05, 0x5a, 0x84, 0x9a, 0x3b, 0x72, 0x7a, 0xde, 0x7e, 0xcf, 0xf7, 0xee, 0x13, 0xd1, 0x82, 0x54,
0xdd, 0x91, 0xf3, 0xbd, 0x7d, 0xdd, 0xbb, 0x4f, 0xa2, 0x2a, 0xb7, 0x7c, 0xc4, 0x2a, 0x77, 0x11,
0x6a, 0x8e, 0xf1, 0x80, 0x72, 0xed, 0xb9, 0x23, 0x87, 0x75, 0x27, 0xaa, 0x5e, 0x75, 0x8c, 0x07,
0xba, 0x77, 0xff, 0xee, 0xc8, 0x41, 0xcb, 0xd0, 0xb2, 0x0d, 0x12, 0xf4, 0xe2, 0xed, 0x4d, 0x85,
0xb5, 0x37, 0x4d, 0x0a, 0x7f, 0x37, 0x6a, 0x71, 0xc6, 0xeb, 0xe5, 0xea, 0x09, 0xea, 0x65, 0xd3,
0xb1, 0x23, 0x46, 0x90, 0xbf, 0x5e, 0x36, 0x1d, 0x5b, 0xb2, 0x79, 0x1d, 0xe6, 0xf6, 0x58, 0x05,
0x44, 0xda, 0xb5, 0x89, 0x11, 0x6a, 0x9d, 0x16, 0x3f, 0xbc, 0x50, 0xd2, 0x43, 0x74, 0xf4, 0x26,
0x54, 0x59, 0xea, 0x61, 0xb4, 0xf5, 0x5c, 0xb4, 0x11, 0x01, 0xa5, 0x36, 0xb1, 0x1d, 0x18, 0x8c,
0xba, 0x91, 0x8f, 0x5a, 0x12, 0xa0, 0x97, 0xe1, 0x4c, 0xdf, 0xc7, 0x46, 0x80, 0xcd, 0xd5, 0x87,
0x77, 0x3c, 0x67, 0x68, 0x30, 0x63, 0x6a, 0x37, 0x97, 0x94, 0xe5, 0x8a, 0x9e, 0xf5, 0x89, 0x06,
0x86, 0xbe, 0x5c, 0xad, 0xfb, 0x9e, 0xd3, 0x9e, 0xe7, 0x81, 0x21, 0x09, 0x45, 0xcf, 0x02, 0x84,
0xa1, 0xdb, 0x08, 0xda, 0x2d, 0x76, 0x8b, 0x55, 0x01, 0x79, 0x27, 0xd0, 0x3e, 0x87, 0xb3, 0x91,
0x85, 0xc4, 0x6e, 0x63, 0xfc, 0x62, 0x95, 0xe3, 0x5e, 0xec, 0xf4, 0xda, 0xf5, 0xaf, 0x45, 0x58,
0xd8, 0x31, 0xee, 0xe1, 0x27, 0x5f, 0x26, 0xe7, 0x8a, 0xc7, 0x9b, 0x70, 0x9a, 0x55, 0xc6, 0x2b,
0x31, 0x79, 0xa6, 0x64, 0xe0, 0xf8, 0x75, 0x8e, 0x13, 0xa2, 0xb7, 0x69, 0xe9, 0x80, 0xfb, 0x87,
0xdb, 0x9e, 0x15, 0x65, 0xdf, 0x67, 0x33, 0xf8, 0xdc, 0x91, 0x58, 0x7a, 0x9c, 0x02, 0x6d, 0x8f,
0x87, 0x36, 0x9e, 0x77, 0xaf, 0x4d, 0xed, 0xbf, 0x22, 0xed, 0xa7, 0x23, 0x1c, 0x6a, 0xc3, 0x9c,
0xc8, 0xee, 0xcc, 0xef, 0x2b, 0x7a, 0xb8, 0x44, 0xdb, 0x70, 0x86, 0x9f, 0x60, 0x47, 0x18, 0x35,
0x3f, 0x7c, 0x25, 0xd7, 0xe1, 0xb3, 0x48, 0x93, 0x3e, 0x51, 0x3d, 0xaa, 0x4f, 0xb4, 0x61, 0x4e,
0xd8, 0x29, 0x8b, 0x05, 0x15, 0x3d, 0x5c, 0xd2, 0x26, 0x02, 0x22, 0x8d, 0xcd, 0x98, 0x05, 0x7c,
0x07, 0x2a, 0xd2, 0x86, 0x0b, 0xb9, 0x6d, 0x58, 0xd2, 0xa4, 0xa3, 0xb0, 0x9a, 0x8a, 0xc2, 0xda,
0xbf, 0x15, 0xa8, 0xaf, 0x51, 0xa1, 0x37, 0xbd, 0x01, 0xcb, 0x19, 0x57, 0xa0, 0xe9, 0xe3, 0xbe,
0xe7, 0x9b, 0x3d, 0xec, 0x06, 0xbe, 0x85, 0x79, 0xbf, 0x59, 0xd4, 0x1b, 0x1c, 0xfa, 0x2e, 0x07,
0x52, 0x34, 0x1a, 0x58, 0x49, 0x60, 0x38, 0xc3, 0xde, 0x3e, 0x75, 0xe0, 0x02, 0x47, 0x93, 0x50,
0xe6, 0xbf, 0x97, 0xa1, 0x1e, 0xa1, 0x05, 0x1e, 0xdb, 0xbf, 0xa8, 0xd7, 0x24, 0x6c, 0xd7, 0x43,
0xcf, 0x43, 0x93, 0x69, 0xad, 0x67, 0x7b, 0x83, 0x1e, 0xed, 0xcd, 0x44, 0x3a, 0xa9, 0x9b, 0x42,
0x2c, 0x7a, 0x1b, 0x49, 0x2c, 0x62, 0x7d, 0x86, 0x45, 0x42, 0x91, 0x58, 0x3b, 0xd6, 0x67, 0x98,
0x66, 0xf3, 0x06, 0xcd, 0x8e, 0x77, 0x3d, 0x13, 0xef, 0x1e, 0xb3, 0x96, 0xc8, 0x31, 0x97, 0xbb,
0x08, 0x55, 0x79, 0x02, 0x71, 0xa4, 0x08, 0x80, 0xd6, 0xa1, 0x19, 0x96, 0x99, 0x3d, 0xde, 0x3d,
0x14, 0x27, 0xd6, 0x76, 0xb1, 0xfc, 0x46, 0xf4, 0x46, 0x48, 0xc6, 0x96, 0xda, 0x3a, 0xd4, 0xe3,
0x9f, 0xe9, 0xae, 0x3b, 0x69, 0x43, 0x91, 0x00, 0x6a, 0x6f, 0x77, 0x47, 0x0e, 0xbd, 0x53, 0x11,
0x3a, 0xc2, 0xa5, 0xf6, 0x85, 0x02, 0x0d, 0x91, 0x94, 0x77, 0xe4, 0xdc, 0x98, 0x1d, 0x4d, 0x61,
0x47, 0x63, 0x7f, 0xa3, 0x6f, 0x27, 0x87, 0x4e, 0xcf, 0x67, 0xba, 0x39, 0x63, 0xc2, 0xea, 0xdf,
0x44, 0x46, 0xce, 0xd3, 0xad, 0x3e, 0xa2, 0x86, 0x26, 0xae, 0x86, 0x19, 0x5a, 0x1b, 0xe6, 0x0c,
0xd3, 0xf4, 0x31, 0x21, 0x42, 0x8e, 0x70, 0x49, 0xbf, 0xdc, 0xc3, 0x3e, 0x09, 0x4d, 0x5e, 0xd5,
0xc3, 0x25, 0x7a, 0x13, 0x2a, 0xb2, 0x60, 0x56, 0xb3, 0x8a, 0xa4, 0xb8, 0x9c, 0xa2, 0xbb, 0x92,
0x14, 0xda, 0x57, 0x05, 0x68, 0x0a, 0x85, 0xad, 0x8a, 0xac, 0x39, 0xdd, 0xf9, 0x56, 0xa1, 0xbe,
0x1f, 0x79, 0xf7, 0xb4, 0x29, 0x4a, 0x3c, 0x08, 0x24, 0x68, 0x66, 0x39, 0x60, 0x32, 0x6f, 0x17,
0x4f, 0x94, 0xb7, 0x4b, 0x47, 0x8c, 0x51, 0xda, 0x0f, 0xa1, 0x16, 0xfb, 0xc2, 0x82, 0x2b, 0x9f,
0xab, 0x08, 0x55, 0x84, 0x4b, 0x74, 0x3b, 0x2a, 0x4b, 0xb8, 0x0e, 0xce, 0x67, 0x6c, 0x92, 0xaa,
0x48, 0xb4, 0x3f, 0x29, 0x50, 0x16, 0x9c, 0x2f, 0x41, 0x4d, 0x44, 0x13, 0x56, 0xb2, 0x71, 0xee,
0x20, 0x40, 0xb4, 0x66, 0x7b, 0x7c, 0xe1, 0xe4, 0x3c, 0x54, 0x52, 0x81, 0x64, 0x4e, 0x44, 0xf4,
0xf0, 0x53, 0x2c, 0x7a, 0xd0, 0x4f, 0x2c, 0x70, 0x7c, 0xad, 0xb0, 0x99, 0xb0, 0x8e, 0xfb, 0xde,
0x3d, 0xec, 0x3f, 0x3c, 0xf9, 0xe4, 0xed, 0x8d, 0x98, 0xa5, 0xe6, 0x6c, 0xed, 0x24, 0x01, 0x7a,
0x23, 0x52, 0xb7, 0x9a, 0x35, 0x78, 0x88, 0x87, 0x0e, 0x61, 0x67, 0x91, 0xda, 0x7f, 0xc1, 0x67,
0x88, 0xc9, 0xa3, 0x1c, 0xb7, 0x24, 0x79, 0x2c, 0x1d, 0x83, 0xf6, 0x2b, 0x05, 0xce, 0x6f, 0xe0,
0x60, 0x3d, 0xd9, 0x97, 0x3f, 0x6d, 0xa9, 0x1c, 0xe8, 0x64, 0x09, 0x75, 0x92, 0x5b, 0xef, 0x40,
0x45, 0x4e, 0x18, 0xf8, 0x74, 0x57, 0xae, 0xb5, 0x9f, 0x28, 0xd0, 0x16, 0xbb, 0xb0, 0x3d, 0x69,
0x35, 0x6c, 0xe3, 0x00, 0x9b, 0xdf, 0x74, 0xcb, 0xfb, 0x4f, 0x05, 0x5a, 0xf1, 0x50, 0xce, 0xa2,
0xf1, 0xab, 0x50, 0x62, 0x93, 0x05, 0x21, 0xc1, 0x4c, 0x63, 0xe5, 0xd8, 0x34, 0x64, 0xb0, 0x0a,
0x6d, 0x57, 0x66, 0x1d, 0xb1, 0x8c, 0xf2, 0x89, 0x7a, 0xf4, 0x7c, 0x22, 0xf2, 0xab, 0x37, 0xa2,
0x7c, 0xf9, 0xe4, 0x2e, 0x02, 0x68, 0x5f, 0x16, 0xa0, 0x1d, 0xb5, 0x12, 0xdf, 0x78, 0x40, 0x9f,
0x50, 0x68, 0xaa, 0x8f, 0xa9, 0xd0, 0x2c, 0x1e, 0x35, 0x88, 0xff, 0xa3, 0x00, 0xcd, 0x48, 0x1d,
0xdb, 0xb6, 0xe1, 0xa2, 0x05, 0x28, 0x0f, 0x6d, 0x23, 0x9a, 0x08, 0x8a, 0x15, 0xda, 0x91, 0x95,
0x49, 0x52, 0x01, 0x2f, 0x66, 0x5d, 0xce, 0x04, 0x0d, 0xeb, 0x29, 0x16, 0xb4, 0x45, 0xe3, 0x45,
0x3e, 0x6b, 0xb4, 0x45, 0x35, 0xc4, 0xad, 0x80, 0xf6, 0xd8, 0x37, 0x00, 0x89, 0xab, 0xeb, 0x59,
0x6e, 0x8f, 0xe0, 0xbe, 0xe7, 0x9a, 0xfc, 0x52, 0x4b, 0x7a, 0x4b, 0x7c, 0xe9, 0xba, 0x3b, 0x1c,
0x8e, 0x5e, 0x85, 0x62, 0xf0, 0x70, 0xc8, 0xc3, 0x73, 0x33, 0x33, 0xec, 0x45, 0x72, 0xed, 0x3e,
0x1c, 0x62, 0x9d, 0xa1, 0xa3, 0x45, 0x00, 0xca, 0x2a, 0xf0, 0x8d, 0x7b, 0xd8, 0x0e, 0x7f, 0xcb,
0x8c, 0x20, 0xd4, 0x4c, 0xc3, 0x59, 0xc5, 0x1c, 0xcf, 0x09, 0x62, 0xa9, 0xfd, 0xad, 0x00, 0xad,
0x88, 0xa5, 0x8e, 0xc9, 0xc8, 0x0e, 0x26, 0xea, 0x6f, 0x7a, 0x83, 0x36, 0x2b, 0xd3, 0xbf, 0x0d,
0x35, 0x31, 0x37, 0x39, 0xc2, 0x45, 0x03, 0x27, 0xd9, 0x9c, 0x62, 0x79, 0xa5, 0xc7, 0x64, 0x79,
0xe5, 0xa3, 0x5a, 0xde, 0x0e, 0x2c, 0x84, 0x01, 0x2d, 0x42, 0xd8, 0xc2, 0x81, 0x31, 0xa5, 0x92,
0xb8, 0x04, 0x35, 0x9e, 0xa8, 0x78, 0x86, 0xe6, 0xc5, 0x35, 0xec, 0xc9, 0xae, 0x53, 0xfb, 0x11,
0x9c, 0x65, 0x01, 0x21, 0x3d, 0x5e, 0xcd, 0x33, 0xeb, 0xd6, 0x64, 0xe9, 0x4e, 0xcb, 0x74, 0x6e,
0xdd, 0x55, 0x3d, 0x01, 0xd3, 0x36, 0xe1, 0x99, 0x14, 0xff, 0x13, 0x04, 0x7c, 0xed, 0xef, 0x0a,
0x9c, 0x5f, 0xf3, 0xbd, 0xe1, 0x87, 0x96, 0x1f, 0x8c, 0x0c, 0x3b, 0x39, 0xb0, 0x7f, 0x32, 0xcd,
0xc7, 0x7b, 0xb1, 0x1c, 0xc3, 0x83, 0xce, 0x8d, 0x8c, 0x2b, 0x1b, 0x17, 0x4a, 0x5c, 0x55, 0x2c,
0x23, 0xfd, 0x57, 0xcd, 0x12, 0x5e, 0xe0, 0xcd, 0x88, 0xa4, 0x79, 0x52, 0x70, 0xe6, 0x34, 0x42,
0x3d, 0xee, 0x34, 0x62, 0x82, 0xf5, 0x17, 0x1f, 0x93, 0xf5, 0x1f, 0xb5, 0x78, 0x46, 0xef, 0x41,
0x72, 0x52, 0xc4, 0xc2, 0xce, 0xb1, 0x46, 0x4c, 0xab, 0x00, 0xd1, 0xd4, 0x44, 0xbc, 0xb6, 0xc8,
0xc3, 0x26, 0x46, 0x45, 0x6f, 0x4b, 0x46, 0x1a, 0x36, 0xed, 0x4c, 0x74, 0xf9, 0xef, 0x43, 0x27,
0xcb, 0x4a, 0x4f, 0x62, 0xf9, 0xbf, 0x2e, 0x00, 0x74, 0x9d, 0xa1, 0xe7, 0x07, 0xbb, 0x06, 0x39,
0x3c, 0x5e, 0xb9, 0x74, 0x0d, 0xe6, 0x23, 0x83, 0x89, 0x1b, 0x7c, 0x33, 0x02, 0x33, 0x9b, 0xbf,
0x02, 0x4d, 0x59, 0xb9, 0x71, 0x3c, 0x95, 0x8f, 0x9c, 0x25, 0x94, 0xa1, 0x5d, 0x80, 0xaa, 0xef,
0xdd, 0xef, 0x51, 0x4f, 0x32, 0x59, 0x8a, 0xa9, 0xe8, 0x15, 0xdf, 0xbb, 0x4f, 0xfd, 0xcb, 0x44,
0xe7, 0x60, 0x2e, 0x30, 0xc8, 0x61, 0xcf, 0x32, 0x45, 0xf1, 0x5f, 0xa6, 0xcb, 0xae, 0x89, 0xce,
0x42, 0x69, 0xdf, 0xb2, 0x31, 0x0f, 0x80, 0x55, 0x9d, 0x2f, 0xd0, 0x6b, 0xe1, 0x4f, 0xe7, 0x73,
0xb9, 0x7f, 0xfa, 0xe3, 0xbf, 0x9e, 0x7f, 0x02, 0x28, 0xd2, 0xcb, 0xc9, 0xca, 0xc9, 0x4b, 0x50,
0xa3, 0x16, 0xe8, 0x7a, 0x26, 0xa6, 0x62, 0x73, 0x1f, 0x83, 0x10, 0xd4, 0x35, 0xaf, 0xff, 0x46,
0x81, 0xd3, 0x63, 0x55, 0x14, 0x6a, 0x02, 0x7c, 0xe0, 0xf6, 0x45, 0x79, 0xd9, 0x3a, 0x85, 0xea,
0x50, 0x09, 0x8b, 0xcd, 0x96, 0x82, 0x6a, 0x30, 0xb7, 0xeb, 0x31, 0xec, 0x56, 0x01, 0xb5, 0xa0,
0xce, 0x09, 0x47, 0xfd, 0x3e, 0x26, 0xa4, 0xa5, 0x4a, 0xc8, 0xba, 0x61, 0xd9, 0x23, 0x1f, 0xb7,
0x8a, 0xa8, 0x01, 0xd5, 0x5d, 0x4f, 0xc7, 0x36, 0x36, 0x08, 0x6e, 0x95, 0x10, 0x82, 0xa6, 0x58,
0x84, 0x44, 0xe5, 0x18, 0x2c, 0x24, 0x9b, 0xbb, 0xbe, 0x1f, 0x2f, 0x4b, 0x68, 0xae, 0x46, 0xe7,
0xe0, 0xcc, 0x07, 0xae, 0x89, 0xf7, 0x2d, 0x17, 0x9b, 0xd1, 0xa7, 0xd6, 0x29, 0x74, 0x06, 0xe6,
0xbb, 0xae, 0x8b, 0xfd, 0x18, 0x50, 0xa1, 0xc0, 0x2d, 0xec, 0x0f, 0x70, 0x0c, 0x58, 0x40, 0xa7,
0xa1, 0xb1, 0x65, 0x3d, 0x88, 0x81, 0xd4, 0x95, 0xff, 0x9c, 0x81, 0xea, 0x9a, 0x11, 0x18, 0x77,
0x3c, 0xcf, 0x37, 0xd1, 0x10, 0x10, 0xfb, 0x4d, 0xdc, 0x19, 0x7a, 0xae, 0x7c, 0x3c, 0x82, 0x5e,
0x9e, 0xe0, 0x4d, 0xe3, 0xa8, 0x22, 0x74, 0x77, 0xae, 0x4e, 0xa0, 0x48, 0xa1, 0x6b, 0xa7, 0x90,
0xc3, 0x76, 0xa4, 0xb5, 0xce, 0xae, 0xd5, 0x3f, 0x0c, 0x7f, 0xfd, 0x98, 0xb2, 0x63, 0x0a, 0x35,
0xdc, 0x31, 0xf5, 0x26, 0x45, 0x2c, 0xf8, 0xc3, 0x85, 0xd0, 0x8c, 0xb4, 0x53, 0xe8, 0x53, 0x38,
0xbb, 0x81, 0x83, 0xe8, 0xb7, 0xea, 0x70, 0xc3, 0x95, 0xc9, 0x1b, 0x8e, 0x21, 0x1f, 0x71, 0xcb,
0x4d, 0x28, 0xb1, 0x8e, 0x05, 0x65, 0x75, 0x05, 0xf1, 0x17, 0x94, 0x9d, 0xa5, 0xc9, 0x08, 0x92,
0xdb, 0x27, 0x30, 0x9f, 0x7a, 0x21, 0x86, 0x5e, 0xc8, 0x20, 0xcb, 0x7e, 0xeb, 0xd7, 0xb9, 0x9e,
0x07, 0x55, 0xee, 0x35, 0x80, 0x66, 0xf2, 0x17, 0x75, 0xb4, 0x9c, 0x41, 0x9f, 0xf9, 0xba, 0xa7,
0xf3, 0x42, 0x0e, 0x4c, 0xb9, 0x91, 0x03, 0xad, 0xf4, 0x8b, 0x25, 0x74, 0x7d, 0x2a, 0x83, 0xa4,
0xb9, 0xbd, 0x98, 0x0b, 0x57, 0x6e, 0xf7, 0x90, 0x19, 0xc1, 0xd8, 0x8b, 0x19, 0x74, 0x33, 0x9b,
0xcd, 0xa4, 0xa7, 0x3c, 0x9d, 0x5b, 0xb9, 0xf1, 0xe5, 0xd6, 0x3f, 0xe6, 0x93, 0x92, 0xac, 0x57,
0x27, 0xe8, 0x95, 0x6c, 0x76, 0x53, 0x9e, 0xcb, 0x74, 0x56, 0x8e, 0x42, 0x22, 0x85, 0xf8, 0x9c,
0x8d, 0x38, 0x32, 0x5e, 0x6e, 0xa4, 0xfd, 0x2e, 0xe4, 0x37, 0xf9, 0x49, 0x4a, 0xe7, 0x95, 0x23,
0x50, 0x48, 0x01, 0xbc, 0xf4, 0x9b, 0xb0, 0xd0, 0x0d, 0x6f, 0xcd, 0xb4, 0x9a, 0xe3, 0xf9, 0xe0,
0xc7, 0x30, 0x9f, 0xfa, 0x9d, 0x29, 0xd3, 0x6b, 0xb2, 0x7f, 0x8b, 0xea, 0x4c, 0xcb, 0x36, 0xdc,
0x25, 0x53, 0x13, 0x23, 0x34, 0xc1, 0xfa, 0x33, 0xa6, 0x4a, 0x9d, 0xeb, 0x79, 0x50, 0xe5, 0x41,
0x08, 0x0b, 0x97, 0xa9, 0xa9, 0x0b, 0xba, 0x91, 0xcd, 0x23, 0x7b, 0x62, 0xd4, 0x79, 0x29, 0x27,
0xb6, 0xdc, 0xb4, 0x07, 0xb0, 0x81, 0x83, 0x2d, 0x1c, 0xf8, 0xd4, 0x46, 0xae, 0x66, 0xaa, 0x3c,
0x42, 0x08, 0xb7, 0xb9, 0x36, 0x13, 0x4f, 0x6e, 0xf0, 0x7d, 0x40, 0x61, 0x8a, 0x8d, 0xfd, 0xca,
0xf9, 0xdc, 0xd4, 0xfe, 0x95, 0x37, 0x9b, 0xb3, 0xee, 0xe6, 0x53, 0x68, 0x6d, 0x19, 0x2e, 0xad,
0xda, 0x22, 0xbe, 0x37, 0x32, 0x05, 0x4b, 0xa3, 0x4d, 0xd0, 0xd6, 0x44, 0x6c, 0x79, 0x98, 0xfb,
0x32, 0x87, 0x1a, 0xd2, 0x05, 0x71, 0x3a, 0xb6, 0x44, 0xda, 0x48, 0x21, 0x4e, 0x88, 0x2d, 0x53,
0xf0, 0xe5, 0xc6, 0x8f, 0x14, 0xf6, 0xf2, 0x30, 0x85, 0xf0, 0x91, 0x15, 0x1c, 0x6c, 0xdb, 0x86,
0x4b, 0xf2, 0x88, 0xc0, 0x10, 0x8f, 0x20, 0x82, 0xc0, 0x97, 0x22, 0x98, 0xd0, 0x48, 0xb4, 0x87,
0x28, 0xeb, 0xa7, 0xca, 0xac, 0x06, 0xb5, 0xb3, 0x3c, 0x1b, 0x51, 0xee, 0x72, 0x00, 0x8d, 0xd0,
0x5e, 0xb9, 0x72, 0x5f, 0x98, 0x24, 0x69, 0x84, 0x33, 0xc1, 0xdd, 0xb2, 0x51, 0xe3, 0xee, 0x36,
0x5e, 0xf9, 0xa3, 0x7c, 0x1d, 0xe3, 0x34, 0x77, 0x9b, 0xdc, 0x4e, 0x68, 0xa7, 0xd0, 0x36, 0x94,
0x79, 0x09, 0x8c, 0xb2, 0x7e, 0x2d, 0x8e, 0xaa, 0xe3, 0xce, 0x95, 0xa9, 0x9f, 0x23, 0x8e, 0x2b,
0xbf, 0x2b, 0x41, 0x25, 0xfc, 0xf5, 0xe8, 0x29, 0xd4, 0x78, 0x4f, 0xa1, 0xe8, 0xfa, 0x18, 0xe6,
0x53, 0x0f, 0xcd, 0x32, 0x63, 0x72, 0xf6, 0x63, 0xb4, 0x59, 0x41, 0xe5, 0x23, 0xf1, 0xef, 0x27,
0x32, 0xfe, 0x5e, 0x9b, 0x54, 0xb8, 0xa5, 0x43, 0xef, 0x0c, 0xc6, 0x4f, 0x3c, 0xd0, 0xde, 0x05,
0x88, 0x05, 0xc2, 0xe9, 0x03, 0x42, 0xea, 0xdb, 0xb3, 0x04, 0x5e, 0xcf, 0x6b, 0xaa, 0xd3, 0xf9,
0xac, 0xde, 0xfe, 0xc1, 0x2b, 0x03, 0x2b, 0x38, 0x18, 0xed, 0xd1, 0x2f, 0xb7, 0x38, 0xea, 0x4b,
0x96, 0x27, 0xfe, 0xba, 0x15, 0x5a, 0xc6, 0x2d, 0x46, 0x7d, 0x8b, 0x32, 0x1f, 0xee, 0xed, 0x95,
0xd9, 0xea, 0xf6, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0xe2, 0xae, 0xfb, 0xcd, 0xe8, 0x34, 0x00,
0x00,
// 3241 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x1b, 0x5d, 0x6f, 0x1c, 0x57,
0x35, 0xb3, 0xdf, 0x7b, 0xf6, 0xc3, 0x9b, 0x9b, 0x34, 0xd9, 0x6c, 0x12, 0xc7, 0x99, 0x36, 0x89,
0x9b, 0xa6, 0x49, 0xeb, 0x50, 0xb5, 0xa2, 0x5f, 0xaa, 0xed, 0xda, 0x5d, 0x61, 0x07, 0x77, 0xec,
0xb6, 0x88, 0x22, 0x56, 0xe3, 0x9d, 0xeb, 0xf5, 0xd4, 0xf3, 0xb1, 0x9d, 0x3b, 0x1b, 0x27, 0x7d,
0x69, 0x44, 0x25, 0x24, 0x50, 0x55, 0x40, 0x48, 0x48, 0x48, 0x08, 0x21, 0x9e, 0x40, 0xbc, 0xc0,
0x23, 0x20, 0xde, 0xab, 0xf2, 0x2f, 0x78, 0xe1, 0x8d, 0xdf, 0x80, 0xee, 0xc7, 0xdc, 0xf9, 0xd8,
0xd9, 0xdd, 0xb1, 0x9d, 0x34, 0x6f, 0xbe, 0x67, 0xce, 0x39, 0xf7, 0xdc, 0x73, 0xcf, 0xf7, 0x5e,
0x43, 0xcb, 0xd0, 0x7d, 0xbd, 0xd7, 0x77, 0x5d, 0xcf, 0xb8, 0x3d, 0xf4, 0x5c, 0xdf, 0x45, 0xa7,
0x6d, 0xd3, 0xba, 0x3f, 0x22, 0x7c, 0x75, 0x9b, 0x7e, 0xee, 0xd4, 0xfb, 0xae, 0x6d, 0xbb, 0x0e,
0x07, 0x75, 0x9a, 0xa6, 0xe3, 0x63, 0xcf, 0xd1, 0x2d, 0xb1, 0xae, 0x47, 0x09, 0x3a, 0x75, 0xd2,
0xdf, 0xc7, 0xb6, 0xce, 0x57, 0xea, 0x03, 0xa8, 0xaf, 0x59, 0x23, 0xb2, 0xaf, 0xe1, 0x4f, 0x47,
0x98, 0xf8, 0xe8, 0x25, 0x28, 0xec, 0xea, 0x04, 0xb7, 0x95, 0x05, 0x65, 0xb1, 0xb6, 0x74, 0xe9,
0x76, 0x6c, 0x2f, 0xb1, 0xcb, 0x26, 0x19, 0x2c, 0xeb, 0x04, 0x6b, 0x0c, 0x13, 0x21, 0x28, 0x18,
0xbb, 0xdd, 0xd5, 0x76, 0x6e, 0x41, 0x59, 0xcc, 0x6b, 0xec, 0x6f, 0xa4, 0x42, 0xbd, 0xef, 0x5a,
0x16, 0xee, 0xfb, 0xa6, 0xeb, 0x74, 0x57, 0xdb, 0x05, 0xf6, 0x2d, 0x06, 0x53, 0x7f, 0xa7, 0x40,
0x43, 0x6c, 0x4d, 0x86, 0xae, 0x43, 0x30, 0xba, 0x0b, 0x25, 0xe2, 0xeb, 0xfe, 0x88, 0x88, 0xdd,
0x2f, 0xa6, 0xee, 0xbe, 0xcd, 0x50, 0x34, 0x81, 0x9a, 0x69, 0xfb, 0xfc, 0xf8, 0xf6, 0x68, 0x1e,
0x80, 0xe0, 0x81, 0x8d, 0x1d, 0xbf, 0xbb, 0x4a, 0xda, 0x85, 0x85, 0xfc, 0x62, 0x5e, 0x8b, 0x40,
0xd4, 0x5f, 0x29, 0xd0, 0xda, 0x0e, 0x96, 0x81, 0x76, 0xce, 0x42, 0xb1, 0xef, 0x8e, 0x1c, 0x9f,
0x09, 0xd8, 0xd0, 0xf8, 0x02, 0x5d, 0x85, 0x7a, 0x7f, 0x5f, 0x77, 0x1c, 0x6c, 0xf5, 0x1c, 0xdd,
0xc6, 0x4c, 0x94, 0xaa, 0x56, 0x13, 0xb0, 0x7b, 0xba, 0x8d, 0x33, 0x49, 0xb4, 0x00, 0xb5, 0xa1,
0xee, 0xf9, 0x66, 0x4c, 0x67, 0x51, 0x90, 0xfa, 0x07, 0x05, 0xce, 0xbd, 0x43, 0x88, 0x39, 0x70,
0xc6, 0x24, 0x3b, 0x07, 0x25, 0xc7, 0x35, 0x70, 0x77, 0x95, 0x89, 0x96, 0xd7, 0xc4, 0x0a, 0x5d,
0x84, 0xea, 0x10, 0x63, 0xaf, 0xe7, 0xb9, 0x56, 0x20, 0x58, 0x85, 0x02, 0x34, 0xd7, 0xc2, 0xe8,
0x7d, 0x38, 0x4d, 0x12, 0x8c, 0x48, 0x3b, 0xbf, 0x90, 0x5f, 0xac, 0x2d, 0x3d, 0x7b, 0x7b, 0xcc,
0xca, 0x6e, 0x27, 0x37, 0xd5, 0xc6, 0xa9, 0xd5, 0x47, 0x39, 0x38, 0x23, 0xf1, 0xb8, 0xac, 0xf4,
0x6f, 0xaa, 0x39, 0x82, 0x07, 0x52, 0x3c, 0xbe, 0xc8, 0xa2, 0x39, 0xa9, 0xf2, 0x7c, 0x54, 0xe5,
0x19, 0x0c, 0x2c, 0xa9, 0xcf, 0xe2, 0x98, 0x3e, 0xd1, 0x15, 0xa8, 0xe1, 0x07, 0x43, 0xd3, 0xc3,
0x3d, 0xdf, 0xb4, 0x71, 0xbb, 0xb4, 0xa0, 0x2c, 0x16, 0x34, 0xe0, 0xa0, 0x1d, 0xd3, 0x8e, 0x5a,
0x64, 0x39, 0xb3, 0x45, 0xaa, 0x7f, 0x54, 0xe0, 0xfc, 0xd8, 0x2d, 0x09, 0x13, 0xd7, 0xa0, 0xc5,
0x4e, 0x1e, 0x6a, 0x86, 0x1a, 0x3b, 0x55, 0xf8, 0xf5, 0x69, 0x0a, 0x0f, 0xd1, 0xb5, 0x31, 0xfa,
0x88, 0x90, 0xb9, 0xec, 0x42, 0x1e, 0xc0, 0xf9, 0x75, 0xec, 0x8b, 0x0d, 0xe8, 0x37, 0x4c, 0x8e,
0x1f, 0x02, 0xe2, 0xbe, 0x94, 0x1b, 0xf3, 0xa5, 0xbf, 0xe6, 0xa4, 0x2f, 0xb1, 0xad, 0xba, 0xce,
0x9e, 0x8b, 0x2e, 0x41, 0x55, 0xa2, 0x08, 0xab, 0x08, 0x01, 0xe8, 0x55, 0x28, 0x52, 0x49, 0xb9,
0x49, 0x34, 0x97, 0xae, 0xa6, 0x9f, 0x29, 0xc2, 0x53, 0xe3, 0xf8, 0xa8, 0x0b, 0x4d, 0xe2, 0xeb,
0x9e, 0xdf, 0x1b, 0xba, 0x84, 0xdd, 0x33, 0x33, 0x9c, 0xda, 0x92, 0x1a, 0xe7, 0x20, 0x43, 0xe4,
0x26, 0x19, 0x6c, 0x09, 0x4c, 0xad, 0xc1, 0x28, 0x83, 0x25, 0x7a, 0x17, 0xea, 0xd8, 0x31, 0x42,
0x46, 0x85, 0xcc, 0x8c, 0x6a, 0xd8, 0x31, 0x24, 0x9b, 0xf0, 0x7e, 0x8a, 0xd9, 0xef, 0xe7, 0x4b,
0x05, 0xda, 0xe3, 0x17, 0x74, 0x92, 0x40, 0xf9, 0x3a, 0x27, 0xc2, 0xfc, 0x82, 0xa6, 0x7a, 0xb8,
0xbc, 0x24, 0x4d, 0x90, 0xa8, 0x26, 0x3c, 0x13, 0x4a, 0xc3, 0xbe, 0x3c, 0x31, 0x63, 0xf9, 0x42,
0x81, 0x73, 0xc9, 0xbd, 0x4e, 0x72, 0xee, 0xef, 0x40, 0xd1, 0x74, 0xf6, 0xdc, 0xe0, 0xd8, 0xf3,
0x53, 0xfc, 0x8c, 0xee, 0xc5, 0x91, 0x55, 0x1b, 0x2e, 0xae, 0x63, 0xbf, 0xeb, 0x10, 0xec, 0xf9,
0xcb, 0xa6, 0x63, 0xb9, 0x83, 0x2d, 0xdd, 0xdf, 0x3f, 0x81, 0x8f, 0xc4, 0xcc, 0x3d, 0x97, 0x30,
0x77, 0xf5, 0x4f, 0x0a, 0x5c, 0x4a, 0xdf, 0x4f, 0x1c, 0xbd, 0x03, 0x95, 0x3d, 0x13, 0x5b, 0x06,
0xd5, 0x99, 0xc2, 0x74, 0x26, 0xd7, 0xd4, 0x57, 0x86, 0x14, 0x59, 0x9c, 0xf0, 0xea, 0x04, 0x03,
0xdd, 0xf6, 0x3d, 0xd3, 0x19, 0x6c, 0x98, 0xc4, 0xd7, 0x38, 0x7e, 0x44, 0x9f, 0xf9, 0xec, 0x96,
0xf9, 0x73, 0x05, 0xe6, 0xd7, 0xb1, 0xbf, 0x22, 0x43, 0x2d, 0xfd, 0x6e, 0x12, 0xdf, 0xec, 0x93,
0x27, 0x5b, 0x44, 0xa4, 0xe4, 0x4c, 0xf5, 0x17, 0x0a, 0x5c, 0x99, 0x28, 0x8c, 0x50, 0x9d, 0x08,
0x25, 0x41, 0xa0, 0x4d, 0x0f, 0x25, 0xdf, 0xc3, 0x0f, 0x3f, 0xd4, 0xad, 0x11, 0xde, 0xd2, 0x4d,
0x8f, 0x87, 0x92, 0x63, 0x06, 0xd6, 0xbf, 0x28, 0x70, 0x79, 0x1d, 0xfb, 0x5b, 0x41, 0x9a, 0x79,
0x8a, 0xda, 0xc9, 0x50, 0x51, 0x7c, 0xc5, 0x2f, 0x33, 0x55, 0xda, 0xa7, 0xa2, 0xbe, 0x79, 0xe6,
0x07, 0x11, 0x87, 0x5c, 0xe1, 0xb5, 0x80, 0x50, 0x9e, 0xfa, 0x28, 0x0f, 0xf5, 0x0f, 0x45, 0x7d,
0xc0, 0xd2, 0x48, 0x52, 0x0f, 0x4a, 0xba, 0x1e, 0x22, 0x25, 0x45, 0x5a, 0x95, 0xb1, 0x0e, 0x0d,
0x82, 0xf1, 0xc1, 0x71, 0x92, 0x46, 0x9d, 0x12, 0xca, 0x60, 0xbf, 0x01, 0xa7, 0x47, 0xce, 0x1e,
0x2d, 0x6b, 0xb1, 0x21, 0x4e, 0xc1, 0xab, 0xcb, 0xd9, 0x91, 0x67, 0x9c, 0x10, 0xbd, 0x07, 0x73,
0x49, 0x5e, 0xc5, 0x4c, 0xbc, 0x92, 0x64, 0xa8, 0x0b, 0x2d, 0xc3, 0x73, 0x87, 0x43, 0x6c, 0xf4,
0x48, 0xc0, 0xaa, 0x94, 0x8d, 0x95, 0xa0, 0x0b, 0x58, 0xa9, 0x3f, 0x53, 0xe0, 0xdc, 0x47, 0xba,
0xdf, 0xdf, 0x5f, 0xb5, 0xc5, 0xe5, 0x9c, 0xc0, 0xb4, 0xdf, 0x84, 0xea, 0x7d, 0x71, 0x11, 0x41,
0xfc, 0xba, 0x92, 0x22, 0x50, 0xf4, 0xca, 0xb5, 0x90, 0x42, 0xfd, 0x5a, 0x81, 0xb3, 0xac, 0x89,
0x08, 0xa4, 0xfb, 0xf6, 0x9d, 0x6c, 0x46, 0x23, 0x81, 0xae, 0x43, 0xd3, 0xd6, 0xbd, 0x83, 0xed,
0x10, 0xa7, 0xc8, 0x70, 0x12, 0x50, 0xf5, 0x01, 0x80, 0x58, 0x6d, 0x92, 0xc1, 0x31, 0xe4, 0x7f,
0x0d, 0xca, 0x62, 0x57, 0xe1, 0x6f, 0xb3, 0x2e, 0x36, 0x40, 0x57, 0xbf, 0x51, 0xa0, 0x19, 0x46,
0x50, 0xe6, 0x55, 0x4d, 0xc8, 0x49, 0x5f, 0xca, 0x75, 0x57, 0xd1, 0x9b, 0x50, 0xe2, 0x6d, 0xa3,
0xe0, 0x7d, 0x2d, 0xce, 0x5b, 0xb4, 0x94, 0x91, 0x30, 0xcc, 0x00, 0x9a, 0x20, 0xa2, 0x3a, 0x92,
0x51, 0x87, 0x77, 0x18, 0x79, 0x2d, 0x02, 0x41, 0x5d, 0x98, 0x8b, 0x17, 0x6d, 0x81, 0xcf, 0x2c,
0x4c, 0x8a, 0x36, 0xab, 0xba, 0xaf, 0xb3, 0x60, 0xd3, 0x8c, 0xd5, 0x6c, 0x44, 0xfd, 0x5f, 0x11,
0x6a, 0x91, 0x53, 0x8e, 0x9d, 0x24, 0x79, 0xa5, 0xb9, 0xd9, 0x71, 0x33, 0x3f, 0xde, 0x39, 0x5c,
0x83, 0xa6, 0xc9, 0x72, 0x75, 0x4f, 0x98, 0x22, 0x0b, 0xae, 0x55, 0xad, 0xc1, 0xa1, 0xc2, 0x2f,
0xd0, 0x3c, 0xd4, 0x9c, 0x91, 0xdd, 0x73, 0xf7, 0x7a, 0x9e, 0x7b, 0x48, 0x44, 0x0b, 0x52, 0x75,
0x46, 0xf6, 0xf7, 0xf7, 0x34, 0xf7, 0x90, 0x84, 0x55, 0x6e, 0xe9, 0x88, 0x55, 0xee, 0x3c, 0xd4,
0x6c, 0xfd, 0x01, 0xe5, 0xda, 0x73, 0x46, 0x36, 0xeb, 0x4e, 0xf2, 0x5a, 0xd5, 0xd6, 0x1f, 0x68,
0xee, 0xe1, 0xbd, 0x91, 0x8d, 0x16, 0xa1, 0x65, 0xe9, 0xc4, 0xef, 0x45, 0xdb, 0x9b, 0x0a, 0x6b,
0x6f, 0x9a, 0x14, 0xfe, 0x6e, 0xd8, 0xe2, 0x8c, 0xd7, 0xcb, 0xd5, 0x13, 0xd4, 0xcb, 0x86, 0x6d,
0x85, 0x8c, 0x20, 0x7b, 0xbd, 0x6c, 0xd8, 0x96, 0x64, 0xf3, 0x1a, 0x94, 0x77, 0x59, 0x05, 0x44,
0xda, 0xb5, 0x89, 0x11, 0x6a, 0x8d, 0x16, 0x3f, 0xbc, 0x50, 0xd2, 0x02, 0x74, 0xf4, 0x06, 0x54,
0x59, 0xea, 0x61, 0xb4, 0xf5, 0x4c, 0xb4, 0x21, 0x01, 0xa5, 0x36, 0xb0, 0xe5, 0xeb, 0x8c, 0xba,
0x91, 0x8d, 0x5a, 0x12, 0xa0, 0x97, 0xe0, 0x4c, 0xdf, 0xc3, 0xba, 0x8f, 0x8d, 0xe5, 0x87, 0x2b,
0xae, 0x3d, 0xd4, 0x99, 0x31, 0xb5, 0x9b, 0x0b, 0xca, 0x62, 0x45, 0x4b, 0xfb, 0x44, 0x03, 0x43,
0x5f, 0xae, 0xd6, 0x3c, 0xd7, 0x6e, 0xcf, 0xf1, 0xc0, 0x10, 0x87, 0xa2, 0xcb, 0x00, 0x41, 0xe8,
0xd6, 0xfd, 0x76, 0x8b, 0xdd, 0x62, 0x55, 0x40, 0xde, 0xf1, 0xd5, 0xcf, 0xe1, 0x6c, 0x68, 0x21,
0x91, 0xdb, 0x18, 0xbf, 0x58, 0xe5, 0xb8, 0x17, 0x3b, 0xbd, 0x76, 0xfd, 0x5b, 0x01, 0xce, 0x6d,
0xeb, 0xf7, 0xf1, 0x93, 0x2f, 0x93, 0x33, 0xc5, 0xe3, 0x0d, 0x38, 0xcd, 0x2a, 0xe3, 0xa5, 0x88,
0x3c, 0x53, 0x32, 0x70, 0xf4, 0x3a, 0xc7, 0x09, 0xd1, 0xdb, 0xb4, 0x74, 0xc0, 0xfd, 0x83, 0x2d,
0xd7, 0x0c, 0xb3, 0xef, 0xe5, 0x14, 0x3e, 0x2b, 0x12, 0x4b, 0x8b, 0x52, 0xa0, 0xad, 0xf1, 0xd0,
0xc6, 0xf3, 0xee, 0x8d, 0xa9, 0xfd, 0x57, 0xa8, 0xfd, 0x64, 0x84, 0x43, 0x6d, 0x28, 0x8b, 0xec,
0xce, 0xfc, 0xbe, 0xa2, 0x05, 0x4b, 0xb4, 0x05, 0x67, 0xf8, 0x09, 0xb6, 0x85, 0x51, 0xf3, 0xc3,
0x57, 0x32, 0x1d, 0x3e, 0x8d, 0x34, 0xee, 0x13, 0xd5, 0xa3, 0xfa, 0x44, 0x1b, 0xca, 0xc2, 0x4e,
0x59, 0x2c, 0xa8, 0x68, 0xc1, 0x92, 0x36, 0x11, 0x10, 0x6a, 0x6c, 0xc6, 0x2c, 0xe0, 0x2d, 0xa8,
0x48, 0x1b, 0xce, 0x65, 0xb6, 0x61, 0x49, 0x93, 0x8c, 0xc2, 0xf9, 0x44, 0x14, 0x56, 0xff, 0xad,
0x40, 0x7d, 0x95, 0x0a, 0xbd, 0xe1, 0x0e, 0x58, 0xce, 0xb8, 0x06, 0x4d, 0x0f, 0xf7, 0x5d, 0xcf,
0xe8, 0x61, 0xc7, 0xf7, 0x4c, 0xcc, 0xfb, 0xcd, 0x82, 0xd6, 0xe0, 0xd0, 0x77, 0x39, 0x90, 0xa2,
0xd1, 0xc0, 0x4a, 0x7c, 0xdd, 0x1e, 0xf6, 0xf6, 0xa8, 0x03, 0xe7, 0x38, 0x9a, 0x84, 0x32, 0xff,
0xbd, 0x0a, 0xf5, 0x10, 0xcd, 0x77, 0xd9, 0xfe, 0x05, 0xad, 0x26, 0x61, 0x3b, 0x2e, 0x7a, 0x0e,
0x9a, 0x4c, 0x6b, 0x3d, 0xcb, 0x1d, 0xf4, 0x68, 0x6f, 0x26, 0xd2, 0x49, 0xdd, 0x10, 0x62, 0xd1,
0xdb, 0x88, 0x63, 0x11, 0xf3, 0x33, 0x2c, 0x12, 0x8a, 0xc4, 0xda, 0x36, 0x3f, 0xc3, 0x34, 0x9b,
0x37, 0x68, 0x76, 0xbc, 0xe7, 0x1a, 0x78, 0xe7, 0x98, 0xb5, 0x44, 0x86, 0xb9, 0xdc, 0x25, 0xa8,
0xca, 0x13, 0x88, 0x23, 0x85, 0x00, 0xb4, 0x06, 0xcd, 0xa0, 0xcc, 0xec, 0xf1, 0xee, 0xa1, 0x30,
0xb1, 0xb6, 0x8b, 0xe4, 0x37, 0xa2, 0x35, 0x02, 0x32, 0xb6, 0x54, 0xd7, 0xa0, 0x1e, 0xfd, 0x4c,
0x77, 0xdd, 0x4e, 0x1a, 0x8a, 0x04, 0x50, 0x7b, 0xbb, 0x37, 0xb2, 0xe9, 0x9d, 0x8a, 0xd0, 0x11,
0x2c, 0xd5, 0x2f, 0x14, 0x68, 0x88, 0xa4, 0xbc, 0x2d, 0xe7, 0xc6, 0xec, 0x68, 0x0a, 0x3b, 0x1a,
0xfb, 0x1b, 0x7d, 0x37, 0x3e, 0x74, 0x7a, 0x2e, 0xd5, 0xcd, 0x19, 0x13, 0x56, 0xff, 0xc6, 0x32,
0x72, 0x96, 0x6e, 0xf5, 0x11, 0x35, 0x34, 0x71, 0x35, 0xcc, 0xd0, 0xda, 0x50, 0xd6, 0x0d, 0xc3,
0xc3, 0x84, 0x08, 0x39, 0x82, 0x25, 0xfd, 0x72, 0x1f, 0x7b, 0x24, 0x30, 0xf9, 0xbc, 0x16, 0x2c,
0xd1, 0x1b, 0x50, 0x91, 0x05, 0x73, 0x3e, 0xad, 0x48, 0x8a, 0xca, 0x29, 0xba, 0x2b, 0x49, 0xa1,
0x7e, 0x95, 0x83, 0xa6, 0x50, 0xd8, 0xb2, 0xc8, 0x9a, 0xd3, 0x9d, 0x6f, 0x19, 0xea, 0x7b, 0xa1,
0x77, 0x4f, 0x9b, 0xa2, 0x44, 0x83, 0x40, 0x8c, 0x66, 0x96, 0x03, 0xc6, 0xf3, 0x76, 0xe1, 0x44,
0x79, 0xbb, 0x78, 0xc4, 0x18, 0xa5, 0xfe, 0x08, 0x6a, 0x91, 0x2f, 0x2c, 0xb8, 0xf2, 0xb9, 0x8a,
0x50, 0x45, 0xb0, 0x44, 0x77, 0xc3, 0xb2, 0x84, 0xeb, 0xe0, 0x42, 0xca, 0x26, 0x89, 0x8a, 0x44,
0xfd, 0xb3, 0x02, 0x25, 0xc1, 0xf9, 0x0a, 0xd4, 0x44, 0x34, 0x61, 0x25, 0x1b, 0xe7, 0x0e, 0x02,
0x44, 0x6b, 0xb6, 0xc7, 0x17, 0x4e, 0x2e, 0x40, 0x25, 0x11, 0x48, 0xca, 0x22, 0xa2, 0x07, 0x9f,
0x22, 0xd1, 0x83, 0x7e, 0x62, 0x81, 0xe3, 0x6b, 0x85, 0xcd, 0x84, 0x35, 0xdc, 0x77, 0xef, 0x63,
0xef, 0xe1, 0xc9, 0x27, 0x6f, 0xaf, 0x47, 0x2c, 0x35, 0x63, 0x6b, 0x27, 0x09, 0xd0, 0xeb, 0xa1,
0xba, 0xf3, 0x69, 0x83, 0x87, 0x68, 0xe8, 0x10, 0x76, 0x16, 0xaa, 0xfd, 0x97, 0x7c, 0x86, 0x18,
0x3f, 0xca, 0x71, 0x4b, 0x92, 0xc7, 0xd2, 0x31, 0xa8, 0xbf, 0x56, 0xe0, 0xc2, 0x3a, 0xf6, 0xd7,
0xe2, 0x7d, 0xf9, 0xd3, 0x96, 0xca, 0x86, 0x4e, 0x9a, 0x50, 0x27, 0xb9, 0xf5, 0x0e, 0x54, 0xe4,
0x84, 0x81, 0x4f, 0x77, 0xe5, 0x5a, 0xfd, 0xa9, 0x02, 0x6d, 0xb1, 0x0b, 0xdb, 0x93, 0x56, 0xc3,
0x16, 0xf6, 0xb1, 0xf1, 0x6d, 0xb7, 0xbc, 0xff, 0x52, 0xa0, 0x15, 0x0d, 0xe5, 0x2c, 0x1a, 0xbf,
0x02, 0x45, 0x36, 0x59, 0x10, 0x12, 0xcc, 0x34, 0x56, 0x8e, 0x4d, 0x43, 0x06, 0xab, 0xd0, 0x76,
0x64, 0xd6, 0x11, 0xcb, 0x30, 0x9f, 0xe4, 0x8f, 0x9e, 0x4f, 0x44, 0x7e, 0x75, 0x47, 0x94, 0x2f,
0x9f, 0xdc, 0x85, 0x00, 0xf5, 0xcb, 0x1c, 0xb4, 0xc3, 0x56, 0xe2, 0x5b, 0x0f, 0xe8, 0x13, 0x0a,
0xcd, 0xfc, 0x63, 0x2a, 0x34, 0x0b, 0x47, 0x0d, 0xe2, 0xff, 0xcc, 0x41, 0x33, 0x54, 0xc7, 0x96,
0xa5, 0x3b, 0xe8, 0x1c, 0x94, 0x86, 0x96, 0x1e, 0x4e, 0x04, 0xc5, 0x0a, 0x6d, 0xcb, 0xca, 0x24,
0xae, 0x80, 0x17, 0xd2, 0x2e, 0x67, 0x82, 0x86, 0xb5, 0x04, 0x0b, 0xda, 0xa2, 0xf1, 0x22, 0x9f,
0x35, 0xda, 0xa2, 0x1a, 0xe2, 0x56, 0x40, 0x7b, 0xec, 0x5b, 0x80, 0xc4, 0xd5, 0xf5, 0x4c, 0xa7,
0x47, 0x70, 0xdf, 0x75, 0x0c, 0x7e, 0xa9, 0x45, 0xad, 0x25, 0xbe, 0x74, 0x9d, 0x6d, 0x0e, 0x47,
0xaf, 0x40, 0xc1, 0x7f, 0x38, 0xe4, 0xe1, 0xb9, 0x99, 0x1a, 0xf6, 0x42, 0xb9, 0x76, 0x1e, 0x0e,
0xb1, 0xc6, 0xd0, 0xd1, 0x3c, 0x00, 0x65, 0xe5, 0x7b, 0xfa, 0x7d, 0x6c, 0x05, 0xbf, 0x65, 0x86,
0x10, 0x6a, 0xa6, 0xc1, 0xac, 0xa2, 0xcc, 0x73, 0x82, 0x58, 0xaa, 0x7f, 0xcf, 0x41, 0x2b, 0x64,
0xa9, 0x61, 0x32, 0xb2, 0xfc, 0x89, 0xfa, 0x9b, 0xde, 0xa0, 0xcd, 0xca, 0xf4, 0x6f, 0x43, 0x4d,
0xcc, 0x4d, 0x8e, 0x70, 0xd1, 0xc0, 0x49, 0x36, 0xa6, 0x58, 0x5e, 0xf1, 0x31, 0x59, 0x5e, 0xe9,
0xa8, 0x96, 0xb7, 0x0d, 0xe7, 0x82, 0x80, 0x16, 0x22, 0x6c, 0x62, 0x5f, 0x9f, 0x52, 0x49, 0x5c,
0x81, 0x1a, 0x4f, 0x54, 0x3c, 0x43, 0xf3, 0xe2, 0x1a, 0x76, 0x65, 0xd7, 0xa9, 0xfe, 0x18, 0xce,
0xb2, 0x80, 0x90, 0x1c, 0xaf, 0x66, 0x99, 0x75, 0xab, 0xb2, 0x74, 0xa7, 0x65, 0x3a, 0xb7, 0xee,
0xaa, 0x16, 0x83, 0xa9, 0x1b, 0xf0, 0x4c, 0x82, 0xff, 0x09, 0x02, 0xbe, 0xfa, 0x0f, 0x05, 0x2e,
0xac, 0x7a, 0xee, 0xf0, 0x43, 0xd3, 0xf3, 0x47, 0xba, 0x15, 0x1f, 0xd8, 0x3f, 0x99, 0xe6, 0xe3,
0xbd, 0x48, 0x8e, 0xe1, 0x41, 0xe7, 0x56, 0xca, 0x95, 0x8d, 0x0b, 0x25, 0xae, 0x2a, 0x92, 0x91,
0xfe, 0x9b, 0x4f, 0x13, 0x5e, 0xe0, 0xcd, 0x88, 0xa4, 0x59, 0x52, 0x70, 0xea, 0x34, 0x22, 0x7f,
0xdc, 0x69, 0xc4, 0x04, 0xeb, 0x2f, 0x3c, 0x26, 0xeb, 0x3f, 0x6a, 0xf1, 0x8c, 0xde, 0x83, 0xf8,
0xa4, 0x88, 0x85, 0x9d, 0x63, 0x8d, 0x98, 0x96, 0x01, 0xc2, 0xa9, 0x89, 0x78, 0x6d, 0x91, 0x85,
0x4d, 0x84, 0x8a, 0xde, 0x96, 0x8c, 0x34, 0x6c, 0xda, 0x19, 0xeb, 0xf2, 0xdf, 0x87, 0x4e, 0x9a,
0x95, 0x9e, 0xc4, 0xf2, 0x7f, 0x93, 0x03, 0xe8, 0xda, 0x43, 0xd7, 0xf3, 0x77, 0x74, 0x72, 0x70,
0xbc, 0x72, 0xe9, 0x06, 0xcc, 0x85, 0x06, 0x13, 0x35, 0xf8, 0x66, 0x08, 0x66, 0x36, 0x7f, 0x0d,
0x9a, 0xb2, 0x72, 0xe3, 0x78, 0x79, 0x3e, 0x72, 0x96, 0x50, 0x86, 0x76, 0x11, 0xaa, 0x9e, 0x7b,
0xd8, 0xa3, 0x9e, 0x64, 0xb0, 0x14, 0x53, 0xd1, 0x2a, 0x9e, 0x7b, 0x48, 0xfd, 0xcb, 0x40, 0xe7,
0xa1, 0xec, 0xeb, 0xe4, 0xa0, 0x67, 0x1a, 0xa2, 0xf8, 0x2f, 0xd1, 0x65, 0xd7, 0x40, 0x67, 0xa1,
0xb8, 0x67, 0x5a, 0x98, 0x07, 0xc0, 0xaa, 0xc6, 0x17, 0xe8, 0xd5, 0xe0, 0xa7, 0xf3, 0x72, 0xe6,
0x9f, 0xfe, 0xf8, 0xaf, 0xe7, 0x5f, 0x2b, 0x30, 0x17, 0x2a, 0x86, 0xd5, 0x35, 0xe8, 0x2d, 0xde,
0xe4, 0xe1, 0x15, 0xd7, 0xe0, 0xd1, 0xa0, 0x39, 0x61, 0xba, 0xcf, 0x09, 0x79, 0x31, 0x14, 0x92,
0x4c, 0xab, 0x2b, 0xe9, 0xb9, 0xe8, 0xa1, 0x4d, 0x23, 0xf8, 0x71, 0xa1, 0xe4, 0xb9, 0x87, 0x5d,
0x83, 0x04, 0xda, 0xe0, 0x2f, 0x88, 0x78, 0x15, 0x45, 0xb5, 0xb1, 0xc2, 0x1e, 0x11, 0x3d, 0x0b,
0x0d, 0xec, 0x79, 0xae, 0xd7, 0xb3, 0x31, 0x21, 0xfa, 0x80, 0x67, 0xdc, 0xaa, 0x56, 0x67, 0xc0,
0x4d, 0x0e, 0x53, 0xbf, 0xc9, 0x41, 0x33, 0x3c, 0x4a, 0xf0, 0x93, 0x82, 0x69, 0x04, 0x3f, 0x29,
0x98, 0x06, 0xcd, 0xfe, 0x1e, 0x8f, 0x76, 0x54, 0xb1, 0x22, 0x27, 0x0a, 0x48, 0xd7, 0xa0, 0xe1,
0x9e, 0xfa, 0x90, 0xe3, 0x1a, 0x98, 0x7e, 0xe7, 0x39, 0x11, 0x02, 0x50, 0xd7, 0xa0, 0x72, 0x44,
0x4c, 0xc0, 0x34, 0x82, 0xe1, 0x4f, 0x24, 0x90, 0x18, 0x34, 0x2a, 0x86, 0xd7, 0x2f, 0xee, 0xaf,
0x1a, 0x2d, 0xe6, 0x0d, 0x9a, 0xb2, 0x77, 0x47, 0xfd, 0x03, 0xec, 0x33, 0x37, 0xac, 0x6a, 0x62,
0x15, 0x37, 0x89, 0x72, 0xc2, 0x24, 0xe4, 0xcd, 0x57, 0xa2, 0x37, 0x7f, 0x11, 0xaa, 0x7c, 0x64,
0xdd, 0xf3, 0x09, 0xfb, 0x41, 0x20, 0xaf, 0x55, 0x38, 0x60, 0x87, 0xa0, 0xd7, 0x82, 0xb2, 0x36,
0x75, 0xc0, 0xcf, 0xe2, 0x45, 0xe2, 0xf2, 0x45, 0x51, 0xab, 0x7e, 0x02, 0x28, 0xfc, 0x72, 0xb2,
0x36, 0x23, 0xa1, 0xd5, 0x5c, 0x52, 0xab, 0x37, 0x7f, 0xab, 0xc0, 0xe9, 0xb1, 0xea, 0x1a, 0x35,
0x01, 0x3e, 0x70, 0xfa, 0xa2, 0xed, 0x68, 0x9d, 0x42, 0x75, 0xa8, 0x04, 0x4d, 0x48, 0x4b, 0x41,
0x35, 0x28, 0xef, 0xb8, 0x0c, 0xbb, 0x95, 0x43, 0x2d, 0xa8, 0x73, 0xc2, 0x51, 0xbf, 0x8f, 0x09,
0x69, 0xe5, 0x25, 0x64, 0x4d, 0x37, 0xad, 0x91, 0x87, 0x5b, 0x05, 0xd4, 0x80, 0xea, 0x8e, 0xab,
0x61, 0x0b, 0xeb, 0x04, 0xb7, 0x8a, 0x08, 0x41, 0x53, 0x2c, 0x02, 0xa2, 0x52, 0x04, 0x16, 0x90,
0x95, 0x6f, 0xee, 0x45, 0xcb, 0x55, 0x5a, 0xc3, 0xa1, 0xf3, 0x70, 0xe6, 0x03, 0xc7, 0xc0, 0x7b,
0xa6, 0x83, 0x8d, 0xf0, 0x53, 0xeb, 0x14, 0x3a, 0x03, 0x73, 0x5d, 0xc7, 0xc1, 0x5e, 0x04, 0xa8,
0x50, 0xe0, 0x26, 0xf6, 0x06, 0x38, 0x02, 0xcc, 0xa1, 0xd3, 0xd0, 0xd8, 0x34, 0x1f, 0x44, 0x40,
0xf9, 0xa5, 0xff, 0x9c, 0x81, 0xea, 0xaa, 0xee, 0xeb, 0x2b, 0xae, 0xeb, 0x19, 0x68, 0x08, 0x88,
0xbd, 0x95, 0xb0, 0x87, 0xae, 0x23, 0x1f, 0x15, 0xa1, 0x97, 0x26, 0x44, 0xd9, 0x71, 0x54, 0x91,
0xd2, 0x3b, 0xd7, 0x27, 0x50, 0x24, 0xd0, 0xd5, 0x53, 0xc8, 0x66, 0x3b, 0xd2, 0x1a, 0x78, 0xc7,
0xec, 0x1f, 0x04, 0xbf, 0x8a, 0x4d, 0xd9, 0x31, 0x81, 0x1a, 0xec, 0x98, 0x78, 0xab, 0x24, 0x16,
0xfc, 0x41, 0x4b, 0x60, 0x46, 0xea, 0x29, 0xf4, 0x29, 0x9c, 0x5d, 0xc7, 0x7e, 0xf8, 0x86, 0x21,
0xd8, 0x70, 0x69, 0xf2, 0x86, 0x63, 0xc8, 0x47, 0xdc, 0x72, 0x03, 0x8a, 0xac, 0x93, 0x45, 0x69,
0xdd, 0x62, 0xf4, 0x65, 0x6d, 0x67, 0x61, 0x32, 0x82, 0xe4, 0xf6, 0x09, 0xcc, 0x25, 0x5e, 0x0e,
0xa2, 0xe7, 0x53, 0xc8, 0xd2, 0xdf, 0x80, 0x76, 0x6e, 0x66, 0x41, 0x95, 0x7b, 0x0d, 0xa0, 0x19,
0x7f, 0x69, 0x81, 0x16, 0x53, 0xe8, 0x53, 0x5f, 0x7d, 0x75, 0x9e, 0xcf, 0x80, 0x29, 0x37, 0xb2,
0xa1, 0x95, 0x7c, 0xc9, 0x86, 0x6e, 0x4e, 0x65, 0x10, 0x37, 0xb7, 0x17, 0x32, 0xe1, 0xca, 0xed,
0x1e, 0x32, 0x23, 0x18, 0x7b, 0x49, 0x85, 0x6e, 0xa7, 0xb3, 0x99, 0xf4, 0xc4, 0xab, 0x73, 0x27,
0x33, 0xbe, 0xdc, 0xfa, 0x27, 0x7c, 0x82, 0x96, 0xf6, 0x1a, 0x09, 0xbd, 0x9c, 0xce, 0x6e, 0xca,
0x33, 0xaa, 0xce, 0xd2, 0x51, 0x48, 0xa4, 0x10, 0x9f, 0xb3, 0xd1, 0x57, 0xca, 0x8b, 0x9e, 0xa4,
0xdf, 0x05, 0xfc, 0x26, 0x3f, 0x55, 0xea, 0xbc, 0x7c, 0x04, 0x0a, 0x29, 0x80, 0x9b, 0x7c, 0x2b,
0x18, 0xb8, 0xe1, 0x9d, 0x99, 0x56, 0x73, 0x3c, 0x1f, 0xfc, 0x18, 0xe6, 0x12, 0xbf, 0x3f, 0xa6,
0x7a, 0x4d, 0xfa, 0x6f, 0x94, 0x9d, 0x69, 0xd9, 0x86, 0xbb, 0x64, 0x62, 0x92, 0x88, 0x26, 0x58,
0x7f, 0xca, 0xb4, 0xb1, 0x73, 0x33, 0x0b, 0xaa, 0x3c, 0x08, 0x61, 0xe1, 0x32, 0x31, 0x8d, 0x43,
0xb7, 0xd2, 0x79, 0xa4, 0x4f, 0x12, 0x3b, 0x2f, 0x66, 0xc4, 0x96, 0x9b, 0xf6, 0x00, 0xd6, 0xb1,
0xbf, 0x89, 0x7d, 0x8f, 0xda, 0xc8, 0xf5, 0x54, 0x95, 0x87, 0x08, 0xc1, 0x36, 0x37, 0x66, 0xe2,
0xc9, 0x0d, 0x7e, 0x00, 0x28, 0x48, 0xb1, 0x91, 0x5f, 0xbf, 0x9f, 0x9d, 0x3a, 0xd7, 0xe0, 0x43,
0x88, 0x59, 0x77, 0xf3, 0x29, 0xb4, 0x36, 0x75, 0x87, 0x56, 0xf3, 0x21, 0xdf, 0x5b, 0xa9, 0x82,
0x25, 0xd1, 0x26, 0x68, 0x6b, 0x22, 0xb6, 0x3c, 0xcc, 0xa1, 0xcc, 0xa1, 0xba, 0x74, 0x41, 0x9c,
0x8c, 0x2d, 0xa1, 0x36, 0x12, 0x88, 0x13, 0x62, 0xcb, 0x14, 0x7c, 0xb9, 0xf1, 0x23, 0x85, 0xbd,
0x48, 0x4d, 0x20, 0x7c, 0x64, 0xfa, 0xfb, 0x5b, 0x96, 0xee, 0x90, 0x2c, 0x22, 0x30, 0xc4, 0x23,
0x88, 0x20, 0xf0, 0xa5, 0x08, 0x06, 0x34, 0x62, 0x63, 0x03, 0x94, 0xf6, 0x13, 0x76, 0xda, 0xe0,
0xa2, 0xb3, 0x38, 0x1b, 0x51, 0xee, 0xb2, 0x0f, 0x8d, 0xc0, 0x5e, 0xb9, 0x72, 0x9f, 0x9f, 0x24,
0x69, 0x88, 0x33, 0xc1, 0xdd, 0xd2, 0x51, 0xa3, 0xee, 0x36, 0xde, 0x11, 0xa2, 0x6c, 0x93, 0x84,
0x69, 0xee, 0x36, 0xb9, 0xcd, 0x54, 0x4f, 0xa1, 0x2d, 0x28, 0xf1, 0x12, 0x18, 0x5d, 0x9e, 0x5a,
0x37, 0x77, 0xae, 0x4d, 0xfd, 0x1c, 0x72, 0x5c, 0xfa, 0x7d, 0x11, 0x2a, 0xc1, 0xaf, 0x8a, 0x4f,
0xa1, 0xc6, 0x7b, 0x0a, 0x45, 0xd7, 0xc7, 0x30, 0x97, 0x78, 0x80, 0x98, 0x1a, 0x93, 0xd3, 0x1f,
0x29, 0xce, 0x0a, 0x2a, 0x1f, 0x89, 0x7f, 0x4b, 0x92, 0xf1, 0xf7, 0xc6, 0xa4, 0xc2, 0x2d, 0x19,
0x7a, 0x67, 0x30, 0x7e, 0xe2, 0x81, 0xf6, 0x1e, 0x40, 0x24, 0x10, 0x4e, 0x1f, 0x1c, 0x53, 0xdf,
0x9e, 0x25, 0xf0, 0x5a, 0x56, 0x53, 0x9d, 0xce, 0x67, 0xf9, 0xee, 0x0f, 0x5f, 0x1e, 0x98, 0xfe,
0xfe, 0x68, 0x97, 0x7e, 0xb9, 0xc3, 0x51, 0x5f, 0x34, 0x5d, 0xf1, 0xd7, 0x9d, 0xc0, 0x32, 0xee,
0x30, 0xea, 0x3b, 0x94, 0xf9, 0x70, 0x77, 0xb7, 0xc4, 0x56, 0x77, 0xff, 0x1f, 0x00, 0x00, 0xff,
0xff, 0x5d, 0x32, 0x73, 0x00, 0x00, 0x37, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -19,16 +19,18 @@ package rootcoord
import (
"context"
"errors"
"fmt"
"strconv"
"sync"
"time"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
)
@ -36,58 +38,45 @@ const (
Bucket = "bucket"
FailedReason = "failed_reason"
MaxPendingCount = 32
delimiter = "/"
)
// import task state
type importTaskState struct {
stateCode commonpb.ImportState // state code
segments []int64 // id list of generated segments
rowIDs []int64 // id list of auto-generated is for auto-id primary key
segments []int64 // ID list of generated segments
rowIDs []int64 // ID list of auto-generated is for auto-id primary key
rowCount int64 // how many rows imported
failedReason string // failed reason
}
// import task
type importTask struct {
id int64 // task id
request int64 // request id
datanode int64 // datanode id which execute this task
collection string // target collection
partition string // target partition
bucket string // target bucket of storage
rowbased bool // row-based or column-based
files []string // import files
timestamp int64 // the timestamp of thie task come in
state importTaskState // task state
}
// importManager manager for import tasks
type importManager struct {
ctx context.Context // reserved
cancel context.CancelFunc // reserved
etcdCli *clientv3.Client // etcd to record import tasks
ctx context.Context // reserved
cancel context.CancelFunc // reserved
taskStore kv.MetaKv // Persistent task info storage.
pendingTasks []*importTask // pending tasks
workingTasks map[int64]*importTask // in-progress tasks
pendingLock sync.Mutex // lock pending task list
workingLock sync.Mutex // lock working task map
nextTaskID int64 // for generating next import task id
lastReqID int64 // for generateing a unique id for import request
pendingTasks []*datapb.ImportTaskInfo // pending tasks
workingTasks map[int64]*datapb.ImportTaskInfo // in-progress tasks
pendingLock sync.RWMutex // lock pending task list
workingLock sync.RWMutex // lock working task map
nextTaskID int64 // for generating next import task ID
lastReqID int64 // for generating a unique ID for import request
callImportService func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse
}
// newImportManager helper function to create a importManager
func newImportManager(ctx context.Context, client *clientv3.Client, importService func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse) *importManager {
func newImportManager(ctx context.Context, client kv.MetaKv, importService func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse) *importManager {
ctx, cancel := context.WithCancel(ctx)
mgr := &importManager{
ctx: ctx,
cancel: cancel,
etcdCli: client,
pendingTasks: make([]*importTask, 0, MaxPendingCount), // currently task queue max size is 32
workingTasks: make(map[int64]*importTask),
pendingLock: sync.Mutex{},
workingLock: sync.Mutex{},
taskStore: client,
pendingTasks: make([]*datapb.ImportTaskInfo, 0, MaxPendingCount), // currently task queue max size is 32
workingTasks: make(map[int64]*datapb.ImportTaskInfo),
pendingLock: sync.RWMutex{},
workingLock: sync.RWMutex{},
nextTaskID: 0,
lastReqID: 0,
callImportService: importService,
@ -97,7 +86,8 @@ func newImportManager(ctx context.Context, client *clientv3.Client, importServic
}
func (m *importManager) init() error {
// TODO: read task list from etcd
// Read tasks from etcd and save them as pendingTasks or workingTasks.
m.load()
// trigger Import() action to DataCoord
m.pushTasks()
@ -117,18 +107,18 @@ func (m *importManager) pushTasks() error {
}
task := m.pendingTasks[0]
log.Debug("import manager send import task", zap.Int64("taskID", task.id))
log.Debug("import manager send import task", zap.Int64("taskID", task.Id))
dbTask := &datapb.ImportTask{
CollectionName: task.collection,
PartitionName: task.partition,
RowBased: task.rowbased,
TaskId: task.id,
Files: task.files,
CollectionName: task.GetCollectionId(),
PartitionName: task.GetPartitionId(),
RowBased: task.GetRowBased(),
TaskId: task.GetId(),
Files: task.GetFiles(),
Infos: []*commonpb.KeyValuePair{
{
Key: Bucket,
Value: task.bucket,
Value: task.GetBucket(),
},
},
}
@ -136,11 +126,11 @@ func (m *importManager) pushTasks() error {
// call DataCoord.Import()
resp := m.callImportService(m.ctx, dbTask)
if resp.Status.ErrorCode == commonpb.ErrorCode_UnexpectedError {
log.Debug("import task is rejected", zap.Int64("task id", dbTask.TaskId))
log.Debug("import task is rejected", zap.Int64("task ID", dbTask.TaskId))
break
}
task.datanode = resp.DatanodeId
log.Debug("import task is assigned", zap.Int64("task id", dbTask.TaskId), zap.Int64("datanode id", task.datanode))
task.DatanodeId = resp.GetDatanodeId()
log.Debug("import task is assigned", zap.Int64("task ID", dbTask.TaskId), zap.Int64("datanode id", task.DatanodeId))
// erase this task from head of pending list if the callImportService succeed
m.pendingTasks = m.pendingTasks[1:]
@ -149,19 +139,19 @@ func (m *importManager) pushTasks() error {
m.workingLock.Lock()
defer m.workingLock.Unlock()
log.Debug("import task was taken to execute", zap.Int64("task id", dbTask.TaskId))
log.Debug("import task was taken to execute", zap.Int64("task ID", dbTask.TaskId))
task.state.stateCode = commonpb.ImportState_ImportPending
m.workingTasks[task.id] = task
// TODO: write this task to etcd
// TODO: Guard nil task state.
task.State.StateCode = commonpb.ImportState_ImportPending
m.workingTasks[task.Id] = task
m.updateImportTask(task)
}()
}
return nil
}
// generate an unique id for import request, this method has no lock, should only be called by importJob()
// genReqID generates a unique id for import request, this method has no lock, should only be called by importJob()
func (m *importManager) genReqID() int64 {
if m.lastReqID == 0 {
m.lastReqID = time.Now().Unix()
@ -238,37 +228,45 @@ func (m *importManager) importJob(req *milvuspb.ImportRequest) *milvuspb.ImportR
// for row-based, each file is a task
taskList := make([]int64, len(req.Files))
for i := 0; i < len(req.Files); i++ {
newTask := &importTask{
id: m.nextTaskID,
request: reqID,
collection: req.CollectionName,
partition: req.PartitionName,
bucket: bucket,
rowbased: req.RowBased,
files: []string{req.Files[i]},
timestamp: time.Now().Unix(),
newTask := &datapb.ImportTaskInfo{
Id: m.nextTaskID,
RequestId: reqID,
CollectionId: req.GetCollectionName(),
PartitionId: req.GetPartitionName(),
Bucket: bucket,
RowBased: req.GetRowBased(),
Files: []string{req.GetFiles()[i]},
CreateTs: time.Now().Unix(),
State: &datapb.ImportTaskState{
StateCode: commonpb.ImportState_ImportPending,
},
}
taskList[i] = newTask.id
taskList[i] = newTask.GetId()
m.nextTaskID++
m.pendingTasks = append(m.pendingTasks, newTask)
m.saveImportTask(newTask)
}
log.Debug("process row-based import request", zap.Int64("reqID", reqID), zap.Any("taskIDs", taskList))
log.Info("process row-based import request", zap.Int64("reqID", reqID), zap.Any("taskIDs", taskList))
} else {
// for column-based, all files is a task
newTask := &importTask{
id: m.nextTaskID,
request: reqID,
collection: req.CollectionName,
partition: req.PartitionName,
bucket: bucket,
rowbased: req.RowBased,
files: req.Files,
timestamp: time.Now().Unix(),
newTask := &datapb.ImportTaskInfo{
Id: m.nextTaskID,
RequestId: reqID,
CollectionId: req.GetCollectionName(),
PartitionId: req.GetPartitionName(),
Bucket: bucket,
RowBased: req.GetRowBased(),
Files: req.GetFiles(),
CreateTs: time.Now().Unix(),
State: &datapb.ImportTaskState{
StateCode: commonpb.ImportState_ImportPending,
},
}
m.nextTaskID++
m.pendingTasks = append(m.pendingTasks, newTask)
log.Debug("process row-based import request", zap.Int64("reqID", reqID), zap.Int64("taskID", newTask.id))
m.saveImportTask(newTask)
log.Info("process column-based import request", zap.Int64("reqID", reqID), zap.Int64("taskID", newTask.Id))
}
}()
@ -292,15 +290,17 @@ func (m *importManager) updateTaskState(state *rootcoordpb.ImportResult) error {
for k, v := range m.workingTasks {
if state.TaskId == k {
found = true
v.state.stateCode = state.State
v.state.segments = state.Segments
v.state.rowCount = state.RowCount
for _, kv := range state.Infos {
if kv.Key == FailedReason {
v.state.failedReason = kv.Value
v.State.StateCode = state.GetState()
v.State.Segments = state.GetSegments()
v.State.RowCount = state.GetRowCount()
for _, kv := range state.GetInfos() {
if kv.GetKey() == FailedReason {
v.State.ErrorMessage = kv.GetValue()
break
}
}
// Update task in task store.
m.updateImportTask(v)
}
}
}()
@ -328,7 +328,7 @@ func (m *importManager) getTaskState(id int64) *milvuspb.GetImportStateResponse
defer m.pendingLock.Unlock()
for i := 0; i < len(m.pendingTasks); i++ {
if id == m.pendingTasks[i].id {
if id == m.pendingTasks[i].Id {
resp.Status = &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}
@ -353,10 +353,10 @@ func (m *importManager) getTaskState(id int64) *milvuspb.GetImportStateResponse
resp.Status = &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}
resp.State = v.state.stateCode
resp.RowCount = v.state.rowCount
resp.IdList = v.state.rowIDs
resp.Infos = append(resp.Infos, &commonpb.KeyValuePair{Key: FailedReason, Value: v.state.failedReason})
resp.State = v.GetState().GetStateCode()
resp.RowCount = v.GetState().GetRowCount()
resp.IdList = v.GetState().GetRowIds()
resp.Infos = append(resp.Infos, &commonpb.KeyValuePair{Key: FailedReason, Value: v.GetState().GetErrorMessage()})
break
}
@ -369,3 +369,76 @@ func (m *importManager) getTaskState(id int64) *milvuspb.GetImportStateResponse
return resp
}
// load Loads task info from Etcd when RootCoord (re)starts.
func (m *importManager) load() error {
log.Info("Import manager starts loading from Etcd")
_, v, err := m.taskStore.LoadWithPrefix(Params.RootCoordCfg.ImportTaskSubPath)
if err != nil {
log.Error("RootCoord Import manager failed to load from Etcd", zap.Error(err))
return err
}
m.workingLock.Lock()
defer m.workingLock.Unlock()
m.pendingLock.Lock()
defer m.pendingLock.Unlock()
for i := range v {
ti := &datapb.ImportTaskInfo{}
if err := proto.Unmarshal([]byte(v[i]), ti); err != nil {
log.Error("Failed to unmarshal proto", zap.String("taskInfo", v[i]), zap.Error(err))
// Ignore bad protos.
continue
}
// Put tasks back to pending or working task list, given their import states.
if ti.GetState().GetStateCode() == commonpb.ImportState_ImportPending {
log.Info("Task has been reloaded as a pending task", zap.Int64("TaskID", ti.Id))
m.pendingTasks = append(m.pendingTasks, ti)
} else {
log.Info("Task has been reloaded as a working tasks", zap.Int64("TaskID", ti.Id))
m.workingTasks[ti.Id] = ti
}
}
return nil
}
// saveImportTask signs a lease and saves import task info into Etcd with this lease.
func (m *importManager) saveImportTask(task *datapb.ImportTaskInfo) error {
log.Info("Saving import task to Etcd", zap.Int64("Task ID", task.Id))
// TODO: Change default lease time and read it into config, once we figure out a proper value.
// Sign a lease.
leaseID, err := m.taskStore.Grant(10800) /*3 hours*/
if err != nil {
log.Error("Failed to grant lease from Etcd for data import.", zap.Int64("Task ID", task.Id), zap.Error(err))
return err
}
log.Info("Lease granted for task", zap.Int64("Task ID", task.Id))
var taskInfo []byte
if taskInfo, err = proto.Marshal(task); err != nil {
log.Error("Failed to marshall task proto", zap.Int64("Task ID", task.Id), zap.Error(err))
return err
} else if err = m.taskStore.SaveWithLease(BuildImportTaskKey(task.Id), string(taskInfo), leaseID); err != nil {
log.Error("Failed to save import task info into Etcd", zap.Int64("Task ID", task.Id), zap.Error(err))
return err
}
log.Info("Task info successfully saved.", zap.Int64("Task ID", task.Id))
return nil
}
// updateImportTask updates the task info in Etcd according to task ID. It won't change the lease on the key.
func (m *importManager) updateImportTask(task *datapb.ImportTaskInfo) error {
log.Info("Updating import task.", zap.Int64("Task ID", task.Id))
if taskInfo, err := proto.Marshal(task); err != nil {
log.Error("Failed to marshall task proto.", zap.Int64("Task ID", task.Id), zap.Error(err))
return err
} else if err = m.taskStore.SaveWithIgnoreLease(BuildImportTaskKey(task.Id), string(taskInfo)); err != nil {
log.Error("Failed to update import task info in Etcd.", zap.Int64("Task ID", task.Id), zap.Error(err))
return err
}
log.Info("Task info successfully updated.", zap.Int64("Task ID", task.Id))
return nil
}
// BuildImportTaskKey constructs and returns an Etcd key with given task ID.
func BuildImportTaskKey(taskID int64) string {
return fmt.Sprintf("%s%s%d", Params.RootCoordCfg.ImportTaskSubPath, delimiter, taskID)
}

View File

@ -20,6 +20,8 @@ import (
"context"
"testing"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/milvuspb"
@ -27,7 +29,33 @@ import (
"github.com/stretchr/testify/assert"
)
type customKV struct {
kv.MockMetaKV
}
func TestImportManager_NewImportManager(t *testing.T) {
Params.RootCoordCfg.ImportTaskSubPath = "test_import_task"
mockKv := &kv.MockMetaKV{}
mockKv.InMemKv = make(map[string]string)
ti1 := &datapb.ImportTaskInfo{
Id: 100,
State: &datapb.ImportTaskState{
StateCode: commonpb.ImportState_ImportPending,
},
}
ti2 := &datapb.ImportTaskInfo{
Id: 200,
State: &datapb.ImportTaskState{
StateCode: commonpb.ImportState_ImportPersisted,
},
}
taskInfo1, err := proto.Marshal(ti1)
assert.NoError(t, err)
taskInfo2, err := proto.Marshal(ti2)
assert.NoError(t, err)
mockKv.SaveWithLease(BuildImportTaskKey(1), "value", 1)
mockKv.SaveWithLease(BuildImportTaskKey(2), string(taskInfo1), 2)
mockKv.SaveWithLease(BuildImportTaskKey(3), string(taskInfo2), 3)
fn := func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
return &datapb.ImportTaskResponse{
Status: &commonpb.Status{
@ -35,12 +63,16 @@ func TestImportManager_NewImportManager(t *testing.T) {
},
}
}
mgr := newImportManager(context.TODO(), nil, fn)
mgr := newImportManager(context.TODO(), mockKv, fn)
assert.NotNil(t, mgr)
mgr.init()
}
func TestImportManager_ImportJob(t *testing.T) {
mgr := newImportManager(context.TODO(), nil, nil)
Params.RootCoordCfg.ImportTaskSubPath = "test_import_task"
mockKv := &kv.MockMetaKV{}
mockKv.InMemKv = make(map[string]string)
mgr := newImportManager(context.TODO(), mockKv, nil)
resp := mgr.importJob(nil)
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
@ -75,12 +107,12 @@ func TestImportManager_ImportJob(t *testing.T) {
}
}
mgr = newImportManager(context.TODO(), nil, fn)
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(rowReq)
assert.Equal(t, len(rowReq.Files), len(mgr.pendingTasks))
assert.Equal(t, 0, len(mgr.workingTasks))
mgr = newImportManager(context.TODO(), nil, fn)
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(colReq)
assert.Equal(t, 1, len(mgr.pendingTasks))
assert.Equal(t, 0, len(mgr.workingTasks))
@ -93,12 +125,12 @@ func TestImportManager_ImportJob(t *testing.T) {
}
}
mgr = newImportManager(context.TODO(), nil, fn)
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(rowReq)
assert.Equal(t, 0, len(mgr.pendingTasks))
assert.Equal(t, len(rowReq.Files), len(mgr.workingTasks))
mgr = newImportManager(context.TODO(), nil, fn)
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(colReq)
assert.Equal(t, 0, len(mgr.pendingTasks))
assert.Equal(t, 1, len(mgr.workingTasks))
@ -120,13 +152,16 @@ func TestImportManager_ImportJob(t *testing.T) {
}
}
mgr = newImportManager(context.TODO(), nil, fn)
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(rowReq)
assert.Equal(t, len(rowReq.Files)-2, len(mgr.pendingTasks))
assert.Equal(t, 2, len(mgr.workingTasks))
}
func TestImportManager_TaskState(t *testing.T) {
Params.RootCoordCfg.ImportTaskSubPath = "test_import_task"
mockKv := &kv.MockMetaKV{}
mockKv.InMemKv = make(map[string]string)
fn := func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
return &datapb.ImportTaskResponse{
Status: &commonpb.Status{
@ -142,7 +177,7 @@ func TestImportManager_TaskState(t *testing.T) {
Files: []string{"f1", "f2", "f3"},
}
mgr := newImportManager(context.TODO(), nil, fn)
mgr := newImportManager(context.TODO(), mockKv, fn)
mgr.importJob(rowReq)
state := &rootcoordpb.ImportResult{
@ -155,6 +190,16 @@ func TestImportManager_TaskState(t *testing.T) {
TaskId: 1,
RowCount: 1000,
State: commonpb.ImportState_ImportCompleted,
Infos: []*commonpb.KeyValuePair{
{
Key: "key1",
Value: "value1",
},
{
Key: "failed_reason",
Value: "some_reason",
},
},
}
err = mgr.updateTaskState(state)
assert.Nil(t, err)

View File

@ -97,17 +97,20 @@ type Core struct {
TSOGetLastSavedTime func() time.Time
//inner members
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
etcdCli *clientv3.Client
kvBase kv.TxnKV //*etcdkv.EtcdKV
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
etcdCli *clientv3.Client
kvBase kv.TxnKV //*etcdkv.EtcdKV
impTaskKv kv.MetaKv
//DDL lock
ddlLock sync.Mutex
kvBaseCreate func(root string) (kv.TxnKV, error)
metaKVCreate func(root string) (kv.MetaKv, error)
//setMsgStreams, send time tick into dd channel and time tick channel
SendTimeTick func(t typeutil.Timestamp, reason string) error
@ -238,6 +241,9 @@ func (c *Core) checkInit() error {
if c.kvBase == nil {
return fmt.Errorf("kvBase is nil")
}
if c.impTaskKv == nil {
return fmt.Errorf("impTaskKv is nil")
}
if c.SendDdCreateCollectionReq == nil {
return fmt.Errorf("sendDdCreateCollectionReq is nil")
}
@ -992,6 +998,11 @@ func (c *Core) Init() error {
return etcdkv.NewEtcdKV(c.etcdCli, root), nil
}
}
if c.metaKVCreate == nil {
c.metaKVCreate = func(root string) (kv.MetaKv, error) {
return etcdkv.NewEtcdKV(c.etcdCli, root), nil
}
}
c.initOnce.Do(func() {
if err := c.initSession(); err != nil {
initError = err
@ -1000,7 +1011,11 @@ func (c *Core) Init() error {
}
connectEtcdFn := func() error {
if c.kvBase, initError = c.kvBaseCreate(Params.EtcdCfg.KvRootPath); initError != nil {
log.Error("RootCoord failed to new EtcdKV", zap.Any("reason", initError))
log.Error("RootCoord failed to new EtcdKV for kvBase", zap.Any("reason", initError))
return initError
}
if c.impTaskKv, initError = c.metaKVCreate(Params.EtcdCfg.KvRootPath); initError != nil {
log.Error("RootCoord failed to new EtcdKV for MetaKV", zap.Any("reason", initError))
return initError
}
var metaKV kv.TxnKV
@ -1083,7 +1098,7 @@ func (c *Core) Init() error {
c.importManager = newImportManager(
c.ctx,
c.etcdCli,
c.impTaskKv,
c.CallImportService,
)
c.importManager.init()

View File

@ -477,6 +477,9 @@ func TestRootCoordInit(t *testing.T) {
core.kvBaseCreate = func(string) (kv.TxnKV, error) {
return nil, retry.Unrecoverable(errors.New("injected"))
}
core.metaKVCreate = func(root string) (kv.MetaKv, error) {
return nil, retry.Unrecoverable(errors.New("injected"))
}
err = core.Init()
assert.NotNil(t, err)
@ -500,6 +503,9 @@ func TestRootCoordInit(t *testing.T) {
}
return memkv.NewMemoryKV(), nil
}
core.metaKVCreate = func(root string) (kv.MetaKv, error) {
return nil, nil
}
err = core.Init()
assert.NotNil(t, err)
@ -520,6 +526,9 @@ func TestRootCoordInit(t *testing.T) {
core.kvBaseCreate = func(string) (kv.TxnKV, error) {
return nil, nil
}
core.metaKVCreate = func(root string) (kv.MetaKv, error) {
return nil, nil
}
err = core.Init()
assert.NotNil(t, err)
@ -541,6 +550,9 @@ func TestRootCoordInit(t *testing.T) {
kv := memkv.NewMemoryKV()
return &loadPrefixFailKV{TxnKV: kv}, nil
}
core.metaKVCreate = func(root string) (kv.MetaKv, error) {
return nil, nil
}
err = core.Init()
assert.NotNil(t, err)
@ -2549,6 +2561,10 @@ func TestCheckInit(t *testing.T) {
err = c.checkInit()
assert.NotNil(t, err)
c.impTaskKv = &etcdkv.EtcdKV{}
err = c.checkInit()
assert.NotNil(t, err)
c.SendDdCreateCollectionReq = func(context.Context, *internalpb.CreateCollectionRequest, []string) (map[string][]byte, error) {
return map[string][]byte{}, nil
}

View File

@ -336,28 +336,19 @@ type rootCoordConfig struct {
MaxPartitionNum int64
MinSegmentSizeToEnableIndex int64
// --- ETCD Path ---
ImportTaskSubPath string
CreatedTime time.Time
UpdatedTime time.Time
}
func (p *rootCoordConfig) init(base *BaseTable) {
p.Base = base
p.initDmlChannelNum()
p.initMaxPartitionNum()
p.initMinSegmentSizeToEnableIndex()
}
func (p *rootCoordConfig) initDmlChannelNum() {
p.DmlChannelNum = p.Base.ParseInt64WithDefault("rootCoord.dmlChannelNum", 256)
}
func (p *rootCoordConfig) initMaxPartitionNum() {
p.MaxPartitionNum = p.Base.ParseInt64WithDefault("rootCoord.maxPartitionNum", 4096)
}
func (p *rootCoordConfig) initMinSegmentSizeToEnableIndex() {
p.MinSegmentSizeToEnableIndex = p.Base.ParseInt64WithDefault("rootCoord.minSegmentSizeToEnableIndex", 1024)
p.ImportTaskSubPath = "importtask"
}
///////////////////////////////////////////////////////////////////////////////