Fill index size and fail reason when query index job (#19026)

Signed-off-by: cai.zhang <cai.zhang@zilliz.com>

Signed-off-by: cai.zhang <cai.zhang@zilliz.com>
pull/19056/head
cai.zhang 2022-09-06 17:19:11 +08:00 committed by GitHub
parent 8dce723327
commit d6b0fdc944
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 246 additions and 207 deletions

View File

@ -168,7 +168,7 @@ func TestIndexNodeClient(t *testing.T) {
t.Run("CreatJob", func(t *testing.T) {
req := &indexpb.CreateJobRequest{
ClusterID: 0,
ClusterID: "0",
BuildID: 0,
}
resp, err := inc.CreateJob(ctx, req)

View File

@ -68,7 +68,7 @@ func TestIndexNodeServer(t *testing.T) {
t.Run("CreateJob", func(t *testing.T) {
req := &indexpb.CreateJobRequest{
ClusterID: 0,
ClusterID: "",
BuildID: 0,
IndexID: 0,
DataPaths: []string{},

View File

@ -243,7 +243,7 @@ func (fsw *flushedSegmentWatcher) childrenRun() {
fsw.childrenTaskMutex.Lock()
defer fsw.childrenTaskMutex.Unlock()
if len(fsw.childrenTasks) > 0 {
log.Debug("IndexCoord flushedSegmentWatcher schedule children tasks", zap.Int("internal task num", len(fsw.childrenTasks)))
log.Debug("IndexCoord flushedSegmentWatcher schedule children tasks", zap.Int("children task num", len(fsw.childrenTasks)))
for segID, tasks := range fsw.childrenTasks {
for _, t := range tasks {
fsw.childrenProcess(t)

View File

@ -240,10 +240,14 @@ func (ib *indexBuilder) process(buildID UniqueID) {
}
indexParams := ib.meta.GetIndexParams(meta.CollectionID, meta.IndexID)
req := &indexpb.CreateJobRequest{
// TODO @xiaocai2333: set clusterID
ClusterID: 0,
StorageConfig: &indexpb.StorageConfig{
var storageConfig *indexpb.StorageConfig
if Params.CommonCfg.StorageType == "local" {
storageConfig = &indexpb.StorageConfig{
RootPath: Params.LocalStorageCfg.Path,
StorageType: Params.CommonCfg.StorageType,
}
} else {
storageConfig = &indexpb.StorageConfig{
Address: Params.MinioCfg.Address,
AccessKeyID: Params.MinioCfg.AccessKeyID,
SecretAccessKey: Params.MinioCfg.SecretAccessKey,
@ -252,13 +256,19 @@ func (ib *indexBuilder) process(buildID UniqueID) {
RootPath: Params.MinioCfg.RootPath,
UseIAM: Params.MinioCfg.UseIAM,
IAMEndpoint: Params.MinioCfg.IAMEndpoint,
},
StorageType: Params.CommonCfg.StorageType,
}
}
req := &indexpb.CreateJobRequest{
ClusterID: Params.CommonCfg.ClusterPrefix,
IndexFilePrefix: path.Join(ib.ic.chunkManager.RootPath(), common.SegmentIndexPath),
BuildID: buildID,
DataPaths: binLogs,
IndexVersion: meta.IndexVersion + 1,
StorageConfig: storageConfig,
IndexParams: indexParams,
TypeParams: typeParams,
NumRows: meta.NumRows,
}
log.Debug("assign task to indexNode", zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
if err := ib.ic.assignTask(client, req); err != nil {
@ -340,7 +350,7 @@ func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) indexTaskState {
client, exist := ib.ic.nodeManager.GetClientByID(nodeID)
if exist {
response, err := client.QueryJobs(ib.ctx, &indexpb.QueryJobsRequest{
ClusterID: 0,
ClusterID: Params.CommonCfg.ClusterPrefix,
BuildIDs: []int64{buildID},
})
if err != nil {
@ -381,7 +391,7 @@ func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
client, exist := ib.ic.nodeManager.GetClientByID(nodeID)
if exist {
status, err := client.DropJobs(ib.ctx, &indexpb.DropJobsRequest{
ClusterID: 0,
ClusterID: Params.CommonCfg.ClusterPrefix,
BuildIDs: []UniqueID{buildID},
})
if err != nil {

View File

@ -201,7 +201,7 @@ func (i *IndexCoord) Init() error {
}
//}()
}
log.Debug("IndexCoord", zap.Int("IndexNode number", len(i.nodeManager.nodeClients)))
log.Debug("IndexCoord", zap.Int("IndexNode number", len(i.nodeManager.GetAllClients())))
i.indexBuilder = newIndexBuilder(i.loopCtx, i, i.metaTable, aliveNodeID)
// TODO silverxia add Rewatch logic

View File

@ -18,28 +18,32 @@ type chunkMgr struct {
}
func (m *chunkMgr) NewChunkManager(ctx context.Context, config *indexpb.StorageConfig) (storage.ChunkManager, error) {
//key := m.cacheKey(bucket, storageAccessKey)
//if v, ok := m.cached.Load(key); ok {
// return v.(storage.ChunkManager), nil
//}
opts := []storage.Option{
storage.Address(config.Address),
storage.AccessKeyID(config.AccessKeyID),
storage.SecretAccessKeyID(config.SecretAccessKey),
storage.UseSSL(config.UseSSL),
storage.BucketName(config.BucketName),
storage.UseIAM(config.UseIAM),
storage.IAMEndpoint(config.IAMEndpoint),
key := m.cacheKey(config.StorageType, config.BucketName, config.Address)
if v, ok := m.cached.Load(key); ok {
return v.(storage.ChunkManager), nil
}
factory := storage.NewChunkManagerFactory("local", "minio", opts...)
opts := make([]storage.Option, 0)
if config.StorageType == "local" {
opts = append(opts, storage.RootPath(config.RootPath))
} else {
opts = append(opts, storage.Address(config.Address),
storage.AccessKeyID(config.AccessKeyID),
storage.SecretAccessKeyID(config.SecretAccessKey),
storage.UseSSL(config.UseSSL),
storage.BucketName(config.BucketName),
storage.UseIAM(config.UseIAM),
storage.IAMEndpoint(config.IAMEndpoint))
}
factory := storage.NewChunkManagerFactory("local", config.StorageType, opts...)
mgr, err := factory.NewVectorStorageChunkManager(ctx)
if err != nil {
return nil, err
}
//v, _ := m.cached.LoadOrStore(key, mgr)
return mgr, nil
v, _ := m.cached.LoadOrStore(key, mgr)
return v.(storage.ChunkManager), nil
}
func (m *chunkMgr) cacheKey(bucket, storageAccessKey string) string {
return fmt.Sprintf("%s/%s", bucket, storageAccessKey)
func (m *chunkMgr) cacheKey(storageType, bucket, address string) string {
return fmt.Sprintf("%s/%s/%s", storageType, bucket, address)
}

View File

@ -68,7 +68,7 @@ var _ types.IndexNodeComponent = (*IndexNode)(nil)
var Params paramtable.ComponentParam
type taskKey struct {
ClusterID UniqueID
ClusterID string
BuildID UniqueID
}

View File

@ -25,14 +25,14 @@ import (
func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest) (*commonpb.Status, error) {
stateCode := i.stateCode.Load().(internalpb.StateCode)
if stateCode != internalpb.StateCode_Healthy {
log.Warn("index node not ready", zap.Int32("state", int32(stateCode)), zap.Int64("ClusterID", req.ClusterID), zap.Int64("IndexBuildID", req.BuildID))
log.Warn("index node not ready", zap.Int32("state", int32(stateCode)), zap.String("ClusterID", req.ClusterID), zap.Int64("IndexBuildID", req.BuildID))
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "state code is not healthy",
}, nil
}
log.Info("IndexNode building index ...",
zap.Int64("ClusterID", req.ClusterID),
zap.String("ClusterID", req.ClusterID),
zap.Int64("IndexBuildID", req.BuildID),
zap.Int64("IndexID", req.IndexID),
zap.String("IndexName", req.IndexName),
@ -44,7 +44,7 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
sp, _ := trace.StartSpanFromContextWithOperationName(ctx, "IndexNode-CreateIndex")
defer sp.Finish()
sp.SetTag("IndexBuildID", strconv.FormatInt(req.BuildID, 10))
sp.SetTag("ClusterID", strconv.FormatInt(req.ClusterID, 10))
sp.SetTag("ClusterID", req.ClusterID)
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.GetNodeID(), 10), metrics.TotalLabel).Inc()
taskCtx := logutil.WithModule(i.loopCtx, typeutil.IndexNodeRole)
@ -52,7 +52,7 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
if oldInfo := i.loadOrStoreTask(req.ClusterID, req.BuildID, &taskInfo{
cancel: taskCancel,
state: commonpb.IndexState_InProgress}); oldInfo != nil {
log.Warn("duplicated index build task", zap.Int64("ClusterID", req.ClusterID), zap.Int64("BuildID", req.BuildID))
log.Warn("duplicated index build task", zap.String("ClusterID", req.ClusterID), zap.Int64("BuildID", req.BuildID))
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_BuildIndexError,
Reason: "duplicated index build task",
@ -62,14 +62,14 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
if err != nil {
log.Error("create chunk manager failed", zap.String("Bucket", req.StorageConfig.BucketName),
zap.String("AccessKey", req.StorageConfig.AccessKeyID),
zap.Int64("ClusterID", req.ClusterID), zap.Int64("IndexBuildID", req.BuildID))
zap.String("ClusterID", req.ClusterID), zap.Int64("IndexBuildID", req.BuildID))
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_BuildIndexError,
Reason: "create chunk manager failed",
}, nil
}
task := &indexBuildTask{
ident: fmt.Sprintf("%d/%d", req.ClusterID, req.BuildID),
ident: fmt.Sprintf("%s/%d", req.ClusterID, req.BuildID),
ctx: taskCtx,
cancel: taskCancel,
BuildID: req.BuildID,
@ -78,7 +78,7 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
req: req,
cm: cm,
nodeID: i.GetNodeID(),
tr: timerecord.NewTimeRecorder(fmt.Sprintf("IndexBuildID: %d, ClusterID: %d", req.BuildID, req.ClusterID)),
tr: timerecord.NewTimeRecorder(fmt.Sprintf("IndexBuildID: %d, ClusterID: %s", req.BuildID, req.ClusterID)),
serializedSize: 0,
}
ret := &commonpb.Status{
@ -86,20 +86,20 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
Reason: "",
}
if err := i.sched.IndexBuildQueue.Enqueue(task); err != nil {
log.Warn("IndexNode failed to schedule", zap.Int64("IndexBuildID", req.BuildID), zap.Int64("ClusterID", req.ClusterID), zap.Error(err))
log.Warn("IndexNode failed to schedule", zap.Int64("IndexBuildID", req.BuildID), zap.String("ClusterID", req.ClusterID), zap.Error(err))
ret.ErrorCode = commonpb.ErrorCode_UnexpectedError
ret.Reason = err.Error()
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.GetNodeID(), 10), metrics.FailLabel).Inc()
return ret, nil
}
log.Info("IndexNode successfully scheduled", zap.Int64("IndexBuildID", req.BuildID), zap.Int64("ClusterID", req.ClusterID), zap.String("indexName", req.IndexName))
log.Info("IndexNode successfully scheduled", zap.Int64("IndexBuildID", req.BuildID), zap.String("ClusterID", req.ClusterID), zap.String("indexName", req.IndexName))
return ret, nil
}
func (i *IndexNode) QueryJobs(ctx context.Context, req *indexpb.QueryJobsRequest) (*indexpb.QueryJobsResponse, error) {
stateCode := i.stateCode.Load().(internalpb.StateCode)
if stateCode != internalpb.StateCode_Healthy {
log.Warn("index node not ready", zap.Int32("state", int32(stateCode)), zap.Int64("ClusterID", req.ClusterID))
log.Warn("index node not ready", zap.Int32("state", int32(stateCode)), zap.String("ClusterID", req.ClusterID))
return &indexpb.QueryJobsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -107,10 +107,10 @@ func (i *IndexNode) QueryJobs(ctx context.Context, req *indexpb.QueryJobsRequest
},
}, nil
}
log.Debug("querying index build task", zap.Int64("ClusterID", req.ClusterID), zap.Int64s("IndexBuildIDs", req.BuildIDs))
log.Debug("querying index build task", zap.String("ClusterID", req.ClusterID), zap.Int64s("IndexBuildIDs", req.BuildIDs))
infos := make(map[UniqueID]*taskInfo)
i.foreachTaskInfo(func(clusterID, buildID UniqueID, info *taskInfo) {
if clusterID == req.ClusterID {
i.foreachTaskInfo(func(ClusterID string, buildID UniqueID, info *taskInfo) {
if ClusterID == req.ClusterID {
infos[buildID] = &taskInfo{
state: info.state,
indexFiles: info.indexFiles[:],
@ -137,16 +137,17 @@ func (i *IndexNode) QueryJobs(ctx context.Context, req *indexpb.QueryJobsRequest
ret.IndexInfos[i].State = info.state
ret.IndexInfos[i].IndexFiles = info.indexFiles
ret.IndexInfos[i].SerializedSize = info.serializedSize
ret.IndexInfos[i].FailReason = info.failReason
}
}
return ret, nil
}
func (i *IndexNode) DropJobs(ctx context.Context, req *indexpb.DropJobsRequest) (*commonpb.Status, error) {
log.Debug("drop index build jobs", zap.Int64("ClusterID", req.ClusterID), zap.Int64s("IndexBuildIDs", req.BuildIDs))
log.Debug("drop index build jobs", zap.String("ClusterID", req.ClusterID), zap.Int64s("IndexBuildIDs", req.BuildIDs))
stateCode := i.stateCode.Load().(internalpb.StateCode)
if stateCode != internalpb.StateCode_Healthy {
log.Warn("index node not ready", zap.Int32("state", int32(stateCode)), zap.Int64("ClusterID", req.ClusterID))
log.Warn("index node not ready", zap.Int32("state", int32(stateCode)), zap.String("ClusterID", req.ClusterID))
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "state code is not healthy",
@ -181,7 +182,7 @@ func (i *IndexNode) GetJobStats(ctx context.Context, req *indexpb.GetJobStatsReq
}
utNum, atNum := i.sched.IndexBuildQueue.GetTaskNum()
jobInfos := make([]*indexpb.JobInfo, 0)
i.foreachTaskInfo(func(clusterID, buildID UniqueID, info *taskInfo) {
i.foreachTaskInfo(func(ClusterID string, buildID UniqueID, info *taskInfo) {
if info.statistic != nil {
jobInfos = append(jobInfos, proto.Clone(info.statistic).(*indexpb.JobInfo))
}

View File

@ -33,7 +33,7 @@ func TestIndexNodeSimple(t *testing.T) {
assert.Nil(t, err, err)
var (
clusterID int64
clusterID = "test-milvus"
idxFilePrefix = "mock_idx"
buildID int64 = 1
collID int64 = 101
@ -164,7 +164,7 @@ func TestIndexNodeComplex(t *testing.T) {
idxParamsPayload, err := json.Marshal(idxParams)
assert.Nil(t, err)
var (
clusterID int64
clusterID string
buildID0 int64
collID0 int64 = 10000
partID0 int64 = 20000

View File

@ -55,6 +55,7 @@ type taskInfo struct {
state commonpb.IndexState
indexFiles []string
serializedSize uint64
failReason string
// task statistics
statistic *indexpb.JobInfo
@ -68,7 +69,7 @@ type task interface {
BuildIndex(context.Context) error
SaveIndexFiles(context.Context) error
OnEnqueue(context.Context) error
SetState(state commonpb.IndexState)
SetState(state commonpb.IndexState, failReason string)
GetState() commonpb.IndexState
Reset()
}
@ -85,7 +86,7 @@ type indexBuildTask struct {
req *indexpb.CreateJobRequest
BuildID UniqueID
nodeID UniqueID
ClusterID UniqueID
ClusterID string
collectionID UniqueID
partitionID UniqueID
segmentID UniqueID
@ -126,8 +127,8 @@ func (it *indexBuildTask) Name() string {
return it.ident
}
func (it *indexBuildTask) SetState(state commonpb.IndexState) {
it.node.storeTaskState(it.ClusterID, it.BuildID, state)
func (it *indexBuildTask) SetState(state commonpb.IndexState, failReason string) {
it.node.storeTaskState(it.ClusterID, it.BuildID, state, failReason)
}
func (it *indexBuildTask) GetState() commonpb.IndexState {

View File

@ -213,16 +213,16 @@ func (sched *TaskScheduler) processTask(t task, q TaskQueue) {
if err := wrap(fn); err != nil {
if err == errCancel {
logutil.Logger(t.Ctx()).Warn("index build task canceled", zap.String("task", t.Name()))
t.SetState(commonpb.IndexState_Failed)
t.SetState(commonpb.IndexState_Failed, err.Error())
} else if errors.Is(err, ErrNoSuchKey) {
t.SetState(commonpb.IndexState_Failed)
t.SetState(commonpb.IndexState_Failed, err.Error())
} else {
t.SetState(commonpb.IndexState_Unissued)
t.SetState(commonpb.IndexState_Unissued, err.Error())
}
return
}
}
t.SetState(commonpb.IndexState_Finished)
t.SetState(commonpb.IndexState_Finished, "")
}
func (sched *TaskScheduler) indexBuildLoop() {

View File

@ -72,6 +72,7 @@ type fakeTask struct {
reterr map[fakeTaskState]error
retstate commonpb.IndexState
expectedState commonpb.IndexState
failReason string
}
var _ task = &fakeTask{}
@ -119,8 +120,9 @@ func (t *fakeTask) Reset() {
_taskwg.Done()
}
func (t *fakeTask) SetState(state commonpb.IndexState) {
func (t *fakeTask) SetState(state commonpb.IndexState, failReason string) {
t.retstate = state
t.failReason = failReason
}
func (t *fakeTask) GetState() commonpb.IndexState {

View File

@ -7,10 +7,10 @@ import (
"github.com/milvus-io/milvus/internal/proto/indexpb"
)
func (i *IndexNode) loadOrStoreTask(clusterID, buildID UniqueID, info *taskInfo) *taskInfo {
func (i *IndexNode) loadOrStoreTask(ClusterID string, buildID UniqueID, info *taskInfo) *taskInfo {
i.stateLock.Lock()
defer i.stateLock.Unlock()
key := taskKey{ClusterID: clusterID, BuildID: buildID}
key := taskKey{ClusterID: ClusterID, BuildID: buildID}
oldInfo, ok := i.tasks[key]
if ok {
return oldInfo
@ -19,24 +19,25 @@ func (i *IndexNode) loadOrStoreTask(clusterID, buildID UniqueID, info *taskInfo)
return nil
}
func (i *IndexNode) loadTaskState(clusterID, buildID UniqueID) (commonpb.IndexState, bool) {
key := taskKey{ClusterID: clusterID, BuildID: buildID}
func (i *IndexNode) loadTaskState(ClusterID string, buildID UniqueID) (commonpb.IndexState, bool) {
key := taskKey{ClusterID: ClusterID, BuildID: buildID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
task, ok := i.tasks[key]
return task.state, ok
}
func (i *IndexNode) storeTaskState(clusterID, buildID UniqueID, state commonpb.IndexState) {
key := taskKey{ClusterID: clusterID, BuildID: buildID}
func (i *IndexNode) storeTaskState(ClusterID string, buildID UniqueID, state commonpb.IndexState, failReason string) {
key := taskKey{ClusterID: ClusterID, BuildID: buildID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
if task, ok := i.tasks[key]; ok {
task.state = state
task.failReason = failReason
}
}
func (i *IndexNode) foreachTaskInfo(fn func(clusterID, buildID UniqueID, info *taskInfo)) {
func (i *IndexNode) foreachTaskInfo(fn func(ClusterID string, buildID UniqueID, info *taskInfo)) {
i.stateLock.Lock()
defer i.stateLock.Unlock()
for key, info := range i.tasks {
@ -44,8 +45,8 @@ func (i *IndexNode) foreachTaskInfo(fn func(clusterID, buildID UniqueID, info *t
}
}
func (i *IndexNode) storeIndexFilesAndStatistic(clusterID, buildID UniqueID, files []string, serializedSize uint64, statistic *indexpb.JobInfo) {
key := taskKey{ClusterID: clusterID, BuildID: buildID}
func (i *IndexNode) storeIndexFilesAndStatistic(ClusterID string, buildID UniqueID, files []string, serializedSize uint64, statistic *indexpb.JobInfo) {
key := taskKey{ClusterID: ClusterID, BuildID: buildID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
if info, ok := i.tasks[key]; ok {

View File

@ -182,10 +182,11 @@ message StorageConfig {
string root_path = 6;
bool useIAM = 7;
string IAMEndpoint = 8;
string storage_type = 9;
}
message CreateJobRequest {
int64 clusterID = 1;
string clusterID = 1;
string index_file_prefix = 2;
int64 buildID = 3;
repeated string data_paths = 4;
@ -195,10 +196,11 @@ message CreateJobRequest {
StorageConfig storage_config = 8;
repeated common.KeyValuePair index_params = 9;
repeated common.KeyValuePair type_params = 10;
int64 num_rows = 11;
}
message QueryJobsRequest {
int64 clusterID = 1;
string clusterID = 1;
repeated int64 buildIDs = 2;
}
@ -212,12 +214,12 @@ message IndexTaskInfo {
message QueryJobsResponse {
common.Status status = 1;
int64 clusterID = 2;
string clusterID = 2;
repeated IndexTaskInfo index_infos = 3;
}
message DropJobsRequest {
int64 clusterID = 1;
string clusterID = 1;
repeated int64 buildIDs = 2;
}

View File

@ -1264,6 +1264,7 @@ type StorageConfig struct {
RootPath string `protobuf:"bytes,6,opt,name=root_path,json=rootPath,proto3" json:"root_path,omitempty"`
UseIAM bool `protobuf:"varint,7,opt,name=useIAM,proto3" json:"useIAM,omitempty"`
IAMEndpoint string `protobuf:"bytes,8,opt,name=IAMEndpoint,proto3" json:"IAMEndpoint,omitempty"`
StorageType string `protobuf:"bytes,9,opt,name=storage_type,json=storageType,proto3" json:"storage_type,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -1350,8 +1351,15 @@ func (m *StorageConfig) GetIAMEndpoint() string {
return ""
}
func (m *StorageConfig) GetStorageType() string {
if m != nil {
return m.StorageType
}
return ""
}
type CreateJobRequest struct {
ClusterID int64 `protobuf:"varint,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"`
ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"`
IndexFilePrefix string `protobuf:"bytes,2,opt,name=index_file_prefix,json=indexFilePrefix,proto3" json:"index_file_prefix,omitempty"`
BuildID int64 `protobuf:"varint,3,opt,name=buildID,proto3" json:"buildID,omitempty"`
DataPaths []string `protobuf:"bytes,4,rep,name=data_paths,json=dataPaths,proto3" json:"data_paths,omitempty"`
@ -1361,6 +1369,7 @@ type CreateJobRequest struct {
StorageConfig *StorageConfig `protobuf:"bytes,8,opt,name=storage_config,json=storageConfig,proto3" json:"storage_config,omitempty"`
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,9,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
TypeParams []*commonpb.KeyValuePair `protobuf:"bytes,10,rep,name=type_params,json=typeParams,proto3" json:"type_params,omitempty"`
NumRows int64 `protobuf:"varint,11,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -1391,11 +1400,11 @@ func (m *CreateJobRequest) XXX_DiscardUnknown() {
var xxx_messageInfo_CreateJobRequest proto.InternalMessageInfo
func (m *CreateJobRequest) GetClusterID() int64 {
func (m *CreateJobRequest) GetClusterID() string {
if m != nil {
return m.ClusterID
}
return 0
return ""
}
func (m *CreateJobRequest) GetIndexFilePrefix() string {
@ -1461,8 +1470,15 @@ func (m *CreateJobRequest) GetTypeParams() []*commonpb.KeyValuePair {
return nil
}
func (m *CreateJobRequest) GetNumRows() int64 {
if m != nil {
return m.NumRows
}
return 0
}
type QueryJobsRequest struct {
ClusterID int64 `protobuf:"varint,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"`
ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"`
BuildIDs []int64 `protobuf:"varint,2,rep,packed,name=buildIDs,proto3" json:"buildIDs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@ -1494,11 +1510,11 @@ func (m *QueryJobsRequest) XXX_DiscardUnknown() {
var xxx_messageInfo_QueryJobsRequest proto.InternalMessageInfo
func (m *QueryJobsRequest) GetClusterID() int64 {
func (m *QueryJobsRequest) GetClusterID() string {
if m != nil {
return m.ClusterID
}
return 0
return ""
}
func (m *QueryJobsRequest) GetBuildIDs() []int64 {
@ -1581,7 +1597,7 @@ func (m *IndexTaskInfo) GetFailReason() string {
type QueryJobsResponse struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
ClusterID int64 `protobuf:"varint,2,opt,name=clusterID,proto3" json:"clusterID,omitempty"`
ClusterID string `protobuf:"bytes,2,opt,name=clusterID,proto3" json:"clusterID,omitempty"`
IndexInfos []*IndexTaskInfo `protobuf:"bytes,3,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@ -1620,11 +1636,11 @@ func (m *QueryJobsResponse) GetStatus() *commonpb.Status {
return nil
}
func (m *QueryJobsResponse) GetClusterID() int64 {
func (m *QueryJobsResponse) GetClusterID() string {
if m != nil {
return m.ClusterID
}
return 0
return ""
}
func (m *QueryJobsResponse) GetIndexInfos() []*IndexTaskInfo {
@ -1635,7 +1651,7 @@ func (m *QueryJobsResponse) GetIndexInfos() []*IndexTaskInfo {
}
type DropJobsRequest struct {
ClusterID int64 `protobuf:"varint,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"`
ClusterID string `protobuf:"bytes,1,opt,name=clusterID,proto3" json:"clusterID,omitempty"`
BuildIDs []int64 `protobuf:"varint,2,rep,packed,name=buildIDs,proto3" json:"buildIDs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@ -1667,11 +1683,11 @@ func (m *DropJobsRequest) XXX_DiscardUnknown() {
var xxx_messageInfo_DropJobsRequest proto.InternalMessageInfo
func (m *DropJobsRequest) GetClusterID() int64 {
func (m *DropJobsRequest) GetClusterID() string {
if m != nil {
return m.ClusterID
}
return 0
return ""
}
func (m *DropJobsRequest) GetBuildIDs() []int64 {
@ -1906,131 +1922,133 @@ func init() {
func init() { proto.RegisterFile("index_coord.proto", fileDescriptor_f9e019eb3fda53c2) }
var fileDescriptor_f9e019eb3fda53c2 = []byte{
// 1983 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x59, 0xcd, 0x8f, 0x1b, 0x49,
0x15, 0x9f, 0x76, 0xcf, 0x87, 0xfb, 0xd9, 0x9e, 0x99, 0x54, 0xb2, 0xc8, 0xeb, 0x24, 0x64, 0xa6,
0x43, 0x12, 0x83, 0xb4, 0x33, 0x61, 0x16, 0xd0, 0x82, 0x00, 0x69, 0x3e, 0x36, 0xc1, 0x93, 0x9d,
0xd1, 0xd0, 0x8e, 0x56, 0x62, 0x85, 0xd4, 0xb4, 0xdd, 0x65, 0xa7, 0x76, 0xec, 0x2e, 0xa7, 0xab,
0x3a, 0xc9, 0x04, 0x09, 0x71, 0x41, 0x02, 0xb4, 0x12, 0x12, 0x07, 0x10, 0x17, 0x4e, 0x9c, 0xf6,
0xc0, 0x15, 0x89, 0x7f, 0x80, 0x03, 0xfc, 0x19, 0xfc, 0x21, 0xa8, 0x3e, 0xba, 0xdd, 0xdd, 0x6e,
0x8f, 0x9d, 0xf1, 0x70, 0xda, 0x9b, 0xeb, 0xf5, 0xab, 0xaf, 0xf7, 0x7b, 0xf5, 0x7e, 0xbf, 0x2a,
0xc3, 0x0d, 0x12, 0xf8, 0xf8, 0x8d, 0xdb, 0xa5, 0x34, 0xf4, 0x77, 0x46, 0x21, 0xe5, 0x14, 0xa1,
0x21, 0x19, 0xbc, 0x8a, 0x98, 0x6a, 0xed, 0xc8, 0xef, 0x8d, 0x6a, 0x97, 0x0e, 0x87, 0x34, 0x50,
0xb6, 0xc6, 0x3a, 0x09, 0x38, 0x0e, 0x03, 0x6f, 0xa0, 0xdb, 0xd5, 0x74, 0x0f, 0xfb, 0x8b, 0x12,
0x58, 0x2d, 0xd1, 0xab, 0x15, 0xf4, 0x28, 0xb2, 0xa1, 0xda, 0xa5, 0x83, 0x01, 0xee, 0x72, 0x42,
0x83, 0xd6, 0x51, 0xdd, 0xd8, 0x32, 0x9a, 0xa6, 0x93, 0xb1, 0xa1, 0x3a, 0xac, 0xf5, 0x08, 0x1e,
0xf8, 0xad, 0xa3, 0x7a, 0x49, 0x7e, 0x8e, 0x9b, 0xe8, 0x2e, 0x80, 0x5a, 0x60, 0xe0, 0x0d, 0x71,
0xdd, 0xdc, 0x32, 0x9a, 0x96, 0x63, 0x49, 0xcb, 0xa9, 0x37, 0xc4, 0xa2, 0xa3, 0x6c, 0xb4, 0x8e,
0xea, 0xcb, 0xaa, 0xa3, 0x6e, 0xa2, 0x03, 0xa8, 0xf0, 0x8b, 0x11, 0x76, 0x47, 0x5e, 0xe8, 0x0d,
0x59, 0x7d, 0x65, 0xcb, 0x6c, 0x56, 0xf6, 0xb6, 0x77, 0x32, 0x5b, 0xd3, 0x7b, 0x7a, 0x86, 0x2f,
0x3e, 0xf5, 0x06, 0x11, 0x3e, 0xf3, 0x48, 0xe8, 0x80, 0xe8, 0x75, 0x26, 0x3b, 0xa1, 0x23, 0xa8,
0xaa, 0xc9, 0xf5, 0x20, 0xab, 0xf3, 0x0e, 0x52, 0x91, 0xdd, 0xd4, 0x28, 0xf6, 0x6f, 0x0c, 0x80,
0x27, 0x72, 0x3b, 0xc2, 0x88, 0x7e, 0x18, 0xef, 0x88, 0x04, 0x3d, 0x2a, 0xa3, 0x51, 0xd9, 0xbb,
0xbb, 0x33, 0x19, 0xf2, 0x9d, 0x24, 0x84, 0x7a, 0xc3, 0x32, 0x9a, 0x75, 0x58, 0xf3, 0xf1, 0x00,
0x73, 0xec, 0xcb, 0x48, 0x95, 0x9d, 0xb8, 0x89, 0xee, 0x41, 0xa5, 0x1b, 0x62, 0x8f, 0x63, 0x97,
0x13, 0x1d, 0xaa, 0x65, 0x07, 0x94, 0xe9, 0x39, 0x19, 0x62, 0xfb, 0xbf, 0x26, 0x54, 0xdb, 0xb8,
0x3f, 0xc4, 0x01, 0x57, 0x2b, 0x99, 0x07, 0x99, 0x2d, 0xa8, 0x8c, 0xbc, 0x90, 0x13, 0xed, 0xa2,
0xd0, 0x49, 0x9b, 0xd0, 0x1d, 0xb0, 0x98, 0x1e, 0xf5, 0x48, 0xce, 0x6a, 0x3a, 0x63, 0x03, 0x7a,
0x1f, 0xca, 0x41, 0x34, 0x74, 0x43, 0xfa, 0x9a, 0xc5, 0x08, 0x05, 0xd1, 0xd0, 0xa1, 0xaf, 0x59,
0x1a, 0xbb, 0x95, 0x2c, 0x76, 0x75, 0x58, 0xeb, 0x44, 0x44, 0xa6, 0xc3, 0xaa, 0xfa, 0xa2, 0x9b,
0xe8, 0x6b, 0xb0, 0x1a, 0x50, 0x1f, 0xb7, 0x8e, 0xea, 0x6b, 0xf2, 0x83, 0x6e, 0xa1, 0xfb, 0x50,
0x53, 0x41, 0x7d, 0x85, 0x43, 0x46, 0x68, 0x50, 0x2f, 0xab, 0xbd, 0x48, 0xe3, 0xa7, 0xca, 0x86,
0xbe, 0x0b, 0x2b, 0x8c, 0x7b, 0x1c, 0xd7, 0xad, 0x2d, 0xa3, 0xb9, 0xbe, 0x77, 0xaf, 0x10, 0x47,
0x19, 0x9a, 0xb6, 0x70, 0x73, 0x94, 0xb7, 0x08, 0x6c, 0xcf, 0x23, 0x03, 0x37, 0xc4, 0x1e, 0xa3,
0x41, 0x1d, 0x64, 0x0e, 0x82, 0x30, 0x39, 0xd2, 0x82, 0xbe, 0x15, 0x1f, 0xa2, 0x1e, 0x19, 0x60,
0xe6, 0x8e, 0x3c, 0xfe, 0x82, 0xd5, 0x2b, 0x5b, 0x66, 0xd3, 0x72, 0x36, 0xe4, 0x87, 0x27, 0xc2,
0x7e, 0x26, 0xcc, 0x69, 0xfc, 0xaa, 0x97, 0xe2, 0x57, 0xcb, 0xe3, 0x87, 0x1e, 0xc0, 0x3a, 0xc3,
0x21, 0xf1, 0x06, 0xe4, 0x2d, 0x76, 0x19, 0x79, 0x8b, 0xeb, 0xeb, 0xd2, 0xa7, 0x96, 0x58, 0xdb,
0xe4, 0x2d, 0xb6, 0xff, 0x6c, 0xc0, 0x4d, 0x07, 0xf7, 0x09, 0xe3, 0x38, 0x3c, 0xa5, 0x3e, 0x76,
0xf0, 0xcb, 0x08, 0x33, 0x8e, 0x1e, 0xc3, 0x72, 0xc7, 0x63, 0x58, 0x67, 0xdc, 0x9d, 0xc2, 0xcd,
0x9f, 0xb0, 0xfe, 0x81, 0xc7, 0xb0, 0x23, 0x3d, 0xd1, 0xf7, 0x60, 0xcd, 0xf3, 0xfd, 0x10, 0x33,
0x26, 0x71, 0x9f, 0xd6, 0x69, 0x5f, 0xf9, 0x38, 0xb1, 0x73, 0x0a, 0x24, 0x33, 0x0d, 0x92, 0xfd,
0x07, 0x03, 0x6e, 0x65, 0x57, 0xc6, 0x46, 0x34, 0x60, 0x18, 0x7d, 0x08, 0xab, 0x22, 0xd4, 0x11,
0xd3, 0x8b, 0xbb, 0x5d, 0x38, 0x4f, 0x5b, 0xba, 0x38, 0xda, 0x55, 0x1c, 0x70, 0x12, 0x10, 0x1e,
0x9f, 0x4d, 0xb5, 0xc2, 0xed, 0xfc, 0x41, 0xd2, 0x65, 0xaa, 0x15, 0x10, 0xae, 0x8e, 0xa3, 0x03,
0x24, 0xf9, 0x6d, 0xff, 0x0c, 0x6e, 0x3d, 0xc5, 0x3c, 0x05, 0xb9, 0x8e, 0xd5, 0x3c, 0x27, 0x23,
0x5b, 0x99, 0x4a, 0xb9, 0xca, 0x64, 0xff, 0xcd, 0x80, 0xf7, 0x72, 0x63, 0x2f, 0xb2, 0xdb, 0x24,
0x77, 0x4b, 0x8b, 0xe4, 0xae, 0x99, 0xcf, 0x5d, 0xfb, 0xd7, 0x06, 0xdc, 0x7e, 0x8a, 0x79, 0xba,
0x2e, 0x5c, 0x73, 0x24, 0xd0, 0xd7, 0x01, 0x92, 0x7a, 0xc0, 0xea, 0xe6, 0x96, 0xd9, 0x34, 0x9d,
0x94, 0xc5, 0xfe, 0x9d, 0x01, 0x37, 0x26, 0xe6, 0xcf, 0x96, 0x15, 0x23, 0x5f, 0x56, 0xfe, 0x5f,
0xe1, 0xf8, 0xa3, 0x01, 0x77, 0x8a, 0xc3, 0xb1, 0x08, 0x78, 0x3f, 0x52, 0x9d, 0xb0, 0xc8, 0x52,
0xc1, 0x20, 0x0f, 0x8a, 0xca, 0xfd, 0xe4, 0x9c, 0xba, 0x93, 0xfd, 0x97, 0x12, 0xa0, 0x43, 0x59,
0x07, 0xe4, 0xc7, 0x77, 0x81, 0xe6, 0xca, 0xc4, 0x9a, 0xa3, 0xcf, 0xe5, 0xeb, 0xa0, 0xcf, 0x95,
0xab, 0xd0, 0xa7, 0x48, 0x04, 0x51, 0x10, 0x19, 0xf7, 0x86, 0x23, 0x49, 0x07, 0xcb, 0xce, 0xd8,
0x60, 0xbf, 0x81, 0x9b, 0xf1, 0x29, 0x93, 0x54, 0xf9, 0x0e, 0xb1, 0xc9, 0xe6, 0x65, 0x29, 0x9f,
0x97, 0x33, 0x22, 0x64, 0xff, 0xa3, 0x04, 0x37, 0x5a, 0x71, 0x75, 0x17, 0xc5, 0x5d, 0xf2, 0xf3,
0xe5, 0x69, 0x3b, 0x1d, 0x8e, 0x14, 0x19, 0x9a, 0x53, 0xc9, 0x70, 0x39, 0x4b, 0x86, 0xd9, 0x05,
0xae, 0xe4, 0x21, 0xbc, 0x16, 0xf5, 0x82, 0x9a, 0xb0, 0x39, 0x26, 0x37, 0xcd, 0x6d, 0x6b, 0x92,
0xdb, 0xd6, 0x49, 0x7a, 0xf7, 0x0c, 0x3d, 0x82, 0x8d, 0x84, 0x89, 0x7c, 0x45, 0x50, 0x65, 0x09,
0xd7, 0x98, 0xb6, 0x7c, 0xc9, 0x50, 0xff, 0x34, 0xa0, 0x92, 0x64, 0xfb, 0x9c, 0x0a, 0x31, 0x13,
0xd7, 0x52, 0x3e, 0xae, 0xdb, 0x50, 0xc5, 0x81, 0xd7, 0x19, 0x60, 0x57, 0xae, 0x49, 0x86, 0xb0,
0xec, 0x54, 0x94, 0x4d, 0x89, 0x9d, 0x27, 0x50, 0x19, 0xcb, 0xae, 0x38, 0xa1, 0x1f, 0x4c, 0xd5,
0x5d, 0x69, 0x50, 0x1d, 0x48, 0xf4, 0x17, 0xb3, 0x7f, 0x5f, 0x1a, 0x73, 0x86, 0xca, 0xb8, 0x45,
0x2a, 0xc3, 0xcf, 0xa1, 0xaa, 0x77, 0xa1, 0xe4, 0xa0, 0xaa, 0x0f, 0xdf, 0x2f, 0x5a, 0x56, 0xd1,
0xa4, 0x3b, 0xa9, 0x30, 0x7e, 0x1c, 0xf0, 0xf0, 0xc2, 0xa9, 0xb0, 0xb1, 0xa5, 0xe1, 0xc2, 0x66,
0xde, 0x01, 0x6d, 0x82, 0x79, 0x8e, 0x2f, 0x74, 0x8c, 0xc5, 0x4f, 0x51, 0x4b, 0x5f, 0x09, 0xec,
0x35, 0x85, 0xde, 0xbb, 0xb4, 0x38, 0xf5, 0xa8, 0xa3, 0xbc, 0x7f, 0x50, 0xfa, 0xc8, 0xb0, 0x29,
0x6c, 0x1e, 0x85, 0x74, 0xf4, 0xce, 0x65, 0x69, 0xb6, 0xaa, 0x2f, 0x3e, 0x26, 0x82, 0xb0, 0x8f,
0x30, 0xeb, 0x86, 0xa4, 0x83, 0x17, 0x9c, 0x74, 0x82, 0xb0, 0xbf, 0x30, 0xe0, 0xbd, 0xdc, 0xd8,
0x8b, 0x20, 0xfb, 0xe3, 0x6c, 0xbe, 0x29, 0x60, 0x67, 0xe8, 0xfc, 0x74, 0x9e, 0x79, 0x92, 0x88,
0xe4, 0xb7, 0x03, 0x71, 0xde, 0xcf, 0x42, 0xda, 0x97, 0x32, 0xeb, 0xfa, 0x76, 0xfc, 0x27, 0x03,
0xee, 0x4e, 0x99, 0x63, 0x91, 0x9d, 0x6f, 0xeb, 0xba, 0x83, 0x7d, 0x25, 0xfb, 0xf5, 0x9d, 0x41,
0xdb, 0xa4, 0xf4, 0xbf, 0x0b, 0xc0, 0x29, 0xf7, 0x06, 0xca, 0x41, 0x5f, 0x1a, 0xa4, 0x45, 0x7c,
0xb6, 0x7f, 0x5b, 0x82, 0x5a, 0x9b, 0xd3, 0xd0, 0xeb, 0xe3, 0x43, 0x1a, 0xf4, 0x48, 0x5f, 0x64,
0x44, 0x2c, 0x45, 0x0d, 0xb9, 0x8d, 0x44, 0x6c, 0x6e, 0x43, 0xd5, 0xeb, 0x76, 0x31, 0x63, 0xee,
0x39, 0xbe, 0xd0, 0x09, 0x63, 0x39, 0x15, 0x65, 0x7b, 0x26, 0x4c, 0x42, 0x9f, 0x33, 0xdc, 0x0d,
0x31, 0x77, 0xc7, 0x9e, 0x3a, 0xe9, 0x36, 0xd4, 0x87, 0xfd, 0xd8, 0x5b, 0x68, 0xd7, 0x88, 0xe1,
0x76, 0xfb, 0x13, 0x59, 0x6c, 0xcb, 0x8e, 0x6e, 0x09, 0xe5, 0xd0, 0x89, 0xba, 0xe7, 0x98, 0xa7,
0x8b, 0x2d, 0x28, 0x93, 0xcc, 0xd9, 0xdb, 0x60, 0x85, 0x94, 0x72, 0x59, 0x21, 0x25, 0x4d, 0x59,
0x4e, 0x59, 0x18, 0x44, 0x11, 0xd1, 0xa3, 0xb6, 0xf6, 0x4f, 0xe4, 0xb5, 0x45, 0x8d, 0xda, 0xda,
0x3f, 0x11, 0xb7, 0xab, 0xd6, 0xfe, 0xc9, 0xc7, 0x81, 0x3f, 0xa2, 0x24, 0xe0, 0xb2, 0x5c, 0x5a,
0x4e, 0xda, 0x64, 0xff, 0xcb, 0x84, 0x4d, 0xc5, 0xfd, 0xc7, 0xb4, 0x13, 0x63, 0x7f, 0x07, 0xac,
0xee, 0x20, 0x12, 0x32, 0x7a, 0x4c, 0x32, 0x89, 0x21, 0x7b, 0x1d, 0x71, 0x47, 0x21, 0xee, 0x91,
0x37, 0x3a, 0x2c, 0xe3, 0xeb, 0xc8, 0x99, 0x34, 0xa7, 0xc9, 0xc5, 0x9c, 0x20, 0x17, 0xdf, 0xe3,
0x9e, 0xae, 0xf8, 0xcb, 0xb2, 0xe2, 0x5b, 0xc2, 0xa2, 0x8a, 0xfd, 0xc4, 0x85, 0x6b, 0xa5, 0xe0,
0xc2, 0x95, 0x22, 0xb5, 0xd5, 0x2c, 0xa9, 0x65, 0x33, 0x73, 0x2d, 0x5f, 0x00, 0x7e, 0x02, 0xeb,
0x4c, 0xe1, 0xef, 0x76, 0x65, 0x02, 0xc8, 0xd0, 0x14, 0xc8, 0x7b, 0x59, 0x9b, 0xd2, 0x99, 0xe2,
0xd4, 0x58, 0x26, 0x71, 0xf2, 0x24, 0x68, 0x5d, 0x89, 0x04, 0x73, 0x6a, 0x08, 0xae, 0xa0, 0x86,
0xec, 0x4f, 0x60, 0xf3, 0xa7, 0x11, 0x0e, 0x2f, 0x8e, 0x69, 0x87, 0xcd, 0x07, 0x64, 0x03, 0xca,
0x1a, 0x8d, 0x58, 0x9e, 0x24, 0x6d, 0xfb, 0xdf, 0x06, 0xd4, 0xe4, 0xc1, 0x7d, 0xee, 0xb1, 0xf3,
0xf8, 0x65, 0x20, 0x86, 0xd2, 0xc8, 0x42, 0x79, 0x75, 0xb1, 0x9c, 0xba, 0xd6, 0x4a, 0xe1, 0x6e,
0xe9, 0x12, 0x25, 0x2f, 0xb4, 0x45, 0x84, 0xbf, 0x5c, 0x44, 0xf8, 0x79, 0xd9, 0xbd, 0x32, 0x21,
0xbb, 0xbf, 0x34, 0xe0, 0x46, 0x2a, 0x38, 0x8b, 0x54, 0x9f, 0x4c, 0x48, 0x4b, 0xf9, 0x90, 0x1e,
0x64, 0xab, 0xb2, 0x59, 0x04, 0x64, 0xaa, 0x2a, 0xc7, 0xc1, 0xcd, 0x54, 0xe6, 0x67, 0xb0, 0x21,
0x48, 0xef, 0x7a, 0x70, 0xfc, 0x8f, 0x01, 0x6b, 0xc7, 0xb4, 0x23, 0x11, 0x4c, 0xbf, 0x95, 0x18,
0xd9, 0xb7, 0x92, 0x4d, 0x30, 0x7d, 0x32, 0xd4, 0xfb, 0x11, 0x3f, 0xc5, 0x09, 0x62, 0xdc, 0x0b,
0xf9, 0xf8, 0xb5, 0x47, 0x28, 0x22, 0x61, 0x91, 0x8f, 0x05, 0xef, 0x43, 0x19, 0x07, 0xbe, 0xfa,
0xa8, 0x65, 0x23, 0x0e, 0x7c, 0xf9, 0xe9, 0x7a, 0x64, 0xf9, 0x2d, 0x58, 0x19, 0xd1, 0xf1, 0x0b,
0x8d, 0x6a, 0xd8, 0xb7, 0x00, 0x3d, 0xc5, 0xfc, 0x98, 0x76, 0x04, 0x2a, 0x71, 0x78, 0xec, 0xbf,
0x96, 0xa4, 0x4a, 0x1f, 0x9b, 0x17, 0x01, 0xd8, 0x86, 0x9a, 0xe2, 0x8e, 0xcf, 0x69, 0xc7, 0x0d,
0xa2, 0x38, 0x28, 0x15, 0x69, 0x3c, 0xa6, 0x9d, 0xd3, 0x68, 0x88, 0x3e, 0x80, 0x9b, 0x24, 0x70,
0x47, 0x9a, 0xce, 0x12, 0x4f, 0x15, 0xa5, 0x4d, 0x12, 0xc4, 0x44, 0xa7, 0xdd, 0x1f, 0xc2, 0x06,
0x0e, 0x5e, 0x46, 0x38, 0xc2, 0x89, 0xab, 0x8a, 0x59, 0x4d, 0x9b, 0xb5, 0x9f, 0xa0, 0x2d, 0x8f,
0x9d, 0xbb, 0x6c, 0x40, 0x39, 0xd3, 0x15, 0xcf, 0x12, 0x96, 0xb6, 0x30, 0xa0, 0x8f, 0xc0, 0x12,
0xdd, 0x55, 0x6a, 0x29, 0xb5, 0x7d, 0xbb, 0x28, 0xb5, 0x34, 0xde, 0x4e, 0xf9, 0x73, 0xf5, 0x83,
0xed, 0x7d, 0x69, 0x01, 0xc8, 0x84, 0x3b, 0xa4, 0x34, 0xf4, 0xd1, 0x48, 0x46, 0xf1, 0x90, 0x0e,
0x47, 0x34, 0xc0, 0x01, 0x97, 0xa7, 0x92, 0xa1, 0xc7, 0x53, 0xde, 0x36, 0x26, 0x5d, 0x75, 0xdc,
0x1b, 0x0f, 0xa7, 0xf4, 0xc8, 0xb9, 0xdb, 0x4b, 0xe8, 0xa5, 0x14, 0xb5, 0xa2, 0x49, 0x18, 0x27,
0x5d, 0x76, 0xf8, 0xc2, 0x0b, 0x02, 0x3c, 0x40, 0x7b, 0xd3, 0xe7, 0x9c, 0x70, 0x8e, 0x67, 0xbd,
0x9f, 0xed, 0xa3, 0x1b, 0x6d, 0x1e, 0x92, 0xa0, 0x1f, 0x43, 0x6f, 0x2f, 0xa1, 0xe7, 0x50, 0x49,
0x5d, 0x6a, 0xd1, 0xc3, 0xa2, 0x48, 0x4d, 0xde, 0x7a, 0x1b, 0x97, 0xe5, 0x88, 0xbd, 0x84, 0x7a,
0x50, 0xcb, 0xbc, 0xba, 0xa0, 0xe6, 0x65, 0x5a, 0x3a, 0xfd, 0xd4, 0xd1, 0xf8, 0xe6, 0x1c, 0x9e,
0xc9, 0xea, 0x7f, 0xa9, 0x02, 0x36, 0xf1, 0x6c, 0xb1, 0x3b, 0x65, 0x90, 0x69, 0x0f, 0x2c, 0x8d,
0xc7, 0xf3, 0x77, 0x48, 0x26, 0xf7, 0xc7, 0x9b, 0x94, 0xf9, 0x83, 0x1e, 0xcd, 0xbe, 0x30, 0xa8,
0xd9, 0x9a, 0xf3, 0xde, 0x2c, 0xec, 0x25, 0x74, 0x06, 0x56, 0x22, 0xee, 0xd1, 0x37, 0x8a, 0x3a,
0xe6, 0xb5, 0xff, 0x1c, 0xe0, 0x64, 0x14, 0x76, 0x31, 0x38, 0x45, 0x02, 0xbf, 0x18, 0x9c, 0x42,
0xb9, 0x6e, 0x2f, 0xa1, 0x5f, 0x8d, 0x9f, 0xde, 0x32, 0xba, 0x16, 0x3d, 0xbe, 0x6c, 0xfb, 0x45,
0x32, 0xbb, 0xf1, 0xed, 0x77, 0xe8, 0x91, 0x4a, 0x0e, 0xd4, 0x7e, 0x41, 0x5f, 0x2b, 0x09, 0x12,
0x85, 0x9e, 0x90, 0xe3, 0xd3, 0xcf, 0xef, 0xa4, 0xeb, 0xd4, 0xc9, 0x2f, 0xe9, 0x91, 0x4c, 0xee,
0x02, 0x3c, 0xc5, 0xfc, 0x04, 0xf3, 0x90, 0x74, 0x59, 0xfe, 0x58, 0xe9, 0xc6, 0xd8, 0x21, 0x9e,
0xea, 0xd1, 0x4c, 0xbf, 0x78, 0x82, 0xbd, 0xbf, 0xaf, 0xea, 0xbf, 0x77, 0x4e, 0xa9, 0x8f, 0xbf,
0x1a, 0xb5, 0xea, 0x0c, 0xac, 0x44, 0x84, 0x17, 0x1f, 0x85, 0xbc, 0x46, 0x9f, 0x75, 0x14, 0x3e,
0x03, 0x2b, 0x11, 0x3c, 0xc5, 0x23, 0xe6, 0xc5, 0x62, 0xe3, 0xc1, 0x0c, 0xaf, 0x64, 0xb5, 0xa7,
0x50, 0x8e, 0x05, 0x0a, 0xba, 0x3f, 0xed, 0xdc, 0xa6, 0x47, 0x9e, 0xb1, 0xd6, 0x5f, 0x40, 0x25,
0xc5, 0xde, 0xc5, 0x95, 0x7a, 0x92, 0xf5, 0x1b, 0x8f, 0x66, 0xfa, 0x7d, 0x35, 0x0e, 0xcc, 0xc1,
0x77, 0x3e, 0xdb, 0xeb, 0x13, 0xfe, 0x22, 0xea, 0x88, 0xc8, 0xee, 0x2a, 0xcf, 0x0f, 0x08, 0xd5,
0xbf, 0x76, 0xe3, 0x55, 0xee, 0xca, 0x91, 0x76, 0x65, 0x9c, 0x46, 0x9d, 0xce, 0xaa, 0x6c, 0x7e,
0xf8, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xee, 0x3b, 0x48, 0x73, 0xa1, 0x1d, 0x00, 0x00,
// 2007 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x59, 0xdd, 0x6f, 0x1b, 0x59,
0x15, 0xcf, 0x78, 0xf2, 0xe1, 0x39, 0x13, 0x27, 0xe9, 0x6d, 0x17, 0x79, 0xdd, 0x96, 0x26, 0x53,
0xda, 0x1a, 0xa4, 0x4d, 0x4a, 0x16, 0xd0, 0x82, 0x00, 0x29, 0x1f, 0xdb, 0xe2, 0x74, 0x13, 0x85,
0x71, 0xb5, 0x12, 0x2b, 0xa4, 0x61, 0xec, 0xb9, 0x4e, 0xef, 0xc6, 0x9e, 0xeb, 0xce, 0xbd, 0xd3,
0x36, 0x45, 0x42, 0xbc, 0xf0, 0x00, 0x5a, 0x09, 0x89, 0x07, 0x10, 0x2f, 0x3c, 0xf1, 0xb4, 0x48,
0xbc, 0x22, 0xf1, 0x2f, 0xc0, 0x5f, 0x81, 0xf8, 0x43, 0xd0, 0xfd, 0x98, 0xf1, 0xcc, 0x78, 0x1c,
0xbb, 0x71, 0x78, 0xda, 0x37, 0xdf, 0x33, 0xe7, 0x7e, 0xfd, 0xce, 0xb9, 0xe7, 0xf7, 0xbb, 0xd7,
0x70, 0x83, 0x84, 0x01, 0x7e, 0xe3, 0x75, 0x29, 0x8d, 0x82, 0xed, 0x61, 0x44, 0x39, 0x45, 0x68,
0x40, 0xfa, 0xaf, 0x62, 0xa6, 0x5a, 0xdb, 0xf2, 0x7b, 0x63, 0xb5, 0x4b, 0x07, 0x03, 0x1a, 0x2a,
0x5b, 0x63, 0x8d, 0x84, 0x1c, 0x47, 0xa1, 0xdf, 0xd7, 0xed, 0xd5, 0x6c, 0x0f, 0xe7, 0x8b, 0x0a,
0x58, 0x2d, 0xd1, 0xab, 0x15, 0xf6, 0x28, 0x72, 0x60, 0xb5, 0x4b, 0xfb, 0x7d, 0xdc, 0xe5, 0x84,
0x86, 0xad, 0xc3, 0xba, 0xb1, 0x69, 0x34, 0x4d, 0x37, 0x67, 0x43, 0x75, 0x58, 0xe9, 0x11, 0xdc,
0x0f, 0x5a, 0x87, 0xf5, 0x8a, 0xfc, 0x9c, 0x34, 0xd1, 0x5d, 0x00, 0xb5, 0xc0, 0xd0, 0x1f, 0xe0,
0xba, 0xb9, 0x69, 0x34, 0x2d, 0xd7, 0x92, 0x96, 0x13, 0x7f, 0x80, 0x45, 0x47, 0xd9, 0x68, 0x1d,
0xd6, 0x17, 0x55, 0x47, 0xdd, 0x44, 0xfb, 0x60, 0xf3, 0x8b, 0x21, 0xf6, 0x86, 0x7e, 0xe4, 0x0f,
0x58, 0x7d, 0x69, 0xd3, 0x6c, 0xda, 0xbb, 0x5b, 0xdb, 0xb9, 0xad, 0xe9, 0x3d, 0x3d, 0xc3, 0x17,
0x9f, 0xfa, 0xfd, 0x18, 0x9f, 0xfa, 0x24, 0x72, 0x41, 0xf4, 0x3a, 0x95, 0x9d, 0xd0, 0x21, 0xac,
0xaa, 0xc9, 0xf5, 0x20, 0xcb, 0xb3, 0x0e, 0x62, 0xcb, 0x6e, 0x6a, 0x14, 0xe7, 0x37, 0x06, 0xc0,
0x13, 0xb9, 0x1d, 0x61, 0x44, 0x3f, 0x4c, 0x76, 0x44, 0xc2, 0x1e, 0x95, 0x68, 0xd8, 0xbb, 0x77,
0xb7, 0xc7, 0x21, 0xdf, 0x4e, 0x21, 0xd4, 0x1b, 0x96, 0x68, 0xd6, 0x61, 0x25, 0xc0, 0x7d, 0xcc,
0x71, 0x20, 0x91, 0xaa, 0xba, 0x49, 0x13, 0xdd, 0x03, 0xbb, 0x1b, 0x61, 0x9f, 0x63, 0x8f, 0x13,
0x0d, 0xd5, 0xa2, 0x0b, 0xca, 0xf4, 0x9c, 0x0c, 0xb0, 0xf3, 0x5f, 0x13, 0x56, 0xdb, 0xf8, 0x6c,
0x80, 0x43, 0xae, 0x56, 0x32, 0x4b, 0x64, 0x36, 0xc1, 0x1e, 0xfa, 0x11, 0x27, 0xda, 0x45, 0x45,
0x27, 0x6b, 0x42, 0x77, 0xc0, 0x62, 0x7a, 0xd4, 0x43, 0x39, 0xab, 0xe9, 0x8e, 0x0c, 0xe8, 0x7d,
0xa8, 0x86, 0xf1, 0xc0, 0x8b, 0xe8, 0x6b, 0x96, 0x44, 0x28, 0x8c, 0x07, 0x2e, 0x7d, 0xcd, 0xb2,
0xb1, 0x5b, 0xca, 0xc7, 0xae, 0x0e, 0x2b, 0x9d, 0x98, 0xc8, 0x74, 0x58, 0x56, 0x5f, 0x74, 0x13,
0x7d, 0x0d, 0x96, 0x43, 0x1a, 0xe0, 0xd6, 0x61, 0x7d, 0x45, 0x7e, 0xd0, 0x2d, 0x74, 0x1f, 0x6a,
0x0a, 0xd4, 0x57, 0x38, 0x62, 0x84, 0x86, 0xf5, 0xaa, 0xda, 0x8b, 0x34, 0x7e, 0xaa, 0x6c, 0xe8,
0xbb, 0xb0, 0xc4, 0xb8, 0xcf, 0x71, 0xdd, 0xda, 0x34, 0x9a, 0x6b, 0xbb, 0xf7, 0x4a, 0xe3, 0x28,
0xa1, 0x69, 0x0b, 0x37, 0x57, 0x79, 0x0b, 0x60, 0x7b, 0x3e, 0xe9, 0x7b, 0x11, 0xf6, 0x19, 0x0d,
0xeb, 0x20, 0x73, 0x10, 0x84, 0xc9, 0x95, 0x16, 0xf4, 0xad, 0xe4, 0x10, 0xf5, 0x48, 0x1f, 0x33,
0x6f, 0xe8, 0xf3, 0x17, 0xac, 0x6e, 0x6f, 0x9a, 0x4d, 0xcb, 0x5d, 0x97, 0x1f, 0x9e, 0x08, 0xfb,
0xa9, 0x30, 0x67, 0xe3, 0xb7, 0x7a, 0x69, 0xfc, 0x6a, 0xc5, 0xf8, 0xa1, 0x07, 0xb0, 0xc6, 0x70,
0x44, 0xfc, 0x3e, 0x79, 0x8b, 0x3d, 0x46, 0xde, 0xe2, 0xfa, 0x9a, 0xf4, 0xa9, 0xa5, 0xd6, 0x36,
0x79, 0x8b, 0x9d, 0x3f, 0x19, 0x70, 0xd3, 0xc5, 0x67, 0x84, 0x71, 0x1c, 0x9d, 0xd0, 0x00, 0xbb,
0xf8, 0x65, 0x8c, 0x19, 0x47, 0x8f, 0x61, 0xb1, 0xe3, 0x33, 0xac, 0x33, 0xee, 0x4e, 0xe9, 0xe6,
0x8f, 0xd9, 0xd9, 0xbe, 0xcf, 0xb0, 0x2b, 0x3d, 0xd1, 0xf7, 0x60, 0xc5, 0x0f, 0x82, 0x08, 0x33,
0x26, 0xe3, 0x3e, 0xa9, 0xd3, 0x9e, 0xf2, 0x71, 0x13, 0xe7, 0x4c, 0x90, 0xcc, 0x6c, 0x90, 0x9c,
0xdf, 0x1b, 0x70, 0x2b, 0xbf, 0x32, 0x36, 0xa4, 0x21, 0xc3, 0xe8, 0x43, 0x58, 0x16, 0x50, 0xc7,
0x4c, 0x2f, 0xee, 0x76, 0xe9, 0x3c, 0x6d, 0xe9, 0xe2, 0x6a, 0x57, 0x71, 0xc0, 0x49, 0x48, 0x78,
0x72, 0x36, 0xd5, 0x0a, 0xb7, 0x8a, 0x07, 0x49, 0x97, 0xa9, 0x56, 0x48, 0xb8, 0x3a, 0x8e, 0x2e,
0x90, 0xf4, 0xb7, 0xf3, 0x33, 0xb8, 0xf5, 0x14, 0xf3, 0x4c, 0xc8, 0x35, 0x56, 0xb3, 0x9c, 0x8c,
0x7c, 0x65, 0xaa, 0x14, 0x2a, 0x93, 0xf3, 0x57, 0x03, 0xde, 0x2b, 0x8c, 0x3d, 0xcf, 0x6e, 0xd3,
0xdc, 0xad, 0xcc, 0x93, 0xbb, 0x66, 0x31, 0x77, 0x9d, 0x5f, 0x1b, 0x70, 0xfb, 0x29, 0xe6, 0xd9,
0xba, 0x70, 0xcd, 0x48, 0xa0, 0xaf, 0x03, 0xa4, 0xf5, 0x80, 0xd5, 0xcd, 0x4d, 0xb3, 0x69, 0xba,
0x19, 0x8b, 0xf3, 0x5b, 0x03, 0x6e, 0x8c, 0xcd, 0x9f, 0x2f, 0x2b, 0x46, 0xb1, 0xac, 0xfc, 0xbf,
0xe0, 0xf8, 0x83, 0x01, 0x77, 0xca, 0xe1, 0x98, 0x27, 0x78, 0x3f, 0x52, 0x9d, 0xb0, 0xc8, 0x52,
0xc1, 0x20, 0x0f, 0xca, 0xca, 0xfd, 0xf8, 0x9c, 0xba, 0x93, 0xf3, 0xe7, 0x0a, 0xa0, 0x03, 0x59,
0x07, 0xe4, 0xc7, 0x77, 0x09, 0xcd, 0x95, 0x89, 0xb5, 0x40, 0x9f, 0x8b, 0xd7, 0x41, 0x9f, 0x4b,
0x57, 0xa1, 0x4f, 0x91, 0x08, 0xa2, 0x20, 0x32, 0xee, 0x0f, 0x86, 0x92, 0x0e, 0x16, 0xdd, 0x91,
0xc1, 0x79, 0x03, 0x37, 0x93, 0x53, 0x26, 0xa9, 0xf2, 0x1d, 0xb0, 0xc9, 0xe7, 0x65, 0xa5, 0x98,
0x97, 0x53, 0x10, 0x72, 0xfe, 0x51, 0x81, 0x1b, 0xad, 0xa4, 0xba, 0x8b, 0xe2, 0x2e, 0xf9, 0xf9,
0xf2, 0xb4, 0x9d, 0x1c, 0x8e, 0x0c, 0x19, 0x9a, 0x13, 0xc9, 0x70, 0x31, 0x4f, 0x86, 0xf9, 0x05,
0x2e, 0x15, 0x43, 0x78, 0x2d, 0xea, 0x05, 0x35, 0x61, 0x63, 0x44, 0x6e, 0x9a, 0xdb, 0x56, 0x24,
0xb7, 0xad, 0x91, 0xec, 0xee, 0x19, 0x7a, 0x04, 0xeb, 0x29, 0x13, 0x05, 0x8a, 0xa0, 0xaa, 0x32,
0x5c, 0x23, 0xda, 0x0a, 0x24, 0x43, 0xfd, 0xd3, 0x00, 0x3b, 0xcd, 0xf6, 0x19, 0x15, 0x62, 0x0e,
0xd7, 0x4a, 0x11, 0xd7, 0x2d, 0x58, 0xc5, 0xa1, 0xdf, 0xe9, 0x63, 0x4f, 0xae, 0x49, 0x42, 0x58,
0x75, 0x6d, 0x65, 0x53, 0x62, 0xe7, 0x09, 0xd8, 0x23, 0xd9, 0x95, 0x24, 0xf4, 0x83, 0x89, 0xba,
0x2b, 0x1b, 0x54, 0x17, 0x52, 0xfd, 0xc5, 0x9c, 0xdf, 0x55, 0x46, 0x9c, 0xa1, 0x32, 0x6e, 0x9e,
0xca, 0xf0, 0x73, 0x58, 0xd5, 0xbb, 0x50, 0x72, 0x50, 0xd5, 0x87, 0xef, 0x97, 0x2d, 0xab, 0x6c,
0xd2, 0xed, 0x0c, 0x8c, 0x1f, 0x87, 0x3c, 0xba, 0x70, 0x6d, 0x36, 0xb2, 0x34, 0x3c, 0xd8, 0x28,
0x3a, 0xa0, 0x0d, 0x30, 0xcf, 0xf1, 0x85, 0xc6, 0x58, 0xfc, 0x14, 0xb5, 0xf4, 0x95, 0x88, 0xbd,
0xa6, 0xd0, 0x7b, 0x97, 0x16, 0xa7, 0x1e, 0x75, 0x95, 0xf7, 0x0f, 0x2a, 0x1f, 0x19, 0x0e, 0x85,
0x8d, 0xc3, 0x88, 0x0e, 0xdf, 0xb9, 0x2c, 0x4d, 0x57, 0xf5, 0xe5, 0xc7, 0x44, 0x10, 0xf6, 0x21,
0x66, 0xdd, 0x88, 0x74, 0xf0, 0x9c, 0x93, 0x8e, 0x11, 0xf6, 0x17, 0x06, 0xbc, 0x57, 0x18, 0x7b,
0x9e, 0xc8, 0xfe, 0x38, 0x9f, 0x6f, 0x2a, 0xb0, 0x53, 0x74, 0x7e, 0x36, 0xcf, 0x7c, 0x49, 0x44,
0xf2, 0xdb, 0xbe, 0x38, 0xef, 0xa7, 0x11, 0x3d, 0x93, 0x32, 0xeb, 0xfa, 0x76, 0xfc, 0x47, 0x03,
0xee, 0x4e, 0x98, 0x63, 0x9e, 0x9d, 0x6f, 0xe9, 0xba, 0x83, 0x03, 0x25, 0xfb, 0xf5, 0x9d, 0x41,
0xdb, 0xa4, 0xf4, 0xbf, 0x0b, 0xc0, 0x29, 0xf7, 0xfb, 0xca, 0x41, 0x5f, 0x1a, 0xa4, 0x45, 0x7c,
0x76, 0xfe, 0x56, 0x81, 0x5a, 0x9b, 0xd3, 0xc8, 0x3f, 0xc3, 0x07, 0x34, 0xec, 0x91, 0x33, 0x91,
0x11, 0x89, 0x14, 0x35, 0xe4, 0x36, 0x52, 0xb1, 0xb9, 0x05, 0xab, 0x7e, 0xb7, 0x8b, 0x19, 0xf3,
0xce, 0xf1, 0x85, 0x4e, 0x18, 0xcb, 0xb5, 0x95, 0xed, 0x99, 0x30, 0x09, 0x7d, 0xce, 0x70, 0x37,
0xc2, 0xdc, 0x1b, 0x79, 0xea, 0xa4, 0x5b, 0x57, 0x1f, 0xf6, 0x12, 0x6f, 0xa1, 0x5d, 0x63, 0x86,
0xdb, 0xed, 0x4f, 0x64, 0xb1, 0xad, 0xba, 0xba, 0x25, 0x94, 0x43, 0x27, 0xee, 0x9e, 0x63, 0x9e,
0x2d, 0xb6, 0xa0, 0x4c, 0x32, 0x67, 0x6f, 0x83, 0x15, 0x51, 0xca, 0x65, 0x85, 0x94, 0x34, 0x65,
0xb9, 0x55, 0x61, 0x10, 0x45, 0x44, 0x8f, 0xda, 0xda, 0x3b, 0x96, 0xd7, 0x16, 0x35, 0x6a, 0x6b,
0xef, 0x58, 0xdc, 0xae, 0x5a, 0x7b, 0xc7, 0x1f, 0x87, 0xc1, 0x90, 0x92, 0x90, 0xcb, 0x72, 0x69,
0xb9, 0x59, 0x93, 0xd8, 0x1e, 0x53, 0x48, 0x78, 0x82, 0x59, 0xe5, 0xd5, 0xc5, 0x72, 0x6d, 0x6d,
0x7b, 0x7e, 0x31, 0xc4, 0xce, 0x7f, 0x4c, 0xd8, 0x50, 0xf2, 0xe0, 0x88, 0x76, 0x92, 0xf4, 0xb8,
0x03, 0x56, 0xb7, 0x1f, 0x0b, 0xa5, 0xad, 0x73, 0xc3, 0x72, 0x47, 0x86, 0xfc, 0x8d, 0xc5, 0x1b,
0x46, 0xb8, 0x47, 0xde, 0x68, 0xe4, 0x46, 0x37, 0x96, 0x53, 0x69, 0xce, 0xf2, 0x8f, 0x39, 0xc6,
0x3f, 0x81, 0xcf, 0x7d, 0x4d, 0x0a, 0x8b, 0x92, 0x14, 0x2c, 0x61, 0x51, 0x7c, 0x30, 0x76, 0x27,
0x5b, 0x2a, 0xb9, 0x93, 0x65, 0x78, 0x6f, 0x39, 0xcf, 0x7b, 0xf9, 0xe4, 0x5d, 0x29, 0xd6, 0x88,
0x9f, 0xc0, 0x5a, 0x02, 0x4c, 0x57, 0xe6, 0x88, 0x44, 0xaf, 0xe4, 0x06, 0x20, 0xcb, 0x57, 0x36,
0x99, 0xdc, 0x1a, 0xcb, 0xe5, 0x56, 0x91, 0x27, 0xad, 0x2b, 0xf1, 0x64, 0x41, 0x30, 0xc1, 0x55,
0x04, 0x53, 0xf6, 0xb2, 0x6c, 0xe7, 0x2e, 0xcb, 0xce, 0x27, 0xb0, 0xf1, 0xd3, 0x18, 0x47, 0x17,
0x47, 0xb4, 0xc3, 0x66, 0x8b, 0x71, 0x03, 0xaa, 0x3a, 0x50, 0x89, 0xb8, 0x49, 0xdb, 0xce, 0xbf,
0x0c, 0xa8, 0xc9, 0x63, 0xff, 0xdc, 0x67, 0xe7, 0xc9, 0xbb, 0x42, 0x12, 0x65, 0x23, 0x1f, 0xe5,
0xab, 0x4b, 0xed, 0xcc, 0xa5, 0x58, 0xca, 0x7e, 0x4b, 0x17, 0x38, 0x79, 0x1d, 0x2e, 0x93, 0x0b,
0x8b, 0x65, 0x72, 0xa1, 0x28, 0xda, 0x97, 0xc6, 0x44, 0xfb, 0x97, 0x06, 0xdc, 0xc8, 0x80, 0x33,
0x4f, 0xed, 0xca, 0x41, 0x5a, 0x29, 0x42, 0xba, 0x9f, 0xaf, 0xe9, 0x66, 0x59, 0x8c, 0x33, 0x35,
0x3d, 0x01, 0x37, 0x57, 0xd7, 0x9f, 0xc1, 0xba, 0xa0, 0xcc, 0xeb, 0x89, 0xe3, 0xbf, 0x0d, 0x58,
0x39, 0xa2, 0x1d, 0x19, 0xc1, 0x6c, 0xf2, 0x18, 0xf9, 0x97, 0x96, 0x0d, 0x30, 0x03, 0x32, 0xd0,
0x85, 0x58, 0xfc, 0x14, 0x87, 0x8b, 0x71, 0x3f, 0xe2, 0xa3, 0xb7, 0x22, 0xa1, 0xa7, 0x84, 0x45,
0x3e, 0x35, 0xbc, 0x0f, 0x55, 0x1c, 0x06, 0xea, 0xa3, 0x16, 0x9d, 0x38, 0x0c, 0xe4, 0xa7, 0xeb,
0x11, 0xf5, 0xb7, 0x60, 0x69, 0x48, 0x47, 0xef, 0x3b, 0xaa, 0xe1, 0xdc, 0x02, 0xf4, 0x14, 0xf3,
0x23, 0xda, 0x11, 0x51, 0x49, 0xe0, 0x71, 0xfe, 0x52, 0x91, 0x1a, 0x7f, 0x64, 0x9e, 0x27, 0xc0,
0x0e, 0xd4, 0x14, 0xf3, 0x7c, 0x4e, 0x3b, 0x5e, 0x18, 0x27, 0xa0, 0xd8, 0xd2, 0x78, 0x44, 0x3b,
0x27, 0xf1, 0x00, 0x7d, 0x00, 0x37, 0x49, 0xe8, 0x0d, 0x35, 0x19, 0xa6, 0x9e, 0x0a, 0xa5, 0x0d,
0x12, 0x26, 0x34, 0xa9, 0xdd, 0x1f, 0xc2, 0x3a, 0x0e, 0x5f, 0xc6, 0x38, 0xc6, 0xa9, 0xab, 0xc2,
0xac, 0xa6, 0xcd, 0xda, 0x4f, 0x90, 0x9e, 0xcf, 0xce, 0x3d, 0xd6, 0xa7, 0x9c, 0xe9, 0x62, 0x68,
0x09, 0x4b, 0x5b, 0x18, 0xd0, 0x47, 0x60, 0x89, 0xee, 0x2a, 0xb5, 0x94, 0x56, 0xbf, 0x5d, 0x96,
0x5a, 0x3a, 0xde, 0x6e, 0xf5, 0x73, 0xf5, 0x83, 0xed, 0x7e, 0x69, 0x01, 0xc8, 0x84, 0x3b, 0xa0,
0x34, 0x0a, 0xd0, 0x50, 0xa2, 0x78, 0x40, 0x07, 0x43, 0x1a, 0xe2, 0x90, 0xcb, 0x53, 0xc9, 0xd0,
0xe3, 0x09, 0x2f, 0x23, 0xe3, 0xae, 0x1a, 0xf7, 0xc6, 0xc3, 0x09, 0x3d, 0x0a, 0xee, 0xce, 0x02,
0x7a, 0x29, 0x25, 0xb1, 0x68, 0x12, 0xc6, 0x49, 0x97, 0x1d, 0xbc, 0xf0, 0xc3, 0x10, 0xf7, 0xd1,
0xee, 0xe4, 0x39, 0xc7, 0x9c, 0x93, 0x59, 0xef, 0xe7, 0xfb, 0xe8, 0x46, 0x9b, 0x47, 0x24, 0x3c,
0x4b, 0x42, 0xef, 0x2c, 0xa0, 0xe7, 0x60, 0x67, 0xae, 0xc4, 0xe8, 0x61, 0x19, 0x52, 0xe3, 0x77,
0xe6, 0xc6, 0x65, 0x39, 0xe2, 0x2c, 0xa0, 0x1e, 0xd4, 0x72, 0x6f, 0x36, 0xa8, 0x79, 0x99, 0x12,
0xcf, 0x3e, 0x94, 0x34, 0xbe, 0x39, 0x83, 0x67, 0xba, 0xfa, 0x5f, 0x2a, 0xc0, 0xc6, 0x1e, 0x3d,
0x76, 0x26, 0x0c, 0x32, 0xe9, 0x79, 0xa6, 0xf1, 0x78, 0xf6, 0x0e, 0xe9, 0xe4, 0xc1, 0x68, 0x93,
0x32, 0x7f, 0xd0, 0xa3, 0xe9, 0xd7, 0x0d, 0x35, 0x5b, 0x73, 0xd6, 0x7b, 0x89, 0xb3, 0x80, 0x4e,
0xc1, 0x4a, 0xaf, 0x06, 0xe8, 0x1b, 0x65, 0x1d, 0x8b, 0x37, 0x87, 0x19, 0x82, 0x93, 0xd3, 0xe7,
0xe5, 0xc1, 0x29, 0xbb, 0x1e, 0x94, 0x07, 0xa7, 0x54, 0xec, 0x3b, 0x0b, 0xe8, 0x57, 0xa3, 0x87,
0xbb, 0x9c, 0x2a, 0x46, 0x8f, 0x2f, 0xdb, 0x7e, 0x99, 0x48, 0x6f, 0x7c, 0xfb, 0x1d, 0x7a, 0x64,
0x92, 0x03, 0xb5, 0x5f, 0xd0, 0xd7, 0x4a, 0x9d, 0xc4, 0x91, 0x2f, 0xc4, 0xfc, 0xe4, 0xf3, 0x3b,
0xee, 0x3a, 0x71, 0xf2, 0x4b, 0x7a, 0xa4, 0x93, 0x7b, 0x00, 0x4f, 0x31, 0x3f, 0xc6, 0x3c, 0x22,
0x5d, 0x56, 0x3c, 0x56, 0xba, 0x31, 0x72, 0x48, 0xa6, 0x7a, 0x34, 0xd5, 0x2f, 0x99, 0x60, 0xf7,
0xef, 0xcb, 0xfa, 0xcf, 0xa1, 0x13, 0x1a, 0xe0, 0xaf, 0x46, 0xad, 0x3a, 0x05, 0x2b, 0xd5, 0xe7,
0xe5, 0x47, 0xa1, 0x28, 0xdf, 0xa7, 0x1d, 0x85, 0xcf, 0xc0, 0x4a, 0x05, 0x4f, 0xf9, 0x88, 0x45,
0xb1, 0xd8, 0x78, 0x30, 0xc5, 0x2b, 0x5d, 0xed, 0x09, 0x54, 0x13, 0x81, 0x82, 0xee, 0x4f, 0x3a,
0xb7, 0xd9, 0x91, 0xa7, 0xac, 0xf5, 0x17, 0x60, 0x67, 0xd8, 0xbb, 0xbc, 0x52, 0x8f, 0xb3, 0x7e,
0xe3, 0xd1, 0x54, 0xbf, 0xaf, 0xc6, 0x81, 0xd9, 0xff, 0xce, 0x67, 0xbb, 0x67, 0x84, 0xbf, 0x88,
0x3b, 0x02, 0xd9, 0x1d, 0xe5, 0xf9, 0x01, 0xa1, 0xfa, 0xd7, 0x4e, 0xb2, 0xca, 0x1d, 0x39, 0xd2,
0x8e, 0xc4, 0x69, 0xd8, 0xe9, 0x2c, 0xcb, 0xe6, 0x87, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0x4a,
0x31, 0x02, 0x5e, 0xdf, 0x1d, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -328,7 +328,7 @@ type IndexNode interface {
QueryJobs(context.Context, *indexpb.QueryJobsRequest) (*indexpb.QueryJobsResponse, error)
// DropJobs cancel index building jobs specified by BuildIDs. Notes that dropping task may have finished.
DropJobs(context.Context, *indexpb.DropJobsRequest) (*commonpb.Status, error)
// GetJobNum returns metrics of indexnode, including available job queue info, available task slots and finished job infos.
// GetJobStats returns metrics of indexnode, including available job queue info, available task slots and finished job infos.
GetJobStats(context.Context, *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error)
ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error)