Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-7285

Erasure Coding Support inside HDFS

    XMLWordPrintableJSON

Details

    • New Feature
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.0.0-alpha1
    • None
    • None
    • Reviewed
    • Hide
      <!-- markdown -->
      HDFS now provides native support for erasure coding (EC) to store data more efficiently. Each individual directory can be configured with an EC policy with command `hdfs erasurecode -setPolicy`. When a file is created, it will inherit the EC policy from its nearest ancestor directory to determine how its blocks are stored. Compared to 3-way replication, the default EC policy saves 50% of storage space while also tolerating more storage failures.

      To support small files, the currently phase of HDFS-EC stores blocks in _striped_ layout, where a logical file block is divided into small units (64KB by default) and distributed to a set of DataNodes. This enables parallel I/O but also decreases data locality. Therefore, the cluster environment and I/O workloads should be considered before configuring EC policies.
      Show
      <!-- markdown --> HDFS now provides native support for erasure coding (EC) to store data more efficiently. Each individual directory can be configured with an EC policy with command `hdfs erasurecode -setPolicy`. When a file is created, it will inherit the EC policy from its nearest ancestor directory to determine how its blocks are stored. Compared to 3-way replication, the default EC policy saves 50% of storage space while also tolerating more storage failures. To support small files, the currently phase of HDFS-EC stores blocks in _striped_ layout, where a logical file block is divided into small units (64KB by default) and distributed to a set of DataNodes. This enables parallel I/O but also decreases data locality. Therefore, the cluster environment and I/O workloads should be considered before configuring EC policies.

    Description

      Erasure Coding (EC) can greatly reduce the storage overhead without sacrifice of data reliability, comparing to the existing HDFS 3-replica approach. For example, if we use a 10+4 Reed Solomon coding, we can allow loss of 4 blocks, with storage overhead only being 40%. This makes EC a quite attractive alternative for big data storage, particularly for cold data.

      Facebook had a related open source project called HDFS-RAID. It used to be one of the contribute packages in HDFS but had been removed since Hadoop 2.0 for maintain reason. The drawbacks are: 1) it is on top of HDFS and depends on MapReduce to do encoding and decoding tasks; 2) it can only be used for cold files that are intended not to be appended anymore; 3) the pure Java EC coding implementation is extremely slow in practical use. Due to these, it might not be a good idea to just bring HDFS-RAID back.

      We (Intel and Cloudera) are working on a design to build EC into HDFS that gets rid of any external dependencies, makes it self-contained and independently maintained. This design lays the EC feature on the storage type support and considers compatible with existing HDFS features like caching, snapshot, encryption, high availability and etc. This design will also support different EC coding schemes, implementations and policies for different deployment scenarios. By utilizing advanced libraries (e.g. Intel ISA-L library), an implementation can greatly improve the performance of EC encoding/decoding and makes the EC solution even more attractive. We will post the design document soon.

      Attachments

        1. HDFSErasureCodingSystemTestReport-20150826.pdf
          218 kB
          Rui Gao
        2. HDFSErasureCodingSystemTestPlan-20150824.pdf
          72 kB
          Rui Gao
        3. HDFSErasureCodingPhaseITestPlan.pdf
          111 kB
          Zhe Zhang
        4. HDFSErasureCodingDesign-20150206.pdf
          1.42 MB
          Tsz-wo Sze
        5. HDFSErasureCodingDesign-20150204.pdf
          1.40 MB
          Tsz-wo Sze
        6. HDFSErasureCodingDesign-20141217.pdf
          1.59 MB
          Zhe Zhang
        7. HDFSErasureCodingDesign-20141028.pdf
          1.98 MB
          Zhe Zhang
        8. HDFS-EC-Merge-PoC-20150624.patch
          811 kB
          Zhe Zhang
        9. HDFS-EC-merge-consolidated-01.patch
          1.06 MB
          Zhe Zhang
        10. HDFS-bistriped.patch
          19 kB
          Zhe Zhang
        11. HDFS-7285-merge-consolidated-trunk-01.patch
          1.06 MB
          Vinayakumar B
        12. HDFS-7285-merge-consolidated-01.patch
          1.06 MB
          Vinayakumar B
        13. HDFS-7285-merge-consolidated.trunk.04.patch
          1.03 MB
          Vinayakumar B
        14. HDFS-7285-merge-consolidated.trunk.03.patch
          1.04 MB
          Vinayakumar B
        15. HDFS-7285-initial-PoC.patch
          470 kB
          Zhe Zhang
        16. HDFS-7285-Consolidated-20150911.patch
          1.20 MB
          Zhe Zhang
        17. fsimage-analysis-20150105.pdf
          82 kB
          Zhe Zhang
        18. ECParser.py
          5 kB
          Zhe Zhang
        19. ECAnalyzer.py
          2 kB
          Zhe Zhang
        20. Consolidated-20150810.patch
          1.23 MB
          Zhe Zhang
        21. Consolidated-20150806.patch
          1.24 MB
          Zhe Zhang
        22. Consolidated-20150707.patch
          1.02 MB
          Zhe Zhang
        23. Compare-consolidated-20150824.diff
          72 kB
          Zhe Zhang
        24. 1619363340018.png
          76 kB
          Tsz-wo Sze

        Issue Links

          1.
          Erasure coding: distribute recovery work for striped blocks to DataNode Sub-task Resolved Zhe Zhang
          2.
          Configurable erasure coding policy for individual files and directories Sub-task Resolved Zhe Zhang
          3.
          Representing striped block groups in NameNode with hierarchical naming protocol Sub-task Resolved Zhe Zhang
          4.
          Process block reports for erasure coded blocks Sub-task Resolved Zhe Zhang
          5.
          [umbrella] Data striping support in HDFS client Sub-task Resolved Li Bo
          6.
          Document the HDFS Erasure Coding feature Sub-task Resolved Uma Maheswara Rao G
          7.
          Erasure Coding: extend BlockInfo to handle EC info Sub-task Resolved Jing Zhao
          8.
          Implement COMPLETE state of erasure coding block groups Sub-task Resolved Zhe Zhang
          9.
          Add a test for BlockGroup support in FSImage Sub-task Resolved Takuya Fukudome
          10.
          Add unit tests for editlog transactions for EC Sub-task Resolved Hui Zheng
          11.
          Change disk quota calculation for EC files Sub-task Resolved Tsz-wo Sze
          12.
          Erasure Coding: update the Balancer/Mover data migration logic Sub-task Resolved Walter Su
          13.
          Erasure Coding: consolidate streamer coordination logic and handle failure when writing striped blocks Sub-task Resolved Tsz-wo Sze
          14.
          Erasure coding: DFSInputStream with decode functionality (pread) Sub-task Resolved Zhe Zhang
          15.
          Change fsck to support EC files Sub-task Resolved Takanobu Asanuma
          16.
          Client side api/config changes to support online encoding Sub-task Resolved Vinayakumar B
          17.
          Add periodic checker to find the corrupted EC blocks/files Sub-task Resolved Vinayakumar B
          18.
          Avoid Block movement in Balancer and Mover for the erasure encoded blocks Sub-task Resolved Vinayakumar B
          19.
          Add logic to DFSOutputStream to support writing a file in striping layout Sub-task Resolved Li Bo
          20.
          Erasure Coding: Add striped block support in INodeFile Sub-task Resolved Jing Zhao
          21.
          Erasure coding: pread from files in striped layout Sub-task Resolved Zhe Zhang
          22.
          Support appending to a striping layout file Sub-task Resolved Li Bo
          23.
          Erasure Coding: Update INodeFile quota computation for striped blocks Sub-task Resolved Kai Sasaki
          24.
          Erasure Coding: allocate and persist striped blocks in NameNode Sub-task Resolved Jing Zhao
          25.
          Erasure Coding: support striped blocks in non-protobuf fsimage Sub-task Resolved Hui Zheng
          26.
          Erasure coding: implement facilities in NameNode to create and manage EC zones Sub-task Resolved Zhe Zhang
          27.
          Erasure coding: extend LocatedBlocks to support reading from striped files Sub-task Resolved Jing Zhao
          28.
          Erasure Coding: Update safemode calculation for striped blocks Sub-task Resolved Rui Gao
          29.
          Erasure Coding: INodeFile.dumpTreeRecursively() supports to print striped blocks Sub-task Resolved Takuya Fukudome
          30.
          Subclass DFSOutputStream to support writing striping layout files Sub-task Resolved Li Bo
          31.
          Erasure Coding: track invalid, corrupt, and under-recovery striped blocks in NameNode Sub-task Resolved Jing Zhao
          32.
          Erasure coding: use BlockInfo[] for both striped and contiguous blocks in INodeFile Sub-task Resolved Zhe Zhang
          33.
          Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks Sub-task Resolved Jing Zhao
          34.
          Erasure coding: resolving conflicts in the branch when merging trunk changes. Sub-task Resolved Zhe Zhang
          35.
          Erasure Coding: INodeFile quota computation unit tests Sub-task Resolved Kai Sasaki
          36.
          WebImageViewer need support file size calculation with striped blocks Sub-task Resolved Rakesh Radhakrishnan
          37.
          Erasure coding: NameNode support for lease recovery of striped block groups Sub-task Resolved Zhe Zhang
          38.
          Erasure coding: Decommission handle for EC blocks. Sub-task Resolved Yi Liu
          39.
          Erasure coding: DataNode support for block recovery of striped block groups Sub-task Resolved Yi Liu
          40.
          getStoragePolicy() regards HOT policy as EC policy Sub-task Resolved Takanobu Asanuma
          41.
          Erasure Coding: simplify striped block recovery work computation and add tests Sub-task Resolved Jing Zhao
          42.
          Erasure coding: extend UnderReplicatedBlocks to accurately handle striped blocks Sub-task Resolved Zhe Zhang
          43.
          Erasure Coding: retrieve eraure coding schema for a file from NameNode Sub-task Resolved Vinayakumar B
          44.
          Erasure Coding: ECworker frame, basics, bootstraping and configuration Sub-task Resolved Uma Maheswara Rao G
          45.
          Erasure Coding: Update CHANGES-HDFS-7285.txt with branch commits Sub-task Resolved Vinayakumar B
          46.
          Erasure coding: stateful (non-positional) read from files in striped layout Sub-task Resolved Zhe Zhang
          47.
          Define a system-wide default EC schema Sub-task Resolved Kai Zheng
          48.
          Erasure coding: fix bug in EC zone and symlinks Sub-task Resolved Jing Zhao
          49.
          Erasure Coding: Add RPC to client-namenode to list all ECSchemas loaded in Namenode. Sub-task Resolved Vinayakumar B
          50.
          Erasure coding: fix bug in TestFSImage Sub-task Resolved Rakesh Radhakrishnan
          51.
          Make hard-coded values consistent with the system default schema first before remove them Sub-task Resolved Kai Zheng
          52.
          Erasure coding: Add auditlog FSNamesystem#createErasureCodingZone if this operation fails Sub-task Resolved Rakesh Radhakrishnan
          53.
          Erasure coding: created util class to analyze striped block groups Sub-task Resolved Zhe Zhang
          54.
          BlockManager treates good blocks in a block group as corrput Sub-task Resolved Li Bo
          55.
          Erasure Coding: Support specifying ECSchema during creation of ECZone Sub-task Resolved Vinayakumar B
          56.
          Erasure Coding: Better to move EC related proto messages to a separate erasurecoding proto file Sub-task Resolved Rakesh Radhakrishnan
          57.
          DFSStripedInputStream fails to read data after one stripe Sub-task Resolved Zhe Zhang
          58.
          Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding Sub-task Resolved Uma Maheswara Rao G
          59.
          Client gets and uses EC schema when reads and writes a stripping file Sub-task Resolved Kai Sasaki
          60.
          Send the EC schema to DataNode via EC encoding/recovering command Sub-task Resolved Uma Maheswara Rao G
          61.
          Fix the editlog corruption exposed by failed TestAddStripedBlocks Sub-task Resolved Jing Zhao
          62.
          Protobuf changes for BlockECRecoveryCommand and its fields for making it ready for transfer to DN Sub-task Resolved Uma Maheswara Rao G
          63.
          Support DFS command for the EC encoding Sub-task Resolved Vinayakumar B
          64.
          Add/implement necessary APIs even we just have the system default schema Sub-task Resolved Kai Zheng
          65.
          Detect if resevered EC Block ID is already used Sub-task Resolved Hui Zheng
          66.
          DFSStripedOutputStream should not create empty blocks Sub-task Resolved Jing Zhao
          67.
          BlockManager.addBlockCollectionWithCheck should check if the block is a striped block Sub-task Resolved Hui Zheng
          68.
          Erasure Coding: Keep default schema's name consistent Sub-task Resolved Unassigned
          69.
          Failure handling: DFSStripedOutputStream continues writing with enough remaining datanodes Sub-task Resolved Li Bo
          70.
          Erasure Coding: DataNode reconstruct striped blocks Sub-task Resolved Yi Liu
          71.
          createErasureCodingZone sets retryCache state as false always Sub-task Resolved Uma Maheswara Rao G
          72.
          Erasure Coding: Improve DFSStripedOutputStream closing of datastreamer threads Sub-task Resolved Rakesh Radhakrishnan
          73.
          Erasure coding: Make block placement policy for EC file configurable Sub-task Resolved Walter Su
          74.
          Erasure coding: refactor client-related code to sync with HDFS-8082 and HDFS-8169 Sub-task Resolved Zhe Zhang
          75.
          ClientProtocol#createErasureCodingZone API was wrongly annotated as Idempotent Sub-task Resolved Vinayakumar B
          76.
          StripedBlockUtil.getInternalBlockLength may have overflow error Sub-task Resolved Tsz-wo Sze
          77.
          Erasure coding: Fix file quota change when we complete/commit the striped blocks Sub-task Resolved Takuya Fukudome
          78.
          Improve end to end striping file test to add erasure recovering test Sub-task Resolved Xinwei Qin
          79.
          Erasure Coding: Seek and other Ops in DFSStripedInputStream. Sub-task Resolved Yi Liu
          80.
          DistributedFileSystem.createErasureCodingZone should pass schema in FileSystemLinkResolver Sub-task Resolved Tsz-wo Sze
          81.
          TestDFSStripedOutputStream should use BlockReaderTestUtil to create BlockReader Sub-task Resolved Tsz-wo Sze
          82.
          Erasure Coding: StripedDataStreamer fails to handle the blocklocations which doesn't satisfy BlockGroupSize Sub-task Resolved Rakesh Radhakrishnan
          83.
          Should calculate checksum for parity blocks in DFSStripedOutputStream Sub-task Resolved Yi Liu
          84.
          Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY commands from standbynode if any Sub-task Resolved Vinayakumar B
          85.
          Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause block id conflicts Sub-task Resolved Jing Zhao
          86.
          Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last stripe is at the block group boundary Sub-task Resolved Jing Zhao
          87.
          Erasure Coding: Create DFSStripedInputStream in DFSClient#open Sub-task Resolved Kai Sasaki
          88.
          Erasure coding: [bug] should always allocate unique striped block group IDs Sub-task Resolved Zhe Zhang
          89.
          Erasure Coding: XML based end-to-end test for ECCli commands Sub-task Resolved Rakesh Radhakrishnan
          90.
          DFSStripedOutputStream.closeThreads releases cellBuffers multiple times Sub-task Resolved Kai Sasaki
          91.
          Avoid assigning a leading streamer in StripedDataStreamer to tolerate datanode failure Sub-task Resolved Tsz-wo Sze
          92.
          Erasure Coding: simplify the retry logic in DFSStripedInputStream (stateful read) Sub-task Resolved Jing Zhao
          93.
          Erasure Coding: Implement batched listing of enrasure coding zones Sub-task Resolved Rakesh Radhakrishnan
          94.
          Erasure Coding: implement parallel stateful reading for striped layout Sub-task Resolved Jing Zhao
          95.
          Erasure coding: move striped reading logic to StripedBlockUtil Sub-task Resolved Zhe Zhang
          96.
          Refactor DFSStripedOutputStream and StripedDataStreamer Sub-task Resolved Tsz-wo Sze
          97.
          Erasure Coding: add ECSchema to HdfsFileStatus Sub-task Resolved Yong Zhang
          98.
          Erasure Coding: Fix Findbug warnings present in erasure coding Sub-task Resolved Rakesh Radhakrishnan
          99.
          Erasure Coding: NameNode may get blocked in waitForLoadingFSImage() when loading editlog Sub-task Resolved Jing Zhao
          100.
          Erasure Coding: DFSStripedOutputStream#close throws NullPointerException exception in some cases Sub-task Resolved Li Bo
          101.
          Erasure coding: refactor EC constants to be consistent with HDFS-8249 Sub-task Resolved Zhe Zhang
          102.
          Erasure Coding: support decoding for stateful read Sub-task Resolved Jing Zhao
          103.
          Erasure coding: consolidate striping-related terminologies Sub-task Resolved Zhe Zhang
          104.
          Bump GenerationStamp for write faliure in DFSStripedOutputStream Sub-task Resolved Tsz-wo Sze
          105.
          Add trace info to DFSClient#getErasureCodingZoneInfo(..) Sub-task Resolved Vinayakumar B
          106.
          Follow-on to update decode for DataNode striped blocks reconstruction Sub-task Resolved Yi Liu
          107.
          Erasure coding: Rename Striped block recovery to reconstruction to eliminate confusion. Sub-task Resolved Yi Liu
          108.
          Erasure coding: rename DFSStripedInputStream related test classes Sub-task Resolved Zhe Zhang
          109.
          Expose some administrative erasure coding operations to HdfsAdmin Sub-task Resolved Uma Maheswara Rao G
          110.
          Erasure Coding: Badly treated when createBlockOutputStream failed in DataStreamer Sub-task Resolved Unassigned
          111.
          Erasure Coding: test skip in TestDFSStripedInputStream Sub-task Resolved Walter Su
          112.
          Erasure Coding: test failed in TestDFSStripedInputStream.testStatefulRead() when use ByteBuffer Sub-task Resolved Walter Su
          113.
          Erasure Coding: whether to use the same chunkSize in decoding with the value in encoding Sub-task Resolved Unassigned
          114.
          Erasure Coding: test webhdfs read write stripe file Sub-task Resolved Walter Su
          115.
          Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction Sub-task Resolved Tsz-wo Sze
          116.
          Fix FindBugs issues introduced by erasure coding Sub-task Resolved Unassigned
          117.
          Erasure Coding: DFSStripedInputStream#seekToNewSource Sub-task Resolved Yi Liu
          118.
          Erasure coding: fix some minor bugs in EC CLI Sub-task Resolved Walter Su
          119.
          Erasure Coding: Badly treated when short of Datanode in StripedDataStreamer Sub-task Resolved Walter Su
          120.
          Erasure Coding: Make the timeout parameter of polling blocking queue configurable in DFSStripedOutputStream Sub-task Resolved Li Bo
          121.
          BlockInfoStriped uses EC schema Sub-task Resolved Kai Sasaki
          122.
          Erasure Coding: DFS opening a non-existent file need to be handled properly Sub-task Resolved Rakesh Radhakrishnan
          123.
          Erasure Coding: TestRecoverStripedFile#testRecoverOneParityBlock is failing Sub-task Resolved Rakesh Radhakrishnan
          124.
          Erasure coding: compute storage type quotas for striped files, to be consistent with HDFS-8327 Sub-task Resolved Zhe Zhang
          125.
          Add cellSize as an XAttr to ECZone Sub-task Resolved Vinayakumar B
          126.
          Erasure Coding: Few improvements for the erasure coding worker Sub-task Resolved Rakesh Radhakrishnan
          127.
          Fix issues like NPE in TestRecoverStripedFile Sub-task Resolved Kai Zheng
          128.
          Remove chunkSize and initialize from erasure coder Sub-task Resolved Kai Zheng
          129.
          NN should consider current EC tasks handling count from DN while assigning new tasks Sub-task Resolved Uma Maheswara Rao G
          130.
          Erasure Coding: unit test the behaviour of BlockManager recovery work for the deleted blocks Sub-task Resolved Rakesh Radhakrishnan
          131.
          Revisit and refactor ErasureCodingInfo Sub-task Resolved Vinayakumar B
          132.
          Erasure Coding: Pread failed to read data starting from not-first stripe Sub-task Resolved Walter Su
          133.
          Fix the isNeededReplication calculation for Striped block in NN Sub-task Resolved Yi Liu
          134.
          Erasure Coding: ECZoneManager#getECZoneInfo is not resolving the path properly if zone dir itself is the snapshottable dir Sub-task Resolved Rakesh Radhakrishnan
          135.
          Remove dataBlockNum and parityBlockNum from BlockInfoStriped Sub-task Resolved Kai Sasaki
          136.
          Erasure Coding: Fix the NullPointerException when deleting file Sub-task Resolved Yi Liu
          137.
          set blockToken in LocatedStripedBlock Sub-task Resolved Walter Su
          138.
          Erasure Coding: make condition check earlier for setReplication Sub-task Resolved Walter Su
          139.
          Erasure Coding: fix cannot rename a zone dir Sub-task Resolved Walter Su
          140.
          Erasure Coding: Consolidate erasure coding zone related implementation into a single class Sub-task Resolved Rakesh Radhakrishnan
          141.
          Erasure coding: properly handle start offset for internal blocks in a block group Sub-task Resolved Zhe Zhang
          142.
          Erasure Coding: stateful read result doesn't match data occasionally because of flawed test Sub-task Resolved Walter Su
          143.
          Erasure coding: fix priority level of UnderReplicatedBlocks for striped block Sub-task Resolved Walter Su
          144.
          Refactor BlockInfoContiguous and fix NPE in TestBlockInfo#testCopyConstructor() Sub-task Resolved Vinayakumar B
          145.
          2 RPC calls for every file read in DFSClient#open(..) resulting in double Audit log entries Sub-task Resolved Vinayakumar B
          146.
          createErasureCodingZone should check whether cellSize is available Sub-task Resolved Yong Zhang
          147.
          Erasure coding: fix striping related logic in FSDirWriteFileOp to sync with HDFS-8421 Sub-task Resolved Zhe Zhang
          148.
          Erasure coding: remove workarounds in client side stripped blocks recovering Sub-task Resolved Zhe Zhang
          149.
          Erasure coding: test DataNode reporting bad/corrupted blocks which belongs to a striped block. Sub-task Resolved Takanobu Asanuma
          150.
          Erasure coding: Two contiguous blocks occupy IDs belong to same striped group Sub-task Resolved Walter Su
          151.
          ErasureCodingWorker fails to do decode work Sub-task Resolved Li Bo
          152.
          Fix a decoding issue in stripped block recovering in client side Sub-task Resolved Kai Zheng
          153.
          Restore ECZone info inside FSImageLoader Sub-task Resolved Kai Sasaki
          154.
          Erasure Coding: processOverReplicatedBlock() handles striped block Sub-task Resolved Walter Su
          155.
          Erasure Coding: Fix FindBugs Multithreaded correctness Warning Sub-task Resolved Rakesh Radhakrishnan
          156.
          Erasure Coding: Fix usage of 'createZone' Sub-task Resolved Vinayakumar B
          157.
          Allow to configure RS and XOR raw coders Sub-task Resolved Kai Zheng
          158.
          Erasure Coding: fix non-protobuf fsimage for striped blocks Sub-task Resolved Jing Zhao
          159.
          Erasure Coding: fsck handles file smaller than a full stripe Sub-task Resolved Walter Su
          160.
          Erasure Coding: SafeMode handles file smaller than a full stripe Sub-task Resolved Walter Su
          161.
          Fix TestErasureCodingCli test Sub-task Resolved Vinayakumar B
          162.
          Erasure coding: Persist cellSize in BlockInfoStriped and StripedBlocksFeature Sub-task Resolved Walter Su
          163.
          Erasure Coding: Remove dataBlockNum and parityBlockNum from StripedBlockProto Sub-task Resolved Yi Liu
          164.
          Erasure Coding: fix the copy constructor of BlockInfoStriped and BlockInfoContiguous Sub-task Resolved Vinayakumar B
          165.
          Erasure Coding: Client can't read(decode) the EC files which have corrupt blocks. Sub-task Resolved Kai Sasaki
          166.
          Erasure Coding: revisit replica counting for striped blocks Sub-task Resolved Jing Zhao
          167.
          Erasure Coding: handle missing internal block locations in DFSStripedInputStream Sub-task Resolved Jing Zhao
          168.
          Erasure Coding: fix some block number calculation for striped block Sub-task Resolved Yi Liu
          169.
          DFSClient hang up when there are not sufficient DataNodes in EC cluster. Sub-task Resolved Kai Sasaki
          170.
          Erasure coding: update BlockManager.blockHasEnoughRacks(..) logic for striped block Sub-task Resolved Kai Sasaki
          171.
          Erasure Coding: client generates too many small packets when writing parity data Sub-task Resolved Li Bo
          172.
          Erasure coding: revisit and simplify BlockInfoStriped and INodeFile Sub-task Resolved Zhe Zhang
          173.
          Erasure coding: For a small file missing and under replicated ec-block calculation is incorrect Sub-task Resolved J.Andreina
          174.
          Erasure Coding: Fail to read a file with corrupted blocks Sub-task Resolved Walter Su
          175.
          Erasure Coding: fix one cell need two packets Sub-task Resolved Walter Su
          176.
          Erasure Coding: the number of chunks in packet is not updated when writing parity data Sub-task Resolved Li Bo
          177.
          Erasure Coding: reuse BlockReader when reading the same block in pread Sub-task Resolved Jing Zhao
          178.
          Erasure Coding: unit test for SequentialBlockGroupIdGenerator Sub-task Resolved Rakesh Radhakrishnan
          179.
          Erasure Coding: Correctly handle BlockManager#InvalidateBlocks for striped block Sub-task Resolved Yi Liu
          180.
          Erasure coding: rename BlockInfoContiguousUC and BlockInfoStripedUC to be consistent with trunk Sub-task Resolved Zhe Zhang
          181.
          Erasure Coding: fix DFSStripedInputStream/DFSStripedOutputStream re-fetch token when expired Sub-task Resolved Walter Su
          182.
          Erasure Coding: use DirectBufferPool in DFSStripedInputStream for buffer allocation Sub-task Resolved Jing Zhao
          183.
          Erasure Coding: Client no need to decode missing parity blocks Sub-task Resolved Walter Su
          184.
          Erasure Coding: add test for namenode process over replicated striped block Sub-task Resolved Takuya Fukudome
          185.
          Erasure Coding: Fix NPE when NameNode processes over-replicated striped blocks Sub-task Resolved Walter Su
          186.
          Erasure coding: store EC schema and cell size in INodeFile and eliminate notion of EC zones Sub-task Resolved Zhe Zhang
          187.
          Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream when the data length is small Sub-task Resolved Tsz-wo Sze
          188.
          Erasure Coding: client occasionally gets less block locations when some datanodes fail Sub-task Resolved Li Bo
          189.
          Erasure Coding: Provide ECSchema validation when setting EC policy Sub-task Resolved J.Andreina
          190.
          Erasure coding: add ECPolicy to replace schema+cellSize in hadoop-hdfs Sub-task Resolved Walter Su
          191.
          Erasure Coding: Fix ArrayIndexOutOfBoundsException in TestWriteStripedFileWithFailure Sub-task Resolved Li Bo
          192.
          Erasure Coding: Use datablocks, parityblocks and cell size from ErasureCodingPolicy Sub-task Resolved Vinayakumar B
          193.
          Erasure Coding: use threadpool for EC recovery tasks on DataNode Sub-task Resolved Rakesh Radhakrishnan
          194.
          Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC to use BlockUnderConstructionFeature Sub-task Resolved Jing Zhao
          195.
          Erasure coding: do not throw exception when setting replication factor on EC files Sub-task Resolved Rui Gao
          196.
          Erasure coding : Fix random failure in TestSafeModeWithStripedFile Sub-task Resolved J.Andreina
          197.
          Erasure coding: fix 2 failed tests of DFSStripedOutputStream Sub-task Resolved Walter Su
          198.
          Erasure coding: MapReduce job failed when I set the / folder to the EC zone Sub-task Resolved Unassigned
          199.
          Rename dfs.datanode.stripedread.threshold.millis to dfs.datanode.stripedread.timeout.millis Sub-task Resolved Andrew Wang
          200.
          Cleanup erasure coding documentation Sub-task Resolved Andrew Wang
          201.
          Erasure Coding: Provide DistributedFilesystem API to getAllErasureCodingPolicies Sub-task Resolved Rakesh Radhakrishnan
          202.
          Erasure coding: update EC command "-s" flag to "-p" when specifying policy Sub-task Resolved Zhe Zhang
          203.
          ErasureCodingWorker#processErasureCodingTasks should not fail to process remaining tasks due to one invalid ECTask Sub-task Resolved Uma Maheswara Rao G
          204.
          Erasure Coding: when recovering lost blocks, logs can be too verbose and hurt performance Sub-task Resolved Rui Li
          205.
          Erasure coding: Refactor DFSStripedOutputStream (Move Namenode RPC Requests to Coordinator) Sub-task Resolved Jing Zhao

          Activity

            People

              zhz Zhe Zhang
              whjiang Weihua Jiang
              Votes:
              4 Vote for this issue
              Watchers:
              129 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: