hdfs.proto 18 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652
  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. }
  121. /**
  122. * Represents a storage available on the datanode
  123. */
  124. message DatanodeStorageProto {
  125. enum StorageState {
  126. NORMAL = 0;
  127. READ_ONLY_SHARED = 1;
  128. }
  129. required string storageUuid = 1;
  130. optional StorageState state = 2 [default = NORMAL];
  131. optional StorageTypeProto storageType = 3 [default = DISK];
  132. }
  133. message StorageReportProto {
  134. required string storageUuid = 1 [ deprecated = true ];
  135. optional bool failed = 2 [ default = false ];
  136. optional uint64 capacity = 3 [ default = 0 ];
  137. optional uint64 dfsUsed = 4 [ default = 0 ];
  138. optional uint64 remaining = 5 [ default = 0 ];
  139. optional uint64 blockPoolUsed = 6 [ default = 0 ];
  140. optional DatanodeStorageProto storage = 7; // supersedes StorageUuid
  141. optional uint64 nonDfsUsed = 8;
  142. }
  143. /**
  144. * Summary of a file or directory
  145. */
  146. message ContentSummaryProto {
  147. required uint64 length = 1;
  148. required uint64 fileCount = 2;
  149. required uint64 directoryCount = 3;
  150. required uint64 quota = 4;
  151. required uint64 spaceConsumed = 5;
  152. required uint64 spaceQuota = 6;
  153. optional StorageTypeQuotaInfosProto typeQuotaInfos = 7;
  154. optional uint64 snapshotLength = 8;
  155. optional uint64 snapshotFileCount = 9;
  156. optional uint64 snapshotDirectoryCount = 10;
  157. optional uint64 snapshotSpaceConsumed = 11;
  158. optional string erasureCodingPolicy = 12;
  159. }
  160. /**
  161. * Summary of quota usage of a directory
  162. */
  163. message QuotaUsageProto {
  164. required uint64 fileAndDirectoryCount = 1;
  165. required uint64 quota = 2;
  166. required uint64 spaceConsumed = 3;
  167. required uint64 spaceQuota = 4;
  168. optional StorageTypeQuotaInfosProto typeQuotaInfos = 5;
  169. }
  170. /**
  171. * Storage type quota and usage information of a file or directory
  172. */
  173. message StorageTypeQuotaInfosProto {
  174. repeated StorageTypeQuotaInfoProto typeQuotaInfo = 1;
  175. }
  176. message StorageTypeQuotaInfoProto {
  177. required StorageTypeProto type = 1;
  178. required uint64 quota = 2;
  179. required uint64 consumed = 3;
  180. }
  181. /**
  182. * Contains a list of paths corresponding to corrupt files and a cookie
  183. * used for iterative calls to NameNode.listCorruptFileBlocks.
  184. *
  185. */
  186. message CorruptFileBlocksProto {
  187. repeated string files = 1;
  188. required string cookie = 2;
  189. }
  190. /**
  191. * Types of recognized storage media.
  192. */
  193. enum StorageTypeProto {
  194. DISK = 1;
  195. SSD = 2;
  196. ARCHIVE = 3;
  197. RAM_DISK = 4;
  198. PROVIDED = 5;
  199. }
  200. /**
  201. * Types of recognized blocks.
  202. */
  203. enum BlockTypeProto {
  204. CONTIGUOUS = 0;
  205. STRIPED = 1;
  206. }
  207. /**
  208. * A list of storage types.
  209. */
  210. message StorageTypesProto {
  211. repeated StorageTypeProto storageTypes = 1;
  212. }
  213. /**
  214. * Block replica storage policy.
  215. */
  216. message BlockStoragePolicyProto {
  217. required uint32 policyId = 1;
  218. required string name = 2;
  219. // a list of storage types for storing the block replicas when creating a
  220. // block.
  221. required StorageTypesProto creationPolicy = 3;
  222. // A list of storage types for creation fallback storage.
  223. optional StorageTypesProto creationFallbackPolicy = 4;
  224. optional StorageTypesProto replicationFallbackPolicy = 5;
  225. }
  226. /**
  227. * A LocatedBlock gives information about a block and its location.
  228. */
  229. message LocatedBlockProto {
  230. required ExtendedBlockProto b = 1;
  231. required uint64 offset = 2; // offset of first byte of block in the file
  232. repeated DatanodeInfoProto locs = 3; // Locations ordered by proximity to client ip
  233. required bool corrupt = 4; // true if all replicas of a block are corrupt, else false
  234. // If block has few corrupt replicas, they are filtered and
  235. // their locations are not part of this object
  236. required hadoop.common.TokenProto blockToken = 5;
  237. repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
  238. repeated StorageTypeProto storageTypes = 7;
  239. repeated string storageIDs = 8;
  240. // striped block related fields
  241. optional bytes blockIndices = 9; // used for striped block to indicate block index for each storage
  242. repeated hadoop.common.TokenProto blockTokens = 10; // each internal block has a block token
  243. }
  244. message DataEncryptionKeyProto {
  245. required uint32 keyId = 1;
  246. required string blockPoolId = 2;
  247. required bytes nonce = 3;
  248. required bytes encryptionKey = 4;
  249. required uint64 expiryDate = 5;
  250. optional string encryptionAlgorithm = 6;
  251. }
  252. /**
  253. * Cipher suite.
  254. */
  255. enum CipherSuiteProto {
  256. UNKNOWN = 1;
  257. AES_CTR_NOPADDING = 2;
  258. }
  259. /**
  260. * Crypto protocol version used to access encrypted files.
  261. */
  262. enum CryptoProtocolVersionProto {
  263. UNKNOWN_PROTOCOL_VERSION = 1;
  264. ENCRYPTION_ZONES = 2;
  265. }
  266. /**
  267. * Encryption information for a file.
  268. */
  269. message FileEncryptionInfoProto {
  270. required CipherSuiteProto suite = 1;
  271. required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
  272. required bytes key = 3;
  273. required bytes iv = 4;
  274. required string keyName = 5;
  275. required string ezKeyVersionName = 6;
  276. }
  277. /**
  278. * Encryption information for an individual
  279. * file within an encryption zone
  280. */
  281. message PerFileEncryptionInfoProto {
  282. required bytes key = 1;
  283. required bytes iv = 2;
  284. required string ezKeyVersionName = 3;
  285. }
  286. /**
  287. * Encryption information for an encryption
  288. * zone
  289. */
  290. message ZoneEncryptionInfoProto {
  291. required CipherSuiteProto suite = 1;
  292. required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
  293. required string keyName = 3;
  294. optional ReencryptionInfoProto reencryptionProto = 4;
  295. }
  296. /**
  297. * Re-encryption information for an encryption zone
  298. */
  299. message ReencryptionInfoProto {
  300. required string ezKeyVersionName = 1;
  301. required uint64 submissionTime = 2;
  302. required bool canceled = 3;
  303. required int64 numReencrypted = 4;
  304. required int64 numFailures = 5;
  305. optional uint64 completionTime = 6;
  306. optional string lastFile = 7;
  307. }
  308. /**
  309. * Cipher option
  310. */
  311. message CipherOptionProto {
  312. required CipherSuiteProto suite = 1;
  313. optional bytes inKey = 2;
  314. optional bytes inIv = 3;
  315. optional bytes outKey = 4;
  316. optional bytes outIv = 5;
  317. }
  318. /**
  319. * A set of file blocks and their locations.
  320. */
  321. message LocatedBlocksProto {
  322. required uint64 fileLength = 1;
  323. repeated LocatedBlockProto blocks = 2;
  324. required bool underConstruction = 3;
  325. optional LocatedBlockProto lastBlock = 4;
  326. required bool isLastBlockComplete = 5;
  327. optional FileEncryptionInfoProto fileEncryptionInfo = 6;
  328. // Optional field for erasure coding
  329. optional ErasureCodingPolicyProto ecPolicy = 7;
  330. }
  331. /**
  332. * ECSchema options entry
  333. */
  334. message ECSchemaOptionEntryProto {
  335. required string key = 1;
  336. required string value = 2;
  337. }
  338. /**
  339. * ECSchema for erasurecoding
  340. */
  341. message ECSchemaProto {
  342. required string codecName = 1;
  343. required uint32 dataUnits = 2;
  344. required uint32 parityUnits = 3;
  345. repeated ECSchemaOptionEntryProto options = 4;
  346. }
  347. /**
  348. * EC policy state.
  349. */
  350. enum ErasureCodingPolicyState {
  351. DISABLED = 1;
  352. ENABLED = 2;
  353. REMOVED = 3;
  354. }
  355. message ErasureCodingPolicyProto {
  356. optional string name = 1;
  357. optional ECSchemaProto schema = 2;
  358. optional uint32 cellSize = 3;
  359. required uint32 id = 4; // Actually a byte - only 8 bits used
  360. optional ErasureCodingPolicyState state = 5 [default = ENABLED];
  361. }
  362. message AddErasureCodingPolicyResponseProto {
  363. required ErasureCodingPolicyProto policy = 1;
  364. required bool succeed = 2;
  365. optional string errorMsg = 3;
  366. }
  367. /**
  368. * Placeholder type for consistent HDFS operations.
  369. */
  370. message HdfsPathHandleProto {
  371. optional uint64 inodeId = 1;
  372. optional uint64 mtime = 2;
  373. optional string path = 3;
  374. }
  375. /**
  376. * Status of a file, directory or symlink
  377. * Optionally includes a file's block locations if requested by client on the rpc call.
  378. */
  379. message HdfsFileStatusProto {
  380. enum FileType {
  381. IS_DIR = 1;
  382. IS_FILE = 2;
  383. IS_SYMLINK = 3;
  384. }
  385. enum Flags {
  386. HAS_ACL = 0x01; // has ACLs
  387. HAS_CRYPT = 0x02; // encrypted
  388. HAS_EC = 0x04; // erasure coded
  389. SNAPSHOT_ENABLED = 0x08; // SNAPSHOT ENABLED
  390. }
  391. required FileType fileType = 1;
  392. required bytes path = 2; // local name of inode encoded java UTF8
  393. required uint64 length = 3;
  394. required FsPermissionProto permission = 4;
  395. required string owner = 5;
  396. required string group = 6;
  397. required uint64 modification_time = 7;
  398. required uint64 access_time = 8;
  399. // Optional fields for symlink
  400. optional bytes symlink = 9; // if symlink, target encoded java UTF8
  401. // Optional fields for file
  402. optional uint32 block_replication = 10 [default = 0]; // only 16bits used
  403. optional uint64 blocksize = 11 [default = 0];
  404. optional LocatedBlocksProto locations = 12; // suppled only if asked by client
  405. // Optional field for fileId
  406. optional uint64 fileId = 13 [default = 0]; // default as an invalid id
  407. optional int32 childrenNum = 14 [default = -1];
  408. // Optional field for file encryption
  409. optional FileEncryptionInfoProto fileEncryptionInfo = 15;
  410. optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
  411. // Optional field for erasure coding
  412. optional ErasureCodingPolicyProto ecPolicy = 17;
  413. // Set of flags
  414. optional uint32 flags = 18 [default = 0];
  415. }
  416. /**
  417. * Checksum algorithms/types used in HDFS
  418. * Make sure this enum's integer values match enum values' id properties defined
  419. * in org.apache.hadoop.util.DataChecksum.Type
  420. */
  421. enum ChecksumTypeProto {
  422. CHECKSUM_NULL = 0;
  423. CHECKSUM_CRC32 = 1;
  424. CHECKSUM_CRC32C = 2;
  425. }
  426. /**
  427. * HDFS Server Defaults
  428. */
  429. message FsServerDefaultsProto {
  430. required uint64 blockSize = 1;
  431. required uint32 bytesPerChecksum = 2;
  432. required uint32 writePacketSize = 3;
  433. required uint32 replication = 4; // Actually a short - only 16 bits used
  434. required uint32 fileBufferSize = 5;
  435. optional bool encryptDataTransfer = 6 [default = false];
  436. optional uint64 trashInterval = 7 [default = 0];
  437. optional ChecksumTypeProto checksumType = 8 [default = CHECKSUM_CRC32];
  438. optional string keyProviderUri = 9;
  439. optional uint32 policyId = 10 [default = 0];
  440. }
  441. /**
  442. * Directory listing
  443. */
  444. message DirectoryListingProto {
  445. repeated HdfsFileStatusProto partialListing = 1;
  446. required uint32 remainingEntries = 2;
  447. }
  448. /**
  449. * Status of a snapshottable directory: besides the normal information for
  450. * a directory status, also include snapshot quota, number of snapshots, and
  451. * the full path of the parent directory.
  452. */
  453. message SnapshottableDirectoryStatusProto {
  454. required HdfsFileStatusProto dirStatus = 1;
  455. // Fields specific for snapshottable directory
  456. required uint32 snapshot_quota = 2;
  457. required uint32 snapshot_number = 3;
  458. required bytes parent_fullpath = 4;
  459. }
  460. /**
  461. * Snapshottable directory listing
  462. */
  463. message SnapshottableDirectoryListingProto {
  464. repeated SnapshottableDirectoryStatusProto snapshottableDirListing = 1;
  465. }
  466. /**
  467. * Snapshot diff report entry
  468. */
  469. message SnapshotDiffReportEntryProto {
  470. required bytes fullpath = 1;
  471. required string modificationLabel = 2;
  472. optional bytes targetPath = 3;
  473. }
  474. /**
  475. * Snapshot diff report
  476. */
  477. message SnapshotDiffReportProto {
  478. // full path of the directory where snapshots were taken
  479. required string snapshotRoot = 1;
  480. required string fromSnapshot = 2;
  481. required string toSnapshot = 3;
  482. repeated SnapshotDiffReportEntryProto diffReportEntries = 4;
  483. }
  484. /**
  485. * Snapshot diff report listing entry
  486. */
  487. message SnapshotDiffReportListingEntryProto {
  488. required bytes fullpath = 1;
  489. required uint64 dirId = 2;
  490. required bool isReference = 3;
  491. optional bytes targetPath = 4;
  492. optional uint64 fileId = 5;
  493. }
  494. message SnapshotDiffReportCursorProto {
  495. required bytes startPath = 1;
  496. required int32 index = 2 [default = -1];
  497. }
  498. /**
  499. * Snapshot diff report listing
  500. */
  501. message SnapshotDiffReportListingProto {
  502. // full path of the directory where snapshots were taken
  503. repeated SnapshotDiffReportListingEntryProto modifiedEntries = 1;
  504. repeated SnapshotDiffReportListingEntryProto createdEntries = 2;
  505. repeated SnapshotDiffReportListingEntryProto deletedEntries = 3;
  506. required bool isFromEarlier = 4;
  507. optional SnapshotDiffReportCursorProto cursor = 5;
  508. }
  509. /**
  510. * Block information
  511. *
  512. * Please be wary of adding additional fields here, since INodeFiles
  513. * need to fit in PB's default max message size of 64MB.
  514. * We restrict the max # of blocks per file
  515. * (dfs.namenode.fs-limits.max-blocks-per-file), but it's better
  516. * to avoid changing this.
  517. */
  518. message BlockProto {
  519. required uint64 blockId = 1;
  520. required uint64 genStamp = 2;
  521. optional uint64 numBytes = 3 [default = 0];
  522. }
  523. /**
  524. * Information related to a snapshot
  525. * TODO: add more information
  526. */
  527. message SnapshotInfoProto {
  528. required string snapshotName = 1;
  529. required string snapshotRoot = 2;
  530. required FsPermissionProto permission = 3;
  531. required string owner = 4;
  532. required string group = 5;
  533. required string createTime = 6;
  534. // TODO: do we need access time?
  535. }
  536. /**
  537. * Rolling upgrade status
  538. */
  539. message RollingUpgradeStatusProto {
  540. required string blockPoolId = 1;
  541. optional bool finalized = 2 [default = false];
  542. }
  543. /**
  544. * A list of storage IDs.
  545. */
  546. message StorageUuidsProto {
  547. repeated string storageUuids = 1;
  548. }
  549. /**
  550. * File access permissions mode.
  551. */
  552. enum AccessModeProto {
  553. READ = 1;
  554. WRITE = 2;
  555. COPY = 3;
  556. REPLACE = 4;
  557. }
  558. /**
  559. * Secret information for the BlockKeyProto. This is not sent on the wire as
  560. * such but is used to pack a byte array and encrypted and put in
  561. * BlockKeyProto.bytes
  562. * When adding further fields, make sure they are optional as they would
  563. * otherwise not be backwards compatible.
  564. *
  565. * Note: As part of the migration from WritableUtils based tokens (aka "legacy")
  566. * to Protocol Buffers, we use the first byte to determine the type. If the
  567. * first byte is <=0 then it is a legacy token. This means that when using
  568. * protobuf tokens, the the first field sent must have a `field_number` less
  569. * than 16 to make sure that the first byte is positive. Otherwise it could be
  570. * parsed as a legacy token. See HDFS-11026 for more discussion.
  571. */
  572. message BlockTokenSecretProto {
  573. optional uint64 expiryDate = 1;
  574. optional uint32 keyId = 2;
  575. optional string userId = 3;
  576. optional string blockPoolId = 4;
  577. optional uint64 blockId = 5;
  578. repeated AccessModeProto modes = 6;
  579. repeated StorageTypeProto storageTypes = 7;
  580. repeated string storageIds = 8;
  581. }