123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280 |
- /**
- * Licensed to the Apache Software Foundation (ASF) 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.
- */
- option java_package = "org.apache.hadoop.hdfs.server.namenode";
- option java_outer_classname = "FsImageProto";
- package hadoop.hdfs.fsimage;
- import "hdfs.proto";
- /**
- * This file defines the on-disk layout of the file system image. The
- * layout is defined by the following EBNF grammar, in which angle
- * brackets mark protobuf definitions. (e.g., <FileSummary>)
- *
- * FILE := MAGIC SECTION* <FileSummary> FileSummaryLength
- * MAGIC := 'HDFSIMG1'
- * SECTION := <NameSystemSection> | ...
- * FileSummaryLength := 4 byte int
- *
- * Some notes:
- *
- * The codec field in FileSummary describes the compression codec used
- * for all sections. The fileheader is always uncompressed.
- *
- * All protobuf messages are serialized in delimited form, which means
- * that there always will be an integer indicates the size of the
- * protobuf message.
- *
- */
- message FileSummary {
- // The version of the above EBNF grammars.
- required uint32 ondiskVersion = 1;
- // layoutVersion describes which features are available in the
- // FSImage.
- required uint32 layoutVersion = 2;
- optional string codec = 3;
- // index for each section
- message Section {
- optional string name = 1;
- optional uint64 length = 2;
- optional uint64 offset = 3;
- }
- repeated Section sections = 4;
- }
- /**
- * Name: NS_INFO
- */
- message NameSystemSection {
- optional uint32 namespaceId = 1;
- optional uint64 genstampV1 = 2;
- optional uint64 genstampV2 = 3;
- optional uint64 genstampV1Limit = 4;
- optional uint64 lastAllocatedBlockId = 5;
- optional uint64 transactionId = 6;
- }
- /**
- * Permission is serialized as a 64-bit long. [0:24):[25:48):[48:64) (in Big Endian).
- * The first and the second parts are the string ids of the user and
- * group name, and the last 16 bits are the permission bits.
- *
- * Name: INODE
- */
- message INodeSection {
- /**
- * under-construction feature for INodeFile
- */
- message FileUnderConstructionFeature {
- optional string clientName = 1;
- optional string clientMachine = 2;
- }
- message INodeFile {
- optional uint32 replication = 1;
- optional uint64 modificationTime = 2;
- optional uint64 accessTime = 3;
- optional uint64 preferredBlockSize = 4;
- optional fixed64 permission = 5;
- repeated BlockProto blocks = 6;
- optional FileUnderConstructionFeature fileUC = 7;
- }
- message INodeDirectory {
- optional uint64 modificationTime = 1;
- // namespace quota
- optional uint64 nsQuota = 2;
- // diskspace quota
- optional uint64 dsQuota = 3;
- optional fixed64 permission = 4;
- }
- message INodeSymlink {
- optional fixed64 permission = 1;
- optional bytes target = 2;
- }
- message INodeReference {
- // id of the referred inode
- optional uint64 referredId = 1;
- // local name recorded in WithName
- optional bytes name = 2;
- // recorded in DstReference
- optional uint32 dstSnapshotId = 3;
- // recorded in WithName
- optional uint32 lastSnapshotId = 4;
- }
- message INode {
- enum Type {
- FILE = 1;
- DIRECTORY = 2;
- SYMLINK = 3;
- };
- required Type type = 1;
- required uint64 id = 2;
- optional bytes name = 3;
- optional INodeFile file = 4;
- optional INodeDirectory directory = 5;
- optional INodeSymlink symlink = 6;
- }
- optional uint64 lastInodeId = 1;
- optional uint64 numInodes = 2;
- // repeated INodes..
- }
- /**
- * This section records information about under-construction files for
- * reconstructing the lease map.
- * NAME: FILES_UNDERCONSTRUCTION
- */
- message FilesUnderConstructionSection {
- message FileUnderConstructionEntry {
- optional uint64 inodeId = 1;
- optional string fullPath = 2;
- }
- // repeated FileUnderConstructionEntry...
- }
- /**
- * This section records the children of each directories
- * NAME: INODE_DIR
- */
- message INodeDirectorySection {
- message DirEntry {
- optional uint64 parent = 1;
- repeated uint64 children = 2 [packed = true];
- optional uint64 numOfRef = 3;
- // repeated INodeReference...
- }
- // repeated DirEntry, ended at the boundary of the section.
- }
- /**
- * This section records the information about snapshot
- * NAME: SNAPSHOT
- */
- message SnapshotSection {
- message Snapshot {
- optional uint32 snapshotId = 1;
- // Snapshot root
- optional INodeSection.INode root = 2;
- }
- optional uint32 snapshotCounter = 1;
- repeated uint64 snapshottableDir = 2 [packed = true];
- // total number of snapshots
- optional uint32 numSnapshots = 3;
- // repeated Snapshot...
- }
- /**
- * This section records information about snapshot diffs
- * NAME: SNAPSHOT_DIFF
- */
- message SnapshotDiffSection {
- message CreatedListEntry {
- optional bytes name = 1;
- }
- message DirectoryDiff {
- optional uint32 snapshotId = 1;
- optional uint32 childrenSize = 2;
- optional bool isSnapshotRoot = 3;
- optional bytes name = 4;
- optional INodeSection.INodeDirectory snapshotCopy = 5;
- optional uint32 createdListSize = 6;
- optional uint32 numOfDeletedRef = 7; // number of reference nodes in deleted list
- repeated uint64 deletedINode = 8 [packed = true]; // id of deleted inode
- // repeated CreatedListEntry (size is specified by createdListSize)
- // repeated INodeReference (reference inodes in deleted list)
- }
- message FileDiff {
- optional uint32 snapshotId = 1;
- optional uint64 fileSize = 2;
- optional bytes name = 3;
- optional INodeSection.INodeFile snapshotCopy = 4;
- }
- message DiffEntry {
- enum Type {
- FILEDIFF = 1;
- DIRECTORYDIFF = 2;
- }
- required Type type = 1;
- optional uint64 inodeId = 2;
- optional uint32 numOfDiff = 3;
- // repeated DirectoryDiff or FileDiff
- }
- // repeated DiffEntry
- }
- /**
- * This section maps string to id
- * NAME: STRING_TABLE
- */
- message StringTableSection {
- message Entry {
- optional uint32 id = 1;
- optional string str = 2;
- }
- optional uint32 numEntry = 1;
- // repeated Entry
- }
- message SecretManagerSection {
- message DelegationKey {
- optional uint32 id = 1;
- optional uint64 expiryDate = 2;
- optional bytes key = 3;
- }
- message PersistToken {
- optional uint32 version = 1;
- optional string owner = 2;
- optional string renewer = 3;
- optional string realUser = 4;
- optional uint64 issueDate = 5;
- optional uint64 maxDate = 6;
- optional uint32 sequenceNumber = 7;
- optional uint32 masterKeyId = 8;
- optional uint64 expiryDate = 9;
- }
- optional uint32 currentId = 1;
- optional uint32 tokenSequenceNumber = 2;
- optional uint32 numKeys = 3;
- optional uint32 numTokens = 4;
- // repeated DelegationKey keys
- // repeated PersistToken tokens
- }
- message CacheManagerSection {
- required uint64 nextDirectiveId = 1;
- required uint32 numPools = 2;
- required uint32 numDirectives = 3;
- // repeated CachePoolInfoProto pools
- // repeated CacheDirectiveInfoProto directives
- }
|