diff options
author | nadya73 <nadya73@yandex-team.com> | 2025-01-16 20:43:51 +0300 |
---|---|---|
committer | nadya73 <nadya73@yandex-team.com> | 2025-01-16 21:51:04 +0300 |
commit | fe6a056db2edf65b3d2360161fefa3f898f874b2 (patch) | |
tree | 463aeb6b5e33f6349aaee5b642dd89dbd1691425 | |
parent | eeb6444e9c9887c23106a80d7652a80dd26422a1 (diff) | |
download | ydb-fe6a056db2edf65b3d2360161fefa3f898f874b2.tar.gz |
YT-23932: Change protobuf type of YPath and RichYPath parameters: `string` -> `bytes`
commit_hash:0f38393e527b8e50c5216bc1147a03ddf23de5ba
-rw-r--r-- | yt/yt_proto/yt/client/api/rpc_proxy/proto/api_service.proto | 188 |
1 files changed, 94 insertions, 94 deletions
diff --git a/yt/yt_proto/yt/client/api/rpc_proxy/proto/api_service.proto b/yt/yt_proto/yt/client/api/rpc_proxy/proto/api_service.proto index f3309a4f6a..7cfd7994d0 100644 --- a/yt/yt_proto/yt/client/api/rpc_proxy/proto/api_service.proto +++ b/yt/yt_proto/yt/client/api/rpc_proxy/proto/api_service.proto @@ -480,7 +480,7 @@ message TRspAbortTransaction message TReqLookupRows { - required string path = 1; + required bytes path = 1; // YPath optional uint64 timestamp = 3 [default = 0x3fffffffffffff01]; optional uint64 retention_timestamp = 10 [default = 0]; @@ -523,7 +523,7 @@ message TRetentionConfig message TReqVersionedLookupRows { - required string path = 1; + required bytes path = 1; // YPath optional uint64 timestamp = 3 [default = 0x3fffffffffffff01]; @@ -557,7 +557,7 @@ message TReqMultiLookup { message TSubrequest { - required string path = 1; + required bytes path = 1; // YPath repeated string columns = 2; optional bool keep_missing_rows = 3 [default = true]; @@ -632,7 +632,7 @@ message TReqSelectRows optional int32 max_subqueries = 10; optional bool allow_full_scan = 11; optional bool allow_join_without_index = 12; - optional string udf_registry_path = 13; + optional bytes udf_registry_path = 13; // YPath optional uint64 memory_limit_per_node = 14; optional string execution_pool = 15; optional EReplicaConsistency replica_consistency = 17; @@ -662,8 +662,8 @@ message TRspSelectRows message TReqAdvanceQueueConsumer { optional NYT.NProto.TGuid transaction_id = 1; - optional string consumer_path = 2; - optional string queue_path = 3; + optional bytes consumer_path = 2; // RichYPath + optional bytes queue_path = 3; // RichYPath optional int32 partition_index = 4; optional int64 old_offset = 5; optional int64 new_offset = 6; @@ -678,8 +678,8 @@ message TRspAdvanceQueueConsumer message TReqPushQueueProducer { optional NYT.NProto.TGuid transaction_id = 1; - optional string producer_path = 2; - optional string queue_path = 3; + optional bytes producer_path = 2; // RichYPath + optional bytes queue_path = 3; // RichYPath optional string session_id = 4; optional int64 epoch = 5; optional bytes user_meta = 6; // YSON @@ -705,7 +705,7 @@ message TRowBatchReadOptions message TReqPullQueue { - optional string queue_path = 1; + optional bytes queue_path = 1; // RichYPath optional int64 offset = 2; optional int32 partition_index = 3; optional TRowBatchReadOptions row_batch_read_options = 4; @@ -722,8 +722,8 @@ message TRspPullQueue message TReqPullQueueConsumer { - optional string consumer_path = 1; - optional string queue_path = 2; + optional bytes consumer_path = 1; // RichYPath + optional bytes queue_path = 2; // RichYPath optional int64 offset = 3; optional int32 partition_index = 4; optional TRowBatchReadOptions row_batch_read_options = 5; @@ -741,8 +741,8 @@ message TRspPullQueueConsumer message TReqRegisterQueueConsumer { - optional string queue_path = 1; - optional string consumer_path = 2; + optional bytes queue_path = 1; // RichYPath + optional bytes consumer_path = 2; // RichYPath optional bool vital = 3; // The most reasonable way to make a std::optional<std::vector<int>> field. @@ -761,8 +761,8 @@ message TRspRegisterQueueConsumer message TReqUnregisterQueueConsumer { - optional string queue_path = 1; - optional string consumer_path = 2; + optional bytes queue_path = 1; // RichYPath + optional bytes consumer_path = 2; // RichYPath } message TRspUnregisterQueueConsumer @@ -773,16 +773,16 @@ message TRspUnregisterQueueConsumer message TReqListQueueConsumerRegistrations { - optional string queue_path = 1; - optional string consumer_path = 2; + optional bytes queue_path = 1; // RichYPath + optional bytes consumer_path = 2; // RichYPath } message TRspListQueueConsumerRegistrations { message TQueueConsumerRegistration { - optional string queue_path = 1; - optional string consumer_path = 2; + optional bytes queue_path = 1; // RichYPath + optional bytes consumer_path = 2; // RichYPath optional bool vital = 3; // The most reasonable way to make a std::optional<std::vector<int>> field. @@ -800,8 +800,8 @@ message TRspListQueueConsumerRegistrations message TReqCreateQueueProducerSession { - optional string producer_path = 1; - optional string queue_path = 2; + optional bytes producer_path = 1; // RichYPath + optional bytes queue_path = 2; // RichYPath optional string session_id = 3; optional bytes user_meta = 4; // YSON } @@ -817,8 +817,8 @@ message TRspCreateQueueProducerSession message TReqRemoveQueueProducerSession { - optional string producer_path = 1; - optional string queue_path = 2; + optional bytes producer_path = 1; // RichYPath + optional bytes queue_path = 2; // RichYPath optional string session_id = 3; } @@ -839,7 +839,7 @@ message TReqExplainQuery optional int32 max_subqueries = 7; optional bool allow_full_scan = 8; optional bool allow_join_without_index = 9; - optional string udf_registry_path = 10; + optional bytes udf_registry_path = 10; // YPath optional string execution_pool = 11; optional bool new_range_inference = 12; optional int32 syntax_version = 13 [default = 1]; @@ -861,7 +861,7 @@ message TReplicationRowIndex message TReqPullRows { - required string path = 1; + required bytes path = 1; // YPath required NYT.NProto.TGuid upstream_replica_id = 2; required bool order_rows_by_timestamp = 3; @@ -885,7 +885,7 @@ message TRspPullRows message TReqGetInSyncReplicas { - required string path = 1; + required bytes path = 1; // YPath optional uint64 timestamp = 2 [default = 0]; @@ -903,7 +903,7 @@ message TRspGetInSyncReplicas message TReqGetTabletInfos { - required string path = 1; + required bytes path = 1; // YPath repeated int32 tablet_indexes = 2; optional bool request_errors = 3 [default = false]; } @@ -940,7 +940,7 @@ message TRspGetTabletInfos message TReqGetTabletErrors { - required string path = 1; + required bytes path = 1; // YPath optional int64 limit = 2; } @@ -978,7 +978,7 @@ message TReqModifyRows optional int64 sequence_number_source_id = 9; required NYT.NProto.TGuid transaction_id = 1; - required string path = 2; + required bytes path = 2; // YPath repeated ERowModificationType row_modification_types = 3; // COMPAT(lukyan): Remove after RPC protocol version update repeated uint32 row_legacy_read_locks = 7; @@ -1239,7 +1239,7 @@ message TPrerequisiteOptions message TRevisionPrerequisite { - required string path = 2; // YPath + required bytes path = 2; // YPath required uint64 revision = 3; reserved 1; // former transaction_id } @@ -1310,7 +1310,7 @@ message TRspGenerateTimestamps message TReqExistsNode { - required string path = 1; // YPath + required bytes path = 1; // YPath optional TTransactionalOptions transactional_options = 100; optional TPrerequisiteOptions prerequisite_options = 101; @@ -1327,7 +1327,7 @@ message TRspExistsNode message TReqGetNode { - required string path = 1; // YPath + required bytes path = 1; // YPath // COMPAT(max42): see comment around TLegacyAttributeKeys. optional TLegacyAttributeKeys legacy_attributes = 2; @@ -1355,7 +1355,7 @@ message TRspGetNode message TReqListNode { - required string path = 1; // YPath + required bytes path = 1; // YPath // COMPAT(max42): see comment around TLegacyAttributeKeys. optional TLegacyAttributeKeys legacy_attributes = 2; @@ -1442,7 +1442,7 @@ message TReplicaInfo { required NYT.NProto.TGuid replica_id = 1; required string cluster_name = 2; - required string replica_path = 3; + required bytes replica_path = 3; // YPath required int32 mode = 4; // ETableReplicaMode } @@ -1457,7 +1457,7 @@ message TIndexInfo message TReqGetTableMountInfo { - required string path = 1; + required bytes path = 1; // RichYPath } message TRspGetTableMountInfo @@ -1468,7 +1468,7 @@ message TRspGetTableMountInfo required bool dynamic = 4; required NYT.NProto.TGuid upstream_replica_id = 5; repeated TReplicaInfo replicas = 6; - optional string physical_path = 7; + optional bytes physical_path = 7; // YPath repeated TIndexInfo indices = 8; optional int32 tablet_count = 9; } @@ -1477,7 +1477,7 @@ message TRspGetTableMountInfo message TReqGetTablePivotKeys { - required string path = 1; + required bytes path = 1; // YPath optional bool represent_key_as_list = 2; } @@ -1490,8 +1490,8 @@ message TRspGetTablePivotKeys message TTableBackupManifest { - required string source_path = 1; // YPath - required string destination_path = 2; // YPath + required bytes source_path = 1; // YPath + required bytes destination_path = 2; // YPath required int32 ordered_mode = 3; } @@ -1541,7 +1541,7 @@ message TRspRestoreTableBackup message TReqCreateNode { - required string path = 1; // YPath + required bytes path = 1; // YPath required int32 type = 2; // NObjectClient::EObjectType optional NYT.NYTree.NProto.TAttributeDictionary attributes = 3; @@ -1565,7 +1565,7 @@ message TRspCreateNode message TReqRemoveNode { - required string path = 1; // YPath + required bytes path = 1; // YPath optional bool recursive = 2 [default = true]; optional bool force = 3 [default = false]; @@ -1583,7 +1583,7 @@ message TRspRemoveNode message TReqSetNode { - required string path = 1; // YPath + required bytes path = 1; // YPath required bytes value = 2; // YSON optional bool recursive = 3; optional bool force = 4; @@ -1602,7 +1602,7 @@ message TRspSetNode message TReqMultisetAttributesNode { - required string path = 1; // YPath + required bytes path = 1; // YPath message TSubrequest { @@ -1626,7 +1626,7 @@ message TRspMultisetAttributesNode message TReqLockNode { - required string path = 1; // YPath + required bytes path = 1; // YPath required int32 mode = 2; // NCypressClient::ELockMode optional bool waitable = 3 [default = false]; @@ -1649,7 +1649,7 @@ message TRspLockNode message TReqUnlockNode { - required string path = 1; // YPath + required bytes path = 1; // YPath optional TTransactionalOptions transactional_options = 100; optional TPrerequisiteOptions prerequisite_options = 101; @@ -1664,8 +1664,8 @@ message TRspUnlockNode message TReqCopyNode { - required string src_path = 1; // YPath - required string dst_path = 2; // YPath + required bytes src_path = 1; // YPath + required bytes dst_path = 2; // YPath optional bool recursive = 3 [default = false]; optional bool force = 4 [default = false]; @@ -1695,8 +1695,8 @@ message TRspCopyNode message TReqMoveNode { - required string src_path = 1; // YPath - required string dst_path = 2; // YPath + required bytes src_path = 1; // YPath + required bytes dst_path = 2; // YPath optional bool recursive = 3 [default = false]; optional bool force = 4 [default = false]; @@ -1724,8 +1724,8 @@ message TRspMoveNode message TReqLinkNode { - required string src_path = 1; // YPath - required string dst_path = 2; // YPath + required bytes src_path = 1; // YPath + required bytes dst_path = 2; // YPath optional bool recursive = 3 [default = false]; optional bool force = 4 [default = false]; @@ -1746,8 +1746,8 @@ message TRspLinkNode message TReqConcatenateNodes { - repeated string src_paths = 1; // YPath - required string dst_path = 2; // YPath + repeated bytes src_paths = 1; // YPath + required bytes dst_path = 2; // YPath message TFetcher { @@ -1769,7 +1769,7 @@ message TRspConcatenateNodes message TReqExternalizeNode { - required string path = 1; // YPath + required bytes path = 1; // YPath required int32 cell_tag = 2; optional TTransactionalOptions transactional_options = 100; @@ -1783,7 +1783,7 @@ message TRspExternalizeNode message TReqInternalizeNode { - required string path = 1; // YPath + required bytes path = 1; // YPath optional TTransactionalOptions transactional_options = 100; } @@ -1838,7 +1838,7 @@ message TRspDetachTransaction message TReqMountTable { - required string path = 1; // YPath + required bytes path = 1; // YPath optional NYT.NProto.TGuid cell_id = 2; optional bool freeze = 3 [default = false]; @@ -1857,7 +1857,7 @@ message TRspMountTable message TReqUnmountTable { - required string path = 1; // YPath + required bytes path = 1; // YPath optional bool force = 2 [default = false]; @@ -1873,7 +1873,7 @@ message TRspUnmountTable message TReqRemountTable { - required string path = 1; // YPath + required bytes path = 1; // YPath optional TMutatingOptions mutating_options = 103; optional TTabletRangeOptions tablet_range_options = 104; @@ -1887,7 +1887,7 @@ message TRspRemountTable message TReqFreezeTable { - required string path = 1; // YPath + required bytes path = 1; // YPath optional TMutatingOptions mutating_options = 103; optional TTabletRangeOptions tablet_range_options = 104; @@ -1901,7 +1901,7 @@ message TRspFreezeTable message TReqUnfreezeTable { - required string path = 1; // YPath + required bytes path = 1; // YPath optional TMutatingOptions mutating_options = 103; optional TTabletRangeOptions tablet_range_options = 104; @@ -1915,7 +1915,7 @@ message TRspUnfreezeTable message TReqReshardTable { - required string path = 1; // YPath + required bytes path = 1; // YPath optional int32 tablet_count = 3; @@ -1938,7 +1938,7 @@ message TRspReshardTable message TReqReshardTableAutomatic { - required string path = 1; // YPath + required bytes path = 1; // YPath required bool keep_actions = 2 [default = false]; optional TMutatingOptions mutating_options = 103; @@ -1954,7 +1954,7 @@ message TRspReshardTableAutomatic message TReqTrimTable { - required string path = 1; // YPath + required bytes path = 1; // YPath required int32 tablet_index = 2; required uint64 trimmed_row_count = 3; @@ -1969,7 +1969,7 @@ message TRspTrimTable message TReqAlterTable { - required string path = 1; // YPath + required bytes path = 1; // YPath optional bytes schema = 2; // YSON optional bool dynamic = 3; @@ -1998,7 +1998,7 @@ message TReqAlterTableReplica optional bool preserve_timestamps = 4; optional EAtomicity atomicity = 5; optional bool enable_replicated_table_tracker = 6; - optional string replica_path = 7; + optional bytes replica_path = 7; // YPath optional TMutatingOptions mutating_options = 100; } @@ -2040,7 +2040,7 @@ message TFetcherConfig message TReqGetColumnarStatistics { - repeated string paths = 1; // RichYPath + repeated bytes paths = 1; // RichYPath optional TFetchChunkSpecConfig fetch_chunk_spec_config = 2; optional TFetcherConfig fetcher_config = 3; optional EColumnarStatisticsFetcherMode fetcher_mode = 4; @@ -2135,7 +2135,7 @@ message TReqPartitionTables optional int32 max_slices_per_fetch = 1; } - repeated string paths = 1; // RichYPath + repeated bytes paths = 1; // RichYPath optional TFetchChunkSpecConfig fetch_chunk_spec_config = 2; optional TFetcherConfig fetcher_config = 3; optional TChunkSliceFetcherConfig chunk_slice_fetcher_config = 4; @@ -2150,7 +2150,7 @@ message TReqPartitionTables message TMultiTablePartition { - repeated string table_ranges = 1; // RichYPath + repeated bytes table_ranges = 1; // RichYPath message TStatistics { @@ -2190,7 +2190,7 @@ message TRspBalanceTabletCells message TReqGetFileFromCache { required string md5 = 1; - required string cache_path = 2; // YPath + required bytes cache_path = 2; // YPath optional TTransactionalOptions transactional_options = 100; optional TMasterReadOptions master_read_options = 102; @@ -2205,9 +2205,9 @@ message TRspGetFileFromCache message TReqPutFileToCache { - required string path = 1; // YPath + required bytes path = 1; // YPath required string md5 = 2; - required string cache_path = 3; // YPath + required bytes cache_path = 3; // YPath optional bool preserve_expiration_timeout = 4 [default = false]; optional TTransactionalOptions transactional_options = 100; @@ -2538,7 +2538,7 @@ message TRspGetJobFailContext message TReqDumpJobContext { required NYT.NProto.TGuid job_id = 1; - required string path = 2; + required bytes path = 2; // YPath } message TRspDumpJobContext @@ -2588,7 +2588,7 @@ message TReqDumpJobProxyLog { required NYT.NProto.TGuid job_id = 1; required NYT.NProto.TGuid operation_id = 2; - required string path = 3; // YPath + required bytes path = 3; // YPath } message TRspDumpJobProxyLog @@ -2601,7 +2601,7 @@ message TRspDumpJobProxyLog message TReqReadFile { - required string path = 1; // YPath + required bytes path = 1; // YPath optional int64 offset = 2; optional int64 length = 3; @@ -2626,7 +2626,7 @@ message TRspReadFile message TReqWriteFile { - required string path = 1; // RichYPath + required bytes path = 1; // RichYPath optional bool compute_md5 = 2 [default = false]; optional bytes config = 3; // YSON-serialized TFileWriterConfig @@ -2643,7 +2643,7 @@ message TRspWriteFile message TReqReadJournal { - required string path = 1; // YPath + required bytes path = 1; // YPath optional int64 first_row_index = 2; optional int64 row_count = 3; @@ -2661,7 +2661,7 @@ message TRspReadJournal message TReqWriteJournal { - required string path = 1; // YPath + required bytes path = 1; // YPath optional bytes config = 2; // YSON-serialized TJournalWriterConfig optional bool enable_multiplexing = 3 [default = true]; @@ -2680,7 +2680,7 @@ message TRspWriteJournal message TReqTruncateJournal { - required string path = 1; // YPath + required bytes path = 1; // YPath required int64 row_count = 2; optional TPrerequisiteOptions prerequisite_options = 101; @@ -2695,7 +2695,7 @@ message TRspTruncateJournal message TReqReadTable { - required string path = 1; // RichYPath + required bytes path = 1; // RichYPath optional bool unordered = 2 [default = false]; optional bool omit_inaccessible_columns = 3 [default = false]; @@ -2728,7 +2728,7 @@ message TRspReadTable message TReqWriteTable { - required string path = 1; // RichYPath + required bytes path = 1; // RichYPath optional bytes config = 2; // YSON-serialized TTableWriterConfig @@ -2795,7 +2795,7 @@ message TRspRemoveMember message TReqCheckPermission { required string user = 1; - required string path = 2; // YSON + required bytes path = 2; // YPath required int32 permission = 3; // bit enum consisting of 7 bits optional TTransactionalOptions transactional_options = 100; @@ -2891,7 +2891,7 @@ message TRspCheckClusterLiveness message TReqGetPipelineSpec { - required string pipeline_path = 1; + required bytes pipeline_path = 1; // YPath } message TRspGetPipelineSpec @@ -2904,7 +2904,7 @@ message TRspGetPipelineSpec message TReqSetPipelineSpec { - required string pipeline_path = 1; + required bytes pipeline_path = 1; // YPath optional int64 expected_version = 2; required bytes spec = 3; // YSON // Update spec even if pipeline is not stopped, just paused. @@ -2920,7 +2920,7 @@ message TRspSetPipelineSpec message TReqGetPipelineDynamicSpec { - required string pipeline_path = 1; + required bytes pipeline_path = 1; // YPath } message TRspGetPipelineDynamicSpec @@ -2933,7 +2933,7 @@ message TRspGetPipelineDynamicSpec message TReqSetPipelineDynamicSpec { - required string pipeline_path = 1; + required bytes pipeline_path = 1; // YPath optional int64 expected_version = 2; required bytes spec = 3; // YSON } @@ -2947,7 +2947,7 @@ message TRspSetPipelineDynamicSpec message TReqStartPipeline { - required string pipeline_path = 1; + required bytes pipeline_path = 1; // YPath } message TRspStartPipeline @@ -2958,7 +2958,7 @@ message TRspStartPipeline message TReqPausePipeline { - required string pipeline_path = 1; + required bytes pipeline_path = 1; // YPath } message TRspPausePipeline @@ -2969,7 +2969,7 @@ message TRspPausePipeline message TReqStopPipeline { - required string pipeline_path = 1; + required bytes pipeline_path = 1; // YPath } message TRspStopPipeline @@ -2980,7 +2980,7 @@ message TRspStopPipeline message TReqGetPipelineState { - required string pipeline_path = 1; + required bytes pipeline_path = 1; // YPath } message TRspGetPipelineState @@ -2993,8 +2993,8 @@ message TRspGetPipelineState message TReqGetFlowView { - required string pipeline_path = 1; - optional string view_path = 2; + required bytes pipeline_path = 1; // YPath + optional bytes view_path = 2; // YPath } message TRspGetFlowView @@ -3208,14 +3208,14 @@ message TJobTraceEvent message TGetFileFromCacheResult { - required string path = 1; // YPath + required bytes path = 1; // YPath } //////////////////////////////////////////////////////////////////////////////// message TPutFileToCacheResult { - required string path = 1; // YPath + required bytes path = 1; // YPath } //////////////////////////////////////////////////////////////////////////////// @@ -3430,7 +3430,7 @@ message TRspGetQueryTrackerInfo message TReqStartDistributedWriteSession { - required string path = 1; + required bytes path = 1; // RichYPath required int32 cookie_count = 2; optional TTransactionalOptions transactional_options = 100; |