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:
congqixia 2025-11-24 19:57:07 +08:00 committed by GitHub
parent a7275e190e
commit c01fd94a6a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
44 changed files with 3764 additions and 2655 deletions

View File

@ -1009,6 +1009,7 @@ common:
splitByAvgSize: splitByAvgSize:
enabled: false # enable split by average size policy in storage v2 enabled: false # enable split by average size policy in storage v2
threshold: 1024 # split by average size policy threshold(in bytes) 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. # 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. # 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 # This helps Milvus-CDC synchronize incremental data

View File

@ -31,6 +31,7 @@ type Params struct {
PreferSegmentSizeRatio float64 `json:"prefer_segment_size_ratio,omitempty"` PreferSegmentSizeRatio float64 `json:"prefer_segment_size_ratio,omitempty"`
BloomFilterApplyBatchSize int `json:"bloom_filter_apply_batch_size,omitempty"` BloomFilterApplyBatchSize int `json:"bloom_filter_apply_batch_size,omitempty"`
StorageConfig *indexpb.StorageConfig `json:"storage_config,omitempty"` StorageConfig *indexpb.StorageConfig `json:"storage_config,omitempty"`
UseLoonFFI bool `json:"use_loon_ffi,omitempty"`
} }
func GenParams() Params { func GenParams() Params {
@ -46,6 +47,7 @@ func GenParams() Params {
PreferSegmentSizeRatio: paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.GetAsFloat(), PreferSegmentSizeRatio: paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.GetAsFloat(),
BloomFilterApplyBatchSize: paramtable.Get().CommonCfg.BloomFilterApplyBatchSize.GetAsInt(), BloomFilterApplyBatchSize: paramtable.Get().CommonCfg.BloomFilterApplyBatchSize.GetAsInt(),
StorageConfig: CreateStorageConfig(), StorageConfig: CreateStorageConfig(),
UseLoonFFI: paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool(),
} }
} }

View 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.

View File

@ -17,11 +17,14 @@
#include <stdexcept> #include <stdexcept>
#include <string> #include <string>
#include <vector> #include <vector>
#include <nlohmann/json.hpp>
#include "common/common_type_c.h" #include "common/common_type_c.h"
#include "common/type_c.h" #include "common/type_c.h"
#include "milvus-storage/properties.h" #include "milvus-storage/properties.h"
#include "storage/loon_ffi/util.h" #include "storage/loon_ffi/util.h"
using json = nlohmann::json;
std::shared_ptr<Properties> std::shared_ptr<Properties>
MakePropertiesFromStorageConfig(CStorageConfig c_storage_config) { MakePropertiesFromStorageConfig(CStorageConfig c_storage_config) {
// Prepare key-value pairs from CStorageConfig // Prepare key-value pairs from CStorageConfig
@ -118,3 +121,65 @@ MakePropertiesFromStorageConfig(CStorageConfig c_storage_config) {
FreeFFIResult(&result); FreeFFIResult(&result);
return properties; 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());
}
}

View File

@ -15,6 +15,7 @@
#include "common/common_type_c.h" #include "common/common_type_c.h"
#include "common/type_c.h" #include "common/type_c.h"
#include "milvus-storage/ffi_c.h" #include "milvus-storage/ffi_c.h"
#include "storage/Types.h"
/** /**
* @brief Creates a shared pointer to Properties from CStorageConfig * @brief Creates a shared pointer to Properties from CStorageConfig
@ -36,3 +37,10 @@
*/ */
std::shared_ptr<Properties> std::shared_ptr<Properties>
MakePropertiesFromStorageConfig(CStorageConfig c_storage_config); 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);

View File

@ -14,7 +14,7 @@
# Update milvus-storage_VERSION for the first occurrence # Update milvus-storage_VERSION for the first occurrence
milvus_add_pkg_config("milvus-storage") milvus_add_pkg_config("milvus-storage")
set_property(DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES "") 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") set( GIT_REPOSITORY "https://github.com/milvus-io/milvus-storage.git")
message(STATUS "milvus-storage repo: ${GIT_REPOSITORY}") message(STATUS "milvus-storage repo: ${GIT_REPOSITORY}")
message(STATUS "milvus-storage version: ${milvus-storage_VERSION}") message(STATUS "milvus-storage version: ${milvus-storage_VERSION}")

View File

@ -377,6 +377,7 @@ func (t *clusteringCompactionTask) BuildCompactionRequest() (*datapb.CompactionP
Deltalogs: segInfo.GetDeltalogs(), Deltalogs: segInfo.GetDeltalogs(),
IsSorted: segInfo.GetIsSorted(), IsSorted: segInfo.GetIsSorted(),
StorageVersion: segInfo.GetStorageVersion(), StorageVersion: segInfo.GetStorageVersion(),
Manifest: segInfo.GetManifestPath(),
}) })
} }
WrapPluginContext(taskProto.GetCollectionID(), taskProto.GetSchema().GetProperties(), plan) WrapPluginContext(taskProto.GetCollectionID(), taskProto.GetSchema().GetProperties(), plan)

View File

@ -302,6 +302,7 @@ func (t *l0CompactionTask) selectFlushedSegment() ([]*SegmentInfo, []*datapb.Com
CollectionID: info.GetCollectionID(), CollectionID: info.GetCollectionID(),
PartitionID: info.GetPartitionID(), PartitionID: info.GetPartitionID(),
IsSorted: info.GetIsSorted(), IsSorted: info.GetIsSorted(),
Manifest: info.GetManifestPath(),
}) })
} }
@ -341,6 +342,7 @@ func (t *l0CompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, err
InsertChannel: segInfo.GetInsertChannel(), InsertChannel: segInfo.GetInsertChannel(),
Deltalogs: segInfo.GetDeltalogs(), Deltalogs: segInfo.GetDeltalogs(),
IsSorted: segInfo.GetIsSorted(), IsSorted: segInfo.GetIsSorted(),
Manifest: segInfo.GetManifestPath(),
}) })
segments = append(segments, segInfo) segments = append(segments, segInfo)
} }

View File

@ -402,6 +402,7 @@ func (t *mixCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, er
Deltalogs: segInfo.GetDeltalogs(), Deltalogs: segInfo.GetDeltalogs(),
IsSorted: segInfo.GetIsSorted(), IsSorted: segInfo.GetIsSorted(),
StorageVersion: segInfo.GetStorageVersion(), StorageVersion: segInfo.GetStorageVersion(),
Manifest: segInfo.GetManifestPath(),
}) })
segIDMap[segID] = segInfo.GetDeltalogs() segIDMap[segID] = segInfo.GetDeltalogs()
segments = append(segments, segInfo) segments = append(segments, segInfo)

View File

@ -1710,6 +1710,7 @@ func (m *meta) completeClusterCompactionMutation(t *datapb.CompactionTask, resul
// visible after stats and index // visible after stats and index
IsInvisible: true, IsInvisible: true,
StorageVersion: seg.GetStorageVersion(), StorageVersion: seg.GetStorageVersion(),
ManifestPath: seg.GetManifest(),
} }
segment := NewSegmentInfo(segmentInfo) segment := NewSegmentInfo(segmentInfo)
compactToSegInfos = append(compactToSegInfos, segment) compactToSegInfos = append(compactToSegInfos, segment)
@ -1810,7 +1811,8 @@ func (m *meta) completeMixCompactionMutation(
DmlPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition { DmlPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition {
return info.GetDmlPosition() return info.GetDmlPosition()
})), })),
IsSorted: compactToSegment.GetIsSorted(), IsSorted: compactToSegment.GetIsSorted(),
ManifestPath: compactToSegment.GetManifest(),
}) })
if compactToSegmentInfo.GetNumOfRows() == 0 { if compactToSegmentInfo.GetNumOfRows() == 0 {
@ -2300,6 +2302,7 @@ func (m *meta) completeSortCompactionMutation(
Deltalogs: resultSegment.GetDeltalogs(), Deltalogs: resultSegment.GetDeltalogs(),
CompactionFrom: []int64{compactFromSegID}, CompactionFrom: []int64{compactFromSegID},
IsSorted: true, IsSorted: true,
ManifestPath: resultSegment.GetManifest(),
} }
segment := NewSegmentInfo(segmentInfo) segment := NewSegmentInfo(segmentInfo)

View File

@ -344,7 +344,8 @@ func (t *clusteringCompactionTask) getScalarAnalyzeResult(ctx context.Context) e
t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows, t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows,
t.partitionID, t.collectionID, t.plan.Channel, 100, t.partitionID, t.collectionID, t.plan.Channel, 100,
storage.WithBufferSize(t.bufferSize), storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig)) storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
if err != nil { if err != nil {
return err 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.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows,
t.partitionID, t.collectionID, t.plan.Channel, 100, t.partitionID, t.collectionID, t.plan.Channel, 100,
storage.WithBufferSize(t.bufferSize), storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig)) storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
if err != nil { if err != nil {
return err 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.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows,
t.partitionID, t.collectionID, t.plan.Channel, 100, t.partitionID, t.collectionID, t.plan.Channel, 100,
storage.WithBufferSize(t.bufferSize), storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig)) storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
if err != nil { if err != nil {
return err return err
} }
@ -497,6 +500,7 @@ func (t *clusteringCompactionTask) mapping(ctx context.Context,
Deltalogs: segment.Deltalogs, Deltalogs: segment.Deltalogs,
FieldBinlogs: segment.FieldBinlogs, FieldBinlogs: segment.FieldBinlogs,
StorageVersion: segment.StorageVersion, StorageVersion: segment.StorageVersion,
Manifest: segment.GetManifest(),
} }
future := t.mappingPool.Submit(func() (any, error) { future := t.mappingPool.Submit(func() (any, error) {
err := t.mappingSegment(ctx, segmentClone) err := t.mappingSegment(ctx, segmentClone)
@ -605,17 +609,33 @@ func (t *clusteringCompactionTask) mappingSegment(
return merr.WrapErrIllegalCompactionPlan() return merr.WrapErrIllegalCompactionPlan()
} }
rr, err := storage.NewBinlogRecordReader(ctx, var rr storage.RecordReader
segment.GetFieldBinlogs(), if segment.GetManifest() != "" {
t.plan.Schema, rr, err = storage.NewManifestRecordReader(ctx,
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) { segment.GetManifest(),
return t.binlogIO.Download(ctx, paths) t.plan.Schema,
}), storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
storage.WithCollectionID(t.GetCollection()), return t.binlogIO.Download(ctx, paths)
storage.WithVersion(segment.StorageVersion), }),
storage.WithBufferSize(t.bufferSize), storage.WithCollectionID(t.GetCollection()),
storage.WithStorageConfig(t.compactionParams.StorageConfig), 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) {
return t.binlogIO.Download(ctx, paths)
}),
storage.WithCollectionID(t.GetCollection()),
storage.WithVersion(segment.StorageVersion),
storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
)
}
if err != nil { if err != nil {
log.Warn("new binlog record reader wrong", zap.Error(err)) log.Warn("new binlog record reader wrong", zap.Error(err))
return err return err
@ -902,18 +922,35 @@ func (t *clusteringCompactionTask) scalarAnalyzeSegment(
log.Warn("unsupported storage version", zap.Int64("storage version", segment.GetStorageVersion())) log.Warn("unsupported storage version", zap.Int64("storage version", segment.GetStorageVersion()))
return nil, fmt.Errorf("unsupported storage version %d", segment.GetStorageVersion()) return nil, fmt.Errorf("unsupported storage version %d", segment.GetStorageVersion())
} }
rr, err := storage.NewBinlogRecordReader(ctx, var rr storage.RecordReader
binlogs, var err error
t.plan.GetSchema(), if segment.GetManifest() != "" {
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) { rr, err = storage.NewManifestRecordReader(ctx,
return t.binlogIO.Download(ctx, paths) segment.GetManifest(),
}), t.plan.GetSchema(),
storage.WithVersion(segment.StorageVersion), storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
storage.WithBufferSize(t.bufferSize), return t.binlogIO.Download(ctx, paths)
storage.WithStorageConfig(t.compactionParams.StorageConfig), }),
storage.WithNeededFields(requiredFields), storage.WithVersion(segment.StorageVersion),
storage.WithCollectionID(t.GetCollection()), 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) {
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()),
)
}
if err != nil { if err != nil {
log.Warn("new binlog record reader wrong", zap.Error(err)) log.Warn("new binlog record reader wrong", zap.Error(err))
return make(map[interface{}]int64), err return make(map[interface{}]int64), err

View File

@ -86,7 +86,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal() {
func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2ToV2Format() { func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2ToV2Format() {
var segmentID int64 = 1001 var segmentID int64 = 1001
fBinlogs, deltalogs, _, _, _, err := s.initStorageV2Segments(10240, segmentID) fBinlogs, deltalogs, _, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
s.NoError(err) s.NoError(err)
dblobs, err := getInt64DeltaBlobs( dblobs, err := getInt64DeltaBlobs(
@ -149,7 +149,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2To
var segmentID int64 = 1001 var segmentID int64 = 1001
fBinlogs, deltalogs, _, _, _, err := s.initStorageV2Segments(10240, segmentID) fBinlogs, deltalogs, _, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
s.NoError(err) s.NoError(err)
dblobs, err := getInt64DeltaBlobs( dblobs, err := getInt64DeltaBlobs(
@ -263,6 +263,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) initStorageV2Segments(rows int,
deltas *datapb.FieldBinlog, deltas *datapb.FieldBinlog,
stats map[int64]*datapb.FieldBinlog, stats map[int64]*datapb.FieldBinlog,
bm25Stats map[int64]*datapb.FieldBinlog, bm25Stats map[int64]*datapb.FieldBinlog,
manifest string,
size int64, size int64,
err error, err error,
) { ) {

View File

@ -44,7 +44,8 @@ func mergeSortMultipleSegments(ctx context.Context,
logIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedLogIDs().GetBegin(), plan.GetPreAllocatedLogIDs().GetEnd()) logIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedLogIDs().GetBegin(), plan.GetPreAllocatedLogIDs().GetEnd())
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc) compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
writer, err := NewMultiSegmentWriter(ctx, binlogIO, compAlloc, plan.GetMaxSize(), plan.GetSchema(), compactionParams, maxRows, partitionID, collectionID, plan.GetChannel(), 4096, 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 { if err != nil {
return nil, err return nil, err
} }
@ -58,14 +59,26 @@ func mergeSortMultipleSegments(ctx context.Context,
segmentReaders := make([]storage.RecordReader, len(binlogs)) segmentReaders := make([]storage.RecordReader, len(binlogs))
segmentFilters := make([]compaction.EntityFilter, len(binlogs)) segmentFilters := make([]compaction.EntityFilter, len(binlogs))
for i, s := range binlogs { for i, s := range binlogs {
reader, err := storage.NewBinlogRecordReader(ctx, var reader storage.RecordReader
s.GetFieldBinlogs(), if s.GetManifest() != "" {
plan.GetSchema(), reader, err = storage.NewManifestRecordReader(ctx,
storage.WithCollectionID(collectionID), s.GetManifest(),
storage.WithDownloader(binlogIO.Download), plan.GetSchema(),
storage.WithVersion(s.StorageVersion), storage.WithCollectionID(collectionID),
storage.WithStorageConfig(compactionParams.StorageConfig), 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),
storage.WithDownloader(binlogIO.Download),
storage.WithVersion(s.StorageVersion),
storage.WithStorageConfig(compactionParams.StorageConfig),
)
}
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -150,7 +150,7 @@ func (t *mixCompactionTask) mergeSplit(
segIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedSegmentIDs().GetBegin(), t.plan.GetPreAllocatedSegmentIDs().GetEnd()) segIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedSegmentIDs().GetBegin(), t.plan.GetPreAllocatedSegmentIDs().GetEnd())
logIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedLogIDs().GetBegin(), t.plan.GetPreAllocatedLogIDs().GetEnd()) logIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedLogIDs().GetBegin(), t.plan.GetPreAllocatedLogIDs().GetEnd())
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc) 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 { if err != nil {
return nil, err return nil, err
} }
@ -217,14 +217,26 @@ func (t *mixCompactionTask) writeSegment(ctx context.Context,
} }
entityFilter := compaction.NewEntityFilter(delta, t.plan.GetCollectionTtl(), t.currentTime) entityFilter := compaction.NewEntityFilter(delta, t.plan.GetCollectionTtl(), t.currentTime)
reader, err := storage.NewBinlogRecordReader(ctx, var reader storage.RecordReader
seg.GetFieldBinlogs(), if seg.GetManifest() != "" {
t.plan.GetSchema(), reader, err = storage.NewManifestRecordReader(ctx,
storage.WithCollectionID(t.collectionID), seg.GetManifest(),
storage.WithDownloader(t.binlogIO.Download), t.plan.GetSchema(),
storage.WithVersion(seg.GetStorageVersion()), storage.WithCollectionID(t.collectionID),
storage.WithStorageConfig(t.compactionParams.StorageConfig), 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),
storage.WithDownloader(t.binlogIO.Download),
storage.WithVersion(seg.GetStorageVersion()),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
)
}
if err != nil { if err != nil {
log.Warn("compact wrong, failed to new insert binlogs reader", zap.Error(err)) log.Warn("compact wrong, failed to new insert binlogs reader", zap.Error(err))
return return

View File

@ -121,7 +121,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_MixToV2Format() {
v2Segments := []int64{10, 11} v2Segments := []int64{10, 11}
for _, segID := range v2Segments { for _, segID := range v2Segments {
binlogs, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc) binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
s.NoError(err) s.NoError(err)
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{ s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
CollectionID: 1, CollectionID: 1,
@ -156,7 +156,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV2Format() {
v2Segments := []int64{10, 11} v2Segments := []int64{10, 11}
for _, segID := range v2Segments { for _, segID := range v2Segments {
binlogs, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc) binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
s.NoError(err) s.NoError(err)
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{ s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
CollectionID: 1, CollectionID: 1,
@ -192,7 +192,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV1Format() {
v2Segments := []int64{10, 11} v2Segments := []int64{10, 11}
for _, segID := range v2Segments { for _, segID := range v2Segments {
binlogs, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc) binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
s.NoError(err) s.NoError(err)
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{ s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
CollectionID: 1, CollectionID: 1,
@ -307,6 +307,7 @@ func (s *MixCompactionTaskStorageV2Suite) initStorageV2Segments(rows int, seed i
deltas *datapb.FieldBinlog, deltas *datapb.FieldBinlog,
stats map[int64]*datapb.FieldBinlog, stats map[int64]*datapb.FieldBinlog,
bm25Stats map[int64]*datapb.FieldBinlog, bm25Stats map[int64]*datapb.FieldBinlog,
manifest string,
size int64, size int64,
err error, err error,
) { ) {

View File

@ -112,7 +112,7 @@ func (s *NamespaceCompactorTestSuite) setupSortedSegments() {
StorageType: "local", StorageType: "local",
RootPath: rootPath, RootPath: rootPath,
}, columnGroups) }, columnGroups)
inserts, _, _, _, _, err := bw.Write(context.Background(), pack) inserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Require().NoError(err) s.Require().NoError(err)
s.sortedSegments = append(s.sortedSegments, &datapb.CompactionSegmentBinlogs{ s.sortedSegments = append(s.sortedSegments, &datapb.CompactionSegmentBinlogs{
SegmentID: int64(i), SegmentID: int64(i),

View File

@ -120,7 +120,7 @@ func (w *MultiSegmentWriter) closeWriter() error {
return err return err
} }
fieldBinlogs, statsLog, bm25Logs := w.writer.GetLogs() fieldBinlogs, statsLog, bm25Logs, manifest := w.writer.GetLogs()
result := &datapb.CompactionSegment{ result := &datapb.CompactionSegment{
SegmentID: w.currentSegmentID, SegmentID: w.currentSegmentID,
@ -130,6 +130,7 @@ func (w *MultiSegmentWriter) closeWriter() error {
Channel: w.channel, Channel: w.channel,
Bm25Logs: lo.Values(bm25Logs), Bm25Logs: lo.Values(bm25Logs),
StorageVersion: w.storageVersion, StorageVersion: w.storageVersion,
Manifest: manifest,
} }
w.res = append(w.res, result) w.res = append(w.res, result)

View File

@ -62,6 +62,8 @@ type sortCompactionTask struct {
insertLogs []*datapb.FieldBinlog insertLogs []*datapb.FieldBinlog
storageVersion int64 storageVersion int64
segmentStorageVersion int64 segmentStorageVersion int64
manifest string
useLoonFFI bool
done chan struct{} done chan struct{}
tr *timerecord.TimeRecorder tr *timerecord.TimeRecorder
@ -132,6 +134,8 @@ func (t *sortCompactionTask) preCompact() error {
t.insertLogs = segment.GetFieldBinlogs() t.insertLogs = segment.GetFieldBinlogs()
t.storageVersion = t.compactionParams.StorageVersion t.storageVersion = t.compactionParams.StorageVersion
t.segmentStorageVersion = segment.GetStorageVersion() t.segmentStorageVersion = segment.GetStorageVersion()
t.manifest = segment.GetManifest()
t.useLoonFFI = t.compactionParams.UseLoonFFI
log.Ctx(t.ctx).Info("preCompaction analyze", log.Ctx(t.ctx).Info("preCompaction analyze",
zap.Int64("planID", t.GetPlanID()), zap.Int64("planID", t.GetPlanID()),
@ -139,6 +143,7 @@ func (t *sortCompactionTask) preCompact() error {
zap.Int64("partitionID", t.partitionID), zap.Int64("partitionID", t.partitionID),
zap.Int64("segmentID", t.segmentID), zap.Int64("segmentID", t.segmentID),
zap.Int64("storageVersion", t.storageVersion), zap.Int64("storageVersion", t.storageVersion),
zap.Bool("useLoonFFI", t.useLoonFFI),
zap.Any("compactionParams", t.compactionParams), zap.Any("compactionParams", t.compactionParams),
) )
@ -175,6 +180,7 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
}), }),
storage.WithVersion(t.storageVersion), storage.WithVersion(t.storageVersion),
storage.WithStorageConfig(t.compactionParams.StorageConfig), storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithUseLoonFFI(t.useLoonFFI),
) )
if err != nil { if err != nil {
log.Warn("sort segment wrong, unable to init segment writer", 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") 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
storage.WithVersion(t.segmentStorageVersion), // use manifest reader if manifest presents
storage.WithDownloader(t.binlogIO.Download), if t.manifest != "" {
storage.WithStorageConfig(t.compactionParams.StorageConfig), rr, err = storage.NewManifestRecordReader(ctx, t.manifest, t.plan.Schema,
storage.WithCollectionID(t.collectionID), 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 { if err != nil {
log.Warn("error creating insert binlog reader", zap.Error(err)) log.Warn("error creating insert binlog reader", zap.Error(err))
return nil, err return nil, err
@ -228,7 +245,7 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
return nil, err return nil, err
} }
binlogs, stats, bm25stats := srw.GetLogs() binlogs, stats, bm25stats, manifest := srw.GetLogs()
insertLogs := storage.SortFieldBinlogs(binlogs) insertLogs := storage.SortFieldBinlogs(binlogs)
if err := binlog.CompressFieldBinlogs(insertLogs); err != nil { if err := binlog.CompressFieldBinlogs(insertLogs); err != nil {
return nil, err return nil, err
@ -264,6 +281,7 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
Channel: t.GetChannelName(), Channel: t.GetChannelName(),
IsSorted: true, IsSorted: true,
StorageVersion: t.storageVersion, StorageVersion: t.storageVersion,
Manifest: manifest,
}, },
} }
planResult := &datapb.CompactionPlanResult{ planResult := &datapb.CompactionPlanResult{

View File

@ -261,7 +261,7 @@ func (st *statsTask) sort(ctx context.Context) ([]*datapb.FieldBinlog, error) {
return nil, err return nil, err
} }
binlogs, stats, bm25stats := srw.GetLogs() binlogs, stats, bm25stats, _ := srw.GetLogs()
insertLogs := storage.SortFieldBinlogs(binlogs) insertLogs := storage.SortFieldBinlogs(binlogs)
if err := binlog.CompressFieldBinlogs(insertLogs); err != nil { if err := binlog.CompressFieldBinlogs(insertLogs); err != nil {
return nil, err return nil, err

View File

@ -20,6 +20,7 @@ import (
"context" "context"
"encoding/base64" "encoding/base64"
"math" "math"
"path"
"github.com/apache/arrow/go/v17/arrow/array" "github.com/apache/arrow/go/v17/arrow/array"
"github.com/apache/arrow/go/v17/arrow/memory" "github.com/apache/arrow/go/v17/arrow/memory"
@ -77,6 +78,7 @@ func (bw *BulkPackWriterV2) Write(ctx context.Context, pack *SyncPack) (
deltas *datapb.FieldBinlog, deltas *datapb.FieldBinlog,
stats map[int64]*datapb.FieldBinlog, stats map[int64]*datapb.FieldBinlog,
bm25Stats map[int64]*datapb.FieldBinlog, bm25Stats map[int64]*datapb.FieldBinlog,
manifest string,
size int64, size int64,
err error, err error,
) { ) {
@ -86,7 +88,7 @@ func (bw *BulkPackWriterV2) Write(ctx context.Context, pack *SyncPack) (
return 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)) log.Error("failed to write insert data", zap.Error(err))
return return
} }
@ -125,24 +127,20 @@ func (bw *BulkPackWriterV2) getBucketName() string {
return paramtable.Get().ServiceParam.MinioCfg.BucketName.GetValue() 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 { if len(pack.insertData) == 0 {
return make(map[int64]*datapb.FieldBinlog), nil return make(map[int64]*datapb.FieldBinlog), "", nil
} }
columnGroups := bw.columnGroups columnGroups := bw.columnGroups
rec, err := bw.serializeBinlog(ctx, pack) rec, err := bw.serializeBinlog(ctx, pack)
if err != nil { if err != nil {
return nil, err return nil, "", err
} }
logs := make(map[int64]*datapb.FieldBinlog) 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) tsArray := rec.Column(common.TimeStampField).(*array.Int64)
rows := rec.Len() rows := rec.Len()
var tsFrom uint64 = math.MaxUint64 var tsFrom uint64 = math.MaxUint64
@ -175,42 +173,84 @@ 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) doWrite := func(w storage.RecordWriter) error {
if err != nil { if err = w.Write(rec); err != nil {
return nil, err return err
}
// close first the get stats & output
return w.Close()
} }
if err = w.Write(rec); err != nil {
return nil, err var manifestPath string
} if paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool() {
// close first to get compressed size k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID)
if err = w.Close(); err != nil { basePath := path.Join(bw.getRootPath(), common.SegmentInsertLogPath, k)
return nil, err w, err := storage.NewPackedRecordManifestWriter(bucketName, basePath, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
} if err != nil {
for _, columnGroup := range columnGroups { return nil, "", err
columnGroupID := columnGroup.GroupID }
logs[columnGroupID] = &datapb.FieldBinlog{ if err = doWrite(w); err != nil {
FieldID: columnGroupID, return nil, "", err
ChildFields: columnGroup.Fields, }
Binlogs: []*datapb.Binlog{ for _, columnGroup := range columnGroups {
{ columnGroupID := columnGroup.GroupID
LogSize: int64(w.GetColumnGroupWrittenCompressed(columnGroup.GroupID)), logs[columnGroupID] = &datapb.FieldBinlog{
MemorySize: int64(w.GetColumnGroupWrittenUncompressed(columnGroup.GroupID)), FieldID: columnGroupID,
LogPath: w.GetWrittenPaths(columnGroupID), ChildFields: columnGroup.Fields,
EntriesNum: w.GetWrittenRowNum(), Binlogs: []*datapb.Binlog{
TimestampFrom: tsFrom, {
TimestampTo: tsTo, LogSize: int64(w.GetColumnGroupWrittenCompressed(columnGroup.GroupID)),
MemorySize: int64(w.GetColumnGroupWrittenUncompressed(columnGroup.GroupID)),
LogPath: w.GetWrittenPaths(columnGroupID),
EntriesNum: w.GetWrittenRowNum(),
TimestampFrom: tsFrom,
TimestampTo: tsTo,
},
}, },
}, }
}
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, nil
return logs, manifestPath, nil
} }
func (bw *BulkPackWriterV2) serializeBinlog(_ context.Context, pack *SyncPack) (storage.Record, error) { func (bw *BulkPackWriterV2) serializeBinlog(_ context.Context, pack *SyncPack) (storage.Record, error) {
if len(pack.insertData) == 0 { if len(pack.insertData) == 0 {
return nil, nil return nil, nil
} }
arrowSchema, err := storage.ConvertToArrowSchema(bw.schema) arrowSchema, err := storage.ConvertToArrowSchema(bw.schema, true)
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -145,7 +145,7 @@ func (s *PackWriterV2Suite) TestPackWriterV2_Write() {
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit) 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.NoError(err)
s.Equal(gotInserts[0].Binlogs[0].GetEntriesNum(), int64(rows)) s.Equal(gotInserts[0].Binlogs[0].GetEntriesNum(), int64(rows))
s.Equal(gotInserts[0].Binlogs[0].GetLogPath(), "/tmp/insert_log/123/456/789/0/1") 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) 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) 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) 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}) 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) 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) 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)) 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) 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) 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}) 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) 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) s.Error(err)
} }

View File

@ -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. // 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, writer := NewBulkPackWriterV2(t.metacache, t.schema, t.chunkManager, t.allocator, 0,
packed.DefaultMultiPartUploadSize, t.storageConfig, columnGroups, t.writeRetryOpts...) 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 { if err != nil {
log.Warn("failed to write sync data with storage v2 format", zap.Error(err)) log.Warn("failed to write sync data with storage v2 format", zap.Error(err))
return err return err

View 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
}

View 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
}

View 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
}

View File

@ -21,6 +21,7 @@ import (
"encoding/base64" "encoding/base64"
"fmt" "fmt"
sio "io" sio "io"
"path"
"sort" "sort"
"github.com/samber/lo" "github.com/samber/lo"
@ -67,6 +68,7 @@ type rwOptions struct {
collectionID int64 collectionID int64
storageConfig *indexpb.StorageConfig storageConfig *indexpb.StorageConfig
neededFields typeutil.Set[int64] neededFields typeutil.Set[int64]
useLoonFFI bool
} }
func (o *rwOptions) validate() error { 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) { func makeBlobsReader(ctx context.Context, binlogs []*datapb.FieldBinlog, downloader downloaderFn) (ChunkedBlobsReader, error) {
if len(binlogs) == 0 { if len(binlogs) == 0 {
return func() ([]*Blob, error) { return func() ([]*Blob, error) {
@ -275,11 +283,22 @@ func NewBinlogRecordReader(ctx context.Context, binlogs []*datapb.FieldBinlog, s
return binlogs[i].GetFieldID() < binlogs[j].GetFieldID() return binlogs[i].GetFieldID() < binlogs[j].GetFieldID()
}) })
var err error binlogLists := lo.Map(binlogs, func(fieldBinlog *datapb.FieldBinlog, _ int) []*datapb.Binlog {
rr, err = NewRecordReaderFromBinlogs(binlogs, schema, rwOptions.bufferSize, rwOptions.storageConfig, pluginContext) return fieldBinlog.GetBinlogs()
if err != nil { })
return nil, err 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: default:
return nil, merr.WrapErrServiceInternal(fmt.Sprintf("unsupported storage version %d", rwOptions.version)) 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 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, func NewBinlogRecordWriter(ctx context.Context, collectionID, partitionID, segmentID UniqueID,
schema *schemapb.CollectionSchema, allocator allocator.Interface, chunkSize uint64, maxRowNum int64, schema *schemapb.CollectionSchema, allocator allocator.Interface, chunkSize uint64, maxRowNum int64,
option ...RwOption, option ...RwOption,
@ -340,12 +384,20 @@ func NewBinlogRecordWriter(ctx context.Context, collectionID, partitionID, segme
blobsWriter, allocator, chunkSize, rootPath, maxRowNum, opts..., blobsWriter, allocator, chunkSize, rootPath, maxRowNum, opts...,
) )
case StorageV2: case StorageV2:
return newPackedBinlogRecordWriter(collectionID, partitionID, segmentID, schema, if rwOptions.useLoonFFI {
blobsWriter, allocator, maxRowNum, return newPackedManifestRecordWriter(collectionID, partitionID, segmentID, schema,
rwOptions.bufferSize, rwOptions.multiPartUploadSize, rwOptions.columnGroups, blobsWriter, allocator, maxRowNum,
rwOptions.storageConfig, rwOptions.bufferSize, rwOptions.multiPartUploadSize, rwOptions.columnGroups,
pluginContext, rwOptions.storageConfig,
) pluginContext)
} else {
return newPackedBinlogRecordWriter(collectionID, partitionID, segmentID, schema,
blobsWriter, allocator, maxRowNum,
rwOptions.bufferSize, rwOptions.multiPartUploadSize, rwOptions.columnGroups,
rwOptions.storageConfig,
pluginContext,
)
}
} }
return nil, merr.WrapErrServiceInternal(fmt.Sprintf("unsupported storage version %d", rwOptions.version)) return nil, merr.WrapErrServiceInternal(fmt.Sprintf("unsupported storage version %d", rwOptions.version))
} }

View File

@ -161,7 +161,7 @@ func (s *PackedBinlogRecordSuite) TestPackedBinlogRecordIntegration() {
rowNum := w.GetRowNum() rowNum := w.GetRowNum()
s.Equal(rowNum, int64(rows)) s.Equal(rowNum, int64(rows))
fieldBinlogs, statsLog, bm25StatsLog := w.GetLogs() fieldBinlogs, statsLog, bm25StatsLog, _ := w.GetLogs()
s.Equal(len(fieldBinlogs), len(columnGroups)) s.Equal(len(fieldBinlogs), len(columnGroups))
for _, columnGroup := range fieldBinlogs { for _, columnGroup := range fieldBinlogs {
s.Equal(len(columnGroup.Binlogs), 1) s.Equal(len(columnGroup.Binlogs), 1)
@ -240,7 +240,7 @@ func (s *PackedBinlogRecordSuite) TestGenerateBM25Stats() {
s.NoError(err) s.NoError(err)
err = w.Close() err = w.Close()
s.NoError(err) s.NoError(err)
fieldBinlogs, statsLog, bm25StatsLog := w.GetLogs() fieldBinlogs, statsLog, bm25StatsLog, _ := w.GetLogs()
s.Equal(len(fieldBinlogs), len(columnGroups)) s.Equal(len(fieldBinlogs), len(columnGroups))
s.Equal(statsLog.Binlogs[0].EntriesNum, int64(1)) s.Equal(statsLog.Binlogs[0].EntriesNum, int64(1))

View File

@ -1,6 +1,7 @@
package storage package storage
import ( import (
"fmt"
"strconv" "strconv"
"github.com/apache/arrow/go/v17/arrow" "github.com/apache/arrow/go/v17/arrow"
@ -11,7 +12,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/util/typeutil" "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) fieldCount := typeutil.GetTotalFieldsNum(schema)
arrowFields := make([]arrow.Field, 0, fieldCount) arrowFields := make([]arrow.Field, 0, fieldCount)
appendArrowField := func(field *schemapb.FieldSchema) error { 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) arrowType := serdeMap[field.DataType].arrowType(dim, elementType)
arrowField := ConvertToArrowField(field, arrowType) arrowField := ConvertToArrowField(field, arrowType, useFieldID)
// Add extra metadata for ArrayOfVector // Add extra metadata for ArrayOfVector
if field.DataType == schemapb.DataType_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 return arrow.NewSchema(arrowFields, nil), nil
} }
func ConvertToArrowField(field *schemapb.FieldSchema, dataType arrow.DataType) arrow.Field { func ConvertToArrowField(field *schemapb.FieldSchema, dataType arrow.DataType, useFieldID bool) arrow.Field {
return arrow.Field{ f := arrow.Field{
Name: field.GetName(), Name: field.GetName(),
Type: dataType, Type: dataType,
Metadata: arrow.NewMetadata([]string{packed.ArrowFieldIdMetadataKey}, []string{strconv.Itoa(int(field.GetFieldID()))}), Metadata: arrow.NewMetadata([]string{packed.ArrowFieldIdMetadataKey}, []string{strconv.Itoa(int(field.GetFieldID()))}),
Nullable: field.GetNullable(), Nullable: field.GetNullable(),
} }
if useFieldID {
field.Name = fmt.Sprintf("%d", field.GetFieldID())
}
return f
} }

View File

@ -56,7 +56,7 @@ func TestConvertArrowSchema(t *testing.T) {
Fields: fieldSchemas, Fields: fieldSchemas,
StructArrayFields: StructArrayFieldSchemas, StructArrayFields: StructArrayFieldSchemas,
} }
arrowSchema, err := ConvertToArrowSchema(schema) arrowSchema, err := ConvertToArrowSchema(schema, false)
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, len(fieldSchemas)+len(StructArrayFieldSchemas[0].Fields), len(arrowSchema.Fields())) assert.Equal(t, len(fieldSchemas)+len(StructArrayFieldSchemas[0].Fields), len(arrowSchema.Fields()))
} }
@ -84,6 +84,6 @@ func TestConvertArrowSchemaWithoutDim(t *testing.T) {
schema := &schemapb.CollectionSchema{ schema := &schemapb.CollectionSchema{
Fields: fieldSchemas, Fields: fieldSchemas,
} }
_, err := ConvertToArrowSchema(schema) _, err := ConvertToArrowSchema(schema, false)
assert.Error(t, err) assert.Error(t, err)
} }

View File

@ -45,11 +45,6 @@ type Record interface {
Retain() Retain()
} }
type RecordReader interface {
Next() (Record, error)
Close() error
}
type RecordWriter interface { type RecordWriter interface {
Write(r Record) error Write(r Record) error
GetWrittenUncompressed() uint64 GetWrittenUncompressed() uint64

View File

@ -22,7 +22,6 @@ import (
"fmt" "fmt"
"io" "io"
"math" "math"
"path"
"sort" "sort"
"strconv" "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/hook"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator" "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/common"
"github.com/milvus-io/milvus/pkg/v2/log" "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/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/merr"
"github.com/milvus-io/milvus/pkg/v2/util/metautil" "github.com/milvus-io/milvus/pkg/v2/util/metautil"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil" "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) { func parseBlobKey(blobKey string) (colId FieldID, logId UniqueID) {
if _, _, _, colId, logId, ok := metautil.ParseInsertLogPath(blobKey); ok { if _, _, _, colId, logId, ok := metautil.ParseInsertLogPath(blobKey); ok {
return colId, logId return colId, logId
@ -738,25 +473,12 @@ func ValueSerializer(v []*Value, schema *schemapb.CollectionSchema) (Record, err
builder := builders[field.FieldID] builder := builders[field.FieldID]
arrays[i] = builder.NewArray() arrays[i] = builder.NewArray()
builder.Release() builder.Release()
fields[i] = ConvertToArrowField(field, arrays[i].DataType()) fields[i] = ConvertToArrowField(field, arrays[i].DataType(), false)
field2Col[field.FieldID] = i field2Col[field.FieldID] = i
} }
return NewSimpleArrowRecord(array.NewRecord(arrow.NewSchema(fields, nil), arrays, int64(len(v))), field2Col), nil 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 ChunkedBlobsWriter func([]*Blob) error
type CompositeBinlogRecordWriter struct { type CompositeBinlogRecordWriter struct {
@ -975,8 +697,9 @@ func (c *CompositeBinlogRecordWriter) GetLogs() (
fieldBinlogs map[FieldID]*datapb.FieldBinlog, fieldBinlogs map[FieldID]*datapb.FieldBinlog,
statsLog *datapb.FieldBinlog, statsLog *datapb.FieldBinlog,
bm25StatsLog map[FieldID]*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 { func (c *CompositeBinlogRecordWriter) GetRowNum() int64 {

View File

@ -177,7 +177,7 @@ func TestBinlogSerializeWriter(t *testing.T) {
err = writer.Close() err = writer.Close()
assert.NoError(t, err) assert.NoError(t, err)
logs, _, _ := writer.GetLogs() logs, _, _, _ := writer.GetLogs()
assert.Equal(t, 18, len(logs)) assert.Equal(t, 18, len(logs))
assert.Equal(t, 5, len(logs[0].Binlogs)) assert.Equal(t, 5, len(logs[0].Binlogs))
}) })

View File

@ -15,519 +15,3 @@
// limitations under the License. // limitations under the License.
package storage 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
}

View File

@ -436,7 +436,7 @@ func TestArrayOfVectorIntegration(t *testing.T) {
}, },
} }
arrowSchema, err := ConvertToArrowSchema(schema) arrowSchema, err := ConvertToArrowSchema(schema, false)
assert.NoError(t, err) assert.NoError(t, err)
assert.NotNil(t, arrowSchema) assert.NotNil(t, arrowSchema)

View 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
}

View File

@ -31,7 +31,9 @@ import (
"github.com/apache/arrow/go/v17/arrow" "github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/cdata" "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/indexcgopb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb" "github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
) )
@ -187,5 +189,31 @@ func (r *FFIPackedReader) Release() {
r.Close() 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 // Ensure FFIPackedReader implements array.RecordReader interface
// var _ array.RecordReader = (*FFIPackedReader)(nil) // var _ array.RecordReader = (*FFIPackedReader)(nil)

View 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
}

View File

@ -34,6 +34,12 @@ type PackedWriter struct {
cPackedWriter C.CPackedWriter cPackedWriter C.CPackedWriter
} }
type FFIPackedWriter struct {
basePath string
cWriterHandle C.WriterHandle
cProperties *C.Properties
}
type PackedReader struct { type PackedReader struct {
cPackedReader C.CPackedReader cPackedReader C.CPackedReader
arr *cdata.CArrowArray arr *cdata.CArrowArray

View File

@ -682,6 +682,7 @@ message CompactionSegmentBinlogs {
int64 partitionID = 8; int64 partitionID = 8;
bool is_sorted = 9; bool is_sorted = 9;
int64 storage_version = 10; int64 storage_version = 10;
string manifest = 11;
} }
message CompactionPlan { message CompactionPlan {
@ -725,6 +726,7 @@ message CompactionSegment {
repeated FieldBinlog bm25logs = 9; repeated FieldBinlog bm25logs = 9;
int64 storage_version = 10; int64 storage_version = 10;
map<int64, data.TextIndexStats> text_stats_logs = 11; map<int64, data.TextIndexStats> text_stats_logs = 11;
string manifest = 12;
} }
message CompactionPlanResult { message CompactionPlanResult {

File diff suppressed because it is too large Load Diff

View File

@ -110,6 +110,7 @@ message CreateJobRequest {
int64 lack_binlog_rows = 30; int64 lack_binlog_rows = 30;
repeated data.FieldBinlog insert_logs = 31; repeated data.FieldBinlog insert_logs = 31;
repeated common.KeyValuePair plugin_context = 32; repeated common.KeyValuePair plugin_context = 32;
string manifest = 33;
} }
message QueryJobsRequest { message QueryJobsRequest {

View File

@ -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"` 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"` 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"` 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() { func (x *CreateJobRequest) Reset() {
@ -550,6 +551,13 @@ func (x *CreateJobRequest) GetPluginContext() []*commonpb.KeyValuePair {
return nil return nil
} }
func (x *CreateJobRequest) GetManifest() string {
if x != nil {
return x.Manifest
}
return ""
}
type QueryJobsRequest struct { type QueryJobsRequest struct {
state protoimpl.MessageState state protoimpl.MessageState
sizeCache protoimpl.SizeCache 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, 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, 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, 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, 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, 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, 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, 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, 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, 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, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x1a, 0x0a, 0x08,
0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x18, 0x21, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x22, 0x4a, 0x0a, 0x10, 0x51, 0x75, 0x65, 0x72,
0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09,
0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x11, 0x51, 0x75, 0x65, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61,
0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73,
0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x6b, 0x49, 0x44, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x11, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74,
0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c,
0x74, 0x75, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12,
0x44, 0x12, 0x42, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01,
0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x42, 0x0a,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03,
0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x49, 0x0a, 0x0f, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73,
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f,
0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x22, 0x49, 0x0a, 0x0f, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71,
0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49,
0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72,
0x22, 0x14, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xea, 0x02, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x22, 0x14, 0x0a, 0x12,
0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x73, 0x74, 0x22, 0xea, 0x02, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74,
0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c,
0x74, 0x75, 0x73, 0x12, 0x22, 0x0a, 0x0d, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x6a, 0x6f, 0x62, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12,
0x6c, 0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x2d, 0x0a, 0x13, 0x69, 0x6e, 0x5f, 0x70, 0x72, 0x22, 0x0a, 0x0d, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d,
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x03, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x4a, 0x6f, 0x62,
0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4e, 0x75, 0x6d, 0x12, 0x2d, 0x0a, 0x13, 0x69, 0x6e, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65,
0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x73, 0x73, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03,
0x65, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x52, 0x10, 0x69, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x6f, 0x62, 0x4e,
0x0d, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x27, 0x75, 0x6d, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x6a, 0x6f,
0x0a, 0x0f, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x62, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x65, 0x6e, 0x71,
0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x75, 0x65, 0x75, 0x65, 0x4a, 0x6f, 0x62, 0x4e, 0x75, 0x6d, 0x12, 0x27, 0x0a, 0x0f, 0x61, 0x76,
0x6c, 0x65, 0x53, 0x6c, 0x6f, 0x74, 0x73, 0x12, 0x38, 0x0a, 0x09, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x73, 0x18, 0x05, 0x20,
0x6e, 0x66, 0x6f, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x6c,
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x6f, 0x74, 0x73, 0x12, 0x38, 0x0a, 0x09, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73,
0x4a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x6a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x69, 0x73, 0x6b, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x49,
0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x69, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x6a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x1f, 0x0a,
0x73, 0x6b, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x69, 0x73, 0x6b, 0x18, 0x07, 0x20, 0x01,
0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x53, 0x6c, 0x28, 0x08, 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x69, 0x73, 0x6b, 0x12, 0x1f,
0x6f, 0x74, 0x73, 0x22, 0xbd, 0x07, 0x0a, 0x0e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 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, 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, 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, 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, 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, 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, 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, 0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x63, 0x68, 0x61,
0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x65,
0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x72, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67,
0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0a, 0x66, 0x69, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65,
0x65, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x3f, 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x72,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d,
0x68, 0x65, 0x6d, 0x61, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x66, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61,
0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x59, 0x0a, 0x0d, 0x73, 0x65, 0x67, 0x6d, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x0a, 0x69, 0x6e,
0x65, 0x6e, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x73, 0x65, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x3d, 0x0a, 0x0a, 0x64, 0x65, 0x6c, 0x74,
0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x61, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d,
0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61,
0x65, 0x73, 0x74, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x52, 0x09, 0x64, 0x65,
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x6c, 0x74, 0x61, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61,
0x61, 0x74, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x0a, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66,
0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x0a, 0x20, 0x01, 0x28,
0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x6d, 0x18, 0x0b, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x64, 0x69, 0x6d, 0x12, 0x2f, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61,
0x5f, 0x74, 0x72, 0x61, 0x69, 0x6e, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x12, 0x3f, 0x0a, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x18, 0x0b,
0x6f, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x01, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x54, 0x72, 0x61, 0x69, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6e, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x75, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53,
0x6d, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70,
0x52, 0x0b, 0x6e, 0x75, 0x6d, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x36, 0x0a, 0x65, 0x12, 0x28, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65,
0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x74, 0x61, 0x72, 0x67,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73,
0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52,
0x66, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x6d, 0x69, 0x6e, 0x5f, 0x63, 0x6c, 0x75, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x12, 0x1a, 0x0a, 0x08, 0x65,
0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x65,
0x0f, 0x20, 0x01, 0x28, 0x01, 0x52, 0x13, 0x6d, 0x69, 0x6e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72,
0x72, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x33, 0x0a, 0x16, 0x6d, 0x61, 0x6f, 0x77, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f,
0x78, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x77, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x61, 0x74, 0x69, 0x6f, 0x18, 0x10, 0x20, 0x01, 0x28, 0x01, 0x52, 0x13, 0x6d, 0x61, 0x78, 0x43, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x10, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c,
0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x75, 0x72,
0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x63,
0x69, 0x7a, 0x65, 0x18, 0x11, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x43, 0x6c, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x73, 0x6b,
0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x12, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b,
0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x12, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x74, 0x61, 0x74, 0x61, 0x73, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x24, 0x0a, 0x0d, 0x62,
0x73, 0x6b, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x48, 0x0a, 0x0e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x61, 0x78, 0x53, 0x69, 0x7a, 0x65, 0x18, 0x13, 0x20, 0x01,
0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x13, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x28, 0x04, 0x52, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x61, 0x78, 0x53, 0x69, 0x7a,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x65, 0x12, 0x31, 0x0a, 0x15, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6a, 0x73, 0x6f, 0x6e,
0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, 0x08,
0x72, 0x52, 0x0d, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x12, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53,
0x1a, 0x61, 0x0a, 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x74, 0x61, 0x74, 0x73, 0x12, 0x40, 0x0a, 0x1d, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79,
0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x74, 0x61, 0x6e, 0x74, 0x69, 0x76, 0x79, 0x5f, 0x6d,
0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x52, 0x19, 0x6a, 0x73, 0x6f,
0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x54, 0x61, 0x6e, 0x74, 0x69, 0x76, 0x79,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x3a, 0x0a, 0x1a, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6b,
0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x66, 0x6f,
0x02, 0x38, 0x01, 0x22, 0xc8, 0x0b, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x74, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x6a, 0x73, 0x6f, 0x6e,
0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x44, 0x61, 0x74, 0x61, 0x46, 0x6f, 0x72, 0x6d,
0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x61, 0x74, 0x12, 0x3f, 0x0a, 0x1d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6a, 0x73, 0x6f,
0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x69, 0x6e, 0x5f, 0x73,
0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x6f, 0x72, 0x74, 0x18, 0x17, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x65, 0x6e, 0x61, 0x62, 0x6c,
0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x65, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x49, 0x6e, 0x53,
0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x72, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x18, 0x18, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b, 0x53, 0x6c, 0x6f, 0x74,
0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73,
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x19, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61,
0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, 0x1c, 0x63, 0x75, 0x72,
0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x1c, 0x0a, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65,
0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x05, 0x52,
0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x3f, 0x0a, 0x0b, 0x69, 0x19, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x49, 0x6e,
0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x0a, 0x0e, 0x70, 0x6c,
0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x75, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x1b, 0x20, 0x03,
0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x52, 0x0a, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x3d, 0x0a, 0x0a, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75,
0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0d, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e,
0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x46, 0x0a, 0x20, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61,
0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x68, 0x72, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67,
0x52, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x0e, 0x73, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c,
0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x09, 0x20, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4d, 0x61, 0x78, 0x53, 0x68, 0x72, 0x65,
0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x4e, 0x0a, 0x24,
0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x73, 0x68, 0x72, 0x65, 0x64,
0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x73,
0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x01, 0x52, 0x20, 0x6a, 0x73, 0x6f, 0x6e,
0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x68, 0x72, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x61,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x74, 0x69, 0x6f, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x3c, 0x0a, 0x1b,
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65,
0x68, 0x65, 0x6d, 0x61, 0x12, 0x3f, 0x0a, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28,
0x70, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x03, 0x52, 0x17, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x57, 0x72, 0x69, 0x74,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xf8, 0x02, 0x0a, 0x12, 0x43,
0x61, 0x74, 0x73, 0x53, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01,
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12,
0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52,
0x1e, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x0d, 0x20, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74,
0x01, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x12, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x1a, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a,
0x03, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x4c, 0x6f, 0x67, 0x49, 0x44, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12,
0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6e, 0x4d, 0x0a, 0x0f, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65,
0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x10, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6e,
0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, 0x12, 0x1d, 0x0a, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0e,
0x0a, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4b,
0x04, 0x52, 0x09, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18,
0x74, 0x61, 0x73, 0x6b, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x12, 0x20, 0x01, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x28, 0x03, 0x52, 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74,
0x24, 0x0a, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x61, 0x78, 0x53, 0x69, 0x7a, 0x65, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x69,
0x18, 0x13, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x4d, 0x61, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x0d, 0x73,
0x78, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x31, 0x0a, 0x15, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x06, 0x20, 0x01,
0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x14, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4a, 0x73, 0x6f, 0x6e, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x74,
0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x40, 0x0a, 0x1d, 0x6a, 0x73, 0x6f, 0x6e, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x74,
0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x74, 0x61, 0x6e, 0x74, 0x69, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65,
0x76, 0x79, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x52, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x84, 0x01, 0x0a, 0x12, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a,
0x19, 0x6a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x54, 0x61, 0x6e, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09,
0x74, 0x69, 0x76, 0x79, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x3a, 0x0a, 0x1a, 0x6a, 0x73, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x64, 0x61, 0x74, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61,
0x61, 0x5f, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73,
0x6a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x73, 0x44, 0x61, 0x74, 0x61, 0x6b, 0x49, 0x44, 0x73, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65,
0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x3f, 0x0a, 0x1d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x5f, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54,
0x69, 0x6e, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x18, 0x17, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x65, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x22, 0x92, 0x03, 0x0a,
0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x74, 0x61, 0x74, 0x0d, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18,
0x73, 0x49, 0x6e, 0x53, 0x6f, 0x72, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52,
0x73, 0x6c, 0x6f, 0x74, 0x18, 0x18, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74,
0x53, 0x6c, 0x6f, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x19, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x49, 0x6e,
0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12,
0x1c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x5f, 0x26, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6b, 0x65,
0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x1a, 0x20, 0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x46,
0x01, 0x28, 0x05, 0x52, 0x19, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x61, 0x6c, 0x69, 0x6c, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x72, 0x69, 0x61,
0x61, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04,
0x0a, 0x0e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65,
0x18, 0x1b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f,
0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0d, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x12, 0x32, 0x0a, 0x15, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x64,
0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x46, 0x0a, 0x20, 0x6a, 0x73, 0x6f, 0x6e, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05,
0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x68, 0x72, 0x65, 0x64, 0x52, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65,
0x64, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x1c, 0x20, 0x01, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x13, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73,
0x28, 0x03, 0x52, 0x1c, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4d, 0x61, 0x78, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01,
0x53, 0x68, 0x72, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x28, 0x03, 0x52, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x56, 0x65,
0x12, 0x4e, 0x0a, 0x24, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x73, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x65, 0x6d, 0x5f, 0x73, 0x69, 0x7a,
0x68, 0x72, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x5f, 0x74, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x6d, 0x65, 0x6d, 0x53, 0x69, 0x7a, 0x65,
0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x01, 0x52, 0x20, 0x12, 0x3f, 0x0a, 0x1c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x63, 0x61, 0x6c,
0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x68, 0x72, 0x65, 0x64, 0x64, 0x69, 0x61, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e,
0x6e, 0x67, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x19, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x53,
0x12, 0x3c, 0x0a, 0x1b, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x77, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f,
0x72, 0x69, 0x74, 0x65, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x6e, 0x22, 0x4e, 0x0a, 0x0f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73,
0x1e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18,
0x57, 0x72, 0x69, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xf8, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x02, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x52, 0x65, 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, 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, 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, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x18, 0x02,
0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x36, 0x0a, 0x08, 0x6a, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x73, 0x12, 0x36, 0x0a,
0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1b, 0x2e, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32,
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x54, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6a, 0x6f,
0x79, 0x70, 0x65, 0x12, 0x4d, 0x0a, 0x0f, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x5f, 0x72, 0x62, 0x54, 0x79, 0x70, 0x65, 0x32, 0xd8, 0x07, 0x0a, 0x09, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4e,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x12, 0x50, 0x0a, 0x09, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x78, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52,
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,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 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, 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, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f,
0x74, 0x61, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x62, 0x73, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62,
0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75,
0x78, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x00, 0x12, 0x4e, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x23, 0x2e,
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, 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, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22,
0x73, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73,
0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74,
0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65,
0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62,
0x73, 0x6b, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 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, 0x54, 0x61, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f,
0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x62, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x51, 0x75, 0x65,
0x79, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75,
0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x56,
0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, 0x54, 0x61, 0x73, 0x6b, 0x32, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x44,
0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x70, 0x4a, 0x6f, 0x62, 0x73, 0x56, 0x32, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 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, 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, 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, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73,
0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x6b, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73, 0x6b,
0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x33, 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 ( var (

View File

@ -288,6 +288,7 @@ type commonConfig struct {
Stv2SystemColumnIncludeClusteringKey ParamItem `refreshable:"true"` Stv2SystemColumnIncludeClusteringKey ParamItem `refreshable:"true"`
Stv2SplitByAvgSize ParamItem `refreshable:"true"` Stv2SplitByAvgSize ParamItem `refreshable:"true"`
Stv2SplitAvgSizeThreshold ParamItem `refreshable:"true"` Stv2SplitAvgSizeThreshold ParamItem `refreshable:"true"`
UseLoonFFI ParamItem `refreshable:"true"`
StoragePathPrefix ParamItem `refreshable:"false"` StoragePathPrefix ParamItem `refreshable:"false"`
StorageZstdConcurrency 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.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{ p.Stv2SplitSystemColumn = ParamItem{
Key: "common.storage.stv2.splitSystemColumn.enabled", Key: "common.storage.stv2.splitSystemColumn.enabled",
Version: "2.6.2", Version: "2.6.2",