hdfs.proto 19 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674
  1. /**
  2. * Licensed to the Apache Software Foundation (ASF) under one
  3. * or more contributor license agreements. See the NOTICE file
  4. * distributed with this work for additional information
  5. * regarding copyright ownership. The ASF licenses this file
  6. * to you under the Apache License, Version 2.0 (the
  7. * "License"); you may not use this file except in compliance
  8. * with the License. You may obtain a copy of the License at
  9. *
  10. * http://www.apache.org/licenses/LICENSE-2.0
  11. *
  12. * Unless required by applicable law or agreed to in writing, software
  13. * distributed under the License is distributed on an "AS IS" BASIS,
  14. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  15. * See the License for the specific language governing permissions and
  16. * limitations under the License.
  17. */
  18. /**
  19. * These .proto interfaces are private and stable.
  20. * Please see http://wiki.apache.org/hadoop/Compatibility
  21. * for what changes are allowed for a *stable* .proto interface.
  22. */
  23. // This file contains protocol buffers that are used throughout HDFS -- i.e.
  24. // by the client, server, and data transfer protocols.
  25. option java_package = "org.apache.hadoop.hdfs.protocol.proto";
  26. option java_outer_classname = "HdfsProtos";
  27. option java_generate_equals_and_hash = true;
  28. package hadoop.hdfs;
  29. import "Security.proto";
  30. import "acl.proto";
  31. /**
  32. * Extended block idenfies a block
  33. */
  34. message ExtendedBlockProto {
  35. required string poolId = 1; // Block pool id - gloablly unique across clusters
  36. required uint64 blockId = 2; // the local id within a pool
  37. required uint64 generationStamp = 3;
  38. optional uint64 numBytes = 4 [default = 0]; // len does not belong in ebid
  39. // here for historical reasons
  40. }
  41. /**
  42. * ProvidedStorageLocation will contain the exact location in the provided
  43. storage. The path, offset and length will result in ranged read. The nonce
  44. is there to verify that you receive what you expect.
  45. */
  46. message ProvidedStorageLocationProto {
  47. required string path = 1;
  48. required int64 offset = 2;
  49. required int64 length = 3;
  50. required bytes nonce = 4;
  51. }
  52. /**
  53. * Identifies a Datanode
  54. */
  55. message DatanodeIDProto {
  56. required string ipAddr = 1; // IP address
  57. required string hostName = 2; // hostname
  58. required string datanodeUuid = 3; // UUID assigned to the Datanode. For
  59. // upgraded clusters this is the same
  60. // as the original StorageID of the
  61. // Datanode.
  62. required uint32 xferPort = 4; // data streaming port
  63. required uint32 infoPort = 5; // datanode http port
  64. required uint32 ipcPort = 6; // ipc server port
  65. optional uint32 infoSecurePort = 7 [default = 0]; // datanode https port
  66. }
  67. /**
  68. * Datanode local information
  69. */
  70. message DatanodeLocalInfoProto {
  71. required string softwareVersion = 1;
  72. required string configVersion = 2;
  73. required uint64 uptime = 3;
  74. }
  75. /**
  76. * Datanode volume information
  77. */
  78. message DatanodeVolumeInfoProto {
  79. required string path = 1;
  80. required StorageTypeProto storageType = 2;
  81. required uint64 usedSpace = 3;
  82. required uint64 freeSpace = 4;
  83. required uint64 reservedSpace = 5;
  84. required uint64 reservedSpaceForReplicas = 6;
  85. required uint64 numBlocks = 7;
  86. }
  87. /**
  88. * DatanodeInfo array
  89. */
  90. message DatanodeInfosProto {
  91. repeated DatanodeInfoProto datanodes = 1;
  92. }
  93. /**
  94. * The status of a Datanode
  95. */
  96. message DatanodeInfoProto {
  97. required DatanodeIDProto id = 1;
  98. optional uint64 capacity = 2 [default = 0];
  99. optional uint64 dfsUsed = 3 [default = 0];
  100. optional uint64 remaining = 4 [default = 0];
  101. optional uint64 blockPoolUsed = 5 [default = 0];
  102. optional uint64 lastUpdate = 6 [default = 0];
  103. optional uint32 xceiverCount = 7 [default = 0];
  104. optional string location = 8;
  105. optional uint64 nonDfsUsed = 9;
  106. enum AdminState {
  107. NORMAL = 0;
  108. DECOMMISSION_INPROGRESS = 1;
  109. DECOMMISSIONED = 2;
  110. ENTERING_MAINTENANCE = 3;
  111. IN_MAINTENANCE = 4;
  112. }
  113. optional AdminState adminState = 10 [default = NORMAL];
  114. optional uint64 cacheCapacity = 11 [default = 0];
  115. optional uint64 cacheUsed = 12 [default = 0];
  116. optional uint64 lastUpdateMonotonic = 13 [default = 0];
  117. optional string upgradeDomain = 14;
  118. optional uint64 lastBlockReportTime = 15 [default = 0];
  119. optional uint64 lastBlockReportMonotonic = 16 [default = 0];
  120. optional uint32 numBlocks = 17 [default = 0];
  121. }
  122. /**
  123. * Represents a storage available on the datanode
  124. */
  125. message DatanodeStorageProto {
  126. enum StorageState {
  127. NORMAL = 0;
  128. READ_ONLY_SHARED = 1;
  129. }
  130. required string storageUuid = 1;
  131. optional StorageState state = 2 [default = NORMAL];
  132. optional StorageTypeProto storageType = 3 [default = DISK];
  133. }
  134. message StorageReportProto {
  135. required string storageUuid = 1 [ deprecated = true ];
  136. optional bool failed = 2 [ default = false ];
  137. optional uint64 capacity = 3 [ default = 0 ];
  138. optional uint64 dfsUsed = 4 [ default = 0 ];
  139. optional uint64 remaining = 5 [ default = 0 ];
  140. optional uint64 blockPoolUsed = 6 [ default = 0 ];
  141. optional DatanodeStorageProto storage = 7; // supersedes StorageUuid
  142. optional uint64 nonDfsUsed = 8;
  143. }
  144. /**
  145. * Summary of a file or directory
  146. */
  147. message ContentSummaryProto {
  148. required uint64 length = 1;
  149. required uint64 fileCount = 2;
  150. required uint64 directoryCount = 3;
  151. required uint64 quota = 4;
  152. required uint64 spaceConsumed = 5;
  153. required uint64 spaceQuota = 6;
  154. optional StorageTypeQuotaInfosProto typeQuotaInfos = 7;
  155. optional uint64 snapshotLength = 8;
  156. optional uint64 snapshotFileCount = 9;
  157. optional uint64 snapshotDirectoryCount = 10;
  158. optional uint64 snapshotSpaceConsumed = 11;
  159. optional string erasureCodingPolicy = 12;
  160. }
  161. /**
  162. * Summary of quota usage of a directory
  163. */
  164. message QuotaUsageProto {
  165. required uint64 fileAndDirectoryCount = 1;
  166. required uint64 quota = 2;
  167. required uint64 spaceConsumed = 3;
  168. required uint64 spaceQuota = 4;
  169. optional StorageTypeQuotaInfosProto typeQuotaInfos = 5;
  170. }
  171. /**
  172. * Storage type quota and usage information of a file or directory
  173. */
  174. message StorageTypeQuotaInfosProto {
  175. repeated StorageTypeQuotaInfoProto typeQuotaInfo = 1;
  176. }
  177. message StorageTypeQuotaInfoProto {
  178. required StorageTypeProto type = 1;
  179. required uint64 quota = 2;
  180. required uint64 consumed = 3;
  181. }
  182. /**
  183. * Contains a list of paths corresponding to corrupt files and a cookie
  184. * used for iterative calls to NameNode.listCorruptFileBlocks.
  185. *
  186. */
  187. message CorruptFileBlocksProto {
  188. repeated string files = 1;
  189. required string cookie = 2;
  190. }
  191. /**
  192. * Types of recognized storage media.
  193. */
  194. enum StorageTypeProto {
  195. DISK = 1;
  196. SSD = 2;
  197. ARCHIVE = 3;
  198. RAM_DISK = 4;
  199. PROVIDED = 5;
  200. }
  201. /**
  202. * Types of recognized blocks.
  203. */
  204. enum BlockTypeProto {
  205. CONTIGUOUS = 0;
  206. STRIPED = 1;
  207. }
  208. /**
  209. * A list of storage types.
  210. */
  211. message StorageTypesProto {
  212. repeated StorageTypeProto storageTypes = 1;
  213. }
  214. /**
  215. * Block replica storage policy.
  216. */
  217. message BlockStoragePolicyProto {
  218. required uint32 policyId = 1;
  219. required string name = 2;
  220. // a list of storage types for storing the block replicas when creating a
  221. // block.
  222. required StorageTypesProto creationPolicy = 3;
  223. // A list of storage types for creation fallback storage.
  224. optional StorageTypesProto creationFallbackPolicy = 4;
  225. optional StorageTypesProto replicationFallbackPolicy = 5;
  226. }
  227. /**
  228. * A LocatedBlock gives information about a block and its location.
  229. */
  230. message LocatedBlockProto {
  231. required ExtendedBlockProto b = 1;
  232. required uint64 offset = 2; // offset of first byte of block in the file
  233. repeated DatanodeInfoProto locs = 3; // Locations ordered by proximity to client ip
  234. required bool corrupt = 4; // true if all replicas of a block are corrupt, else false
  235. // If block has few corrupt replicas, they are filtered and
  236. // their locations are not part of this object
  237. required hadoop.common.TokenProto blockToken = 5;
  238. repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
  239. repeated StorageTypeProto storageTypes = 7;
  240. repeated string storageIDs = 8;
  241. // striped block related fields
  242. optional bytes blockIndices = 9; // used for striped block to indicate block index for each storage
  243. repeated hadoop.common.TokenProto blockTokens = 10; // each internal block has a block token
  244. }
  245. message DataEncryptionKeyProto {
  246. required uint32 keyId = 1;
  247. required string blockPoolId = 2;
  248. required bytes nonce = 3;
  249. required bytes encryptionKey = 4;
  250. required uint64 expiryDate = 5;
  251. optional string encryptionAlgorithm = 6;
  252. }
  253. /**
  254. * Cipher suite.
  255. */
  256. enum CipherSuiteProto {
  257. UNKNOWN = 1;
  258. AES_CTR_NOPADDING = 2;
  259. }
  260. /**
  261. * Crypto protocol version used to access encrypted files.
  262. */
  263. enum CryptoProtocolVersionProto {
  264. UNKNOWN_PROTOCOL_VERSION = 1;
  265. ENCRYPTION_ZONES = 2;
  266. }
  267. /**
  268. * Encryption information for a file.
  269. */
  270. message FileEncryptionInfoProto {
  271. required CipherSuiteProto suite = 1;
  272. required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
  273. required bytes key = 3;
  274. required bytes iv = 4;
  275. required string keyName = 5;
  276. required string ezKeyVersionName = 6;
  277. }
  278. /**
  279. * Encryption information for an individual
  280. * file within an encryption zone
  281. */
  282. message PerFileEncryptionInfoProto {
  283. required bytes key = 1;
  284. required bytes iv = 2;
  285. required string ezKeyVersionName = 3;
  286. }
  287. /**
  288. * Encryption information for an encryption
  289. * zone
  290. */
  291. message ZoneEncryptionInfoProto {
  292. required CipherSuiteProto suite = 1;
  293. required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
  294. required string keyName = 3;
  295. optional ReencryptionInfoProto reencryptionProto = 4;
  296. }
  297. /**
  298. * Re-encryption information for an encryption zone
  299. */
  300. message ReencryptionInfoProto {
  301. required string ezKeyVersionName = 1;
  302. required uint64 submissionTime = 2;
  303. required bool canceled = 3;
  304. required int64 numReencrypted = 4;
  305. required int64 numFailures = 5;
  306. optional uint64 completionTime = 6;
  307. optional string lastFile = 7;
  308. }
  309. /**
  310. * Cipher option
  311. */
  312. message CipherOptionProto {
  313. required CipherSuiteProto suite = 1;
  314. optional bytes inKey = 2;
  315. optional bytes inIv = 3;
  316. optional bytes outKey = 4;
  317. optional bytes outIv = 5;
  318. }
  319. /**
  320. * A set of file blocks and their locations.
  321. */
  322. message LocatedBlocksProto {
  323. required uint64 fileLength = 1;
  324. repeated LocatedBlockProto blocks = 2;
  325. required bool underConstruction = 3;
  326. optional LocatedBlockProto lastBlock = 4;
  327. required bool isLastBlockComplete = 5;
  328. optional FileEncryptionInfoProto fileEncryptionInfo = 6;
  329. // Optional field for erasure coding
  330. optional ErasureCodingPolicyProto ecPolicy = 7;
  331. }
  332. /**
  333. * ECSchema options entry
  334. */
  335. message ECSchemaOptionEntryProto {
  336. required string key = 1;
  337. required string value = 2;
  338. }
  339. /**
  340. * ECSchema for erasurecoding
  341. */
  342. message ECSchemaProto {
  343. required string codecName = 1;
  344. required uint32 dataUnits = 2;
  345. required uint32 parityUnits = 3;
  346. repeated ECSchemaOptionEntryProto options = 4;
  347. }
  348. /**
  349. * EC policy state.
  350. */
  351. enum ErasureCodingPolicyState {
  352. DISABLED = 1;
  353. ENABLED = 2;
  354. REMOVED = 3;
  355. }
  356. message ErasureCodingPolicyProto {
  357. optional string name = 1;
  358. optional ECSchemaProto schema = 2;
  359. optional uint32 cellSize = 3;
  360. required uint32 id = 4; // Actually a byte - only 8 bits used
  361. optional ErasureCodingPolicyState state = 5 [default = ENABLED];
  362. }
  363. message AddErasureCodingPolicyResponseProto {
  364. required ErasureCodingPolicyProto policy = 1;
  365. required bool succeed = 2;
  366. optional string errorMsg = 3;
  367. }
  368. /**
  369. * Placeholder type for consistent HDFS operations.
  370. */
  371. message HdfsPathHandleProto {
  372. optional uint64 inodeId = 1;
  373. optional uint64 mtime = 2;
  374. optional string path = 3;
  375. }
  376. /**
  377. * Status of a file, directory or symlink
  378. * Optionally includes a file's block locations if requested by client on the rpc call.
  379. */
  380. message HdfsFileStatusProto {
  381. enum FileType {
  382. IS_DIR = 1;
  383. IS_FILE = 2;
  384. IS_SYMLINK = 3;
  385. }
  386. enum Flags {
  387. HAS_ACL = 0x01; // has ACLs
  388. HAS_CRYPT = 0x02; // encrypted
  389. HAS_EC = 0x04; // erasure coded
  390. SNAPSHOT_ENABLED = 0x08; // SNAPSHOT ENABLED
  391. }
  392. required FileType fileType = 1;
  393. required bytes path = 2; // local name of inode encoded java UTF8
  394. required uint64 length = 3;
  395. required FsPermissionProto permission = 4;
  396. required string owner = 5;
  397. required string group = 6;
  398. required uint64 modification_time = 7;
  399. required uint64 access_time = 8;
  400. // Optional fields for symlink
  401. optional bytes symlink = 9; // if symlink, target encoded java UTF8
  402. // Optional fields for file
  403. optional uint32 block_replication = 10 [default = 0]; // only 16bits used
  404. optional uint64 blocksize = 11 [default = 0];
  405. optional LocatedBlocksProto locations = 12; // suppled only if asked by client
  406. // Optional field for fileId
  407. optional uint64 fileId = 13 [default = 0]; // default as an invalid id
  408. optional int32 childrenNum = 14 [default = -1];
  409. // Optional field for file encryption
  410. optional FileEncryptionInfoProto fileEncryptionInfo = 15;
  411. optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
  412. // Optional field for erasure coding
  413. optional ErasureCodingPolicyProto ecPolicy = 17;
  414. // Set of flags
  415. optional uint32 flags = 18 [default = 0];
  416. }
  417. /**
  418. * Checksum algorithms/types used in HDFS
  419. * Make sure this enum's integer values match enum values' id properties defined
  420. * in org.apache.hadoop.util.DataChecksum.Type
  421. */
  422. enum ChecksumTypeProto {
  423. CHECKSUM_NULL = 0;
  424. CHECKSUM_CRC32 = 1;
  425. CHECKSUM_CRC32C = 2;
  426. }
  427. enum BlockChecksumTypeProto {
  428. MD5CRC = 1; // BlockChecksum obtained by taking the MD5 digest of chunk CRCs
  429. COMPOSITE_CRC = 2; // Chunk-independent CRC, optionally striped
  430. }
  431. /**
  432. * Algorithms/types denoting how block-level checksums are computed using
  433. * lower-level chunk checksums/CRCs.
  434. * These options should be kept in sync with
  435. * org.apache.hadoop.hdfs.protocol.BlockChecksumOptions.
  436. */
  437. message BlockChecksumOptionsProto {
  438. optional BlockChecksumTypeProto blockChecksumType = 1 [default = MD5CRC];
  439. // Only used if blockChecksumType specifies a striped format, such as
  440. // COMPOSITE_CRC. If so, then the blockChecksum in the response is expected
  441. // to be the concatenation of N crcs, where
  442. // N == ((requestedLength - 1) / stripedLength) + 1
  443. optional uint64 stripeLength = 2;
  444. }
  445. /**
  446. * HDFS Server Defaults
  447. */
  448. message FsServerDefaultsProto {
  449. required uint64 blockSize = 1;
  450. required uint32 bytesPerChecksum = 2;
  451. required uint32 writePacketSize = 3;
  452. required uint32 replication = 4; // Actually a short - only 16 bits used
  453. required uint32 fileBufferSize = 5;
  454. optional bool encryptDataTransfer = 6 [default = false];
  455. optional uint64 trashInterval = 7 [default = 0];
  456. optional ChecksumTypeProto checksumType = 8 [default = CHECKSUM_CRC32];
  457. optional string keyProviderUri = 9;
  458. optional uint32 policyId = 10 [default = 0];
  459. }
  460. /**
  461. * Directory listing
  462. */
  463. message DirectoryListingProto {
  464. repeated HdfsFileStatusProto partialListing = 1;
  465. required uint32 remainingEntries = 2;
  466. }
  467. /**
  468. * Status of a snapshottable directory: besides the normal information for
  469. * a directory status, also include snapshot quota, number of snapshots, and
  470. * the full path of the parent directory.
  471. */
  472. message SnapshottableDirectoryStatusProto {
  473. required HdfsFileStatusProto dirStatus = 1;
  474. // Fields specific for snapshottable directory
  475. required uint32 snapshot_quota = 2;
  476. required uint32 snapshot_number = 3;
  477. required bytes parent_fullpath = 4;
  478. }
  479. /**
  480. * Snapshottable directory listing
  481. */
  482. message SnapshottableDirectoryListingProto {
  483. repeated SnapshottableDirectoryStatusProto snapshottableDirListing = 1;
  484. }
  485. /**
  486. * Snapshot diff report entry
  487. */
  488. message SnapshotDiffReportEntryProto {
  489. required bytes fullpath = 1;
  490. required string modificationLabel = 2;
  491. optional bytes targetPath = 3;
  492. }
  493. /**
  494. * Snapshot diff report
  495. */
  496. message SnapshotDiffReportProto {
  497. // full path of the directory where snapshots were taken
  498. required string snapshotRoot = 1;
  499. required string fromSnapshot = 2;
  500. required string toSnapshot = 3;
  501. repeated SnapshotDiffReportEntryProto diffReportEntries = 4;
  502. }
  503. /**
  504. * Snapshot diff report listing entry
  505. */
  506. message SnapshotDiffReportListingEntryProto {
  507. required bytes fullpath = 1;
  508. required uint64 dirId = 2;
  509. required bool isReference = 3;
  510. optional bytes targetPath = 4;
  511. optional uint64 fileId = 5;
  512. }
  513. message SnapshotDiffReportCursorProto {
  514. required bytes startPath = 1;
  515. required int32 index = 2 [default = -1];
  516. }
  517. /**
  518. * Snapshot diff report listing
  519. */
  520. message SnapshotDiffReportListingProto {
  521. // full path of the directory where snapshots were taken
  522. repeated SnapshotDiffReportListingEntryProto modifiedEntries = 1;
  523. repeated SnapshotDiffReportListingEntryProto createdEntries = 2;
  524. repeated SnapshotDiffReportListingEntryProto deletedEntries = 3;
  525. required bool isFromEarlier = 4;
  526. optional SnapshotDiffReportCursorProto cursor = 5;
  527. }
  528. /**
  529. * Block information
  530. *
  531. * Please be wary of adding additional fields here, since INodeFiles
  532. * need to fit in PB's default max message size of 64MB.
  533. * We restrict the max # of blocks per file
  534. * (dfs.namenode.fs-limits.max-blocks-per-file), but it's better
  535. * to avoid changing this.
  536. */
  537. message BlockProto {
  538. required uint64 blockId = 1;
  539. required uint64 genStamp = 2;
  540. optional uint64 numBytes = 3 [default = 0];
  541. }
  542. /**
  543. * Information related to a snapshot
  544. * TODO: add more information
  545. */
  546. message SnapshotInfoProto {
  547. required string snapshotName = 1;
  548. required string snapshotRoot = 2;
  549. required FsPermissionProto permission = 3;
  550. required string owner = 4;
  551. required string group = 5;
  552. required string createTime = 6;
  553. // TODO: do we need access time?
  554. }
  555. /**
  556. * Rolling upgrade status
  557. */
  558. message RollingUpgradeStatusProto {
  559. required string blockPoolId = 1;
  560. optional bool finalized = 2 [default = false];
  561. }
  562. /**
  563. * A list of storage IDs.
  564. */
  565. message StorageUuidsProto {
  566. repeated string storageUuids = 1;
  567. }
  568. /**
  569. * File access permissions mode.
  570. */
  571. enum AccessModeProto {
  572. READ = 1;
  573. WRITE = 2;
  574. COPY = 3;
  575. REPLACE = 4;
  576. }
  577. /**
  578. * Secret information for the BlockKeyProto. This is not sent on the wire as
  579. * such but is used to pack a byte array and encrypted and put in
  580. * BlockKeyProto.bytes
  581. * When adding further fields, make sure they are optional as they would
  582. * otherwise not be backwards compatible.
  583. *
  584. * Note: As part of the migration from WritableUtils based tokens (aka "legacy")
  585. * to Protocol Buffers, we use the first byte to determine the type. If the
  586. * first byte is <=0 then it is a legacy token. This means that when using
  587. * protobuf tokens, the the first field sent must have a `field_number` less
  588. * than 16 to make sure that the first byte is positive. Otherwise it could be
  589. * parsed as a legacy token. See HDFS-11026 for more discussion.
  590. */
  591. message BlockTokenSecretProto {
  592. optional uint64 expiryDate = 1;
  593. optional uint32 keyId = 2;
  594. optional string userId = 3;
  595. optional string blockPoolId = 4;
  596. optional uint64 blockId = 5;
  597. repeated AccessModeProto modes = 6;
  598. repeated StorageTypeProto storageTypes = 7;
  599. repeated string storageIds = 8;
  600. }