mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 09:08:43 +08:00
enhance: integrate Storage V2 FFI interface for unified storage access (#45723)
Related #44956 This commit integrates the Storage V2 FFI (Foreign Function Interface) interface throughout the Milvus codebase, enabling unified storage access through the Loon FFI layer. This is a significant step towards standardizing storage operations across different storage versions. 1. Configuration Support - **configs/milvus.yaml**: Added `useLoonFFI` configuration flag under `common.storage.file.splitByAvgSize` section - Allows runtime toggle between traditional binlog readers and new FFI-based manifest readers - Default: `false` (maintains backward compatibility) 2. Core FFI Infrastructure Enhanced Utilities (internal/core/src/storage/loon_ffi/util.cpp/h) - **ToCStorageConfig()**: Converts Go's `StorageConfig` to C's `CStorageConfig` struct for FFI calls - **GetManifest()**: Parses manifest JSON and retrieves latest column groups using FFI - Accepts manifest path with `base_path` and `ver` fields - Calls `get_latest_column_groups()` FFI function - Returns column group information as string - Comprehensive error handling for JSON parsing and FFI errors 3. Dependency Updates - **internal/core/thirdparty/milvus-storage/CMakeLists.txt**: - Updated milvus-storage version from `0883026` to `302143c` - Ensures compatibility with latest FFI interfaces 4. Data Coordinator Changes All compaction task builders now include manifest path in segment binlogs: - **compaction_task_clustering.go**: Added `Manifest: segInfo.GetManifestPath()` to segment binlogs - **compaction_task_l0.go**: Added manifest path to both L0 segment selection and compaction plan building - **compaction_task_mix.go**: Added manifest path to mixed compaction segment binlogs - **meta.go**: Updated metadata completion logic: - `completeClusterCompactionMutation()`: Set `ManifestPath` in new segment info - `completeMixCompactionMutation()`: Preserve manifest path in compacted segments - `completeSortCompactionMutation()`: Include manifest path in sorted segments 5. Data Node Compactor Enhancements All compactors updated to support dual-mode reading (binlog vs manifest): 6. Flush & Sync Manager Updates Pack Writer V2 (pack_writer_v2.go) - **BulkPackWriterV2.Write()**: Extended return signature to include `manifest string` - Implementation: - Generate manifest path: `path.Join(pack.segmentID, "manifest.json")` - Write packed data using FFI-based writer - Return manifest path along with binlogs, deltas, and stats Task Handling (task.go) - Updated all sync task result handling to accommodate new manifest return value - Ensured backward compatibility for callers not using manifest 7. Go Storage Layer Integration New Interfaces and Implementations - **record_reader.go**: Interface for unified record reading across storage versions - **record_writer.go**: Interface for unified record writing across storage versions - **binlog_record_writer.go**: Concrete implementation for traditional binlog-based writing Enhanced Schema Support (schema.go, schema_test.go) - Schema conversion utilities to support FFI-based storage operations - Ensures proper Arrow schema mapping for V2 storage Serialization Updates - **serde.go, serde_events.go, serde_events_v2.go**: Updated to work with new reader/writer interfaces - Test files updated to validate dual-mode serialization 8. Storage V2 Packed Format FFI Common (storagev2/packed/ffi_common.go) - Common FFI utilities and type conversions for packed storage format Packed Writer FFI (storagev2/packed/packed_writer_ffi.go) - FFI-based implementation of packed writer - Integrates with Loon storage layer for efficient columnar writes Packed Reader FFI (storagev2/packed/packed_reader_ffi.go) - Already existed, now complemented by writer implementation 9. Protocol Buffer Updates data_coord.proto & datapb/data_coord.pb.go - Added `manifest` field to compaction segment messages - Enables passing manifest metadata through compaction pipeline worker.proto & workerpb/worker.pb.go - Added compaction parameter for `useLoonFFI` flag - Allows workers to receive FFI configuration from coordinator 10. Parameter Configuration component_param.go - Added `UseLoonFFI` parameter to compaction configuration - Reads from `common.storage.file.useLoonFFI` config path - Default: `false` for safe rollout 11. Test Updates - **clustering_compactor_storage_v2_test.go**: Updated signatures to handle manifest return value - **mix_compactor_storage_v2_test.go**: Updated test helpers for manifest support - **namespace_compactor_test.go**: Adjusted writer calls to expect manifest - **pack_writer_v2_test.go**: Validated manifest generation in pack writing This integration follows a **dual-mode approach**: 1. **Legacy Path**: Traditional binlog-based reading/writing (when `useLoonFFI=false` or no manifest) 2. **FFI Path**: Manifest-based reading/writing through Loon FFI (when `useLoonFFI=true` and manifest exists) --------- Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
This commit is contained in:
parent
a7275e190e
commit
c01fd94a6a
@ -1009,6 +1009,7 @@ common:
|
||||
splitByAvgSize:
|
||||
enabled: false # enable split by average size policy in storage v2
|
||||
threshold: 1024 # split by average size policy threshold(in bytes) in storage v2
|
||||
useLoonFFI: false
|
||||
# Whether to disable the internal time messaging mechanism for the system.
|
||||
# If disabled (set to false), the system will not allow DML operations, including insertion, deletion, queries, and searches.
|
||||
# This helps Milvus-CDC synchronize incremental data
|
||||
|
||||
@ -31,6 +31,7 @@ type Params struct {
|
||||
PreferSegmentSizeRatio float64 `json:"prefer_segment_size_ratio,omitempty"`
|
||||
BloomFilterApplyBatchSize int `json:"bloom_filter_apply_batch_size,omitempty"`
|
||||
StorageConfig *indexpb.StorageConfig `json:"storage_config,omitempty"`
|
||||
UseLoonFFI bool `json:"use_loon_ffi,omitempty"`
|
||||
}
|
||||
|
||||
func GenParams() Params {
|
||||
@ -46,6 +47,7 @@ func GenParams() Params {
|
||||
PreferSegmentSizeRatio: paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.GetAsFloat(),
|
||||
BloomFilterApplyBatchSize: paramtable.Get().CommonCfg.BloomFilterApplyBatchSize.GetAsInt(),
|
||||
StorageConfig: CreateStorageConfig(),
|
||||
UseLoonFFI: paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
0
internal/core/src/storage/loon_ffi/ffi_writer_c.cpp
Normal file
0
internal/core/src/storage/loon_ffi/ffi_writer_c.cpp
Normal file
13
internal/core/src/storage/loon_ffi/ffi_writer_c.h
Normal file
13
internal/core/src/storage/loon_ffi/ffi_writer_c.h
Normal file
@ -0,0 +1,13 @@
|
||||
// Copyright 2023 Zilliz
|
||||
//
|
||||
// Licensed 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.
|
||||
@ -17,11 +17,14 @@
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <nlohmann/json.hpp>
|
||||
#include "common/common_type_c.h"
|
||||
#include "common/type_c.h"
|
||||
#include "milvus-storage/properties.h"
|
||||
#include "storage/loon_ffi/util.h"
|
||||
|
||||
using json = nlohmann::json;
|
||||
|
||||
std::shared_ptr<Properties>
|
||||
MakePropertiesFromStorageConfig(CStorageConfig c_storage_config) {
|
||||
// Prepare key-value pairs from CStorageConfig
|
||||
@ -118,3 +121,65 @@ MakePropertiesFromStorageConfig(CStorageConfig c_storage_config) {
|
||||
FreeFFIResult(&result);
|
||||
return properties;
|
||||
}
|
||||
|
||||
CStorageConfig
|
||||
ToCStorageConfig(const milvus::storage::StorageConfig& config) {
|
||||
return CStorageConfig{config.address.c_str(),
|
||||
config.bucket_name.c_str(),
|
||||
config.access_key_id.c_str(),
|
||||
config.access_key_value.c_str(),
|
||||
config.root_path.c_str(),
|
||||
config.storage_type.c_str(),
|
||||
config.cloud_provider.c_str(),
|
||||
config.iam_endpoint.c_str(),
|
||||
config.log_level.c_str(),
|
||||
config.region.c_str(),
|
||||
config.useSSL,
|
||||
config.sslCACert.c_str(),
|
||||
config.useIAM,
|
||||
config.useVirtualHost,
|
||||
config.requestTimeoutMs,
|
||||
config.gcp_credential_json.c_str(),
|
||||
false, // this field does not exist in StorageConfig
|
||||
config.max_connections};
|
||||
}
|
||||
|
||||
std::string
|
||||
GetManifest(const std::string& path,
|
||||
const std::shared_ptr<Properties>& properties) {
|
||||
try {
|
||||
// Parse the JSON string
|
||||
json j = json::parse(path);
|
||||
|
||||
// Extract base_path and ver fields
|
||||
std::string base_path = j.at("base_path").get<std::string>();
|
||||
int64_t ver = j.at("ver").get<int64_t>();
|
||||
|
||||
// return std::make_pair(base_path, ver);
|
||||
char* out_column_groups = nullptr;
|
||||
int64_t out_read_version = 0;
|
||||
FFIResult result = get_latest_column_groups(base_path.c_str(),
|
||||
properties.get(),
|
||||
&out_column_groups,
|
||||
&out_read_version);
|
||||
if (!IsSuccess(&result)) {
|
||||
auto message = GetErrorMessage(&result);
|
||||
// Copy the error message before freeing the FFIResult
|
||||
std::string error_msg = message ? message : "Unknown error";
|
||||
FreeFFIResult(&result);
|
||||
throw std::runtime_error(error_msg);
|
||||
}
|
||||
|
||||
FreeFFIResult(&result);
|
||||
return {out_column_groups};
|
||||
} catch (const json::parse_error& e) {
|
||||
throw std::runtime_error(
|
||||
std::string("Failed to parse manifest JSON: ") + e.what());
|
||||
} catch (const json::out_of_range& e) {
|
||||
throw std::runtime_error(
|
||||
std::string("Missing required field in manifest: ") + e.what());
|
||||
} catch (const json::type_error& e) {
|
||||
throw std::runtime_error(
|
||||
std::string("Invalid field type in manifest: ") + e.what());
|
||||
}
|
||||
}
|
||||
@ -15,6 +15,7 @@
|
||||
#include "common/common_type_c.h"
|
||||
#include "common/type_c.h"
|
||||
#include "milvus-storage/ffi_c.h"
|
||||
#include "storage/Types.h"
|
||||
|
||||
/**
|
||||
* @brief Creates a shared pointer to Properties from CStorageConfig
|
||||
@ -36,3 +37,10 @@
|
||||
*/
|
||||
std::shared_ptr<Properties>
|
||||
MakePropertiesFromStorageConfig(CStorageConfig c_storage_config);
|
||||
|
||||
CStorageConfig
|
||||
ToCStorageConfig(const milvus::storage::StorageConfig& config);
|
||||
|
||||
std::string
|
||||
GetManifest(const std::string& path,
|
||||
const std::shared_ptr<Properties>& properties);
|
||||
@ -14,7 +14,7 @@
|
||||
# Update milvus-storage_VERSION for the first occurrence
|
||||
milvus_add_pkg_config("milvus-storage")
|
||||
set_property(DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES "")
|
||||
set( milvus-storage_VERSION 0883026)
|
||||
set( milvus-storage_VERSION 302143c)
|
||||
set( GIT_REPOSITORY "https://github.com/milvus-io/milvus-storage.git")
|
||||
message(STATUS "milvus-storage repo: ${GIT_REPOSITORY}")
|
||||
message(STATUS "milvus-storage version: ${milvus-storage_VERSION}")
|
||||
|
||||
@ -377,6 +377,7 @@ func (t *clusteringCompactionTask) BuildCompactionRequest() (*datapb.CompactionP
|
||||
Deltalogs: segInfo.GetDeltalogs(),
|
||||
IsSorted: segInfo.GetIsSorted(),
|
||||
StorageVersion: segInfo.GetStorageVersion(),
|
||||
Manifest: segInfo.GetManifestPath(),
|
||||
})
|
||||
}
|
||||
WrapPluginContext(taskProto.GetCollectionID(), taskProto.GetSchema().GetProperties(), plan)
|
||||
|
||||
@ -302,6 +302,7 @@ func (t *l0CompactionTask) selectFlushedSegment() ([]*SegmentInfo, []*datapb.Com
|
||||
CollectionID: info.GetCollectionID(),
|
||||
PartitionID: info.GetPartitionID(),
|
||||
IsSorted: info.GetIsSorted(),
|
||||
Manifest: info.GetManifestPath(),
|
||||
})
|
||||
}
|
||||
|
||||
@ -341,6 +342,7 @@ func (t *l0CompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, err
|
||||
InsertChannel: segInfo.GetInsertChannel(),
|
||||
Deltalogs: segInfo.GetDeltalogs(),
|
||||
IsSorted: segInfo.GetIsSorted(),
|
||||
Manifest: segInfo.GetManifestPath(),
|
||||
})
|
||||
segments = append(segments, segInfo)
|
||||
}
|
||||
|
||||
@ -402,6 +402,7 @@ func (t *mixCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, er
|
||||
Deltalogs: segInfo.GetDeltalogs(),
|
||||
IsSorted: segInfo.GetIsSorted(),
|
||||
StorageVersion: segInfo.GetStorageVersion(),
|
||||
Manifest: segInfo.GetManifestPath(),
|
||||
})
|
||||
segIDMap[segID] = segInfo.GetDeltalogs()
|
||||
segments = append(segments, segInfo)
|
||||
|
||||
@ -1710,6 +1710,7 @@ func (m *meta) completeClusterCompactionMutation(t *datapb.CompactionTask, resul
|
||||
// visible after stats and index
|
||||
IsInvisible: true,
|
||||
StorageVersion: seg.GetStorageVersion(),
|
||||
ManifestPath: seg.GetManifest(),
|
||||
}
|
||||
segment := NewSegmentInfo(segmentInfo)
|
||||
compactToSegInfos = append(compactToSegInfos, segment)
|
||||
@ -1811,6 +1812,7 @@ func (m *meta) completeMixCompactionMutation(
|
||||
return info.GetDmlPosition()
|
||||
})),
|
||||
IsSorted: compactToSegment.GetIsSorted(),
|
||||
ManifestPath: compactToSegment.GetManifest(),
|
||||
})
|
||||
|
||||
if compactToSegmentInfo.GetNumOfRows() == 0 {
|
||||
@ -2300,6 +2302,7 @@ func (m *meta) completeSortCompactionMutation(
|
||||
Deltalogs: resultSegment.GetDeltalogs(),
|
||||
CompactionFrom: []int64{compactFromSegID},
|
||||
IsSorted: true,
|
||||
ManifestPath: resultSegment.GetManifest(),
|
||||
}
|
||||
|
||||
segment := NewSegmentInfo(segmentInfo)
|
||||
|
||||
@ -344,7 +344,8 @@ func (t *clusteringCompactionTask) getScalarAnalyzeResult(ctx context.Context) e
|
||||
t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows,
|
||||
t.partitionID, t.collectionID, t.plan.Channel, 100,
|
||||
storage.WithBufferSize(t.bufferSize),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig))
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -367,7 +368,8 @@ func (t *clusteringCompactionTask) getScalarAnalyzeResult(ctx context.Context) e
|
||||
t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows,
|
||||
t.partitionID, t.collectionID, t.plan.Channel, 100,
|
||||
storage.WithBufferSize(t.bufferSize),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig))
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -427,7 +429,8 @@ func (t *clusteringCompactionTask) generatedVectorPlan(ctx context.Context, buff
|
||||
t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows,
|
||||
t.partitionID, t.collectionID, t.plan.Channel, 100,
|
||||
storage.WithBufferSize(t.bufferSize),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig))
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -497,6 +500,7 @@ func (t *clusteringCompactionTask) mapping(ctx context.Context,
|
||||
Deltalogs: segment.Deltalogs,
|
||||
FieldBinlogs: segment.FieldBinlogs,
|
||||
StorageVersion: segment.StorageVersion,
|
||||
Manifest: segment.GetManifest(),
|
||||
}
|
||||
future := t.mappingPool.Submit(func() (any, error) {
|
||||
err := t.mappingSegment(ctx, segmentClone)
|
||||
@ -605,7 +609,21 @@ func (t *clusteringCompactionTask) mappingSegment(
|
||||
return merr.WrapErrIllegalCompactionPlan()
|
||||
}
|
||||
|
||||
rr, err := storage.NewBinlogRecordReader(ctx,
|
||||
var rr storage.RecordReader
|
||||
if segment.GetManifest() != "" {
|
||||
rr, err = storage.NewManifestRecordReader(ctx,
|
||||
segment.GetManifest(),
|
||||
t.plan.Schema,
|
||||
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
|
||||
return t.binlogIO.Download(ctx, paths)
|
||||
}),
|
||||
storage.WithCollectionID(t.GetCollection()),
|
||||
storage.WithVersion(segment.StorageVersion),
|
||||
storage.WithBufferSize(t.bufferSize),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
)
|
||||
} else {
|
||||
rr, err = storage.NewBinlogRecordReader(ctx,
|
||||
segment.GetFieldBinlogs(),
|
||||
t.plan.Schema,
|
||||
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
|
||||
@ -616,6 +634,8 @@ func (t *clusteringCompactionTask) mappingSegment(
|
||||
storage.WithBufferSize(t.bufferSize),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
log.Warn("new binlog record reader wrong", zap.Error(err))
|
||||
return err
|
||||
@ -902,7 +922,23 @@ func (t *clusteringCompactionTask) scalarAnalyzeSegment(
|
||||
log.Warn("unsupported storage version", zap.Int64("storage version", segment.GetStorageVersion()))
|
||||
return nil, fmt.Errorf("unsupported storage version %d", segment.GetStorageVersion())
|
||||
}
|
||||
rr, err := storage.NewBinlogRecordReader(ctx,
|
||||
var rr storage.RecordReader
|
||||
var err error
|
||||
if segment.GetManifest() != "" {
|
||||
rr, err = storage.NewManifestRecordReader(ctx,
|
||||
segment.GetManifest(),
|
||||
t.plan.GetSchema(),
|
||||
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
|
||||
return t.binlogIO.Download(ctx, paths)
|
||||
}),
|
||||
storage.WithVersion(segment.StorageVersion),
|
||||
storage.WithBufferSize(t.bufferSize),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
storage.WithNeededFields(requiredFields),
|
||||
storage.WithCollectionID(t.GetCollection()),
|
||||
)
|
||||
} else {
|
||||
rr, err = storage.NewBinlogRecordReader(ctx,
|
||||
binlogs,
|
||||
t.plan.GetSchema(),
|
||||
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
|
||||
@ -914,6 +950,7 @@ func (t *clusteringCompactionTask) scalarAnalyzeSegment(
|
||||
storage.WithNeededFields(requiredFields),
|
||||
storage.WithCollectionID(t.GetCollection()),
|
||||
)
|
||||
}
|
||||
if err != nil {
|
||||
log.Warn("new binlog record reader wrong", zap.Error(err))
|
||||
return make(map[interface{}]int64), err
|
||||
|
||||
@ -86,7 +86,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal() {
|
||||
func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2ToV2Format() {
|
||||
var segmentID int64 = 1001
|
||||
|
||||
fBinlogs, deltalogs, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
|
||||
fBinlogs, deltalogs, _, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
|
||||
s.NoError(err)
|
||||
|
||||
dblobs, err := getInt64DeltaBlobs(
|
||||
@ -149,7 +149,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2To
|
||||
|
||||
var segmentID int64 = 1001
|
||||
|
||||
fBinlogs, deltalogs, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
|
||||
fBinlogs, deltalogs, _, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
|
||||
s.NoError(err)
|
||||
|
||||
dblobs, err := getInt64DeltaBlobs(
|
||||
@ -263,6 +263,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) initStorageV2Segments(rows int,
|
||||
deltas *datapb.FieldBinlog,
|
||||
stats map[int64]*datapb.FieldBinlog,
|
||||
bm25Stats map[int64]*datapb.FieldBinlog,
|
||||
manifest string,
|
||||
size int64,
|
||||
err error,
|
||||
) {
|
||||
|
||||
@ -44,7 +44,8 @@ func mergeSortMultipleSegments(ctx context.Context,
|
||||
logIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedLogIDs().GetBegin(), plan.GetPreAllocatedLogIDs().GetEnd())
|
||||
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
|
||||
writer, err := NewMultiSegmentWriter(ctx, binlogIO, compAlloc, plan.GetMaxSize(), plan.GetSchema(), compactionParams, maxRows, partitionID, collectionID, plan.GetChannel(), 4096,
|
||||
storage.WithStorageConfig(compactionParams.StorageConfig))
|
||||
storage.WithStorageConfig(compactionParams.StorageConfig),
|
||||
storage.WithUseLoonFFI(compactionParams.UseLoonFFI))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -58,7 +59,18 @@ func mergeSortMultipleSegments(ctx context.Context,
|
||||
segmentReaders := make([]storage.RecordReader, len(binlogs))
|
||||
segmentFilters := make([]compaction.EntityFilter, len(binlogs))
|
||||
for i, s := range binlogs {
|
||||
reader, err := storage.NewBinlogRecordReader(ctx,
|
||||
var reader storage.RecordReader
|
||||
if s.GetManifest() != "" {
|
||||
reader, err = storage.NewManifestRecordReader(ctx,
|
||||
s.GetManifest(),
|
||||
plan.GetSchema(),
|
||||
storage.WithCollectionID(collectionID),
|
||||
storage.WithDownloader(binlogIO.Download),
|
||||
storage.WithVersion(s.StorageVersion),
|
||||
storage.WithStorageConfig(compactionParams.StorageConfig),
|
||||
)
|
||||
} else {
|
||||
reader, err = storage.NewBinlogRecordReader(ctx,
|
||||
s.GetFieldBinlogs(),
|
||||
plan.GetSchema(),
|
||||
storage.WithCollectionID(collectionID),
|
||||
@ -66,6 +78,7 @@ func mergeSortMultipleSegments(ctx context.Context,
|
||||
storage.WithVersion(s.StorageVersion),
|
||||
storage.WithStorageConfig(compactionParams.StorageConfig),
|
||||
)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
@ -150,7 +150,7 @@ func (t *mixCompactionTask) mergeSplit(
|
||||
segIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedSegmentIDs().GetBegin(), t.plan.GetPreAllocatedSegmentIDs().GetEnd())
|
||||
logIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedLogIDs().GetBegin(), t.plan.GetPreAllocatedLogIDs().GetEnd())
|
||||
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
|
||||
mWriter, err := NewMultiSegmentWriter(ctx, t.binlogIO, compAlloc, t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.maxRows, t.partitionID, t.collectionID, t.GetChannelName(), 4096, storage.WithStorageConfig(t.compactionParams.StorageConfig))
|
||||
mWriter, err := NewMultiSegmentWriter(ctx, t.binlogIO, compAlloc, t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.maxRows, t.partitionID, t.collectionID, t.GetChannelName(), 4096, storage.WithStorageConfig(t.compactionParams.StorageConfig), storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -217,7 +217,18 @@ func (t *mixCompactionTask) writeSegment(ctx context.Context,
|
||||
}
|
||||
entityFilter := compaction.NewEntityFilter(delta, t.plan.GetCollectionTtl(), t.currentTime)
|
||||
|
||||
reader, err := storage.NewBinlogRecordReader(ctx,
|
||||
var reader storage.RecordReader
|
||||
if seg.GetManifest() != "" {
|
||||
reader, err = storage.NewManifestRecordReader(ctx,
|
||||
seg.GetManifest(),
|
||||
t.plan.GetSchema(),
|
||||
storage.WithCollectionID(t.collectionID),
|
||||
storage.WithDownloader(t.binlogIO.Download),
|
||||
storage.WithVersion(seg.GetStorageVersion()),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
)
|
||||
} else {
|
||||
reader, err = storage.NewBinlogRecordReader(ctx,
|
||||
seg.GetFieldBinlogs(),
|
||||
t.plan.GetSchema(),
|
||||
storage.WithCollectionID(t.collectionID),
|
||||
@ -225,6 +236,7 @@ func (t *mixCompactionTask) writeSegment(ctx context.Context,
|
||||
storage.WithVersion(seg.GetStorageVersion()),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
)
|
||||
}
|
||||
if err != nil {
|
||||
log.Warn("compact wrong, failed to new insert binlogs reader", zap.Error(err))
|
||||
return
|
||||
|
||||
@ -121,7 +121,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_MixToV2Format() {
|
||||
|
||||
v2Segments := []int64{10, 11}
|
||||
for _, segID := range v2Segments {
|
||||
binlogs, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
|
||||
binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
|
||||
s.NoError(err)
|
||||
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
|
||||
CollectionID: 1,
|
||||
@ -156,7 +156,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV2Format() {
|
||||
|
||||
v2Segments := []int64{10, 11}
|
||||
for _, segID := range v2Segments {
|
||||
binlogs, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
|
||||
binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
|
||||
s.NoError(err)
|
||||
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
|
||||
CollectionID: 1,
|
||||
@ -192,7 +192,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV1Format() {
|
||||
|
||||
v2Segments := []int64{10, 11}
|
||||
for _, segID := range v2Segments {
|
||||
binlogs, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
|
||||
binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
|
||||
s.NoError(err)
|
||||
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
|
||||
CollectionID: 1,
|
||||
@ -307,6 +307,7 @@ func (s *MixCompactionTaskStorageV2Suite) initStorageV2Segments(rows int, seed i
|
||||
deltas *datapb.FieldBinlog,
|
||||
stats map[int64]*datapb.FieldBinlog,
|
||||
bm25Stats map[int64]*datapb.FieldBinlog,
|
||||
manifest string,
|
||||
size int64,
|
||||
err error,
|
||||
) {
|
||||
|
||||
@ -112,7 +112,7 @@ func (s *NamespaceCompactorTestSuite) setupSortedSegments() {
|
||||
StorageType: "local",
|
||||
RootPath: rootPath,
|
||||
}, columnGroups)
|
||||
inserts, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
inserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
s.Require().NoError(err)
|
||||
s.sortedSegments = append(s.sortedSegments, &datapb.CompactionSegmentBinlogs{
|
||||
SegmentID: int64(i),
|
||||
|
||||
@ -120,7 +120,7 @@ func (w *MultiSegmentWriter) closeWriter() error {
|
||||
return err
|
||||
}
|
||||
|
||||
fieldBinlogs, statsLog, bm25Logs := w.writer.GetLogs()
|
||||
fieldBinlogs, statsLog, bm25Logs, manifest := w.writer.GetLogs()
|
||||
|
||||
result := &datapb.CompactionSegment{
|
||||
SegmentID: w.currentSegmentID,
|
||||
@ -130,6 +130,7 @@ func (w *MultiSegmentWriter) closeWriter() error {
|
||||
Channel: w.channel,
|
||||
Bm25Logs: lo.Values(bm25Logs),
|
||||
StorageVersion: w.storageVersion,
|
||||
Manifest: manifest,
|
||||
}
|
||||
|
||||
w.res = append(w.res, result)
|
||||
|
||||
@ -62,6 +62,8 @@ type sortCompactionTask struct {
|
||||
insertLogs []*datapb.FieldBinlog
|
||||
storageVersion int64
|
||||
segmentStorageVersion int64
|
||||
manifest string
|
||||
useLoonFFI bool
|
||||
|
||||
done chan struct{}
|
||||
tr *timerecord.TimeRecorder
|
||||
@ -132,6 +134,8 @@ func (t *sortCompactionTask) preCompact() error {
|
||||
t.insertLogs = segment.GetFieldBinlogs()
|
||||
t.storageVersion = t.compactionParams.StorageVersion
|
||||
t.segmentStorageVersion = segment.GetStorageVersion()
|
||||
t.manifest = segment.GetManifest()
|
||||
t.useLoonFFI = t.compactionParams.UseLoonFFI
|
||||
|
||||
log.Ctx(t.ctx).Info("preCompaction analyze",
|
||||
zap.Int64("planID", t.GetPlanID()),
|
||||
@ -139,6 +143,7 @@ func (t *sortCompactionTask) preCompact() error {
|
||||
zap.Int64("partitionID", t.partitionID),
|
||||
zap.Int64("segmentID", t.segmentID),
|
||||
zap.Int64("storageVersion", t.storageVersion),
|
||||
zap.Bool("useLoonFFI", t.useLoonFFI),
|
||||
zap.Any("compactionParams", t.compactionParams),
|
||||
)
|
||||
|
||||
@ -175,6 +180,7 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
|
||||
}),
|
||||
storage.WithVersion(t.storageVersion),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
storage.WithUseLoonFFI(t.useLoonFFI),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("sort segment wrong, unable to init segment writer",
|
||||
@ -207,12 +213,23 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
|
||||
log.Warn("sort task only support int64 and varchar pk field")
|
||||
}
|
||||
|
||||
rr, err := storage.NewBinlogRecordReader(ctx, t.insertLogs, t.plan.Schema,
|
||||
var rr storage.RecordReader
|
||||
// use manifest reader if manifest presents
|
||||
if t.manifest != "" {
|
||||
rr, err = storage.NewManifestRecordReader(ctx, t.manifest, t.plan.Schema,
|
||||
storage.WithVersion(t.segmentStorageVersion),
|
||||
storage.WithDownloader(t.binlogIO.Download),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
storage.WithCollectionID(t.collectionID),
|
||||
)
|
||||
} else {
|
||||
rr, err = storage.NewBinlogRecordReader(ctx, t.insertLogs, t.plan.Schema,
|
||||
storage.WithVersion(t.segmentStorageVersion),
|
||||
storage.WithDownloader(t.binlogIO.Download),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
storage.WithCollectionID(t.collectionID),
|
||||
)
|
||||
}
|
||||
if err != nil {
|
||||
log.Warn("error creating insert binlog reader", zap.Error(err))
|
||||
return nil, err
|
||||
@ -228,7 +245,7 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
|
||||
return nil, err
|
||||
}
|
||||
|
||||
binlogs, stats, bm25stats := srw.GetLogs()
|
||||
binlogs, stats, bm25stats, manifest := srw.GetLogs()
|
||||
insertLogs := storage.SortFieldBinlogs(binlogs)
|
||||
if err := binlog.CompressFieldBinlogs(insertLogs); err != nil {
|
||||
return nil, err
|
||||
@ -264,6 +281,7 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
|
||||
Channel: t.GetChannelName(),
|
||||
IsSorted: true,
|
||||
StorageVersion: t.storageVersion,
|
||||
Manifest: manifest,
|
||||
},
|
||||
}
|
||||
planResult := &datapb.CompactionPlanResult{
|
||||
|
||||
@ -261,7 +261,7 @@ func (st *statsTask) sort(ctx context.Context) ([]*datapb.FieldBinlog, error) {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
binlogs, stats, bm25stats := srw.GetLogs()
|
||||
binlogs, stats, bm25stats, _ := srw.GetLogs()
|
||||
insertLogs := storage.SortFieldBinlogs(binlogs)
|
||||
if err := binlog.CompressFieldBinlogs(insertLogs); err != nil {
|
||||
return nil, err
|
||||
|
||||
@ -20,6 +20,7 @@ import (
|
||||
"context"
|
||||
"encoding/base64"
|
||||
"math"
|
||||
"path"
|
||||
|
||||
"github.com/apache/arrow/go/v17/arrow/array"
|
||||
"github.com/apache/arrow/go/v17/arrow/memory"
|
||||
@ -77,6 +78,7 @@ func (bw *BulkPackWriterV2) Write(ctx context.Context, pack *SyncPack) (
|
||||
deltas *datapb.FieldBinlog,
|
||||
stats map[int64]*datapb.FieldBinlog,
|
||||
bm25Stats map[int64]*datapb.FieldBinlog,
|
||||
manifest string,
|
||||
size int64,
|
||||
err error,
|
||||
) {
|
||||
@ -86,7 +88,7 @@ func (bw *BulkPackWriterV2) Write(ctx context.Context, pack *SyncPack) (
|
||||
return
|
||||
}
|
||||
|
||||
if inserts, err = bw.writeInserts(ctx, pack); err != nil {
|
||||
if inserts, manifest, err = bw.writeInserts(ctx, pack); err != nil {
|
||||
log.Error("failed to write insert data", zap.Error(err))
|
||||
return
|
||||
}
|
||||
@ -125,24 +127,20 @@ func (bw *BulkPackWriterV2) getBucketName() string {
|
||||
return paramtable.Get().ServiceParam.MinioCfg.BucketName.GetValue()
|
||||
}
|
||||
|
||||
func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (map[int64]*datapb.FieldBinlog, error) {
|
||||
func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (map[int64]*datapb.FieldBinlog, string, error) {
|
||||
if len(pack.insertData) == 0 {
|
||||
return make(map[int64]*datapb.FieldBinlog), nil
|
||||
return make(map[int64]*datapb.FieldBinlog), "", nil
|
||||
}
|
||||
|
||||
columnGroups := bw.columnGroups
|
||||
|
||||
rec, err := bw.serializeBinlog(ctx, pack)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, "", err
|
||||
}
|
||||
|
||||
logs := make(map[int64]*datapb.FieldBinlog)
|
||||
paths := make([]string, 0)
|
||||
for _, columnGroup := range columnGroups {
|
||||
path := metautil.BuildInsertLogPath(bw.getRootPath(), pack.collectionID, pack.partitionID, pack.segmentID, columnGroup.GroupID, bw.nextID())
|
||||
paths = append(paths, path)
|
||||
}
|
||||
|
||||
tsArray := rec.Column(common.TimeStampField).(*array.Int64)
|
||||
rows := rec.Len()
|
||||
var tsFrom uint64 = math.MaxUint64
|
||||
@ -175,16 +173,24 @@ func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (m
|
||||
}
|
||||
}
|
||||
|
||||
w, err := storage.NewPackedRecordWriter(bucketName, paths, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
doWrite := func(w storage.RecordWriter) error {
|
||||
if err = w.Write(rec); err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
// close first to get compressed size
|
||||
if err = w.Close(); err != nil {
|
||||
return nil, err
|
||||
// close first the get stats & output
|
||||
return w.Close()
|
||||
}
|
||||
|
||||
var manifestPath string
|
||||
if paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool() {
|
||||
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID)
|
||||
basePath := path.Join(bw.getRootPath(), common.SegmentInsertLogPath, k)
|
||||
w, err := storage.NewPackedRecordManifestWriter(bucketName, basePath, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
|
||||
if err != nil {
|
||||
return nil, "", err
|
||||
}
|
||||
if err = doWrite(w); err != nil {
|
||||
return nil, "", err
|
||||
}
|
||||
for _, columnGroup := range columnGroups {
|
||||
columnGroupID := columnGroup.GroupID
|
||||
@ -203,14 +209,48 @@ func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (m
|
||||
},
|
||||
}
|
||||
}
|
||||
return logs, nil
|
||||
manifestPath = w.GetWrittenManifest()
|
||||
} else {
|
||||
paths := make([]string, 0)
|
||||
for _, columnGroup := range columnGroups {
|
||||
path := metautil.BuildInsertLogPath(bw.getRootPath(), pack.collectionID, pack.partitionID, pack.segmentID, columnGroup.GroupID, bw.nextID())
|
||||
paths = append(paths, path)
|
||||
}
|
||||
w, err := storage.NewPackedRecordWriter(bucketName, paths, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
|
||||
if err != nil {
|
||||
return nil, "", err
|
||||
}
|
||||
if err = doWrite(w); err != nil {
|
||||
return nil, "", err
|
||||
}
|
||||
// workaround to store row num
|
||||
for _, columnGroup := range columnGroups {
|
||||
columnGroupID := columnGroup.GroupID
|
||||
logs[columnGroupID] = &datapb.FieldBinlog{
|
||||
FieldID: columnGroupID,
|
||||
ChildFields: columnGroup.Fields,
|
||||
Binlogs: []*datapb.Binlog{
|
||||
{
|
||||
LogSize: int64(w.GetColumnGroupWrittenCompressed(columnGroup.GroupID)),
|
||||
MemorySize: int64(w.GetColumnGroupWrittenUncompressed(columnGroup.GroupID)),
|
||||
LogPath: w.GetWrittenPaths(columnGroupID),
|
||||
EntriesNum: w.GetWrittenRowNum(),
|
||||
TimestampFrom: tsFrom,
|
||||
TimestampTo: tsTo,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return logs, manifestPath, nil
|
||||
}
|
||||
|
||||
func (bw *BulkPackWriterV2) serializeBinlog(_ context.Context, pack *SyncPack) (storage.Record, error) {
|
||||
if len(pack.insertData) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
arrowSchema, err := storage.ConvertToArrowSchema(bw.schema)
|
||||
arrowSchema, err := storage.ConvertToArrowSchema(bw.schema, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
@ -145,7 +145,7 @@ func (s *PackWriterV2Suite) TestPackWriterV2_Write() {
|
||||
|
||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
||||
|
||||
gotInserts, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
gotInserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
s.NoError(err)
|
||||
s.Equal(gotInserts[0].Binlogs[0].GetEntriesNum(), int64(rows))
|
||||
s.Equal(gotInserts[0].Binlogs[0].GetLogPath(), "/tmp/insert_log/123/456/789/0/1")
|
||||
@ -164,7 +164,7 @@ func (s *PackWriterV2Suite) TestWriteEmptyInsertData() {
|
||||
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName)
|
||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
||||
|
||||
_, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
s.NoError(err)
|
||||
}
|
||||
|
||||
@ -193,7 +193,7 @@ func (s *PackWriterV2Suite) TestNoPkField() {
|
||||
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
|
||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
||||
|
||||
_, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
s.Error(err)
|
||||
}
|
||||
|
||||
@ -210,7 +210,7 @@ func (s *PackWriterV2Suite) TestAllocIDExhausedError() {
|
||||
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema))
|
||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
||||
|
||||
_, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
s.Error(err)
|
||||
}
|
||||
|
||||
@ -231,7 +231,7 @@ func (s *PackWriterV2Suite) TestWriteInsertDataError() {
|
||||
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
|
||||
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
|
||||
|
||||
_, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
|
||||
s.Error(err)
|
||||
}
|
||||
|
||||
|
||||
@ -140,7 +140,7 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
|
||||
// New sync task means needs to flush data immediately, so do not need to buffer data in writer again.
|
||||
writer := NewBulkPackWriterV2(t.metacache, t.schema, t.chunkManager, t.allocator, 0,
|
||||
packed.DefaultMultiPartUploadSize, t.storageConfig, columnGroups, t.writeRetryOpts...)
|
||||
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.flushedSize, err = writer.Write(ctx, t.pack)
|
||||
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.manifestPath, t.flushedSize, err = writer.Write(ctx, t.pack)
|
||||
if err != nil {
|
||||
log.Warn("failed to write sync data with storage v2 format", zap.Error(err))
|
||||
return err
|
||||
|
||||
456
internal/storage/binlog_record_writer.go
Normal file
456
internal/storage/binlog_record_writer.go
Normal file
@ -0,0 +1,456 @@
|
||||
// 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 storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"path"
|
||||
|
||||
"github.com/apache/arrow/go/v17/arrow"
|
||||
"github.com/apache/arrow/go/v17/arrow/array"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/storagecommon"
|
||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
type BinlogRecordWriter interface {
|
||||
RecordWriter
|
||||
GetLogs() (
|
||||
fieldBinlogs map[FieldID]*datapb.FieldBinlog,
|
||||
statsLog *datapb.FieldBinlog,
|
||||
bm25StatsLog map[FieldID]*datapb.FieldBinlog,
|
||||
manifest string,
|
||||
)
|
||||
GetRowNum() int64
|
||||
FlushChunk() error
|
||||
GetBufferUncompressed() uint64
|
||||
Schema() *schemapb.CollectionSchema
|
||||
}
|
||||
|
||||
type packedBinlogRecordWriterBase struct {
|
||||
// attributes
|
||||
collectionID UniqueID
|
||||
partitionID UniqueID
|
||||
segmentID UniqueID
|
||||
schema *schemapb.CollectionSchema
|
||||
BlobsWriter ChunkedBlobsWriter
|
||||
allocator allocator.Interface
|
||||
maxRowNum int64
|
||||
arrowSchema *arrow.Schema
|
||||
bufferSize int64
|
||||
multiPartUploadSize int64
|
||||
columnGroups []storagecommon.ColumnGroup
|
||||
storageConfig *indexpb.StorageConfig
|
||||
storagePluginContext *indexcgopb.StoragePluginContext
|
||||
|
||||
pkCollector *PkStatsCollector
|
||||
bm25Collector *Bm25StatsCollector
|
||||
tsFrom typeutil.Timestamp
|
||||
tsTo typeutil.Timestamp
|
||||
rowNum int64
|
||||
writtenUncompressed uint64
|
||||
|
||||
// results
|
||||
fieldBinlogs map[FieldID]*datapb.FieldBinlog
|
||||
statsLog *datapb.FieldBinlog
|
||||
bm25StatsLog map[FieldID]*datapb.FieldBinlog
|
||||
manifest string
|
||||
}
|
||||
|
||||
func (pw *packedBinlogRecordWriterBase) getColumnStatsFromRecord(r Record, allFields []*schemapb.FieldSchema) map[int64]storagecommon.ColumnStats {
|
||||
result := make(map[int64]storagecommon.ColumnStats)
|
||||
for _, field := range allFields {
|
||||
if arr := r.Column(field.FieldID); arr != nil {
|
||||
result[field.FieldID] = storagecommon.ColumnStats{
|
||||
AvgSize: int64(arr.Data().SizeInBytes()) / int64(arr.Len()),
|
||||
}
|
||||
}
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
func (pw *packedBinlogRecordWriterBase) GetWrittenUncompressed() uint64 {
|
||||
return pw.writtenUncompressed
|
||||
}
|
||||
|
||||
func (pw *packedBinlogRecordWriterBase) writeStats() error {
|
||||
// Write PK stats
|
||||
pkStatsMap, err := pw.pkCollector.Digest(
|
||||
pw.collectionID,
|
||||
pw.partitionID,
|
||||
pw.segmentID,
|
||||
pw.storageConfig.GetRootPath(),
|
||||
pw.rowNum,
|
||||
pw.allocator,
|
||||
pw.BlobsWriter,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// Extract single PK stats from map
|
||||
for _, statsLog := range pkStatsMap {
|
||||
pw.statsLog = statsLog
|
||||
break
|
||||
}
|
||||
|
||||
// Write BM25 stats
|
||||
bm25StatsLog, err := pw.bm25Collector.Digest(
|
||||
pw.collectionID,
|
||||
pw.partitionID,
|
||||
pw.segmentID,
|
||||
pw.storageConfig.GetRootPath(),
|
||||
pw.rowNum,
|
||||
pw.allocator,
|
||||
pw.BlobsWriter,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
pw.bm25StatsLog = bm25StatsLog
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pw *packedBinlogRecordWriterBase) GetLogs() (
|
||||
fieldBinlogs map[FieldID]*datapb.FieldBinlog,
|
||||
statsLog *datapb.FieldBinlog,
|
||||
bm25StatsLog map[FieldID]*datapb.FieldBinlog,
|
||||
manifest string,
|
||||
) {
|
||||
return pw.fieldBinlogs, pw.statsLog, pw.bm25StatsLog, pw.manifest
|
||||
}
|
||||
|
||||
func (pw *packedBinlogRecordWriterBase) GetRowNum() int64 {
|
||||
return pw.rowNum
|
||||
}
|
||||
|
||||
func (pw *packedBinlogRecordWriterBase) FlushChunk() error {
|
||||
return nil // do nothing
|
||||
}
|
||||
|
||||
func (pw *packedBinlogRecordWriterBase) Schema() *schemapb.CollectionSchema {
|
||||
return pw.schema
|
||||
}
|
||||
|
||||
func (pw *packedBinlogRecordWriterBase) GetBufferUncompressed() uint64 {
|
||||
return uint64(pw.multiPartUploadSize)
|
||||
}
|
||||
|
||||
var _ BinlogRecordWriter = (*PackedBinlogRecordWriter)(nil)
|
||||
|
||||
type PackedBinlogRecordWriter struct {
|
||||
packedBinlogRecordWriterBase
|
||||
writer *packedRecordWriter
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) Write(r Record) error {
|
||||
if err := pw.initWriters(r); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Track timestamps
|
||||
tsArray := r.Column(common.TimeStampField).(*array.Int64)
|
||||
rows := r.Len()
|
||||
for i := 0; i < rows; i++ {
|
||||
ts := typeutil.Timestamp(tsArray.Value(i))
|
||||
if ts < pw.tsFrom {
|
||||
pw.tsFrom = ts
|
||||
}
|
||||
if ts > pw.tsTo {
|
||||
pw.tsTo = ts
|
||||
}
|
||||
}
|
||||
|
||||
// Collect statistics
|
||||
if err := pw.pkCollector.Collect(r); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := pw.bm25Collector.Collect(r); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err := pw.writer.Write(r)
|
||||
if err != nil {
|
||||
return merr.WrapErrServiceInternal(fmt.Sprintf("write record batch error: %s", err.Error()))
|
||||
}
|
||||
pw.writtenUncompressed = pw.writer.GetWrittenUncompressed()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) initWriters(r Record) error {
|
||||
if pw.writer == nil {
|
||||
if len(pw.columnGroups) == 0 {
|
||||
allFields := typeutil.GetAllFieldSchemas(pw.schema)
|
||||
pw.columnGroups = storagecommon.SplitColumns(allFields, pw.getColumnStatsFromRecord(r, allFields), storagecommon.DefaultPolicies()...)
|
||||
}
|
||||
logIdStart, _, err := pw.allocator.Alloc(uint32(len(pw.columnGroups)))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
paths := []string{}
|
||||
for _, columnGroup := range pw.columnGroups {
|
||||
path := metautil.BuildInsertLogPath(pw.storageConfig.GetRootPath(), pw.collectionID, pw.partitionID, pw.segmentID, columnGroup.GroupID, logIdStart)
|
||||
paths = append(paths, path)
|
||||
logIdStart++
|
||||
}
|
||||
pw.writer, err = NewPackedRecordWriter(pw.storageConfig.GetBucketName(), paths, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext)
|
||||
if err != nil {
|
||||
return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) finalizeBinlogs() {
|
||||
if pw.writer == nil {
|
||||
return
|
||||
}
|
||||
pw.rowNum = pw.writer.GetWrittenRowNum()
|
||||
if pw.fieldBinlogs == nil {
|
||||
pw.fieldBinlogs = make(map[FieldID]*datapb.FieldBinlog, len(pw.columnGroups))
|
||||
}
|
||||
for _, columnGroup := range pw.columnGroups {
|
||||
columnGroupID := columnGroup.GroupID
|
||||
if _, exists := pw.fieldBinlogs[columnGroupID]; !exists {
|
||||
pw.fieldBinlogs[columnGroupID] = &datapb.FieldBinlog{
|
||||
FieldID: columnGroupID,
|
||||
ChildFields: columnGroup.Fields,
|
||||
}
|
||||
}
|
||||
pw.fieldBinlogs[columnGroupID].Binlogs = append(pw.fieldBinlogs[columnGroupID].Binlogs, &datapb.Binlog{
|
||||
LogSize: int64(pw.writer.GetColumnGroupWrittenCompressed(columnGroupID)),
|
||||
MemorySize: int64(pw.writer.GetColumnGroupWrittenUncompressed(columnGroupID)),
|
||||
LogPath: pw.writer.GetWrittenPaths(columnGroupID),
|
||||
EntriesNum: pw.writer.GetWrittenRowNum(),
|
||||
TimestampFrom: pw.tsFrom,
|
||||
TimestampTo: pw.tsTo,
|
||||
})
|
||||
}
|
||||
pw.manifest = pw.writer.GetWrittenManifest()
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) Close() error {
|
||||
if pw.writer != nil {
|
||||
if err := pw.writer.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
pw.finalizeBinlogs()
|
||||
if err := pw.writeStats(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func newPackedBinlogRecordWriter(collectionID, partitionID, segmentID UniqueID, schema *schemapb.CollectionSchema,
|
||||
blobsWriter ChunkedBlobsWriter, allocator allocator.Interface, maxRowNum int64, bufferSize, multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*PackedBinlogRecordWriter, error) {
|
||||
arrowSchema, err := ConvertToArrowSchema(schema, true)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
|
||||
}
|
||||
|
||||
writer := &PackedBinlogRecordWriter{
|
||||
packedBinlogRecordWriterBase: packedBinlogRecordWriterBase{
|
||||
collectionID: collectionID,
|
||||
partitionID: partitionID,
|
||||
segmentID: segmentID,
|
||||
schema: schema,
|
||||
arrowSchema: arrowSchema,
|
||||
BlobsWriter: blobsWriter,
|
||||
allocator: allocator,
|
||||
maxRowNum: maxRowNum,
|
||||
bufferSize: bufferSize,
|
||||
multiPartUploadSize: multiPartUploadSize,
|
||||
columnGroups: columnGroups,
|
||||
storageConfig: storageConfig,
|
||||
storagePluginContext: storagePluginContext,
|
||||
tsFrom: typeutil.MaxTimestamp,
|
||||
tsTo: 0,
|
||||
},
|
||||
}
|
||||
|
||||
// Create stats collectors
|
||||
writer.pkCollector, err = NewPkStatsCollector(
|
||||
collectionID,
|
||||
schema,
|
||||
maxRowNum,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
writer.bm25Collector = NewBm25StatsCollector(schema)
|
||||
|
||||
return writer, nil
|
||||
}
|
||||
|
||||
var _ BinlogRecordWriter = (*PackedManifestRecordWriter)(nil)
|
||||
|
||||
type PackedManifestRecordWriter struct {
|
||||
packedBinlogRecordWriterBase
|
||||
// writer and stats generated at runtime
|
||||
writer *packedRecordManifestWriter
|
||||
}
|
||||
|
||||
func (pw *PackedManifestRecordWriter) Write(r Record) error {
|
||||
if err := pw.initWriters(r); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Track timestamps
|
||||
tsArray := r.Column(common.TimeStampField).(*array.Int64)
|
||||
rows := r.Len()
|
||||
for i := 0; i < rows; i++ {
|
||||
ts := typeutil.Timestamp(tsArray.Value(i))
|
||||
if ts < pw.tsFrom {
|
||||
pw.tsFrom = ts
|
||||
}
|
||||
if ts > pw.tsTo {
|
||||
pw.tsTo = ts
|
||||
}
|
||||
}
|
||||
|
||||
// Collect statistics
|
||||
if err := pw.pkCollector.Collect(r); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := pw.bm25Collector.Collect(r); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err := pw.writer.Write(r)
|
||||
if err != nil {
|
||||
return merr.WrapErrServiceInternal(fmt.Sprintf("write record batch error: %s", err.Error()))
|
||||
}
|
||||
pw.writtenUncompressed = pw.writer.GetWrittenUncompressed()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pw *PackedManifestRecordWriter) initWriters(r Record) error {
|
||||
if pw.writer == nil {
|
||||
if len(pw.columnGroups) == 0 {
|
||||
allFields := typeutil.GetAllFieldSchemas(pw.schema)
|
||||
pw.columnGroups = storagecommon.SplitColumns(allFields, pw.getColumnStatsFromRecord(r, allFields), storagecommon.DefaultPolicies()...)
|
||||
}
|
||||
|
||||
var err error
|
||||
k := metautil.JoinIDPath(pw.collectionID, pw.partitionID, pw.segmentID)
|
||||
basePath := path.Join(pw.storageConfig.GetRootPath(), common.SegmentInsertLogPath, k)
|
||||
pw.writer, err = NewPackedRecordManifestWriter(pw.storageConfig.GetBucketName(), basePath, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext)
|
||||
if err != nil {
|
||||
return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pw *PackedManifestRecordWriter) finalizeBinlogs() {
|
||||
if pw.writer == nil {
|
||||
return
|
||||
}
|
||||
pw.rowNum = pw.writer.GetWrittenRowNum()
|
||||
if pw.fieldBinlogs == nil {
|
||||
pw.fieldBinlogs = make(map[FieldID]*datapb.FieldBinlog, len(pw.columnGroups))
|
||||
}
|
||||
for _, columnGroup := range pw.columnGroups {
|
||||
columnGroupID := columnGroup.GroupID
|
||||
if _, exists := pw.fieldBinlogs[columnGroupID]; !exists {
|
||||
pw.fieldBinlogs[columnGroupID] = &datapb.FieldBinlog{
|
||||
FieldID: columnGroupID,
|
||||
ChildFields: columnGroup.Fields,
|
||||
}
|
||||
}
|
||||
pw.fieldBinlogs[columnGroupID].Binlogs = append(pw.fieldBinlogs[columnGroupID].Binlogs, &datapb.Binlog{
|
||||
LogSize: int64(pw.writer.GetColumnGroupWrittenCompressed(columnGroupID)),
|
||||
MemorySize: int64(pw.writer.GetColumnGroupWrittenUncompressed(columnGroupID)),
|
||||
LogPath: pw.writer.GetWrittenPaths(columnGroupID),
|
||||
EntriesNum: pw.writer.GetWrittenRowNum(),
|
||||
TimestampFrom: pw.tsFrom,
|
||||
TimestampTo: pw.tsTo,
|
||||
})
|
||||
}
|
||||
pw.manifest = pw.writer.GetWrittenManifest()
|
||||
}
|
||||
|
||||
func (pw *PackedManifestRecordWriter) Close() error {
|
||||
if pw.writer != nil {
|
||||
if err := pw.writer.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
pw.finalizeBinlogs()
|
||||
if err := pw.writeStats(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func newPackedManifestRecordWriter(collectionID, partitionID, segmentID UniqueID, schema *schemapb.CollectionSchema,
|
||||
blobsWriter ChunkedBlobsWriter, allocator allocator.Interface, maxRowNum int64, bufferSize, multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*PackedManifestRecordWriter, error) {
|
||||
arrowSchema, err := ConvertToArrowSchema(schema, true)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
|
||||
}
|
||||
|
||||
writer := &PackedManifestRecordWriter{
|
||||
packedBinlogRecordWriterBase: packedBinlogRecordWriterBase{
|
||||
collectionID: collectionID,
|
||||
partitionID: partitionID,
|
||||
segmentID: segmentID,
|
||||
schema: schema,
|
||||
arrowSchema: arrowSchema,
|
||||
BlobsWriter: blobsWriter,
|
||||
allocator: allocator,
|
||||
maxRowNum: maxRowNum,
|
||||
bufferSize: bufferSize,
|
||||
multiPartUploadSize: multiPartUploadSize,
|
||||
columnGroups: columnGroups,
|
||||
storageConfig: storageConfig,
|
||||
storagePluginContext: storagePluginContext,
|
||||
tsFrom: typeutil.MaxTimestamp,
|
||||
tsTo: 0,
|
||||
},
|
||||
}
|
||||
|
||||
// Create stats collectors
|
||||
writer.pkCollector, err = NewPkStatsCollector(
|
||||
collectionID,
|
||||
schema,
|
||||
maxRowNum,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
writer.bm25Collector = NewBm25StatsCollector(schema)
|
||||
|
||||
return writer, nil
|
||||
}
|
||||
334
internal/storage/record_reader.go
Normal file
334
internal/storage/record_reader.go
Normal file
@ -0,0 +1,334 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
|
||||
"github.com/apache/arrow/go/v17/arrow"
|
||||
"github.com/apache/arrow/go/v17/arrow/array"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storagev2/packed"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
type RecordReader interface {
|
||||
Next() (Record, error)
|
||||
Close() error
|
||||
}
|
||||
|
||||
type packedRecordReader struct {
|
||||
reader *packed.PackedReader
|
||||
field2Col map[FieldID]int
|
||||
}
|
||||
|
||||
var _ RecordReader = (*packedRecordReader)(nil)
|
||||
|
||||
func (pr *packedRecordReader) Next() (Record, error) {
|
||||
rec, err := pr.reader.ReadNext()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewSimpleArrowRecord(rec, pr.field2Col), nil
|
||||
}
|
||||
|
||||
func (pr *packedRecordReader) Close() error {
|
||||
if pr.reader != nil {
|
||||
return pr.reader.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func newPackedRecordReader(
|
||||
paths []string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*packedRecordReader, error) {
|
||||
arrowSchema, err := ConvertToArrowSchema(schema, true)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
|
||||
}
|
||||
field2Col := make(map[FieldID]int)
|
||||
allFields := typeutil.GetAllFieldSchemas(schema)
|
||||
for i, field := range allFields {
|
||||
field2Col[field.FieldID] = i
|
||||
}
|
||||
reader, err := packed.NewPackedReader(paths, arrowSchema, bufferSize, storageConfig, storagePluginContext)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &packedRecordReader{
|
||||
reader: reader,
|
||||
field2Col: field2Col,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func NewRecordReaderFromManifest(manifest string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (RecordReader, error) {
|
||||
return NewManifestReader(manifest, schema, bufferSize, storageConfig, storagePluginContext)
|
||||
}
|
||||
|
||||
var _ RecordReader = (*IterativeRecordReader)(nil)
|
||||
|
||||
type IterativeRecordReader struct {
|
||||
cur RecordReader
|
||||
iterate func() (RecordReader, error)
|
||||
}
|
||||
|
||||
// Close implements RecordReader.
|
||||
func (ir *IterativeRecordReader) Close() error {
|
||||
if ir.cur != nil {
|
||||
return ir.cur.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ir *IterativeRecordReader) Next() (Record, error) {
|
||||
if ir.cur == nil {
|
||||
r, err := ir.iterate()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
ir.cur = r
|
||||
}
|
||||
rec, err := ir.cur.Next()
|
||||
if err == io.EOF {
|
||||
closeErr := ir.cur.Close()
|
||||
if closeErr != nil {
|
||||
return nil, closeErr
|
||||
}
|
||||
ir.cur, err = ir.iterate()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
rec, err = ir.cur.Next()
|
||||
}
|
||||
return rec, err
|
||||
}
|
||||
|
||||
func newIterativePackedRecordReader(
|
||||
paths [][]string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) *IterativeRecordReader {
|
||||
chunk := 0
|
||||
return &IterativeRecordReader{
|
||||
iterate: func() (RecordReader, error) {
|
||||
if chunk >= len(paths) {
|
||||
return nil, io.EOF
|
||||
}
|
||||
currentPaths := paths[chunk]
|
||||
chunk++
|
||||
return newPackedRecordReader(currentPaths, schema, bufferSize, storageConfig, storagePluginContext)
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
type ManifestReader struct {
|
||||
fieldBinlogs []*datapb.FieldBinlog
|
||||
manifest string
|
||||
reader *packed.FFIPackedReader
|
||||
|
||||
bufferSize int64
|
||||
arrowSchema *arrow.Schema
|
||||
schema *schemapb.CollectionSchema
|
||||
schemaHelper *typeutil.SchemaHelper
|
||||
field2Col map[FieldID]int
|
||||
storageConfig *indexpb.StorageConfig
|
||||
storagePluginContext *indexcgopb.StoragePluginContext
|
||||
|
||||
neededColumns []string
|
||||
}
|
||||
|
||||
// NewManifestReaderFromBinlogs creates a ManifestReader from binlogs
|
||||
func NewManifestReaderFromBinlogs(fieldBinlogs []*datapb.FieldBinlog,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*ManifestReader, error) {
|
||||
arrowSchema, err := ConvertToArrowSchema(schema, false)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
|
||||
}
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
field2Col := make(map[FieldID]int)
|
||||
allFields := typeutil.GetAllFieldSchemas(schema)
|
||||
neededColumns := make([]string, 0, len(allFields))
|
||||
for i, field := range allFields {
|
||||
field2Col[field.FieldID] = i
|
||||
neededColumns = append(neededColumns, field.Name)
|
||||
}
|
||||
prr := &ManifestReader{
|
||||
fieldBinlogs: fieldBinlogs,
|
||||
bufferSize: bufferSize,
|
||||
arrowSchema: arrowSchema,
|
||||
schema: schema,
|
||||
schemaHelper: schemaHelper,
|
||||
field2Col: field2Col,
|
||||
storageConfig: storageConfig,
|
||||
storagePluginContext: storagePluginContext,
|
||||
|
||||
neededColumns: neededColumns,
|
||||
}
|
||||
|
||||
err = prr.init()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return prr, nil
|
||||
}
|
||||
|
||||
func NewManifestReader(manifest string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*ManifestReader, error) {
|
||||
arrowSchema, err := ConvertToArrowSchema(schema, true)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
|
||||
}
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
field2Col := make(map[FieldID]int)
|
||||
allFields := typeutil.GetAllFieldSchemas(schema)
|
||||
neededColumns := make([]string, 0, len(allFields))
|
||||
for i, field := range allFields {
|
||||
field2Col[field.FieldID] = i
|
||||
neededColumns = append(neededColumns, field.Name)
|
||||
}
|
||||
prr := &ManifestReader{
|
||||
manifest: manifest,
|
||||
bufferSize: bufferSize,
|
||||
arrowSchema: arrowSchema,
|
||||
schema: schema,
|
||||
schemaHelper: schemaHelper,
|
||||
field2Col: field2Col,
|
||||
storageConfig: storageConfig,
|
||||
storagePluginContext: storagePluginContext,
|
||||
|
||||
neededColumns: neededColumns,
|
||||
}
|
||||
|
||||
err = prr.init()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return prr, nil
|
||||
}
|
||||
|
||||
func (mr *ManifestReader) init() error {
|
||||
// TODO add needed column option
|
||||
manifest, err := packed.GetManifest(mr.manifest, mr.storageConfig)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
reader, err := packed.NewFFIPackedReader(manifest, mr.arrowSchema, mr.neededColumns, mr.bufferSize, mr.storageConfig, mr.storagePluginContext)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
mr.reader = reader
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mr ManifestReader) Next() (Record, error) {
|
||||
rec, err := mr.reader.ReadNext()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewSimpleArrowRecord(rec, mr.field2Col), nil
|
||||
}
|
||||
|
||||
func (mr ManifestReader) Close() error {
|
||||
if mr.reader != nil {
|
||||
return mr.reader.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ChunkedBlobsReader returns a chunk composed of blobs, or io.EOF if no more data
|
||||
type ChunkedBlobsReader func() ([]*Blob, error)
|
||||
|
||||
type CompositeBinlogRecordReader struct {
|
||||
fields map[FieldID]*schemapb.FieldSchema
|
||||
index map[FieldID]int16
|
||||
brs []*BinlogReader
|
||||
rrs []array.RecordReader
|
||||
}
|
||||
|
||||
var _ RecordReader = (*CompositeBinlogRecordReader)(nil)
|
||||
|
||||
func (crr *CompositeBinlogRecordReader) Next() (Record, error) {
|
||||
recs := make([]arrow.Array, len(crr.fields))
|
||||
nonExistingFields := make([]*schemapb.FieldSchema, 0)
|
||||
nRows := 0
|
||||
for _, f := range crr.fields {
|
||||
idx := crr.index[f.FieldID]
|
||||
if crr.rrs[idx] != nil {
|
||||
if ok := crr.rrs[idx].Next(); !ok {
|
||||
return nil, io.EOF
|
||||
}
|
||||
r := crr.rrs[idx].Record()
|
||||
recs[idx] = r.Column(0)
|
||||
if nRows == 0 {
|
||||
nRows = int(r.NumRows())
|
||||
}
|
||||
if nRows != int(r.NumRows()) {
|
||||
return nil, merr.WrapErrServiceInternal(fmt.Sprintf("number of rows mismatch for field %d", f.FieldID))
|
||||
}
|
||||
} else {
|
||||
nonExistingFields = append(nonExistingFields, f)
|
||||
}
|
||||
}
|
||||
for _, f := range nonExistingFields {
|
||||
// If the field is not in the current batch, fill with null array
|
||||
arr, err := GenerateEmptyArrayFromSchema(f, nRows)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
recs[crr.index[f.FieldID]] = arr
|
||||
}
|
||||
return &compositeRecord{
|
||||
index: crr.index,
|
||||
recs: recs,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (crr *CompositeBinlogRecordReader) Close() error {
|
||||
if crr.brs != nil {
|
||||
for _, er := range crr.brs {
|
||||
if er != nil {
|
||||
er.Close()
|
||||
}
|
||||
}
|
||||
}
|
||||
if crr.rrs != nil {
|
||||
for _, rr := range crr.rrs {
|
||||
if rr != nil {
|
||||
rr.Release()
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
370
internal/storage/record_writer.go
Normal file
370
internal/storage/record_writer.go
Normal file
@ -0,0 +1,370 @@
|
||||
// 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 storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"path"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/apache/arrow/go/v17/arrow"
|
||||
"github.com/apache/arrow/go/v17/arrow/array"
|
||||
"github.com/samber/lo"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storagecommon"
|
||||
"github.com/milvus-io/milvus/internal/storagev2/packed"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
var _ RecordWriter = (*packedRecordWriter)(nil)
|
||||
|
||||
type packedRecordWriter struct {
|
||||
writer *packed.PackedWriter
|
||||
bufferSize int64
|
||||
columnGroups []storagecommon.ColumnGroup
|
||||
bucketName string
|
||||
pathsMap map[typeutil.UniqueID]string
|
||||
schema *schemapb.CollectionSchema
|
||||
arrowSchema *arrow.Schema
|
||||
rowNum int64
|
||||
writtenUncompressed uint64
|
||||
columnGroupUncompressed map[typeutil.UniqueID]uint64
|
||||
columnGroupCompressed map[typeutil.UniqueID]uint64
|
||||
outputManifest string
|
||||
storageConfig *indexpb.StorageConfig
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) Write(r Record) error {
|
||||
var rec arrow.Record
|
||||
sar, ok := r.(*simpleArrowRecord)
|
||||
if !ok {
|
||||
// Get all fields including struct sub-fields
|
||||
allFields := typeutil.GetAllFieldSchemas(pw.schema)
|
||||
arrays := make([]arrow.Array, len(allFields))
|
||||
for i, field := range allFields {
|
||||
arrays[i] = r.Column(field.FieldID)
|
||||
}
|
||||
rec = array.NewRecord(pw.arrowSchema, arrays, int64(r.Len()))
|
||||
} else {
|
||||
rec = sar.r
|
||||
}
|
||||
pw.rowNum += int64(r.Len())
|
||||
for col, arr := range rec.Columns() {
|
||||
// size := arr.Data().SizeInBytes()
|
||||
size := calculateActualDataSize(arr)
|
||||
pw.writtenUncompressed += size
|
||||
for _, columnGroup := range pw.columnGroups {
|
||||
if lo.Contains(columnGroup.Columns, col) {
|
||||
pw.columnGroupUncompressed[columnGroup.GroupID] += size
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
defer rec.Release()
|
||||
return pw.writer.WriteRecordBatch(rec)
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetWrittenUncompressed() uint64 {
|
||||
return pw.writtenUncompressed
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetColumnGroupWrittenUncompressed(columnGroup typeutil.UniqueID) uint64 {
|
||||
if size, ok := pw.columnGroupUncompressed[columnGroup]; ok {
|
||||
return size
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetColumnGroupWrittenCompressed(columnGroup typeutil.UniqueID) uint64 {
|
||||
if size, ok := pw.columnGroupCompressed[columnGroup]; ok {
|
||||
return size
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetWrittenPaths(columnGroup typeutil.UniqueID) string {
|
||||
if path, ok := pw.pathsMap[columnGroup]; ok {
|
||||
return path
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetWrittenManifest() string {
|
||||
return pw.outputManifest
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetWrittenRowNum() int64 {
|
||||
return pw.rowNum
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) Close() error {
|
||||
if pw.writer != nil {
|
||||
err := pw.writer.Close()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for id, fpath := range pw.pathsMap {
|
||||
truePath := path.Join(pw.bucketName, fpath)
|
||||
size, err := packed.GetFileSize(truePath, pw.storageConfig)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
pw.columnGroupCompressed[id] = uint64(size)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func NewPackedRecordWriter(
|
||||
bucketName string,
|
||||
paths []string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
multiPartUploadSize int64,
|
||||
columnGroups []storagecommon.ColumnGroup,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*packedRecordWriter, error) {
|
||||
// Validate PK field exists before proceeding
|
||||
_, err := typeutil.GetPrimaryFieldSchema(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
arrowSchema, err := ConvertToArrowSchema(schema, false)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrServiceInternal(
|
||||
fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error()))
|
||||
}
|
||||
// if storage config is not passed, use common config
|
||||
storageType := paramtable.Get().CommonCfg.StorageType.GetValue()
|
||||
if storageConfig != nil {
|
||||
storageType = storageConfig.GetStorageType()
|
||||
}
|
||||
// compose true path before create packed writer here
|
||||
// and returned writtenPaths shall remain untouched
|
||||
truePaths := lo.Map(paths, func(p string, _ int) string {
|
||||
if storageType == "local" {
|
||||
return p
|
||||
}
|
||||
return path.Join(bucketName, p)
|
||||
})
|
||||
writer, err := packed.NewPackedWriter(truePaths, arrowSchema, bufferSize, multiPartUploadSize, columnGroups, storageConfig, storagePluginContext)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrServiceInternal(
|
||||
fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||
}
|
||||
columnGroupUncompressed := make(map[typeutil.UniqueID]uint64)
|
||||
columnGroupCompressed := make(map[typeutil.UniqueID]uint64)
|
||||
pathsMap := make(map[typeutil.UniqueID]string)
|
||||
if len(paths) != len(columnGroups) {
|
||||
return nil, merr.WrapErrParameterInvalid(len(paths), len(columnGroups),
|
||||
"paths length is not equal to column groups length for packed record writer")
|
||||
}
|
||||
for i, columnGroup := range columnGroups {
|
||||
columnGroupUncompressed[columnGroup.GroupID] = 0
|
||||
columnGroupCompressed[columnGroup.GroupID] = 0
|
||||
pathsMap[columnGroup.GroupID] = paths[i]
|
||||
}
|
||||
return &packedRecordWriter{
|
||||
writer: writer,
|
||||
schema: schema,
|
||||
arrowSchema: arrowSchema,
|
||||
bufferSize: bufferSize,
|
||||
bucketName: bucketName,
|
||||
pathsMap: pathsMap,
|
||||
columnGroups: columnGroups,
|
||||
columnGroupUncompressed: columnGroupUncompressed,
|
||||
columnGroupCompressed: columnGroupCompressed,
|
||||
storageConfig: storageConfig,
|
||||
}, nil
|
||||
}
|
||||
|
||||
type packedRecordManifestWriter struct {
|
||||
writer *packed.FFIPackedWriter
|
||||
bufferSize int64
|
||||
columnGroups []storagecommon.ColumnGroup
|
||||
bucketName string
|
||||
pathsMap map[typeutil.UniqueID]string
|
||||
schema *schemapb.CollectionSchema
|
||||
arrowSchema *arrow.Schema
|
||||
rowNum int64
|
||||
writtenUncompressed uint64
|
||||
columnGroupUncompressed map[typeutil.UniqueID]uint64
|
||||
columnGroupCompressed map[typeutil.UniqueID]uint64
|
||||
outputManifest string
|
||||
storageConfig *indexpb.StorageConfig
|
||||
}
|
||||
|
||||
func (pw *packedRecordManifestWriter) Write(r Record) error {
|
||||
var rec arrow.Record
|
||||
sar, ok := r.(*simpleArrowRecord)
|
||||
if !ok {
|
||||
// Get all fields including struct sub-fields
|
||||
allFields := typeutil.GetAllFieldSchemas(pw.schema)
|
||||
arrays := make([]arrow.Array, len(allFields))
|
||||
for i, field := range allFields {
|
||||
arrays[i] = r.Column(field.FieldID)
|
||||
}
|
||||
rec = array.NewRecord(pw.arrowSchema, arrays, int64(r.Len()))
|
||||
} else {
|
||||
rec = sar.r
|
||||
}
|
||||
pw.rowNum += int64(r.Len())
|
||||
for col, arr := range rec.Columns() {
|
||||
// size := arr.Data().SizeInBytes()
|
||||
size := calculateActualDataSize(arr)
|
||||
pw.writtenUncompressed += size
|
||||
for _, columnGroup := range pw.columnGroups {
|
||||
if lo.Contains(columnGroup.Columns, col) {
|
||||
pw.columnGroupUncompressed[columnGroup.GroupID] += size
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
defer rec.Release()
|
||||
return pw.writer.WriteRecordBatch(rec)
|
||||
}
|
||||
|
||||
func (pw *packedRecordManifestWriter) GetWrittenUncompressed() uint64 {
|
||||
return pw.writtenUncompressed
|
||||
}
|
||||
|
||||
func (pw *packedRecordManifestWriter) GetColumnGroupWrittenUncompressed(columnGroup typeutil.UniqueID) uint64 {
|
||||
if size, ok := pw.columnGroupUncompressed[columnGroup]; ok {
|
||||
return size
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (pw *packedRecordManifestWriter) GetColumnGroupWrittenCompressed(columnGroup typeutil.UniqueID) uint64 {
|
||||
if size, ok := pw.columnGroupCompressed[columnGroup]; ok {
|
||||
return size
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (pw *packedRecordManifestWriter) GetWrittenPaths(columnGroup typeutil.UniqueID) string {
|
||||
if path, ok := pw.pathsMap[columnGroup]; ok {
|
||||
return path
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (pw *packedRecordManifestWriter) GetWrittenManifest() string {
|
||||
return pw.outputManifest
|
||||
}
|
||||
|
||||
func (pw *packedRecordManifestWriter) GetWrittenRowNum() int64 {
|
||||
return pw.rowNum
|
||||
}
|
||||
|
||||
func (pw *packedRecordManifestWriter) Close() error {
|
||||
if pw.writer != nil {
|
||||
manifest, err := pw.writer.Close()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
pw.outputManifest = manifest
|
||||
for id := range pw.pathsMap {
|
||||
pw.columnGroupCompressed[id] = uint64(0)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func NewPackedRecordManifestWriter(
|
||||
bucketName string,
|
||||
basePath string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
multiPartUploadSize int64,
|
||||
columnGroups []storagecommon.ColumnGroup,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*packedRecordManifestWriter, error) {
|
||||
// Validate PK field exists before proceeding
|
||||
_, err := typeutil.GetPrimaryFieldSchema(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
arrowSchema, err := ConvertToArrowSchema(schema, true)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrServiceInternal(
|
||||
fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error()))
|
||||
}
|
||||
// if storage config is not passed, use common config
|
||||
storageType := paramtable.Get().CommonCfg.StorageType.GetValue()
|
||||
if storageConfig != nil {
|
||||
storageType = storageConfig.GetStorageType()
|
||||
}
|
||||
ffiBasePath := basePath
|
||||
if storageType != "local" {
|
||||
ffiBasePath = path.Join(bucketName, basePath)
|
||||
}
|
||||
writer, err := packed.NewFFIPackedWriter(ffiBasePath, arrowSchema, columnGroups, storageConfig, storagePluginContext)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrServiceInternal(
|
||||
fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||
}
|
||||
columnGroupUncompressed := make(map[typeutil.UniqueID]uint64)
|
||||
columnGroupCompressed := make(map[typeutil.UniqueID]uint64)
|
||||
|
||||
// provide mock path
|
||||
pathsMap := make(map[typeutil.UniqueID]string)
|
||||
start := time.Now().UnixNano()
|
||||
for _, columnGroup := range columnGroups {
|
||||
columnGroupUncompressed[columnGroup.GroupID] = 0
|
||||
columnGroupCompressed[columnGroup.GroupID] = 0
|
||||
start++
|
||||
pathsMap[columnGroup.GroupID] = path.Join(basePath, strconv.FormatInt(columnGroup.GroupID, 10), strconv.FormatInt(start, 10))
|
||||
}
|
||||
|
||||
return &packedRecordManifestWriter{
|
||||
writer: writer,
|
||||
schema: schema,
|
||||
arrowSchema: arrowSchema,
|
||||
bufferSize: bufferSize,
|
||||
bucketName: bucketName,
|
||||
pathsMap: pathsMap,
|
||||
columnGroups: columnGroups,
|
||||
columnGroupUncompressed: columnGroupUncompressed,
|
||||
columnGroupCompressed: columnGroupCompressed,
|
||||
storageConfig: storageConfig,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Deprecated, todo remove
|
||||
func NewPackedSerializeWriter(bucketName string, paths []string, schema *schemapb.CollectionSchema, bufferSize int64,
|
||||
multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup, batchSize int,
|
||||
) (*SerializeWriterImpl[*Value], error) {
|
||||
packedRecordWriter, err := NewPackedRecordWriter(bucketName, paths, schema, bufferSize, multiPartUploadSize, columnGroups, nil, nil)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrServiceInternal(
|
||||
fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||
}
|
||||
return NewSerializeRecordWriter(packedRecordWriter, func(v []*Value) (Record, error) {
|
||||
return ValueSerializer(v, schema)
|
||||
}, batchSize), nil
|
||||
}
|
||||
@ -21,6 +21,7 @@ import (
|
||||
"encoding/base64"
|
||||
"fmt"
|
||||
sio "io"
|
||||
"path"
|
||||
"sort"
|
||||
|
||||
"github.com/samber/lo"
|
||||
@ -67,6 +68,7 @@ type rwOptions struct {
|
||||
collectionID int64
|
||||
storageConfig *indexpb.StorageConfig
|
||||
neededFields typeutil.Set[int64]
|
||||
useLoonFFI bool
|
||||
}
|
||||
|
||||
func (o *rwOptions) validate() error {
|
||||
@ -163,6 +165,12 @@ func WithNeededFields(neededFields typeutil.Set[int64]) RwOption {
|
||||
}
|
||||
}
|
||||
|
||||
func WithUseLoonFFI(useLoonFFI bool) RwOption {
|
||||
return func(options *rwOptions) {
|
||||
options.useLoonFFI = useLoonFFI
|
||||
}
|
||||
}
|
||||
|
||||
func makeBlobsReader(ctx context.Context, binlogs []*datapb.FieldBinlog, downloader downloaderFn) (ChunkedBlobsReader, error) {
|
||||
if len(binlogs) == 0 {
|
||||
return func() ([]*Blob, error) {
|
||||
@ -275,11 +283,22 @@ func NewBinlogRecordReader(ctx context.Context, binlogs []*datapb.FieldBinlog, s
|
||||
return binlogs[i].GetFieldID() < binlogs[j].GetFieldID()
|
||||
})
|
||||
|
||||
var err error
|
||||
rr, err = NewRecordReaderFromBinlogs(binlogs, schema, rwOptions.bufferSize, rwOptions.storageConfig, pluginContext)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
binlogLists := lo.Map(binlogs, func(fieldBinlog *datapb.FieldBinlog, _ int) []*datapb.Binlog {
|
||||
return fieldBinlog.GetBinlogs()
|
||||
})
|
||||
bucketName := rwOptions.storageConfig.BucketName
|
||||
paths := make([][]string, len(binlogLists[0]))
|
||||
for _, binlogs := range binlogLists {
|
||||
for j, binlog := range binlogs {
|
||||
logPath := binlog.GetLogPath()
|
||||
if rwOptions.storageConfig.StorageType != "local" {
|
||||
logPath = path.Join(bucketName, logPath)
|
||||
}
|
||||
paths[j] = append(paths[j], logPath)
|
||||
}
|
||||
}
|
||||
// FIXME: add needed fields support
|
||||
rr = newIterativePackedRecordReader(paths, schema, rwOptions.bufferSize, rwOptions.storageConfig, pluginContext)
|
||||
default:
|
||||
return nil, merr.WrapErrServiceInternal(fmt.Sprintf("unsupported storage version %d", rwOptions.version))
|
||||
}
|
||||
@ -289,6 +308,31 @@ func NewBinlogRecordReader(ctx context.Context, binlogs []*datapb.FieldBinlog, s
|
||||
return rr, nil
|
||||
}
|
||||
|
||||
func NewManifestRecordReader(ctx context.Context, manifestPath string, schema *schemapb.CollectionSchema, option ...RwOption) (rr RecordReader, err error) {
|
||||
rwOptions := DefaultReaderOptions()
|
||||
for _, opt := range option {
|
||||
opt(rwOptions)
|
||||
}
|
||||
if err := rwOptions.validate(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var pluginContext *indexcgopb.StoragePluginContext
|
||||
if hookutil.IsClusterEncyptionEnabled() {
|
||||
if ez := hookutil.GetEzByCollProperties(schema.GetProperties(), rwOptions.collectionID); ez != nil {
|
||||
unsafe := hookutil.GetCipher().GetUnsafeKey(ez.EzID, ez.CollectionID)
|
||||
if len(unsafe) > 0 {
|
||||
pluginContext = &indexcgopb.StoragePluginContext{
|
||||
EncryptionZoneId: ez.EzID,
|
||||
CollectionId: ez.CollectionID,
|
||||
EncryptionKey: string(unsafe),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return NewRecordReaderFromManifest(manifestPath, schema, rwOptions.bufferSize, rwOptions.storageConfig, pluginContext)
|
||||
}
|
||||
|
||||
func NewBinlogRecordWriter(ctx context.Context, collectionID, partitionID, segmentID UniqueID,
|
||||
schema *schemapb.CollectionSchema, allocator allocator.Interface, chunkSize uint64, maxRowNum int64,
|
||||
option ...RwOption,
|
||||
@ -340,6 +384,13 @@ func NewBinlogRecordWriter(ctx context.Context, collectionID, partitionID, segme
|
||||
blobsWriter, allocator, chunkSize, rootPath, maxRowNum, opts...,
|
||||
)
|
||||
case StorageV2:
|
||||
if rwOptions.useLoonFFI {
|
||||
return newPackedManifestRecordWriter(collectionID, partitionID, segmentID, schema,
|
||||
blobsWriter, allocator, maxRowNum,
|
||||
rwOptions.bufferSize, rwOptions.multiPartUploadSize, rwOptions.columnGroups,
|
||||
rwOptions.storageConfig,
|
||||
pluginContext)
|
||||
} else {
|
||||
return newPackedBinlogRecordWriter(collectionID, partitionID, segmentID, schema,
|
||||
blobsWriter, allocator, maxRowNum,
|
||||
rwOptions.bufferSize, rwOptions.multiPartUploadSize, rwOptions.columnGroups,
|
||||
@ -347,6 +398,7 @@ func NewBinlogRecordWriter(ctx context.Context, collectionID, partitionID, segme
|
||||
pluginContext,
|
||||
)
|
||||
}
|
||||
}
|
||||
return nil, merr.WrapErrServiceInternal(fmt.Sprintf("unsupported storage version %d", rwOptions.version))
|
||||
}
|
||||
|
||||
|
||||
@ -161,7 +161,7 @@ func (s *PackedBinlogRecordSuite) TestPackedBinlogRecordIntegration() {
|
||||
rowNum := w.GetRowNum()
|
||||
s.Equal(rowNum, int64(rows))
|
||||
|
||||
fieldBinlogs, statsLog, bm25StatsLog := w.GetLogs()
|
||||
fieldBinlogs, statsLog, bm25StatsLog, _ := w.GetLogs()
|
||||
s.Equal(len(fieldBinlogs), len(columnGroups))
|
||||
for _, columnGroup := range fieldBinlogs {
|
||||
s.Equal(len(columnGroup.Binlogs), 1)
|
||||
@ -240,7 +240,7 @@ func (s *PackedBinlogRecordSuite) TestGenerateBM25Stats() {
|
||||
s.NoError(err)
|
||||
err = w.Close()
|
||||
s.NoError(err)
|
||||
fieldBinlogs, statsLog, bm25StatsLog := w.GetLogs()
|
||||
fieldBinlogs, statsLog, bm25StatsLog, _ := w.GetLogs()
|
||||
s.Equal(len(fieldBinlogs), len(columnGroups))
|
||||
|
||||
s.Equal(statsLog.Binlogs[0].EntriesNum, int64(1))
|
||||
|
||||
@ -1,6 +1,7 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
|
||||
"github.com/apache/arrow/go/v17/arrow"
|
||||
@ -11,7 +12,7 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
func ConvertToArrowSchema(schema *schemapb.CollectionSchema) (*arrow.Schema, error) {
|
||||
func ConvertToArrowSchema(schema *schemapb.CollectionSchema, useFieldID bool) (*arrow.Schema, error) {
|
||||
fieldCount := typeutil.GetTotalFieldsNum(schema)
|
||||
arrowFields := make([]arrow.Field, 0, fieldCount)
|
||||
appendArrowField := func(field *schemapb.FieldSchema) error {
|
||||
@ -37,7 +38,7 @@ func ConvertToArrowSchema(schema *schemapb.CollectionSchema) (*arrow.Schema, err
|
||||
}
|
||||
|
||||
arrowType := serdeMap[field.DataType].arrowType(dim, elementType)
|
||||
arrowField := ConvertToArrowField(field, arrowType)
|
||||
arrowField := ConvertToArrowField(field, arrowType, useFieldID)
|
||||
|
||||
// Add extra metadata for ArrayOfVector
|
||||
if field.DataType == schemapb.DataType_ArrayOfVector {
|
||||
@ -67,11 +68,15 @@ func ConvertToArrowSchema(schema *schemapb.CollectionSchema) (*arrow.Schema, err
|
||||
return arrow.NewSchema(arrowFields, nil), nil
|
||||
}
|
||||
|
||||
func ConvertToArrowField(field *schemapb.FieldSchema, dataType arrow.DataType) arrow.Field {
|
||||
return arrow.Field{
|
||||
func ConvertToArrowField(field *schemapb.FieldSchema, dataType arrow.DataType, useFieldID bool) arrow.Field {
|
||||
f := arrow.Field{
|
||||
Name: field.GetName(),
|
||||
Type: dataType,
|
||||
Metadata: arrow.NewMetadata([]string{packed.ArrowFieldIdMetadataKey}, []string{strconv.Itoa(int(field.GetFieldID()))}),
|
||||
Nullable: field.GetNullable(),
|
||||
}
|
||||
if useFieldID {
|
||||
field.Name = fmt.Sprintf("%d", field.GetFieldID())
|
||||
}
|
||||
return f
|
||||
}
|
||||
|
||||
@ -56,7 +56,7 @@ func TestConvertArrowSchema(t *testing.T) {
|
||||
Fields: fieldSchemas,
|
||||
StructArrayFields: StructArrayFieldSchemas,
|
||||
}
|
||||
arrowSchema, err := ConvertToArrowSchema(schema)
|
||||
arrowSchema, err := ConvertToArrowSchema(schema, false)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(fieldSchemas)+len(StructArrayFieldSchemas[0].Fields), len(arrowSchema.Fields()))
|
||||
}
|
||||
@ -84,6 +84,6 @@ func TestConvertArrowSchemaWithoutDim(t *testing.T) {
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Fields: fieldSchemas,
|
||||
}
|
||||
_, err := ConvertToArrowSchema(schema)
|
||||
_, err := ConvertToArrowSchema(schema, false)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
@ -45,11 +45,6 @@ type Record interface {
|
||||
Retain()
|
||||
}
|
||||
|
||||
type RecordReader interface {
|
||||
Next() (Record, error)
|
||||
Close() error
|
||||
}
|
||||
|
||||
type RecordWriter interface {
|
||||
Write(r Record) error
|
||||
GetWrittenUncompressed() uint64
|
||||
|
||||
@ -22,7 +22,6 @@ import (
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"path"
|
||||
"sort"
|
||||
"strconv"
|
||||
|
||||
@ -35,278 +34,14 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/hook"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/json"
|
||||
"github.com/milvus-io/milvus/internal/storagev2/packed"
|
||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
func NewRecordReaderFromBinlogs(fieldBinlogs []*datapb.FieldBinlog,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (RecordReader, error) {
|
||||
// check legacy or import binlog struct
|
||||
for _, fieldBinlog := range fieldBinlogs {
|
||||
if len(fieldBinlog.ChildFields) == 0 {
|
||||
binlogLists := lo.Map(fieldBinlogs, func(fieldBinlog *datapb.FieldBinlog, _ int) []*datapb.Binlog {
|
||||
return fieldBinlog.GetBinlogs()
|
||||
})
|
||||
bucketName := storageConfig.BucketName
|
||||
paths := make([][]string, len(binlogLists[0]))
|
||||
for _, binlogs := range binlogLists {
|
||||
for j, binlog := range binlogs {
|
||||
logPath := binlog.GetLogPath()
|
||||
if storageConfig.StorageType != "local" {
|
||||
logPath = path.Join(bucketName, logPath)
|
||||
}
|
||||
paths[j] = append(paths[j], logPath)
|
||||
}
|
||||
}
|
||||
return newIterativePackedRecordReader(paths, schema, bufferSize, storageConfig, storagePluginContext), nil
|
||||
}
|
||||
}
|
||||
return NewManifestReaderFromBinlogs(fieldBinlogs, schema, bufferSize, storageConfig, storagePluginContext)
|
||||
}
|
||||
|
||||
var _ RecordReader = (*IterativeRecordReader)(nil)
|
||||
|
||||
type IterativeRecordReader struct {
|
||||
cur RecordReader
|
||||
iterate func() (RecordReader, error)
|
||||
}
|
||||
|
||||
// Close implements RecordReader.
|
||||
func (ir *IterativeRecordReader) Close() error {
|
||||
if ir.cur != nil {
|
||||
return ir.cur.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ir *IterativeRecordReader) Next() (Record, error) {
|
||||
if ir.cur == nil {
|
||||
r, err := ir.iterate()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
ir.cur = r
|
||||
}
|
||||
rec, err := ir.cur.Next()
|
||||
if err == io.EOF {
|
||||
closeErr := ir.cur.Close()
|
||||
if closeErr != nil {
|
||||
return nil, closeErr
|
||||
}
|
||||
ir.cur, err = ir.iterate()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
rec, err = ir.cur.Next()
|
||||
}
|
||||
return rec, err
|
||||
}
|
||||
|
||||
type Manifest struct {
|
||||
Version int `json:"version"`
|
||||
ColumnGroups []*ColumnGroup `json:"column_groups"`
|
||||
}
|
||||
|
||||
type ColumnGroup struct {
|
||||
Columns []string `json:"columns"`
|
||||
Format string `json:"format"`
|
||||
Paths []string `json:"paths"`
|
||||
}
|
||||
|
||||
type ManifestReader struct {
|
||||
fieldBinlogs []*datapb.FieldBinlog
|
||||
reader *packed.FFIPackedReader
|
||||
|
||||
bufferSize int64
|
||||
arrowSchema *arrow.Schema
|
||||
schema *schemapb.CollectionSchema
|
||||
schemaHelper *typeutil.SchemaHelper
|
||||
field2Col map[FieldID]int
|
||||
storageConfig *indexpb.StorageConfig
|
||||
storagePluginContext *indexcgopb.StoragePluginContext
|
||||
|
||||
neededColumns []string
|
||||
}
|
||||
|
||||
// NewManifestReaderFromBinlogs creates a ManifestReader from binlogs
|
||||
func NewManifestReaderFromBinlogs(fieldBinlogs []*datapb.FieldBinlog,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*ManifestReader, error) {
|
||||
arrowSchema, err := ConvertToArrowSchema(schema)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
|
||||
}
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
field2Col := make(map[FieldID]int)
|
||||
allFields := typeutil.GetAllFieldSchemas(schema)
|
||||
neededColumns := make([]string, 0, len(allFields))
|
||||
for i, field := range allFields {
|
||||
field2Col[field.FieldID] = i
|
||||
neededColumns = append(neededColumns, field.Name)
|
||||
}
|
||||
prr := &ManifestReader{
|
||||
fieldBinlogs: fieldBinlogs,
|
||||
bufferSize: bufferSize,
|
||||
arrowSchema: arrowSchema,
|
||||
schema: schema,
|
||||
schemaHelper: schemaHelper,
|
||||
field2Col: field2Col,
|
||||
storageConfig: storageConfig,
|
||||
storagePluginContext: storagePluginContext,
|
||||
|
||||
neededColumns: neededColumns,
|
||||
}
|
||||
|
||||
err = prr.init()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return prr, nil
|
||||
}
|
||||
|
||||
func (mr *ManifestReader) generateManifest() (string, error) {
|
||||
m := &Manifest{
|
||||
Version: 0,
|
||||
ColumnGroups: lo.Map(mr.fieldBinlogs, func(binlog *datapb.FieldBinlog, _ int) *ColumnGroup {
|
||||
return &ColumnGroup{
|
||||
Columns: lo.Map(binlog.ChildFields, func(fieldID int64, _ int) string {
|
||||
field, err := mr.schemaHelper.GetFieldFromID(fieldID)
|
||||
if err != nil {
|
||||
// return empty string if field not found
|
||||
return ""
|
||||
}
|
||||
return field.GetName()
|
||||
}),
|
||||
Format: "parquet",
|
||||
Paths: lo.Map(binlog.Binlogs, func(binlog *datapb.Binlog, _ int) string {
|
||||
p := binlog.GetLogPath()
|
||||
if mr.storageConfig.StorageType != "local" {
|
||||
p = path.Join(mr.storageConfig.BucketName, p)
|
||||
}
|
||||
return p
|
||||
}),
|
||||
}
|
||||
}),
|
||||
}
|
||||
bs, err := json.Marshal(m)
|
||||
return string(bs), err
|
||||
}
|
||||
|
||||
func (mr *ManifestReader) init() error {
|
||||
manifest, err := mr.generateManifest()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// TODO add needed column option
|
||||
|
||||
reader, err := packed.NewFFIPackedReader(manifest, mr.arrowSchema, mr.neededColumns, mr.bufferSize, mr.storageConfig, mr.storagePluginContext)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
mr.reader = reader
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mr ManifestReader) Next() (Record, error) {
|
||||
rec, err := mr.reader.ReadNext()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewSimpleArrowRecord(rec, mr.field2Col), nil
|
||||
}
|
||||
|
||||
func (mr ManifestReader) Close() error {
|
||||
if mr.reader != nil {
|
||||
return mr.reader.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ChunkedBlobsReader returns a chunk composed of blobs, or io.EOF if no more data
|
||||
type ChunkedBlobsReader func() ([]*Blob, error)
|
||||
|
||||
type CompositeBinlogRecordReader struct {
|
||||
fields map[FieldID]*schemapb.FieldSchema
|
||||
index map[FieldID]int16
|
||||
brs []*BinlogReader
|
||||
rrs []array.RecordReader
|
||||
}
|
||||
|
||||
var _ RecordReader = (*CompositeBinlogRecordReader)(nil)
|
||||
|
||||
func (crr *CompositeBinlogRecordReader) Next() (Record, error) {
|
||||
recs := make([]arrow.Array, len(crr.fields))
|
||||
nonExistingFields := make([]*schemapb.FieldSchema, 0)
|
||||
nRows := 0
|
||||
for _, f := range crr.fields {
|
||||
idx := crr.index[f.FieldID]
|
||||
if crr.rrs[idx] != nil {
|
||||
if ok := crr.rrs[idx].Next(); !ok {
|
||||
return nil, io.EOF
|
||||
}
|
||||
r := crr.rrs[idx].Record()
|
||||
recs[idx] = r.Column(0)
|
||||
if nRows == 0 {
|
||||
nRows = int(r.NumRows())
|
||||
}
|
||||
if nRows != int(r.NumRows()) {
|
||||
return nil, merr.WrapErrServiceInternal(fmt.Sprintf("number of rows mismatch for field %d", f.FieldID))
|
||||
}
|
||||
} else {
|
||||
nonExistingFields = append(nonExistingFields, f)
|
||||
}
|
||||
}
|
||||
for _, f := range nonExistingFields {
|
||||
// If the field is not in the current batch, fill with null array
|
||||
arr, err := GenerateEmptyArrayFromSchema(f, nRows)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
recs[crr.index[f.FieldID]] = arr
|
||||
}
|
||||
return &compositeRecord{
|
||||
index: crr.index,
|
||||
recs: recs,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (crr *CompositeBinlogRecordReader) Close() error {
|
||||
if crr.brs != nil {
|
||||
for _, er := range crr.brs {
|
||||
if er != nil {
|
||||
er.Close()
|
||||
}
|
||||
}
|
||||
}
|
||||
if crr.rrs != nil {
|
||||
for _, rr := range crr.rrs {
|
||||
if rr != nil {
|
||||
rr.Release()
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func parseBlobKey(blobKey string) (colId FieldID, logId UniqueID) {
|
||||
if _, _, _, colId, logId, ok := metautil.ParseInsertLogPath(blobKey); ok {
|
||||
return colId, logId
|
||||
@ -738,25 +473,12 @@ func ValueSerializer(v []*Value, schema *schemapb.CollectionSchema) (Record, err
|
||||
builder := builders[field.FieldID]
|
||||
arrays[i] = builder.NewArray()
|
||||
builder.Release()
|
||||
fields[i] = ConvertToArrowField(field, arrays[i].DataType())
|
||||
fields[i] = ConvertToArrowField(field, arrays[i].DataType(), false)
|
||||
field2Col[field.FieldID] = i
|
||||
}
|
||||
return NewSimpleArrowRecord(array.NewRecord(arrow.NewSchema(fields, nil), arrays, int64(len(v))), field2Col), nil
|
||||
}
|
||||
|
||||
type BinlogRecordWriter interface {
|
||||
RecordWriter
|
||||
GetLogs() (
|
||||
fieldBinlogs map[FieldID]*datapb.FieldBinlog,
|
||||
statsLog *datapb.FieldBinlog,
|
||||
bm25StatsLog map[FieldID]*datapb.FieldBinlog,
|
||||
)
|
||||
GetRowNum() int64
|
||||
FlushChunk() error
|
||||
GetBufferUncompressed() uint64
|
||||
Schema() *schemapb.CollectionSchema
|
||||
}
|
||||
|
||||
type ChunkedBlobsWriter func([]*Blob) error
|
||||
|
||||
type CompositeBinlogRecordWriter struct {
|
||||
@ -975,8 +697,9 @@ func (c *CompositeBinlogRecordWriter) GetLogs() (
|
||||
fieldBinlogs map[FieldID]*datapb.FieldBinlog,
|
||||
statsLog *datapb.FieldBinlog,
|
||||
bm25StatsLog map[FieldID]*datapb.FieldBinlog,
|
||||
manifest string,
|
||||
) {
|
||||
return c.fieldBinlogs, c.statsLog, c.bm25StatsLog
|
||||
return c.fieldBinlogs, c.statsLog, c.bm25StatsLog, ""
|
||||
}
|
||||
|
||||
func (c *CompositeBinlogRecordWriter) GetRowNum() int64 {
|
||||
|
||||
@ -177,7 +177,7 @@ func TestBinlogSerializeWriter(t *testing.T) {
|
||||
err = writer.Close()
|
||||
assert.NoError(t, err)
|
||||
|
||||
logs, _, _ := writer.GetLogs()
|
||||
logs, _, _, _ := writer.GetLogs()
|
||||
assert.Equal(t, 18, len(logs))
|
||||
assert.Equal(t, 5, len(logs[0].Binlogs))
|
||||
})
|
||||
|
||||
@ -15,519 +15,3 @@
|
||||
// limitations under the License.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"path"
|
||||
|
||||
"github.com/apache/arrow/go/v17/arrow"
|
||||
"github.com/apache/arrow/go/v17/arrow/array"
|
||||
"github.com/samber/lo"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/storagecommon"
|
||||
"github.com/milvus-io/milvus/internal/storagev2/packed"
|
||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
type packedRecordReader struct {
|
||||
reader *packed.PackedReader
|
||||
field2Col map[FieldID]int
|
||||
}
|
||||
|
||||
var _ RecordReader = (*packedRecordReader)(nil)
|
||||
|
||||
func (pr *packedRecordReader) Next() (Record, error) {
|
||||
rec, err := pr.reader.ReadNext()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewSimpleArrowRecord(rec, pr.field2Col), nil
|
||||
}
|
||||
|
||||
func (pr *packedRecordReader) Close() error {
|
||||
if pr.reader != nil {
|
||||
return pr.reader.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func newPackedRecordReader(
|
||||
paths []string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*packedRecordReader, error) {
|
||||
arrowSchema, err := ConvertToArrowSchema(schema)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
|
||||
}
|
||||
field2Col := make(map[FieldID]int)
|
||||
allFields := typeutil.GetAllFieldSchemas(schema)
|
||||
for i, field := range allFields {
|
||||
field2Col[field.FieldID] = i
|
||||
}
|
||||
reader, err := packed.NewPackedReader(paths, arrowSchema, bufferSize, storageConfig, storagePluginContext)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &packedRecordReader{
|
||||
reader: reader,
|
||||
field2Col: field2Col,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func newIterativePackedRecordReader(
|
||||
paths [][]string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) *IterativeRecordReader {
|
||||
chunk := 0
|
||||
return &IterativeRecordReader{
|
||||
iterate: func() (RecordReader, error) {
|
||||
if chunk >= len(paths) {
|
||||
return nil, io.EOF
|
||||
}
|
||||
currentPaths := paths[chunk]
|
||||
chunk++
|
||||
return newPackedRecordReader(currentPaths, schema, bufferSize, storageConfig, storagePluginContext)
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
var _ RecordWriter = (*packedRecordWriter)(nil)
|
||||
|
||||
type packedRecordWriter struct {
|
||||
writer *packed.PackedWriter
|
||||
bufferSize int64
|
||||
columnGroups []storagecommon.ColumnGroup
|
||||
bucketName string
|
||||
pathsMap map[typeutil.UniqueID]string
|
||||
schema *schemapb.CollectionSchema
|
||||
arrowSchema *arrow.Schema
|
||||
rowNum int64
|
||||
writtenUncompressed uint64
|
||||
columnGroupUncompressed map[typeutil.UniqueID]uint64
|
||||
columnGroupCompressed map[typeutil.UniqueID]uint64
|
||||
storageConfig *indexpb.StorageConfig
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) Write(r Record) error {
|
||||
var rec arrow.Record
|
||||
sar, ok := r.(*simpleArrowRecord)
|
||||
if !ok {
|
||||
// Get all fields including struct sub-fields
|
||||
allFields := typeutil.GetAllFieldSchemas(pw.schema)
|
||||
arrays := make([]arrow.Array, len(allFields))
|
||||
for i, field := range allFields {
|
||||
arrays[i] = r.Column(field.FieldID)
|
||||
}
|
||||
rec = array.NewRecord(pw.arrowSchema, arrays, int64(r.Len()))
|
||||
} else {
|
||||
rec = sar.r
|
||||
}
|
||||
pw.rowNum += int64(r.Len())
|
||||
for col, arr := range rec.Columns() {
|
||||
// size := arr.Data().SizeInBytes()
|
||||
size := calculateActualDataSize(arr)
|
||||
pw.writtenUncompressed += size
|
||||
for _, columnGroup := range pw.columnGroups {
|
||||
if lo.Contains(columnGroup.Columns, col) {
|
||||
pw.columnGroupUncompressed[columnGroup.GroupID] += size
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
defer rec.Release()
|
||||
return pw.writer.WriteRecordBatch(rec)
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetWrittenUncompressed() uint64 {
|
||||
return pw.writtenUncompressed
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetColumnGroupWrittenUncompressed(columnGroup typeutil.UniqueID) uint64 {
|
||||
if size, ok := pw.columnGroupUncompressed[columnGroup]; ok {
|
||||
return size
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetColumnGroupWrittenCompressed(columnGroup typeutil.UniqueID) uint64 {
|
||||
if size, ok := pw.columnGroupCompressed[columnGroup]; ok {
|
||||
return size
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetWrittenPaths(columnGroup typeutil.UniqueID) string {
|
||||
if path, ok := pw.pathsMap[columnGroup]; ok {
|
||||
return path
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) GetWrittenRowNum() int64 {
|
||||
return pw.rowNum
|
||||
}
|
||||
|
||||
func (pw *packedRecordWriter) Close() error {
|
||||
if pw.writer != nil {
|
||||
err := pw.writer.Close()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for id, fpath := range pw.pathsMap {
|
||||
truePath := path.Join(pw.bucketName, fpath)
|
||||
size, err := packed.GetFileSize(truePath, pw.storageConfig)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
pw.columnGroupCompressed[id] = uint64(size)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func NewPackedRecordWriter(
|
||||
bucketName string,
|
||||
paths []string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
bufferSize int64,
|
||||
multiPartUploadSize int64,
|
||||
columnGroups []storagecommon.ColumnGroup,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*packedRecordWriter, error) {
|
||||
// Validate PK field exists before proceeding
|
||||
_, err := typeutil.GetPrimaryFieldSchema(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
arrowSchema, err := ConvertToArrowSchema(schema)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrServiceInternal(
|
||||
fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error()))
|
||||
}
|
||||
// if storage config is not passed, use common config
|
||||
storageType := paramtable.Get().CommonCfg.StorageType.GetValue()
|
||||
if storageConfig != nil {
|
||||
storageType = storageConfig.GetStorageType()
|
||||
}
|
||||
// compose true path before create packed writer here
|
||||
// and returned writtenPaths shall remain untouched
|
||||
truePaths := lo.Map(paths, func(p string, _ int) string {
|
||||
if storageType == "local" {
|
||||
return p
|
||||
}
|
||||
return path.Join(bucketName, p)
|
||||
})
|
||||
writer, err := packed.NewPackedWriter(truePaths, arrowSchema, bufferSize, multiPartUploadSize, columnGroups, storageConfig, storagePluginContext)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrServiceInternal(
|
||||
fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||
}
|
||||
columnGroupUncompressed := make(map[typeutil.UniqueID]uint64)
|
||||
columnGroupCompressed := make(map[typeutil.UniqueID]uint64)
|
||||
pathsMap := make(map[typeutil.UniqueID]string)
|
||||
if len(paths) != len(columnGroups) {
|
||||
return nil, merr.WrapErrParameterInvalid(len(paths), len(columnGroups),
|
||||
"paths length is not equal to column groups length for packed record writer")
|
||||
}
|
||||
for i, columnGroup := range columnGroups {
|
||||
columnGroupUncompressed[columnGroup.GroupID] = 0
|
||||
columnGroupCompressed[columnGroup.GroupID] = 0
|
||||
pathsMap[columnGroup.GroupID] = paths[i]
|
||||
}
|
||||
return &packedRecordWriter{
|
||||
writer: writer,
|
||||
schema: schema,
|
||||
arrowSchema: arrowSchema,
|
||||
bufferSize: bufferSize,
|
||||
bucketName: bucketName,
|
||||
pathsMap: pathsMap,
|
||||
columnGroups: columnGroups,
|
||||
columnGroupUncompressed: columnGroupUncompressed,
|
||||
columnGroupCompressed: columnGroupCompressed,
|
||||
storageConfig: storageConfig,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Deprecated, todo remove
|
||||
func NewPackedSerializeWriter(bucketName string, paths []string, schema *schemapb.CollectionSchema, bufferSize int64,
|
||||
multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup, batchSize int,
|
||||
) (*SerializeWriterImpl[*Value], error) {
|
||||
packedRecordWriter, err := NewPackedRecordWriter(bucketName, paths, schema, bufferSize, multiPartUploadSize, columnGroups, nil, nil)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrServiceInternal(
|
||||
fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||
}
|
||||
return NewSerializeRecordWriter(packedRecordWriter, func(v []*Value) (Record, error) {
|
||||
return ValueSerializer(v, schema)
|
||||
}, batchSize), nil
|
||||
}
|
||||
|
||||
var _ BinlogRecordWriter = (*PackedBinlogRecordWriter)(nil)
|
||||
|
||||
type PackedBinlogRecordWriter struct {
|
||||
// attributes
|
||||
collectionID UniqueID
|
||||
partitionID UniqueID
|
||||
segmentID UniqueID
|
||||
schema *schemapb.CollectionSchema
|
||||
BlobsWriter ChunkedBlobsWriter
|
||||
allocator allocator.Interface
|
||||
maxRowNum int64
|
||||
arrowSchema *arrow.Schema
|
||||
bufferSize int64
|
||||
multiPartUploadSize int64
|
||||
columnGroups []storagecommon.ColumnGroup
|
||||
storageConfig *indexpb.StorageConfig
|
||||
storagePluginContext *indexcgopb.StoragePluginContext
|
||||
|
||||
// writer and stats generated at runtime
|
||||
writer *packedRecordWriter
|
||||
pkCollector *PkStatsCollector
|
||||
bm25Collector *Bm25StatsCollector
|
||||
tsFrom typeutil.Timestamp
|
||||
tsTo typeutil.Timestamp
|
||||
rowNum int64
|
||||
writtenUncompressed uint64
|
||||
|
||||
// results
|
||||
fieldBinlogs map[FieldID]*datapb.FieldBinlog
|
||||
statsLog *datapb.FieldBinlog
|
||||
bm25StatsLog map[FieldID]*datapb.FieldBinlog
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) Write(r Record) error {
|
||||
if err := pw.initWriters(r); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Track timestamps
|
||||
tsArray := r.Column(common.TimeStampField).(*array.Int64)
|
||||
rows := r.Len()
|
||||
for i := 0; i < rows; i++ {
|
||||
ts := typeutil.Timestamp(tsArray.Value(i))
|
||||
if ts < pw.tsFrom {
|
||||
pw.tsFrom = ts
|
||||
}
|
||||
if ts > pw.tsTo {
|
||||
pw.tsTo = ts
|
||||
}
|
||||
}
|
||||
|
||||
// Collect statistics
|
||||
if err := pw.pkCollector.Collect(r); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := pw.bm25Collector.Collect(r); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err := pw.writer.Write(r)
|
||||
if err != nil {
|
||||
return merr.WrapErrServiceInternal(fmt.Sprintf("write record batch error: %s", err.Error()))
|
||||
}
|
||||
pw.writtenUncompressed = pw.writer.GetWrittenUncompressed()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) initWriters(r Record) error {
|
||||
if pw.writer == nil {
|
||||
if len(pw.columnGroups) == 0 {
|
||||
allFields := typeutil.GetAllFieldSchemas(pw.schema)
|
||||
pw.columnGroups = storagecommon.SplitColumns(allFields, pw.getColumnStatsFromRecord(r, allFields), storagecommon.DefaultPolicies()...)
|
||||
}
|
||||
logIdStart, _, err := pw.allocator.Alloc(uint32(len(pw.columnGroups)))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
paths := []string{}
|
||||
for _, columnGroup := range pw.columnGroups {
|
||||
path := metautil.BuildInsertLogPath(pw.storageConfig.GetRootPath(), pw.collectionID, pw.partitionID, pw.segmentID, columnGroup.GroupID, logIdStart)
|
||||
paths = append(paths, path)
|
||||
logIdStart++
|
||||
}
|
||||
pw.writer, err = NewPackedRecordWriter(pw.storageConfig.GetBucketName(), paths, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext)
|
||||
if err != nil {
|
||||
return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error()))
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) getColumnStatsFromRecord(r Record, allFields []*schemapb.FieldSchema) map[int64]storagecommon.ColumnStats {
|
||||
result := make(map[int64]storagecommon.ColumnStats)
|
||||
for _, field := range allFields {
|
||||
if arr := r.Column(field.FieldID); arr != nil {
|
||||
result[field.FieldID] = storagecommon.ColumnStats{
|
||||
AvgSize: int64(arr.Data().SizeInBytes()) / int64(arr.Len()),
|
||||
}
|
||||
}
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) GetWrittenUncompressed() uint64 {
|
||||
return pw.writtenUncompressed
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) Close() error {
|
||||
if pw.writer != nil {
|
||||
if err := pw.writer.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
pw.finalizeBinlogs()
|
||||
if err := pw.writeStats(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) finalizeBinlogs() {
|
||||
if pw.writer == nil {
|
||||
return
|
||||
}
|
||||
pw.rowNum = pw.writer.GetWrittenRowNum()
|
||||
if pw.fieldBinlogs == nil {
|
||||
pw.fieldBinlogs = make(map[FieldID]*datapb.FieldBinlog, len(pw.columnGroups))
|
||||
}
|
||||
for _, columnGroup := range pw.columnGroups {
|
||||
columnGroupID := columnGroup.GroupID
|
||||
if _, exists := pw.fieldBinlogs[columnGroupID]; !exists {
|
||||
pw.fieldBinlogs[columnGroupID] = &datapb.FieldBinlog{
|
||||
FieldID: columnGroupID,
|
||||
ChildFields: columnGroup.Fields,
|
||||
}
|
||||
}
|
||||
pw.fieldBinlogs[columnGroupID].Binlogs = append(pw.fieldBinlogs[columnGroupID].Binlogs, &datapb.Binlog{
|
||||
LogSize: int64(pw.writer.GetColumnGroupWrittenCompressed(columnGroupID)),
|
||||
MemorySize: int64(pw.writer.GetColumnGroupWrittenUncompressed(columnGroupID)),
|
||||
LogPath: pw.writer.GetWrittenPaths(columnGroupID),
|
||||
EntriesNum: pw.writer.GetWrittenRowNum(),
|
||||
TimestampFrom: pw.tsFrom,
|
||||
TimestampTo: pw.tsTo,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) writeStats() error {
|
||||
// Write PK stats
|
||||
pkStatsMap, err := pw.pkCollector.Digest(
|
||||
pw.collectionID,
|
||||
pw.partitionID,
|
||||
pw.segmentID,
|
||||
pw.storageConfig.GetRootPath(),
|
||||
pw.rowNum,
|
||||
pw.allocator,
|
||||
pw.BlobsWriter,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// Extract single PK stats from map
|
||||
for _, statsLog := range pkStatsMap {
|
||||
pw.statsLog = statsLog
|
||||
break
|
||||
}
|
||||
|
||||
// Write BM25 stats
|
||||
bm25StatsLog, err := pw.bm25Collector.Digest(
|
||||
pw.collectionID,
|
||||
pw.partitionID,
|
||||
pw.segmentID,
|
||||
pw.storageConfig.GetRootPath(),
|
||||
pw.rowNum,
|
||||
pw.allocator,
|
||||
pw.BlobsWriter,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
pw.bm25StatsLog = bm25StatsLog
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) GetLogs() (
|
||||
fieldBinlogs map[FieldID]*datapb.FieldBinlog,
|
||||
statsLog *datapb.FieldBinlog,
|
||||
bm25StatsLog map[FieldID]*datapb.FieldBinlog,
|
||||
) {
|
||||
return pw.fieldBinlogs, pw.statsLog, pw.bm25StatsLog
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) GetRowNum() int64 {
|
||||
return pw.rowNum
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) FlushChunk() error {
|
||||
return nil // do nothing
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) Schema() *schemapb.CollectionSchema {
|
||||
return pw.schema
|
||||
}
|
||||
|
||||
func (pw *PackedBinlogRecordWriter) GetBufferUncompressed() uint64 {
|
||||
return uint64(pw.multiPartUploadSize)
|
||||
}
|
||||
|
||||
func newPackedBinlogRecordWriter(collectionID, partitionID, segmentID UniqueID, schema *schemapb.CollectionSchema,
|
||||
blobsWriter ChunkedBlobsWriter, allocator allocator.Interface, maxRowNum int64, bufferSize, multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup,
|
||||
storageConfig *indexpb.StorageConfig,
|
||||
storagePluginContext *indexcgopb.StoragePluginContext,
|
||||
) (*PackedBinlogRecordWriter, error) {
|
||||
arrowSchema, err := ConvertToArrowSchema(schema)
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
|
||||
}
|
||||
|
||||
writer := &PackedBinlogRecordWriter{
|
||||
collectionID: collectionID,
|
||||
partitionID: partitionID,
|
||||
segmentID: segmentID,
|
||||
schema: schema,
|
||||
arrowSchema: arrowSchema,
|
||||
BlobsWriter: blobsWriter,
|
||||
allocator: allocator,
|
||||
maxRowNum: maxRowNum,
|
||||
bufferSize: bufferSize,
|
||||
multiPartUploadSize: multiPartUploadSize,
|
||||
columnGroups: columnGroups,
|
||||
storageConfig: storageConfig,
|
||||
storagePluginContext: storagePluginContext,
|
||||
tsFrom: typeutil.MaxTimestamp,
|
||||
tsTo: 0,
|
||||
}
|
||||
|
||||
// Create stats collectors
|
||||
writer.pkCollector, err = NewPkStatsCollector(
|
||||
collectionID,
|
||||
schema,
|
||||
maxRowNum,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
writer.bm25Collector = NewBm25StatsCollector(schema)
|
||||
|
||||
return writer, nil
|
||||
}
|
||||
|
||||
@ -436,7 +436,7 @@ func TestArrayOfVectorIntegration(t *testing.T) {
|
||||
},
|
||||
}
|
||||
|
||||
arrowSchema, err := ConvertToArrowSchema(schema)
|
||||
arrowSchema, err := ConvertToArrowSchema(schema, false)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, arrowSchema)
|
||||
|
||||
|
||||
215
internal/storagev2/packed/ffi_common.go
Normal file
215
internal/storagev2/packed/ffi_common.go
Normal file
@ -0,0 +1,215 @@
|
||||
package packed
|
||||
|
||||
/*
|
||||
#cgo pkg-config: milvus_core milvus-storage
|
||||
#include <stdlib.h>
|
||||
#include "milvus-storage/ffi_c.h"
|
||||
#include "arrow/c/abi.h"
|
||||
#include "arrow/c/helpers.h"
|
||||
*/
|
||||
import "C"
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"strconv"
|
||||
"unsafe"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
)
|
||||
|
||||
// Property keys - matching milvus-storage/properties.h
|
||||
const (
|
||||
PropertyFSAddress = "fs.address"
|
||||
PropertyFSBucketName = "fs.bucket_name"
|
||||
PropertyFSAccessKeyID = "fs.access_key_id"
|
||||
PropertyFSAccessKeyValue = "fs.access_key_value"
|
||||
PropertyFSRootPath = "fs.root_path"
|
||||
PropertyFSStorageType = "fs.storage_type"
|
||||
PropertyFSCloudProvider = "fs.cloud_provider"
|
||||
PropertyFSIAMEndpoint = "fs.iam_endpoint"
|
||||
PropertyFSLogLevel = "fs.log_level"
|
||||
PropertyFSRegion = "fs.region"
|
||||
PropertyFSUseSSL = "fs.use_ssl"
|
||||
PropertyFSSSLCACert = "fs.ssl_ca_cert"
|
||||
PropertyFSUseIAM = "fs.use_iam"
|
||||
PropertyFSUseVirtualHost = "fs.use_virtual_host"
|
||||
PropertyFSRequestTimeoutMS = "fs.request_timeout_ms"
|
||||
PropertyFSGCPCredentialJSON = "fs.gcp_credential_json"
|
||||
PropertyFSUseCustomPartUpload = "fs.use_custom_part_upload"
|
||||
|
||||
PropertyWriterPolicy = "writer.policy"
|
||||
PropertyWriterSchemaBasedPattern = "writer.split.schema_based.patterns"
|
||||
)
|
||||
|
||||
// MakePropertiesFromStorageConfig creates a Properties object from StorageConfig
|
||||
// This function converts a StorageConfig structure into a Properties object by
|
||||
// calling the FFI properties_create function. All configuration fields from
|
||||
// StorageConfig are mapped to corresponding key-value pairs in Properties.
|
||||
func MakePropertiesFromStorageConfig(storageConfig *indexpb.StorageConfig, extraKVs map[string]string) (*C.Properties, error) {
|
||||
if storageConfig == nil {
|
||||
return nil, fmt.Errorf("storageConfig is required")
|
||||
}
|
||||
|
||||
// Prepare key-value pairs from StorageConfig
|
||||
var keys []string
|
||||
var values []string
|
||||
|
||||
// Add non-empty string fields
|
||||
if storageConfig.GetAddress() != "" {
|
||||
keys = append(keys, PropertyFSAddress)
|
||||
values = append(values, storageConfig.GetAddress())
|
||||
}
|
||||
if storageConfig.GetBucketName() != "" {
|
||||
keys = append(keys, PropertyFSBucketName)
|
||||
values = append(values, storageConfig.GetBucketName())
|
||||
}
|
||||
if storageConfig.GetAccessKeyID() != "" {
|
||||
keys = append(keys, PropertyFSAccessKeyID)
|
||||
values = append(values, storageConfig.GetAccessKeyID())
|
||||
}
|
||||
if storageConfig.GetSecretAccessKey() != "" {
|
||||
keys = append(keys, PropertyFSAccessKeyValue)
|
||||
values = append(values, storageConfig.GetSecretAccessKey())
|
||||
}
|
||||
if storageConfig.GetRootPath() != "" {
|
||||
keys = append(keys, PropertyFSRootPath)
|
||||
values = append(values, storageConfig.GetRootPath())
|
||||
}
|
||||
if storageConfig.GetStorageType() != "" {
|
||||
keys = append(keys, PropertyFSStorageType)
|
||||
values = append(values, storageConfig.GetStorageType())
|
||||
}
|
||||
if storageConfig.GetCloudProvider() != "" {
|
||||
keys = append(keys, PropertyFSCloudProvider)
|
||||
values = append(values, storageConfig.GetCloudProvider())
|
||||
}
|
||||
if storageConfig.GetIAMEndpoint() != "" {
|
||||
keys = append(keys, PropertyFSIAMEndpoint)
|
||||
values = append(values, storageConfig.GetIAMEndpoint())
|
||||
}
|
||||
// Always add log level if any string field is set (matching C++ behavior)
|
||||
keys = append(keys, PropertyFSLogLevel)
|
||||
values = append(values, "Warn")
|
||||
|
||||
if storageConfig.GetRegion() != "" {
|
||||
keys = append(keys, PropertyFSRegion)
|
||||
values = append(values, storageConfig.GetRegion())
|
||||
}
|
||||
if storageConfig.GetSslCACert() != "" {
|
||||
keys = append(keys, PropertyFSSSLCACert)
|
||||
values = append(values, storageConfig.GetSslCACert())
|
||||
}
|
||||
if storageConfig.GetGcpCredentialJSON() != "" {
|
||||
keys = append(keys, PropertyFSGCPCredentialJSON)
|
||||
values = append(values, storageConfig.GetGcpCredentialJSON())
|
||||
}
|
||||
|
||||
// Add boolean fields
|
||||
keys = append(keys, PropertyFSUseSSL)
|
||||
if storageConfig.GetUseSSL() {
|
||||
values = append(values, "true")
|
||||
} else {
|
||||
values = append(values, "false")
|
||||
}
|
||||
|
||||
keys = append(keys, PropertyFSUseIAM)
|
||||
if storageConfig.GetUseIAM() {
|
||||
values = append(values, "true")
|
||||
} else {
|
||||
values = append(values, "false")
|
||||
}
|
||||
|
||||
keys = append(keys, PropertyFSUseVirtualHost)
|
||||
if storageConfig.GetUseVirtualHost() {
|
||||
values = append(values, "true")
|
||||
} else {
|
||||
values = append(values, "false")
|
||||
}
|
||||
|
||||
keys = append(keys, PropertyFSUseCustomPartUpload)
|
||||
values = append(values, "true") // hardcoded to true as in the original code
|
||||
|
||||
// Add integer field
|
||||
keys = append(keys, PropertyFSRequestTimeoutMS)
|
||||
values = append(values, strconv.FormatInt(storageConfig.GetRequestTimeoutMs(), 10))
|
||||
|
||||
// Add extra kvs
|
||||
for k, v := range extraKVs {
|
||||
keys = append(keys, k)
|
||||
values = append(values, v)
|
||||
}
|
||||
|
||||
// Convert to C arrays
|
||||
cKeys := make([]*C.char, len(keys))
|
||||
cValues := make([]*C.char, len(values))
|
||||
for i := range keys {
|
||||
cKeys[i] = C.CString(keys[i])
|
||||
cValues[i] = C.CString(values[i])
|
||||
}
|
||||
// Defer cleanup of all C strings
|
||||
defer func() {
|
||||
for i := range cKeys {
|
||||
C.free(unsafe.Pointer(cKeys[i]))
|
||||
C.free(unsafe.Pointer(cValues[i]))
|
||||
}
|
||||
}()
|
||||
|
||||
// Create Properties using FFI
|
||||
properties := &C.Properties{}
|
||||
var cKeysPtr **C.char
|
||||
var cValuesPtr **C.char
|
||||
if len(cKeys) > 0 {
|
||||
cKeysPtr = &cKeys[0]
|
||||
cValuesPtr = &cValues[0]
|
||||
}
|
||||
|
||||
result := C.properties_create(
|
||||
(**C.char)(unsafe.Pointer(cKeysPtr)),
|
||||
(**C.char)(unsafe.Pointer(cValuesPtr)),
|
||||
C.size_t(len(keys)),
|
||||
properties,
|
||||
)
|
||||
|
||||
err := HandleFFIResult(result)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return properties, nil
|
||||
}
|
||||
|
||||
func HandleFFIResult(ffiResult C.FFIResult) error {
|
||||
defer C.FreeFFIResult(&ffiResult)
|
||||
if C.IsSuccess(&ffiResult) == 0 {
|
||||
errMsg := C.GetErrorMessage(&ffiResult)
|
||||
errStr := "Unknown error"
|
||||
if errMsg != nil {
|
||||
errStr = C.GoString(errMsg)
|
||||
}
|
||||
|
||||
return fmt.Errorf("failed to create properties: %s", errStr)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type ManifestJSON struct {
|
||||
ManifestVersion int64 `json:"ver"`
|
||||
BasePath string `json:"base_path"`
|
||||
}
|
||||
|
||||
func MarshalManifestPath(basePath string, version int64) string {
|
||||
bs, _ := json.Marshal(ManifestJSON{
|
||||
ManifestVersion: version,
|
||||
BasePath: basePath,
|
||||
})
|
||||
return string(bs)
|
||||
}
|
||||
|
||||
func UnmarshalManfestPath(manifestPath string) (string, int64, error) {
|
||||
var manifestJSON ManifestJSON
|
||||
err := json.Unmarshal([]byte(manifestPath), &manifestJSON)
|
||||
if err != nil {
|
||||
return "", 0, err
|
||||
}
|
||||
return manifestJSON.BasePath, manifestJSON.ManifestVersion, nil
|
||||
}
|
||||
@ -31,7 +31,9 @@ import (
|
||||
|
||||
"github.com/apache/arrow/go/v17/arrow"
|
||||
"github.com/apache/arrow/go/v17/arrow/cdata"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
)
|
||||
@ -187,5 +189,31 @@ func (r *FFIPackedReader) Release() {
|
||||
r.Close()
|
||||
}
|
||||
|
||||
func GetManifest(manifestPath string, storageConfig *indexpb.StorageConfig) (manifest string, err error) {
|
||||
basePath, version, err := UnmarshalManfestPath(manifestPath)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
log.Info("GetManifest", zap.String("manifestPath", manifestPath), zap.String("basePath", basePath), zap.Int64("version", version))
|
||||
|
||||
cProperties, err := MakePropertiesFromStorageConfig(storageConfig, nil)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
cBasePath := C.CString(basePath)
|
||||
defer C.free(unsafe.Pointer(cBasePath))
|
||||
|
||||
var cManifest *C.char
|
||||
var cVersion C.int64_t
|
||||
result := C.get_latest_column_groups(cBasePath, cProperties, &cManifest, &cVersion)
|
||||
err = HandleFFIResult(result)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
manifest = C.GoString(cManifest)
|
||||
return manifest, nil
|
||||
}
|
||||
|
||||
// Ensure FFIPackedReader implements array.RecordReader interface
|
||||
// var _ array.RecordReader = (*FFIPackedReader)(nil)
|
||||
|
||||
181
internal/storagev2/packed/packed_writer_ffi.go
Normal file
181
internal/storagev2/packed/packed_writer_ffi.go
Normal file
@ -0,0 +1,181 @@
|
||||
// Copyright 2023 Zilliz
|
||||
//
|
||||
// Licensed 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 packed
|
||||
|
||||
/*
|
||||
#cgo pkg-config: milvus_core milvus-storage
|
||||
|
||||
#include <stdlib.h>
|
||||
#include "milvus-storage/ffi_c.h"
|
||||
#include "segcore/packed_writer_c.h"
|
||||
#include "segcore/column_groups_c.h"
|
||||
#include "arrow/c/abi.h"
|
||||
#include "arrow/c/helpers.h"
|
||||
*/
|
||||
import "C"
|
||||
|
||||
import (
|
||||
"strings"
|
||||
"unsafe"
|
||||
|
||||
"github.com/apache/arrow/go/v17/arrow"
|
||||
"github.com/apache/arrow/go/v17/arrow/cdata"
|
||||
"github.com/samber/lo"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/storagecommon"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
)
|
||||
|
||||
func createStorageConfig() *indexpb.StorageConfig {
|
||||
var storageConfig *indexpb.StorageConfig
|
||||
|
||||
if paramtable.Get().CommonCfg.StorageType.GetValue() == "local" {
|
||||
storageConfig = &indexpb.StorageConfig{
|
||||
RootPath: paramtable.Get().LocalStorageCfg.Path.GetValue(),
|
||||
StorageType: paramtable.Get().CommonCfg.StorageType.GetValue(),
|
||||
}
|
||||
} else {
|
||||
storageConfig = &indexpb.StorageConfig{
|
||||
Address: paramtable.Get().MinioCfg.Address.GetValue(),
|
||||
AccessKeyID: paramtable.Get().MinioCfg.AccessKeyID.GetValue(),
|
||||
SecretAccessKey: paramtable.Get().MinioCfg.SecretAccessKey.GetValue(),
|
||||
UseSSL: paramtable.Get().MinioCfg.UseSSL.GetAsBool(),
|
||||
SslCACert: paramtable.Get().MinioCfg.SslCACert.GetValue(),
|
||||
BucketName: paramtable.Get().MinioCfg.BucketName.GetValue(),
|
||||
RootPath: paramtable.Get().MinioCfg.RootPath.GetValue(),
|
||||
UseIAM: paramtable.Get().MinioCfg.UseIAM.GetAsBool(),
|
||||
IAMEndpoint: paramtable.Get().MinioCfg.IAMEndpoint.GetValue(),
|
||||
StorageType: paramtable.Get().CommonCfg.StorageType.GetValue(),
|
||||
Region: paramtable.Get().MinioCfg.Region.GetValue(),
|
||||
UseVirtualHost: paramtable.Get().MinioCfg.UseVirtualHost.GetAsBool(),
|
||||
CloudProvider: paramtable.Get().MinioCfg.CloudProvider.GetValue(),
|
||||
RequestTimeoutMs: paramtable.Get().MinioCfg.RequestTimeoutMs.GetAsInt64(),
|
||||
GcpCredentialJSON: paramtable.Get().MinioCfg.GcpCredentialJSON.GetValue(),
|
||||
}
|
||||
}
|
||||
|
||||
return storageConfig
|
||||
}
|
||||
|
||||
func NewFFIPackedWriter(basePath string, schema *arrow.Schema, columnGroups []storagecommon.ColumnGroup, storageConfig *indexpb.StorageConfig, storagePluginContext *indexcgopb.StoragePluginContext) (*FFIPackedWriter, error) {
|
||||
cBasePath := C.CString(basePath)
|
||||
defer C.free(unsafe.Pointer(cBasePath))
|
||||
|
||||
var cas cdata.CArrowSchema
|
||||
cdata.ExportArrowSchema(schema, &cas)
|
||||
cSchema := (*C.struct_ArrowSchema)(unsafe.Pointer(&cas))
|
||||
defer cdata.ReleaseCArrowSchema(&cas)
|
||||
|
||||
if storageConfig == nil {
|
||||
storageConfig = createStorageConfig()
|
||||
}
|
||||
|
||||
pattern := strings.Join(lo.Map(columnGroups, func(columnGroup storagecommon.ColumnGroup, _ int) string {
|
||||
return strings.Join(lo.Map(columnGroup.Columns, func(index int, _ int) string {
|
||||
return schema.Field(index).Name
|
||||
}), "|")
|
||||
}), ",")
|
||||
|
||||
cProperties, err := MakePropertiesFromStorageConfig(storageConfig, map[string]string{
|
||||
PropertyWriterPolicy: "schema_based",
|
||||
PropertyWriterSchemaBasedPattern: pattern,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var writerHandle C.WriterHandle
|
||||
|
||||
result := C.writer_new(cBasePath, cSchema, cProperties, &writerHandle)
|
||||
|
||||
err = HandleFFIResult(result)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &FFIPackedWriter{
|
||||
basePath: basePath,
|
||||
cWriterHandle: writerHandle,
|
||||
cProperties: cProperties,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (pw *FFIPackedWriter) WriteRecordBatch(recordBatch arrow.Record) error {
|
||||
var caa cdata.CArrowArray
|
||||
var cas cdata.CArrowSchema
|
||||
|
||||
// Export the record batch to C Arrow format
|
||||
cdata.ExportArrowRecordBatch(recordBatch, &caa, &cas)
|
||||
defer cdata.ReleaseCArrowArray(&caa)
|
||||
defer cdata.ReleaseCArrowSchema(&cas)
|
||||
|
||||
// Convert to C struct
|
||||
cArray := (*C.struct_ArrowArray)(unsafe.Pointer(&caa))
|
||||
|
||||
result := C.writer_write(pw.cWriterHandle, cArray)
|
||||
return HandleFFIResult(result)
|
||||
}
|
||||
|
||||
func (pw *FFIPackedWriter) Close() (string, error) {
|
||||
var manifest *C.char
|
||||
|
||||
result := C.writer_close(pw.cWriterHandle, nil, nil, 0, &manifest)
|
||||
if err := HandleFFIResult(result); err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
cBasePath := C.CString(pw.basePath)
|
||||
defer C.free(unsafe.Pointer(cBasePath))
|
||||
var transationHandle C.TransactionHandle
|
||||
result = C.transaction_begin(cBasePath, pw.cProperties, &transationHandle)
|
||||
if err := HandleFFIResult(result); err != nil {
|
||||
return "", err
|
||||
}
|
||||
defer C.transaction_destroy(transationHandle)
|
||||
|
||||
// #define LOON_TRANSACTION_UPDATE_ADDFILES 0
|
||||
// #define LOON_TRANSACTION_UPDATE_ADDFEILD 1
|
||||
// #define LOON_TRANSACTION_UPDATE_MAX 2
|
||||
|
||||
// #define LOON_TRANSACTION_RESOLVE_FAIL 0
|
||||
// #define LOON_TRANSACTION_RESOLVE_MERGE 1
|
||||
// #define LOON_TRANSACTION_RESOLVE_MAX 2
|
||||
|
||||
var commitResult C.bool
|
||||
result = C.transaction_commit(transationHandle, C.int16_t(0), C.int16_t(0), manifest, &commitResult)
|
||||
if err := HandleFFIResult(result); err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
defer C.transaction_destroy(transationHandle)
|
||||
|
||||
var readVersion C.int64_t
|
||||
|
||||
// TODO: not atomic, need to get version from transaction
|
||||
var cOutManifest *C.char
|
||||
result = C.get_latest_column_groups(cBasePath, pw.cProperties, &cOutManifest, &readVersion)
|
||||
if err := HandleFFIResult(result); err != nil {
|
||||
return "", err
|
||||
}
|
||||
outManifest := C.GoString(cOutManifest)
|
||||
log.Info("FFI writer closed with output manifest", zap.String("manifest", outManifest), zap.Int64("version", int64(readVersion)))
|
||||
|
||||
defer C.properties_free(pw.cProperties)
|
||||
return MarshalManifestPath(pw.basePath, int64(readVersion)), nil
|
||||
}
|
||||
@ -34,6 +34,12 @@ type PackedWriter struct {
|
||||
cPackedWriter C.CPackedWriter
|
||||
}
|
||||
|
||||
type FFIPackedWriter struct {
|
||||
basePath string
|
||||
cWriterHandle C.WriterHandle
|
||||
cProperties *C.Properties
|
||||
}
|
||||
|
||||
type PackedReader struct {
|
||||
cPackedReader C.CPackedReader
|
||||
arr *cdata.CArrowArray
|
||||
|
||||
@ -682,6 +682,7 @@ message CompactionSegmentBinlogs {
|
||||
int64 partitionID = 8;
|
||||
bool is_sorted = 9;
|
||||
int64 storage_version = 10;
|
||||
string manifest = 11;
|
||||
}
|
||||
|
||||
message CompactionPlan {
|
||||
@ -725,6 +726,7 @@ message CompactionSegment {
|
||||
repeated FieldBinlog bm25logs = 9;
|
||||
int64 storage_version = 10;
|
||||
map<int64, data.TextIndexStats> text_stats_logs = 11;
|
||||
string manifest = 12;
|
||||
}
|
||||
|
||||
message CompactionPlanResult {
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@ -110,6 +110,7 @@ message CreateJobRequest {
|
||||
int64 lack_binlog_rows = 30;
|
||||
repeated data.FieldBinlog insert_logs = 31;
|
||||
repeated common.KeyValuePair plugin_context = 32;
|
||||
string manifest = 33;
|
||||
}
|
||||
|
||||
message QueryJobsRequest {
|
||||
|
||||
@ -292,6 +292,7 @@ type CreateJobRequest struct {
|
||||
LackBinlogRows int64 `protobuf:"varint,30,opt,name=lack_binlog_rows,json=lackBinlogRows,proto3" json:"lack_binlog_rows,omitempty"`
|
||||
InsertLogs []*datapb.FieldBinlog `protobuf:"bytes,31,rep,name=insert_logs,json=insertLogs,proto3" json:"insert_logs,omitempty"`
|
||||
PluginContext []*commonpb.KeyValuePair `protobuf:"bytes,32,rep,name=plugin_context,json=pluginContext,proto3" json:"plugin_context,omitempty"`
|
||||
Manifest string `protobuf:"bytes,33,opt,name=manifest,proto3" json:"manifest,omitempty"`
|
||||
}
|
||||
|
||||
func (x *CreateJobRequest) Reset() {
|
||||
@ -550,6 +551,13 @@ func (x *CreateJobRequest) GetPluginContext() []*commonpb.KeyValuePair {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *CreateJobRequest) GetManifest() string {
|
||||
if x != nil {
|
||||
return x.Manifest
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type QueryJobsRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
@ -2221,7 +2229,7 @@ var file_worker_proto_rawDesc = []byte{
|
||||
0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61,
|
||||
0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
|
||||
0x3a, 0x02, 0x38, 0x01, 0x22, 0x95, 0x0b, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a,
|
||||
0x3a, 0x02, 0x38, 0x01, 0x22, 0xb1, 0x0b, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a,
|
||||
0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75,
|
||||
0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c,
|
||||
0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78,
|
||||
@ -2310,52 +2318,114 @@ var file_worker_proto_rawDesc = []byte{
|
||||
0x74, 0x65, 0x78, 0x74, 0x18, 0x20, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
|
||||
0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0d, 0x70,
|
||||
0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x22, 0x4a, 0x0a, 0x10,
|
||||
0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18,
|
||||
0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52,
|
||||
0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x11, 0x51, 0x75, 0x65,
|
||||
0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33,
|
||||
0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f,
|
||||
0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61,
|
||||
0x74, 0x75, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49,
|
||||
0x44, 0x12, 0x42, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73,
|
||||
0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65,
|
||||
0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78,
|
||||
0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x49, 0x0a, 0x0f, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62,
|
||||
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73,
|
||||
0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75,
|
||||
0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44,
|
||||
0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73,
|
||||
0x22, 0x14, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xea, 0x02, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x4a, 0x6f,
|
||||
0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33,
|
||||
0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f,
|
||||
0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61,
|
||||
0x74, 0x75, 0x73, 0x12, 0x22, 0x0a, 0x0d, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x6a, 0x6f, 0x62,
|
||||
0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x74, 0x6f, 0x74, 0x61,
|
||||
0x6c, 0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x2d, 0x0a, 0x13, 0x69, 0x6e, 0x5f, 0x70, 0x72,
|
||||
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73,
|
||||
0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75,
|
||||
0x65, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||
0x0d, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x27,
|
||||
0x0a, 0x0f, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
|
||||
0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62,
|
||||
0x6c, 0x65, 0x53, 0x6c, 0x6f, 0x74, 0x73, 0x12, 0x38, 0x0a, 0x09, 0x6a, 0x6f, 0x62, 0x5f, 0x69,
|
||||
0x6e, 0x66, 0x6f, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e,
|
||||
0x4a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x6a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f,
|
||||
0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x69, 0x73, 0x6b,
|
||||
0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x69,
|
||||
0x73, 0x6b, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
|
||||
0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x53, 0x6c,
|
||||
0x6f, 0x74, 0x73, 0x22, 0xbd, 0x07, 0x0a, 0x0e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52,
|
||||
0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x1a, 0x0a, 0x08,
|
||||
0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x18, 0x21, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
|
||||
0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x22, 0x4a, 0x0a, 0x10, 0x51, 0x75, 0x65, 0x72,
|
||||
0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09,
|
||||
0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61,
|
||||
0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73,
|
||||
0x6b, 0x49, 0x44, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x11, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f,
|
||||
0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74,
|
||||
0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
|
||||
0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12,
|
||||
0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x42, 0x0a,
|
||||
0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03,
|
||||
0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73,
|
||||
0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f,
|
||||
0x73, 0x22, 0x49, 0x0a, 0x0f, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71,
|
||||
0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49,
|
||||
0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72,
|
||||
0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20,
|
||||
0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x22, 0x14, 0x0a, 0x12,
|
||||
0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x22, 0xea, 0x02, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61,
|
||||
0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74,
|
||||
0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
|
||||
0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12,
|
||||
0x22, 0x0a, 0x0d, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x4a, 0x6f, 0x62,
|
||||
0x4e, 0x75, 0x6d, 0x12, 0x2d, 0x0a, 0x13, 0x69, 0x6e, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65,
|
||||
0x73, 0x73, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x10, 0x69, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x6f, 0x62, 0x4e,
|
||||
0x75, 0x6d, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x6a, 0x6f,
|
||||
0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x65, 0x6e, 0x71,
|
||||
0x75, 0x65, 0x75, 0x65, 0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x27, 0x0a, 0x0f, 0x61, 0x76,
|
||||
0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x73, 0x18, 0x05, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x0e, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x6c,
|
||||
0x6f, 0x74, 0x73, 0x12, 0x38, 0x0a, 0x09, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73,
|
||||
0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x49,
|
||||
0x6e, 0x66, 0x6f, 0x52, 0x08, 0x6a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x1f, 0x0a,
|
||||
0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x69, 0x73, 0x6b, 0x18, 0x07, 0x20, 0x01,
|
||||
0x28, 0x08, 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x69, 0x73, 0x6b, 0x12, 0x1f,
|
||||
0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x73, 0x18, 0x08, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x53, 0x6c, 0x6f, 0x74, 0x73, 0x22,
|
||||
0xbd, 0x07, 0x0a, 0x0e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44,
|
||||
0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c,
|
||||
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c,
|
||||
0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b,
|
||||
0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28,
|
||||
0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18,
|
||||
0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||
0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x66, 0x69, 0x65, 0x6c,
|
||||
0x64, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x65,
|
||||
0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f,
|
||||
0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61,
|
||||
0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64,
|
||||
0x54, 0x79, 0x70, 0x65, 0x12, 0x59, 0x0a, 0x0d, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f,
|
||||
0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78,
|
||||
0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e,
|
||||
0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x79, 0x52, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12,
|
||||
0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x0a, 0x0e, 0x73, 0x74, 0x6f,
|
||||
0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x0a, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f,
|
||||
0x6e, 0x66, 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e,
|
||||
0x66, 0x69, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x03, 0x64, 0x69, 0x6d, 0x12, 0x2f, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x74, 0x72, 0x61,
|
||||
0x69, 0x6e, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x0c, 0x20,
|
||||
0x01, 0x28, 0x01, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x54, 0x72, 0x61, 0x69, 0x6e, 0x53, 0x69, 0x7a,
|
||||
0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x75, 0x6d, 0x5f, 0x63, 0x6c,
|
||||
0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x6e, 0x75,
|
||||
0x6d, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x36, 0x0a, 0x05, 0x66, 0x69, 0x65,
|
||||
0x6c, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46,
|
||||
0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x66, 0x69, 0x65, 0x6c,
|
||||
0x64, 0x12, 0x33, 0x0a, 0x16, 0x6d, 0x69, 0x6e, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72,
|
||||
0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x0f, 0x20, 0x01, 0x28,
|
||||
0x01, 0x52, 0x13, 0x6d, 0x69, 0x6e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x69, 0x7a,
|
||||
0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x33, 0x0a, 0x16, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6c,
|
||||
0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f,
|
||||
0x18, 0x10, 0x20, 0x01, 0x28, 0x01, 0x52, 0x13, 0x6d, 0x61, 0x78, 0x43, 0x6c, 0x75, 0x73, 0x74,
|
||||
0x65, 0x72, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x28, 0x0a, 0x10, 0x6d,
|
||||
0x61, 0x78, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18,
|
||||
0x11, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65,
|
||||
0x72, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x73, 0x6c,
|
||||
0x6f, 0x74, 0x18, 0x12, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b, 0x53, 0x6c,
|
||||
0x6f, 0x74, 0x12, 0x48, 0x0a, 0x0e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x6f, 0x6e,
|
||||
0x74, 0x65, 0x78, 0x74, 0x18, 0x13, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
|
||||
0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0d, 0x70,
|
||||
0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x1a, 0x61, 0x0a, 0x11,
|
||||
0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03,
|
||||
0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53,
|
||||
0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22,
|
||||
0xc8, 0x0b, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65,
|
||||
0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74,
|
||||
0x65, 0x72, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02,
|
||||
@ -2364,384 +2434,323 @@ var file_worker_proto_rawDesc = []byte{
|
||||
0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44,
|
||||
0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18,
|
||||
0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x05, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09,
|
||||
0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0a, 0x66, 0x69,
|
||||
0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63,
|
||||
0x68, 0x65, 0x6d, 0x61, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x66,
|
||||
0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x59, 0x0a, 0x0d, 0x73, 0x65, 0x67, 0x6d,
|
||||
0x65, 0x6e, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32,
|
||||
0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
|
||||
0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75,
|
||||
0x65, 0x73, 0x74, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73,
|
||||
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74,
|
||||
0x61, 0x74, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09,
|
||||
0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x0a,
|
||||
0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18,
|
||||
0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61,
|
||||
0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67,
|
||||
0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, 0x18, 0x0b,
|
||||
0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x64, 0x69, 0x6d, 0x12, 0x2f, 0x0a, 0x14, 0x6d, 0x61, 0x78,
|
||||
0x5f, 0x74, 0x72, 0x61, 0x69, 0x6e, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69,
|
||||
0x6f, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x01, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x54, 0x72, 0x61, 0x69,
|
||||
0x6e, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x75,
|
||||
0x6d, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x0b, 0x6e, 0x75, 0x6d, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x36, 0x0a,
|
||||
0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65,
|
||||
0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05,
|
||||
0x66, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x6d, 0x69, 0x6e, 0x5f, 0x63, 0x6c, 0x75,
|
||||
0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18,
|
||||
0x0f, 0x20, 0x01, 0x28, 0x01, 0x52, 0x13, 0x6d, 0x69, 0x6e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65,
|
||||
0x72, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x33, 0x0a, 0x16, 0x6d, 0x61,
|
||||
0x78, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72,
|
||||
0x61, 0x74, 0x69, 0x6f, 0x18, 0x10, 0x20, 0x01, 0x28, 0x01, 0x52, 0x13, 0x6d, 0x61, 0x78, 0x43,
|
||||
0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12,
|
||||
0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73,
|
||||
0x69, 0x7a, 0x65, 0x18, 0x11, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x43, 0x6c,
|
||||
0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73,
|
||||
0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x12, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x74, 0x61,
|
||||
0x73, 0x6b, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x48, 0x0a, 0x0e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e,
|
||||
0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x13, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f,
|
||||
0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69,
|
||||
0x72, 0x52, 0x0d, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74,
|
||||
0x1a, 0x61, 0x0a, 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73,
|
||||
0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,
|
||||
0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x65, 0x67, 0x6d,
|
||||
0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a,
|
||||
0x02, 0x38, 0x01, 0x22, 0xc8, 0x0b, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x74,
|
||||
0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c,
|
||||
0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63,
|
||||
0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b,
|
||||
0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44,
|
||||
0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44,
|
||||
0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||
0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
|
||||
0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69,
|
||||
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74,
|
||||
0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d,
|
||||
0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x1c, 0x0a,
|
||||
0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x3f, 0x0a, 0x0b, 0x69,
|
||||
0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b,
|
||||
0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67,
|
||||
0x52, 0x0a, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x3d, 0x0a, 0x0a,
|
||||
0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b,
|
||||
0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67,
|
||||
0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x0e, 0x73,
|
||||
0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x09, 0x20,
|
||||
0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65,
|
||||
0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43,
|
||||
0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18,
|
||||
0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c,
|
||||
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63,
|
||||
0x68, 0x65, 0x6d, 0x61, 0x12, 0x3f, 0x0a, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79,
|
||||
0x70, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74,
|
||||
0x61, 0x74, 0x73, 0x53, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f,
|
||||
0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53,
|
||||
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f,
|
||||
0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12,
|
||||
0x1e, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x0d, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x12,
|
||||
0x1a, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x0e, 0x20, 0x01, 0x28,
|
||||
0x03, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x12, 0x19, 0x0a, 0x08, 0x6e,
|
||||
0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e,
|
||||
0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
|
||||
0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x10, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d,
|
||||
0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, 0x12, 0x1d, 0x0a,
|
||||
0x0a, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28,
|
||||
0x04, 0x52, 0x09, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x73, 0x12, 0x21, 0x0a, 0x0c,
|
||||
0x74, 0x61, 0x73, 0x6b, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x12, 0x20, 0x01,
|
||||
0x28, 0x03, 0x52, 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12,
|
||||
0x24, 0x0a, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x61, 0x78, 0x53, 0x69, 0x7a, 0x65,
|
||||
0x18, 0x13, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x61,
|
||||
0x78, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x31, 0x0a, 0x15, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f,
|
||||
0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x14,
|
||||
0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4a, 0x73, 0x6f, 0x6e,
|
||||
0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x40, 0x0a, 0x1d, 0x6a, 0x73, 0x6f, 0x6e,
|
||||
0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x74, 0x61, 0x6e, 0x74, 0x69,
|
||||
0x76, 0x79, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||
0x19, 0x6a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x54, 0x61, 0x6e,
|
||||
0x74, 0x69, 0x76, 0x79, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x3a, 0x0a, 0x1a, 0x6a, 0x73,
|
||||
0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x64, 0x61, 0x74,
|
||||
0x61, 0x5f, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16,
|
||||
0x6a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x44, 0x61, 0x74, 0x61,
|
||||
0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x3f, 0x0a, 0x1d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65,
|
||||
0x5f, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f,
|
||||
0x69, 0x6e, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x18, 0x17, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x65,
|
||||
0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74,
|
||||
0x73, 0x49, 0x6e, 0x53, 0x6f, 0x72, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f,
|
||||
0x73, 0x6c, 0x6f, 0x74, 0x18, 0x18, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b,
|
||||
0x53, 0x6c, 0x6f, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f,
|
||||
0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x19, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73,
|
||||
0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a,
|
||||
0x1c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x5f,
|
||||
0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x1a, 0x20,
|
||||
0x01, 0x28, 0x05, 0x52, 0x19, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x61, 0x6c,
|
||||
0x61, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x48,
|
||||
0x0a, 0x0e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74,
|
||||
0x18, 0x1b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79,
|
||||
0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0d, 0x70, 0x6c, 0x75, 0x67, 0x69,
|
||||
0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x46, 0x0a, 0x20, 0x6a, 0x73, 0x6f, 0x6e,
|
||||
0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x68, 0x72, 0x65, 0x64,
|
||||
0x64, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x1c, 0x20, 0x01,
|
||||
0x28, 0x03, 0x52, 0x1c, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4d, 0x61, 0x78,
|
||||
0x53, 0x68, 0x72, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73,
|
||||
0x12, 0x4e, 0x0a, 0x24, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x73,
|
||||
0x68, 0x72, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x5f, 0x74,
|
||||
0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x01, 0x52, 0x20,
|
||||
0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x68, 0x72, 0x65, 0x64, 0x64, 0x69,
|
||||
0x6e, 0x67, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64,
|
||||
0x12, 0x3c, 0x0a, 0x1b, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x77,
|
||||
0x72, 0x69, 0x74, 0x65, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18,
|
||||
0x1e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73,
|
||||
0x57, 0x72, 0x69, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xf8,
|
||||
0x02, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x52, 0x65,
|
||||
0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x63, 0x68, 0x61,
|
||||
0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x65,
|
||||
0x72, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67,
|
||||
0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65,
|
||||
0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x3f, 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72,
|
||||
0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61,
|
||||
0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0a, 0x69, 0x6e,
|
||||
0x73, 0x65, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x3d, 0x0a, 0x0a, 0x64, 0x65, 0x6c, 0x74,
|
||||
0x61, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61,
|
||||
0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65,
|
||||
0x6c, 0x74, 0x61, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61,
|
||||
0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32,
|
||||
0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
|
||||
0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66,
|
||||
0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69,
|
||||
0x67, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x0a, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||
0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61,
|
||||
0x12, 0x3f, 0x0a, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x18, 0x0b,
|
||||
0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53,
|
||||
0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70,
|
||||
0x65, 0x12, 0x28, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65,
|
||||
0x6e, 0x74, 0x49, 0x44, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x74, 0x61, 0x72, 0x67,
|
||||
0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73,
|
||||
0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||
0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x12, 0x1a, 0x0a, 0x08, 0x65,
|
||||
0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x65,
|
||||
0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72,
|
||||
0x6f, 0x77, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f,
|
||||
0x77, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x5f, 0x74, 0x74, 0x6c, 0x18, 0x10, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c,
|
||||
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x75, 0x72,
|
||||
0x72, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x63,
|
||||
0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x73, 0x6b,
|
||||
0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x12, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b,
|
||||
0x74, 0x61, 0x73, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x24, 0x0a, 0x0d, 0x62,
|
||||
0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x61, 0x78, 0x53, 0x69, 0x7a, 0x65, 0x18, 0x13, 0x20, 0x01,
|
||||
0x28, 0x04, 0x52, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x61, 0x78, 0x53, 0x69, 0x7a,
|
||||
0x65, 0x12, 0x31, 0x0a, 0x15, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6a, 0x73, 0x6f, 0x6e,
|
||||
0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, 0x08,
|
||||
0x52, 0x12, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53,
|
||||
0x74, 0x61, 0x74, 0x73, 0x12, 0x40, 0x0a, 0x1d, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79,
|
||||
0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x74, 0x61, 0x6e, 0x74, 0x69, 0x76, 0x79, 0x5f, 0x6d,
|
||||
0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x52, 0x19, 0x6a, 0x73, 0x6f,
|
||||
0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x54, 0x61, 0x6e, 0x74, 0x69, 0x76, 0x79,
|
||||
0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x3a, 0x0a, 0x1a, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6b,
|
||||
0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x66, 0x6f,
|
||||
0x72, 0x6d, 0x61, 0x74, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x6a, 0x73, 0x6f, 0x6e,
|
||||
0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x44, 0x61, 0x74, 0x61, 0x46, 0x6f, 0x72, 0x6d,
|
||||
0x61, 0x74, 0x12, 0x3f, 0x0a, 0x1d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6a, 0x73, 0x6f,
|
||||
0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x69, 0x6e, 0x5f, 0x73,
|
||||
0x6f, 0x72, 0x74, 0x18, 0x17, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x65, 0x6e, 0x61, 0x62, 0x6c,
|
||||
0x65, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x49, 0x6e, 0x53,
|
||||
0x6f, 0x72, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
|
||||
0x18, 0x18, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b, 0x53, 0x6c, 0x6f, 0x74,
|
||||
0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73,
|
||||
0x69, 0x6f, 0x6e, 0x18, 0x19, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61,
|
||||
0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, 0x1c, 0x63, 0x75, 0x72,
|
||||
0x72, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65,
|
||||
0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x05, 0x52,
|
||||
0x19, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x49, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x0a, 0x0e, 0x70, 0x6c,
|
||||
0x75, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x1b, 0x20, 0x03,
|
||||
0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75,
|
||||
0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0d, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e,
|
||||
0x74, 0x65, 0x78, 0x74, 0x12, 0x46, 0x0a, 0x20, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61,
|
||||
0x74, 0x73, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x68, 0x72, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67,
|
||||
0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c,
|
||||
0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4d, 0x61, 0x78, 0x53, 0x68, 0x72, 0x65,
|
||||
0x64, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x4e, 0x0a, 0x24,
|
||||
0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x73, 0x68, 0x72, 0x65, 0x64,
|
||||
0x64, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x73,
|
||||
0x68, 0x6f, 0x6c, 0x64, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x01, 0x52, 0x20, 0x6a, 0x73, 0x6f, 0x6e,
|
||||
0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x68, 0x72, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x61,
|
||||
0x74, 0x69, 0x6f, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x3c, 0x0a, 0x1b,
|
||||
0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65,
|
||||
0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28,
|
||||
0x03, 0x52, 0x17, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x57, 0x72, 0x69, 0x74,
|
||||
0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xf8, 0x02, 0x0a, 0x12, 0x43,
|
||||
0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12,
|
||||
0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||
0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74,
|
||||
0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a,
|
||||
0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12,
|
||||
0x4d, 0x0a, 0x0f, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e,
|
||||
0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0e,
|
||||
0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4b,
|
||||
0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18,
|
||||
0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74,
|
||||
0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x69,
|
||||
0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x0d, 0x73,
|
||||
0x74, 0x61, 0x74, 0x73, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x06, 0x20, 0x01,
|
||||
0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x74,
|
||||
0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x74,
|
||||
0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x84, 0x01, 0x0a, 0x12, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a,
|
||||
0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09,
|
||||
0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61,
|
||||
0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73,
|
||||
0x6b, 0x49, 0x44, 0x73, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65,
|
||||
0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54,
|
||||
0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x22, 0x92, 0x03, 0x0a,
|
||||
0x0d, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18,
|
||||
0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||
0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74,
|
||||
0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x49, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12,
|
||||
0x26, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6b, 0x65,
|
||||
0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x46,
|
||||
0x69, 0x6c, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x72, 0x69, 0x61,
|
||||
0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04,
|
||||
0x52, 0x0e, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65,
|
||||
0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18,
|
||||
0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f,
|
||||
0x6e, 0x12, 0x32, 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x64,
|
||||
0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05,
|
||||
0x52, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65,
|
||||
0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x13, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73,
|
||||
0x74, 0x6f, 0x72, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01,
|
||||
0x28, 0x03, 0x52, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x56, 0x65,
|
||||
0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x65, 0x6d, 0x5f, 0x73, 0x69, 0x7a,
|
||||
0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x6d, 0x65, 0x6d, 0x53, 0x69, 0x7a, 0x65,
|
||||
0x12, 0x3f, 0x0a, 0x1c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x63, 0x61, 0x6c,
|
||||
0x61, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e,
|
||||
0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x19, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x53,
|
||||
0x63, 0x61, 0x6c, 0x61, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f,
|
||||
0x6e, 0x22, 0x4e, 0x0a, 0x0f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73,
|
||||
0x75, 0x6c, 0x74, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18,
|
||||
0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78,
|
||||
0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74,
|
||||
0x73, 0x22, 0xa3, 0x01, 0x0a, 0x0d, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73,
|
||||
0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73,
|
||||
0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e,
|
||||
0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12,
|
||||
0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e,
|
||||
0x12, 0x25, 0x0a, 0x0e, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x5f, 0x66, 0x69,
|
||||
0x6c, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f,
|
||||
0x69, 0x64, 0x73, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x4d, 0x0a, 0x0e, 0x41, 0x6e, 0x61, 0x6c, 0x79,
|
||||
0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x72, 0x65, 0x73,
|
||||
0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e,
|
||||
0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72,
|
||||
0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xdd, 0x06, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x74, 0x73,
|
||||
0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44,
|
||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x32,
|
||||
0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e,
|
||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64,
|
||||
0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61,
|
||||
0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f,
|
||||
0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61,
|
||||
0x73, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f,
|
||||
0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
|
||||
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69,
|
||||
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61,
|
||||
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67,
|
||||
0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65,
|
||||
0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e,
|
||||
0x65, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65,
|
||||
0x6c, 0x12, 0x3f, 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x73,
|
||||
0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64,
|
||||
0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0a, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x4c, 0x6f,
|
||||
0x67, 0x73, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, 0x67, 0x73,
|
||||
0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64,
|
||||
0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67,
|
||||
0x73, 0x12, 0x5a, 0x0a, 0x0f, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f,
|
||||
0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e,
|
||||
0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x54, 0x65, 0x78, 0x74,
|
||||
0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d,
|
||||
0x74, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x19, 0x0a,
|
||||
0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||
0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x3b, 0x0a, 0x09, 0x62, 0x6d, 0x32, 0x35,
|
||||
0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e,
|
||||
0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x08, 0x62, 0x6d, 0x32,
|
||||
0x35, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x64, 0x0a, 0x13, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6b, 0x65,
|
||||
0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0d, 0x20, 0x03,
|
||||
0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73,
|
||||
0x75, 0x6c, 0x74, 0x2e, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73,
|
||||
0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x6a, 0x73, 0x6f, 0x6e, 0x4b,
|
||||
0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x1a, 0x63, 0x0a, 0x12, 0x54,
|
||||
0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03,
|
||||
0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x65, 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78,
|
||||
0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01,
|
||||
0x1a, 0x64, 0x0a, 0x15, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73,
|
||||
0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,
|
||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x35, 0x0a, 0x05, 0x76,
|
||||
0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4a,
|
||||
0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c,
|
||||
0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x49, 0x0a, 0x0c, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52,
|
||||
0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x39, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74,
|
||||
0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61,
|
||||
0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74,
|
||||
0x73, 0x22, 0xeb, 0x02, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56,
|
||||
0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61,
|
||||
0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e,
|
||||
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1c,
|
||||
0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x51, 0x0a, 0x11,
|
||||
0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74,
|
||||
0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64,
|
||||
0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x48, 0x00, 0x52, 0x0f,
|
||||
0x69, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12,
|
||||
0x54, 0x0a, 0x13, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x72,
|
||||
0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65,
|
||||
0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73,
|
||||
0x48, 0x00, 0x52, 0x11, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65,
|
||||
0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x4e, 0x0a, 0x11, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6a,
|
||||
0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c,
|
||||
0x74, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x4a, 0x6f, 0x62, 0x52, 0x65,
|
||||
0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x08, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22,
|
||||
0x83, 0x01, 0x0a, 0x11, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72,
|
||||
0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65,
|
||||
0x72, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x36, 0x0a, 0x08, 0x6a,
|
||||
0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e,
|
||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64,
|
||||
0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54,
|
||||
0x79, 0x70, 0x65, 0x12, 0x4d, 0x0a, 0x0f, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x72,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65,
|
||||
0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x48, 0x00, 0x52, 0x0e, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x12, 0x4b, 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x72, 0x65, 0x71, 0x75,
|
||||
0x65, 0x73, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43,
|
||||
0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48,
|
||||
0x00, 0x52, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
|
||||
0x4d, 0x0a, 0x0d, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61,
|
||||
0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00,
|
||||
0x52, 0x0c, 0x73, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x09,
|
||||
0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x84, 0x01, 0x0a, 0x12, 0x51, 0x75,
|
||||
0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18,
|
||||
0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52,
|
||||
0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f,
|
||||
0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e,
|
||||
0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65,
|
||||
0x22, 0x92, 0x03, 0x0a, 0x0d, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e,
|
||||
0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x05,
|
||||
0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f,
|
||||
0x6e, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74,
|
||||
0x61, 0x74, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c,
|
||||
0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73,
|
||||
0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04,
|
||||
0x20, 0x01, 0x28, 0x04, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64,
|
||||
0x53, 0x69, 0x7a, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61,
|
||||
0x73, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52,
|
||||
0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x32, 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74,
|
||||
0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06,
|
||||
0x20, 0x01, 0x28, 0x05, 0x52, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64,
|
||||
0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x13, 0x69, 0x6e, 0x64,
|
||||
0x65, 0x78, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e,
|
||||
0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f,
|
||||
0x72, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x65, 0x6d,
|
||||
0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x6d, 0x65, 0x6d,
|
||||
0x53, 0x69, 0x7a, 0x65, 0x12, 0x3f, 0x0a, 0x1c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f,
|
||||
0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72,
|
||||
0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x19, 0x63, 0x75, 0x72, 0x72,
|
||||
0x65, 0x6e, 0x74, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65,
|
||||
0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x4e, 0x0a, 0x0f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f,
|
||||
0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75,
|
||||
0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49,
|
||||
0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x72, 0x65,
|
||||
0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xa3, 0x01, 0x0a, 0x0d, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a,
|
||||
0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49,
|
||||
0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12,
|
||||
0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74,
|
||||
0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73,
|
||||
0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65,
|
||||
0x61, 0x73, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x65, 0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64,
|
||||
0x73, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x65,
|
||||
0x6e, 0x74, 0x72, 0x6f, 0x69, 0x64, 0x73, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x4d, 0x0a, 0x0e, 0x41,
|
||||
0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x3b, 0x0a,
|
||||
0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c,
|
||||
0x74, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xdd, 0x06, 0x0a, 0x0b, 0x53,
|
||||
0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61,
|
||||
0x73, 0x6b, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b,
|
||||
0x49, 0x44, 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||
0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52,
|
||||
0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72,
|
||||
0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69,
|
||||
0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
|
||||
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63,
|
||||
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70,
|
||||
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a,
|
||||
0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x63,
|
||||
0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68,
|
||||
0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x3f, 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f,
|
||||
0x6c, 0x6f, 0x67, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46,
|
||||
0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0a, 0x69, 0x6e, 0x73, 0x65,
|
||||
0x72, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f,
|
||||
0x6c, 0x6f, 0x67, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46,
|
||||
0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74,
|
||||
0x73, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x5a, 0x0a, 0x0f, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x73, 0x74,
|
||||
0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e,
|
||||
0x54, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74,
|
||||
0x72, 0x79, 0x52, 0x0d, 0x74, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67,
|
||||
0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0b, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x3b, 0x0a, 0x09,
|
||||
0x62, 0x6d, 0x32, 0x35, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32,
|
||||
0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64,
|
||||
0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52,
|
||||
0x08, 0x62, 0x6d, 0x32, 0x35, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x64, 0x0a, 0x13, 0x6a, 0x73, 0x6f,
|
||||
0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6c, 0x6f, 0x67, 0x73,
|
||||
0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74,
|
||||
0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53,
|
||||
0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x6a,
|
||||
0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x1a,
|
||||
0x63, 0x0a, 0x12, 0x54, 0x65, 0x78, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73,
|
||||
0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,
|
||||
0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x65, 0x78, 0x74, 0x49,
|
||||
0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
|
||||
0x3a, 0x02, 0x38, 0x01, 0x1a, 0x64, 0x0a, 0x15, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53,
|
||||
0x74, 0x61, 0x74, 0x73, 0x4c, 0x6f, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a,
|
||||
0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12,
|
||||
0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61,
|
||||
0x74, 0x61, 0x2e, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52,
|
||||
0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x49, 0x0a, 0x0c, 0x53, 0x74,
|
||||
0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x39, 0x0a, 0x07, 0x72, 0x65,
|
||||
0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78,
|
||||
0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, 0x65,
|
||||
0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xeb, 0x02, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a,
|
||||
0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a,
|
||||
0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e,
|
||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d,
|
||||
0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74,
|
||||
0x75, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18,
|
||||
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44,
|
||||
0x12, 0x51, 0x0a, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x72, 0x65,
|
||||
0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78,
|
||||
0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73,
|
||||
0x48, 0x00, 0x52, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x75,
|
||||
0x6c, 0x74, 0x73, 0x12, 0x54, 0x0a, 0x13, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x6a,
|
||||
0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x73,
|
||||
0x75, 0x6c, 0x74, 0x73, 0x48, 0x00, 0x52, 0x11, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x4a,
|
||||
0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x4e, 0x0a, 0x11, 0x73, 0x74, 0x61,
|
||||
0x74, 0x73, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x05,
|
||||
0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52,
|
||||
0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x4a,
|
||||
0x6f, 0x62, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x08, 0x0a, 0x06, 0x72, 0x65, 0x73,
|
||||
0x75, 0x6c, 0x74, 0x22, 0x83, 0x01, 0x0a, 0x11, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73,
|
||||
0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75,
|
||||
0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c,
|
||||
0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49,
|
||||
0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44,
|
||||
0x73, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65,
|
||||
0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x32, 0xd8, 0x07, 0x0a, 0x09, 0x49, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x50, 0x0a, 0x09, 0x43, 0x72, 0x65, 0x61, 0x74,
|
||||
0x65, 0x4a, 0x6f, 0x62, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65,
|
||||
0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
|
||||
0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x09, 0x51, 0x75, 0x65,
|
||||
0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72,
|
||||
0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65,
|
||||
0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f,
|
||||
0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62,
|
||||
0x73, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x52,
|
||||
0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02,
|
||||
0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x12, 0x36, 0x0a,
|
||||
0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32,
|
||||
0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
|
||||
0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f,
|
||||
0x62, 0x54, 0x79, 0x70, 0x65, 0x32, 0xd8, 0x07, 0x0a, 0x09, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4e,
|
||||
0x6f, 0x64, 0x65, 0x12, 0x50, 0x0a, 0x09, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62,
|
||||
0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61,
|
||||
0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53,
|
||||
0x74, 0x61, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62,
|
||||
0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65,
|
||||
0x78, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73,
|
||||
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74,
|
||||
0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61,
|
||||
0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f,
|
||||
0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a,
|
||||
0x0b, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x26, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65,
|
||||
0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71,
|
||||
0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a,
|
||||
0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
|
||||
0x52, 0x0a, 0x0a, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x25, 0x2e,
|
||||
0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f,
|
||||
0x62, 0x73, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62,
|
||||
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75,
|
||||
0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
|
||||
0x00, 0x12, 0x4e, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x23, 0x2e,
|
||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64,
|
||||
0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71,
|
||||
0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75,
|
||||
0x73, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63,
|
||||
0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69,
|
||||
0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61,
|
||||
0x73, 0x6b, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61,
|
||||
0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e,
|
||||
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72,
|
||||
0x79, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79,
|
||||
0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78,
|
||||
0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, 0x54, 0x61, 0x73, 0x6b,
|
||||
0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65,
|
||||
0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22,
|
||||
0x00, 0x12, 0x60, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73,
|
||||
0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74,
|
||||
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65,
|
||||
0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
|
||||
0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62,
|
||||
0x56, 0x32, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f,
|
||||
0x62, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
|
||||
0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x51, 0x75, 0x65,
|
||||
0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75,
|
||||
0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56,
|
||||
0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x44,
|
||||
0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44,
|
||||
0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12,
|
||||
0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e,
|
||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d,
|
||||
0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
||||
0x12, 0x52, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x25,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74,
|
||||
0x75, 0x73, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63,
|
||||
0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x33,
|
||||
0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73,
|
||||
0x6b, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73, 0x6b,
|
||||
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65,
|
||||
0x72, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
||||
0x12, 0x4e, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x23, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65,
|
||||
0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00,
|
||||
0x42, 0x33, 0x5a, 0x31, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f,
|
||||
0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x77, 0x6f, 0x72,
|
||||
0x6b, 0x65, 0x72, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
||||
@ -288,6 +288,7 @@ type commonConfig struct {
|
||||
Stv2SystemColumnIncludeClusteringKey ParamItem `refreshable:"true"`
|
||||
Stv2SplitByAvgSize ParamItem `refreshable:"true"`
|
||||
Stv2SplitAvgSizeThreshold ParamItem `refreshable:"true"`
|
||||
UseLoonFFI ParamItem `refreshable:"true"`
|
||||
|
||||
StoragePathPrefix ParamItem `refreshable:"false"`
|
||||
StorageZstdConcurrency ParamItem `refreshable:"false"`
|
||||
@ -938,6 +939,14 @@ Large numeric passwords require double quotes to avoid yaml parsing precision is
|
||||
}
|
||||
p.EnableStorageV2.Init(base.mgr)
|
||||
|
||||
p.UseLoonFFI = ParamItem{
|
||||
Key: "common.storage.useLoonFFI",
|
||||
Version: "2.6.7",
|
||||
DefaultValue: "false",
|
||||
Export: true,
|
||||
}
|
||||
p.UseLoonFFI.Init(base.mgr)
|
||||
|
||||
p.Stv2SplitSystemColumn = ParamItem{
|
||||
Key: "common.storage.stv2.splitSystemColumn.enabled",
|
||||
Version: "2.6.2",
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user