You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by vi...@apache.org on 2015/07/08 13:34:03 UTC

[1/2] hadoop git commit: Revert "HDFS-8712. Remove 'public' and 'abstract' modifiers in FsVolumeSpi and FsDatasetSpi (Contributed by Lei (Eddy) Xu)"

Repository: hadoop
Updated Branches:
  refs/heads/branch-2 38ee9247c -> 6fe2e7c3f


Revert "HDFS-8712. Remove 'public' and 'abstract' modifiers in FsVolumeSpi and FsDatasetSpi (Contributed by Lei (Eddy) Xu)"

This reverts commit 38ee9247cd3d01ffba88018ad1fd37f40e5969f5.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/01e12b7d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/01e12b7d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/01e12b7d

Branch: refs/heads/branch-2
Commit: 01e12b7d6985b46ec019985e9ee9d3eaaa5e161b
Parents: 38ee924
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Jul 8 16:59:22 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Wed Jul 8 16:59:22 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 -
 .../server/datanode/fsdataset/FsDatasetSpi.java | 149 +++++++++----------
 .../server/datanode/fsdataset/FsVolumeSpi.java  |  45 +++---
 3 files changed, 97 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/01e12b7d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 3d1caab..ac5887f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -368,9 +368,6 @@ Release 2.8.0 - UNRELEASED
     HDFS-8620. Clean up the checkstyle warinings about ClientProtocol.
     (Takanobu Asanuma via wheat9)
 
-    HDFS-8712. Remove 'public' and 'abstracta modifiers in FsVolumeSpi and
-    FsDatasetSpi (Lei (Eddy) Xu via vinayakumarb)
-
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01e12b7d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index af6a532..51bd07f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -72,7 +72,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   /**
    * A factory for creating {@link FsDatasetSpi} objects.
    */
-  abstract class Factory<D extends FsDatasetSpi<?>> {
+  public static abstract class Factory<D extends FsDatasetSpi<?>> {
     /** @return the configured factory. */
     public static Factory<?> getFactory(Configuration conf) {
       @SuppressWarnings("rawtypes")
@@ -182,7 +182,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * The caller must release the reference of each volume by calling
    * {@link FsVolumeReferences#close()}.
    */
-  FsVolumeReferences getFsVolumeReferences();
+  public FsVolumeReferences getFsVolumeReferences();
 
   /**
    * Add a new volume to the FsDataset.<p/>
@@ -193,7 +193,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param location      The storage location for the new volume.
    * @param nsInfos       Namespace information for the new volume.
    */
-  void addVolume(
+  public void addVolume(
       final StorageLocation location,
       final List<NamespaceInfo> nsInfos) throws IOException;
 
@@ -207,20 +207,20 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param clearFailure set true to clear the failure information about the
    *                     volumes.
    */
-  void removeVolumes(Set<File> volumes, boolean clearFailure);
+  public void removeVolumes(Set<File> volumes, boolean clearFailure);
 
   /** @return a storage with the given storage ID */
-  DatanodeStorage getStorage(final String storageUuid);
+  public DatanodeStorage getStorage(final String storageUuid);
 
   /** @return one or more storage reports for attached volumes. */
-  StorageReport[] getStorageReports(String bpid)
+  public StorageReport[] getStorageReports(String bpid)
       throws IOException;
 
   /** @return the volume that contains a replica of the block. */
-  V getVolume(ExtendedBlock b);
+  public V getVolume(ExtendedBlock b);
 
   /** @return a volume information map (name => info). */
-  Map<String, Object> getVolumeInfoMap();
+  public Map<String, Object> getVolumeInfoMap();
 
   /**
    * Returns info about volume failures.
@@ -230,17 +230,17 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   VolumeFailureSummary getVolumeFailureSummary();
 
   /** @return a list of finalized blocks for the given block pool. */
-  List<FinalizedReplica> getFinalizedBlocks(String bpid);
+  public List<FinalizedReplica> getFinalizedBlocks(String bpid);
 
   /** @return a list of finalized blocks for the given block pool. */
-  List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid);
+  public List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid);
 
   /**
    * Check whether the in-memory block record matches the block on the disk,
    * and, in case that they are not matched, update the record or mark it
    * as corrupted.
    */
-  void checkAndUpdate(String bpid, long blockId, File diskFile,
+  public void checkAndUpdate(String bpid, long blockId, File diskFile,
       File diskMetaFile, FsVolumeSpi vol) throws IOException;
 
   /**
@@ -249,15 +249,15 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    *         otherwise, return null.
    * @throws IOException
    */
-  LengthInputStream getMetaDataInputStream(ExtendedBlock b
+  public LengthInputStream getMetaDataInputStream(ExtendedBlock b
       ) throws IOException;
 
   /**
-   * Returns the specified block's on-disk length (excluding metadata).
+   * Returns the specified block's on-disk length (excluding metadata)
    * @return   the specified block's on-disk length (excluding metadta)
    * @throws IOException on error
    */
-  long getLength(ExtendedBlock b) throws IOException;
+  public long getLength(ExtendedBlock b) throws IOException;
 
   /**
    * Get reference to the replica meta info in the replicasMap. 
@@ -265,48 +265,47 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return replica from the replicas map
    */
   @Deprecated
-  Replica getReplica(String bpid, long blockId);
+  public Replica getReplica(String bpid, long blockId);
 
   /**
    * @return replica meta information
    */
-  String getReplicaString(String bpid, long blockId);
+  public String getReplicaString(String bpid, long blockId);
 
   /**
    * @return the generation stamp stored with the block.
    */
-  Block getStoredBlock(String bpid, long blkid) throws IOException;
-
+  public Block getStoredBlock(String bpid, long blkid) throws IOException;
+  
   /**
-   * Returns an input stream at specified offset of the specified block.
+   * Returns an input stream at specified offset of the specified block
    * @param b block
    * @param seekOffset offset with in the block to seek to
    * @return an input stream to read the contents of the specified block,
    *  starting at the offset
    * @throws IOException
    */
-  InputStream getBlockInputStream(ExtendedBlock b, long seekOffset)
+  public InputStream getBlockInputStream(ExtendedBlock b, long seekOffset)
             throws IOException;
 
   /**
-   * Returns an input stream at specified offset of the specified block.
+   * Returns an input stream at specified offset of the specified block
    * The block is still in the tmp directory and is not finalized
    * @return an input stream to read the contents of the specified block,
    *  starting at the offset
    * @throws IOException
    */
-  ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
+  public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
       long ckoff) throws IOException;
 
   /**
    * Creates a temporary replica and returns the meta information of the replica
-   * .
    * 
    * @param b block
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  ReplicaHandler createTemporary(StorageType storageType,
+  public ReplicaHandler createTemporary(StorageType storageType,
       ExtendedBlock b) throws IOException;
 
   /**
@@ -316,11 +315,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  ReplicaHandler createRbw(StorageType storageType,
+  public ReplicaHandler createRbw(StorageType storageType,
       ExtendedBlock b, boolean allowLazyPersist) throws IOException;
 
   /**
-   * Recovers a RBW replica and returns the meta info of the replica.
+   * Recovers a RBW replica and returns the meta info of the replica
    * 
    * @param b block
    * @param newGS the new generation stamp for the replica
@@ -329,7 +328,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  ReplicaHandler recoverRbw(ExtendedBlock b,
+  public ReplicaHandler recoverRbw(ExtendedBlock b,
       long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException;
 
   /**
@@ -337,11 +336,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param temporary the temporary replica being converted
    * @return the result RBW
    */
-  ReplicaInPipelineInterface convertTemporaryToRbw(
+  public ReplicaInPipelineInterface convertTemporaryToRbw(
       ExtendedBlock temporary) throws IOException;
 
   /**
-   * Append to a finalized replica and returns the meta info of the replica.
+   * Append to a finalized replica and returns the meta info of the replica
    * 
    * @param b block
    * @param newGS the new generation stamp for the replica
@@ -349,12 +348,12 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meata info of the replica which is being written to
    * @throws IOException
    */
-  ReplicaHandler append(ExtendedBlock b, long newGS,
+  public ReplicaHandler append(ExtendedBlock b, long newGS,
       long expectedBlockLen) throws IOException;
 
   /**
-   * Recover a failed append to a finalized replica and returns the meta
-   * info of the replica.
+   * Recover a failed append to a finalized replica
+   * and returns the meta info of the replica
    * 
    * @param b block
    * @param newGS the new generation stamp for the replica
@@ -362,11 +361,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException
    */
-  ReplicaHandler recoverAppend(
+  public ReplicaHandler recoverAppend(
       ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException;
   
   /**
-   * Recover a failed pipeline close.
+   * Recover a failed pipeline close
    * It bumps the replica's generation stamp and finalize it if RBW replica
    * 
    * @param b block
@@ -375,7 +374,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the storage uuid of the replica.
    * @throws IOException
    */
-  String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
+  public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
       ) throws IOException;
   
   /**
@@ -387,21 +386,21 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * block is been finalized. For instance, the block resides on an HDFS volume
    * that has been removed.
    */
-  void finalizeBlock(ExtendedBlock b) throws IOException;
+  public void finalizeBlock(ExtendedBlock b) throws IOException;
 
   /**
    * Unfinalizes the block previously opened for writing using writeToBlock.
    * The temporary file associated with this block is deleted.
    * @throws IOException
    */
-  void unfinalizeBlock(ExtendedBlock b) throws IOException;
+  public void unfinalizeBlock(ExtendedBlock b) throws IOException;
 
   /**
    * Returns one block report per volume.
    * @param bpid Block Pool Id
    * @return - a map of DatanodeStorage to block report for the volume.
    */
-  Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
+  public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
 
   /**
    * Returns the cache report - the full list of cached block IDs of a
@@ -409,10 +408,10 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param   bpid Block Pool Id
    * @return  the cache report - the full list of cached block IDs.
    */
-  List<Long> getCacheReport(String bpid);
+  public List<Long> getCacheReport(String bpid);
 
   /** Does the dataset contain the block? */
-  boolean contains(ExtendedBlock block);
+  public boolean contains(ExtendedBlock block);
 
   /**
    * Check if a block is valid.
@@ -432,7 +431,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * 
    * @throws IOException                       May be thrown from the methods called. 
    */
-  void checkBlock(ExtendedBlock b, long minLength, ReplicaState state)
+  public void checkBlock(ExtendedBlock b, long minLength, ReplicaState state)
       throws ReplicaNotFoundException, UnexpectedReplicaStateException,
       FileNotFoundException, EOFException, IOException;
       
@@ -441,13 +440,13 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * Is the block valid?
    * @return - true if the specified block is valid
    */
-  boolean isValidBlock(ExtendedBlock b);
+  public boolean isValidBlock(ExtendedBlock b);
 
   /**
    * Is the block a valid RBW?
    * @return - true if the specified block is a valid RBW
    */
-  boolean isValidRbw(ExtendedBlock b);
+  public boolean isValidRbw(ExtendedBlock b);
 
   /**
    * Invalidates the specified blocks
@@ -455,21 +454,21 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param invalidBlks - the blocks to be invalidated
    * @throws IOException
    */
-  void invalidate(String bpid, Block invalidBlks[]) throws IOException;
+  public void invalidate(String bpid, Block invalidBlks[]) throws IOException;
 
   /**
    * Caches the specified blocks
    * @param bpid Block pool id
    * @param blockIds - block ids to cache
    */
-  void cache(String bpid, long[] blockIds);
+  public void cache(String bpid, long[] blockIds);
 
   /**
    * Uncaches the specified blocks
    * @param bpid Block pool id
    * @param blockIds - blocks ids to uncache
    */
-  void uncache(String bpid, long[] blockIds);
+  public void uncache(String bpid, long[] blockIds);
 
   /**
    * Determine if the specified block is cached.
@@ -477,18 +476,18 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param blockIds - block id
    * @return true if the block is cached
    */
-  boolean isCached(String bpid, long blockId);
+  public boolean isCached(String bpid, long blockId);
 
     /**
      * Check if all the data directories are healthy
      * @return A set of unhealthy data directories.
      */
-  Set<File> checkDataDir();
+  public Set<File> checkDataDir();
 
   /**
    * Shutdown the FSDataset
    */
-  void shutdown();
+  public void shutdown();
 
   /**
    * Sets the file pointer of the checksum stream so that the last checksum
@@ -498,7 +497,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param checksumSize number of bytes each checksum has
    * @throws IOException
    */
-  void adjustCrcChannelPosition(ExtendedBlock b,
+  public void adjustCrcChannelPosition(ExtendedBlock b,
       ReplicaOutputStreams outs, int checksumSize) throws IOException;
 
   /**
@@ -506,7 +505,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return true if more than the minimum number of valid volumes are left 
    * in the FSDataSet.
    */
-  boolean hasEnoughResource();
+  public boolean hasEnoughResource();
 
   /**
    * Get visible length of the specified replica.
@@ -518,14 +517,14 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return actual state of the replica on this data-node or 
    * null if data-node does not have the replica.
    */
-  ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock
+  public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock
       ) throws IOException;
 
   /**
    * Update replica's generation stamp and length and finalize it.
    * @return the ID of storage that stores the block
    */
-  String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
+  public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
       long recoveryId, long newBlockId, long newLength) throws IOException;
 
   /**
@@ -533,14 +532,14 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param bpid Block pool Id
    * @param conf Configuration
    */
-  void addBlockPool(String bpid, Configuration conf) throws IOException;
-
+  public void addBlockPool(String bpid, Configuration conf) throws IOException;
+  
   /**
    * Shutdown and remove the block pool from underlying storage.
    * @param bpid Block pool Id to be removed
    */
-  void shutdownBlockPool(String bpid) ;
-
+  public void shutdownBlockPool(String bpid) ;
+  
   /**
    * Deletes the block pool directories. If force is false, directories are 
    * deleted only if no block files exist for the block pool. If force 
@@ -552,12 +551,12 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    *        directory for the blockpool is deleted along with its contents.
    * @throws IOException
    */
-  void deleteBlockPool(String bpid, boolean force) throws IOException;
-
+  public void deleteBlockPool(String bpid, boolean force) throws IOException;
+  
   /**
    * Get {@link BlockLocalPathInfo} for the given block.
    */
-  BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b
+  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b
       ) throws IOException;
 
   /**
@@ -569,7 +568,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return metadata Metadata for the list of blocks
    * @throws IOException
    */
-  HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid,
+  public HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid,
       long[] blockIds) throws IOException;
 
   /**
@@ -577,51 +576,51 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * moved to a separate trash directory instead of being deleted immediately.
    * This can be useful for example during rolling upgrades.
    */
-  void enableTrash(String bpid);
+  public void enableTrash(String bpid);
 
   /**
    * Clear trash
    */
-  void clearTrash(String bpid);
+  public void clearTrash(String bpid);
 
   /**
    * @return true when trash is enabled
    */
-  boolean trashEnabled(String bpid);
+  public boolean trashEnabled(String bpid);
 
   /**
    * Create a marker file indicating that a rolling upgrade is in progress.
    */
-  void setRollingUpgradeMarker(String bpid) throws IOException;
+  public void setRollingUpgradeMarker(String bpid) throws IOException;
 
   /**
    * Delete the rolling upgrade marker file if it exists.
    * @param bpid
    */
-  void clearRollingUpgradeMarker(String bpid) throws IOException;
+  public void clearRollingUpgradeMarker(String bpid) throws IOException;
 
   /**
-   * submit a sync_file_range request to AsyncDiskService.
+   * submit a sync_file_range request to AsyncDiskService
    */
-  void submitBackgroundSyncFileRangeRequest(final ExtendedBlock block,
+  public void submitBackgroundSyncFileRangeRequest(final ExtendedBlock block,
       final FileDescriptor fd, final long offset, final long nbytes,
       final int flags);
 
   /**
    * Callback from RamDiskAsyncLazyPersistService upon async lazy persist task end
    */
-  void onCompleteLazyPersist(String bpId, long blockId,
+   public void onCompleteLazyPersist(String bpId, long blockId,
       long creationTime, File[] savedFiles, V targetVolume);
 
    /**
     * Callback from RamDiskAsyncLazyPersistService upon async lazy persist task fail
     */
-   void onFailLazyPersist(String bpId, long blockId);
+   public void onFailLazyPersist(String bpId, long blockId);
 
     /**
      * Move block from one storage to another storage
      */
-   ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
+    public ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
         StorageType targetStorageType) throws IOException;
 
   /**
@@ -630,15 +629,15 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    *
    * It is a no-op when dfs.datanode.block-pinning.enabled is set to false.
    */
-  void setPinning(ExtendedBlock block) throws IOException;
+  public void setPinning(ExtendedBlock block) throws IOException;
 
   /**
    * Check whether the block was pinned
    */
-  boolean getPinning(ExtendedBlock block) throws IOException;
-
+  public boolean getPinning(ExtendedBlock block) throws IOException;
+  
   /**
    * Confirm whether the block is deleting
    */
-  boolean isDeletingBlock(String bpid, long blockId);
+  public boolean isDeletingBlock(String bpid, long blockId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01e12b7d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index 97a93bb..d34022d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@ -39,35 +39,35 @@ public interface FsVolumeSpi {
   FsVolumeReference obtainReference() throws ClosedChannelException;
 
   /** @return the StorageUuid of the volume */
-  String getStorageID();
+  public String getStorageID();
 
   /** @return a list of block pools. */
-  String[] getBlockPoolList();
+  public String[] getBlockPoolList();
 
   /** @return the available storage space in bytes. */
-  long getAvailable() throws IOException;
+  public long getAvailable() throws IOException;
 
   /** @return the base path to the volume */
-  String getBasePath();
+  public String getBasePath();
 
   /** @return the path to the volume */
-  String getPath(String bpid) throws IOException;
+  public String getPath(String bpid) throws IOException;
 
   /** @return the directory for the finalized blocks in the block pool. */
-  File getFinalizedDir(String bpid) throws IOException;
+  public File getFinalizedDir(String bpid) throws IOException;
   
-  StorageType getStorageType();
+  public StorageType getStorageType();
 
   /**
    * Reserve disk space for an RBW block so a writer does not run out of
    * space before the block is full.
    */
-  void reserveSpaceForRbw(long bytesToReserve);
+  public void reserveSpaceForRbw(long bytesToReserve);
 
   /**
    * Release disk space previously reserved for RBW block.
    */
-  void releaseReservedSpace(long bytesToRelease);
+  public void releaseReservedSpace(long bytesToRelease);
 
   /** Returns true if the volume is NOT backed by persistent storage. */
   public boolean isTransientStorage();
@@ -78,7 +78,7 @@ public interface FsVolumeSpi {
    * bytesToRelease will be rounded down to the OS page size since locked
    * memory reservation must always be a multiple of the page size.
    */
-  void releaseLockedMemory(long bytesToRelease);
+  public void releaseLockedMemory(long bytesToRelease);
 
   /**
    * BlockIterator will return ExtendedBlock entries from a block pool in
@@ -90,7 +90,7 @@ public interface FsVolumeSpi {
    *
    * Closing the iterator does not save it.  You must call save to save it.
    */
-  interface BlockIterator extends Closeable {
+  public interface BlockIterator extends Closeable {
     /**
      * Get the next block.<p/>
      *
@@ -107,17 +107,17 @@ public interface FsVolumeSpi {
      *                         this volume.  In this case, EOF will be set on
      *                         the iterator.
      */
-    ExtendedBlock nextBlock() throws IOException;
+    public ExtendedBlock nextBlock() throws IOException;
 
     /**
      * Returns true if we got to the end of the block pool.
      */
-    boolean atEnd();
+    public boolean atEnd();
 
     /**
      * Repositions the iterator at the beginning of the block pool.
      */
-    void rewind();
+    public void rewind();
 
     /**
      * Save this block iterator to the underlying volume.
@@ -127,7 +127,7 @@ public interface FsVolumeSpi {
      * @throws IOException   If there was an error when saving the block
      *                         iterator.
      */
-    void save() throws IOException;
+    public void save() throws IOException;
 
     /**
      * Set the maximum staleness of entries that we will return.<p/>
@@ -138,25 +138,25 @@ public interface FsVolumeSpi {
      * to 0, consumers of this API must handle race conditions where block
      * disappear before they can be processed.
      */
-    void setMaxStalenessMs(long maxStalenessMs);
+    public void setMaxStalenessMs(long maxStalenessMs);
 
     /**
      * Get the wall-clock time, measured in milliseconds since the Epoch,
      * when this iterator was created.
      */
-    long getIterStartMs();
+    public long getIterStartMs();
 
     /**
      * Get the wall-clock time, measured in milliseconds since the Epoch,
      * when this iterator was last saved.  Returns iterStartMs if the
      * iterator was never saved.
      */
-    long getLastSavedMs();
+    public long getLastSavedMs();
 
     /**
      * Get the id of the block pool which this iterator traverses.
      */
-    String getBlockPoolId();
+    public String getBlockPoolId();
   }
 
   /**
@@ -168,7 +168,7 @@ public interface FsVolumeSpi {
    *
    * @return                 The new block iterator.
    */
-  BlockIterator newBlockIterator(String bpid, String name);
+  public BlockIterator newBlockIterator(String bpid, String name);
 
   /**
    * Load a saved block iterator.
@@ -180,10 +180,11 @@ public interface FsVolumeSpi {
    * @throws IOException     If there was an IO error loading the saved
    *                           block iterator.
    */
-  BlockIterator loadBlockIterator(String bpid, String name) throws IOException;
+  public BlockIterator loadBlockIterator(String bpid, String name)
+      throws IOException;
 
   /**
    * Get the FSDatasetSpi which this volume is a part of.
    */
-  FsDatasetSpi getDataset();
+  public FsDatasetSpi getDataset();
 }


[2/2] hadoop git commit: HDFS-8712. Remove 'public' and 'abstract' modifiers in FsVolumeSpi and FsDatasetSpi (Contributed by Lei (Eddy) Xu)

Posted by vi...@apache.org.
HDFS-8712. Remove 'public' and 'abstract' modifiers in FsVolumeSpi and FsDatasetSpi (Contributed by Lei (Eddy) Xu)

(cherry picked from commit bd4e10900cc53a2768c31cc29fdb3698684bc2a0)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6fe2e7c3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6fe2e7c3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6fe2e7c3

Branch: refs/heads/branch-2
Commit: 6fe2e7c3f398b53f78ebedec55bab4bb06c5b72b
Parents: 01e12b7
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Jul 8 16:34:54 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Wed Jul 8 17:03:18 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../server/datanode/fsdataset/FsDatasetSpi.java | 149 ++++++++++---------
 .../server/datanode/fsdataset/FsVolumeSpi.java  |  51 ++++---
 3 files changed, 103 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fe2e7c3/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ac5887f..3d1caab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -368,6 +368,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8620. Clean up the checkstyle warinings about ClientProtocol.
     (Takanobu Asanuma via wheat9)
 
+    HDFS-8712. Remove 'public' and 'abstracta modifiers in FsVolumeSpi and
+    FsDatasetSpi (Lei (Eddy) Xu via vinayakumarb)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fe2e7c3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 51bd07f..af6a532 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -72,7 +72,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   /**
    * A factory for creating {@link FsDatasetSpi} objects.
    */
-  public static abstract class Factory<D extends FsDatasetSpi<?>> {
+  abstract class Factory<D extends FsDatasetSpi<?>> {
     /** @return the configured factory. */
     public static Factory<?> getFactory(Configuration conf) {
       @SuppressWarnings("rawtypes")
@@ -182,7 +182,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * The caller must release the reference of each volume by calling
    * {@link FsVolumeReferences#close()}.
    */
-  public FsVolumeReferences getFsVolumeReferences();
+  FsVolumeReferences getFsVolumeReferences();
 
   /**
    * Add a new volume to the FsDataset.<p/>
@@ -193,7 +193,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param location      The storage location for the new volume.
    * @param nsInfos       Namespace information for the new volume.
    */
-  public void addVolume(
+  void addVolume(
       final StorageLocation location,
       final List<NamespaceInfo> nsInfos) throws IOException;
 
@@ -207,20 +207,20 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param clearFailure set true to clear the failure information about the
    *                     volumes.
    */
-  public void removeVolumes(Set<File> volumes, boolean clearFailure);
+  void removeVolumes(Set<File> volumes, boolean clearFailure);
 
   /** @return a storage with the given storage ID */
-  public DatanodeStorage getStorage(final String storageUuid);
+  DatanodeStorage getStorage(final String storageUuid);
 
   /** @return one or more storage reports for attached volumes. */
-  public StorageReport[] getStorageReports(String bpid)
+  StorageReport[] getStorageReports(String bpid)
       throws IOException;
 
   /** @return the volume that contains a replica of the block. */
-  public V getVolume(ExtendedBlock b);
+  V getVolume(ExtendedBlock b);
 
   /** @return a volume information map (name => info). */
-  public Map<String, Object> getVolumeInfoMap();
+  Map<String, Object> getVolumeInfoMap();
 
   /**
    * Returns info about volume failures.
@@ -230,17 +230,17 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   VolumeFailureSummary getVolumeFailureSummary();
 
   /** @return a list of finalized blocks for the given block pool. */
-  public List<FinalizedReplica> getFinalizedBlocks(String bpid);
+  List<FinalizedReplica> getFinalizedBlocks(String bpid);
 
   /** @return a list of finalized blocks for the given block pool. */
-  public List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid);
+  List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid);
 
   /**
    * Check whether the in-memory block record matches the block on the disk,
    * and, in case that they are not matched, update the record or mark it
    * as corrupted.
    */
-  public void checkAndUpdate(String bpid, long blockId, File diskFile,
+  void checkAndUpdate(String bpid, long blockId, File diskFile,
       File diskMetaFile, FsVolumeSpi vol) throws IOException;
 
   /**
@@ -249,15 +249,15 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    *         otherwise, return null.
    * @throws IOException
    */
-  public LengthInputStream getMetaDataInputStream(ExtendedBlock b
+  LengthInputStream getMetaDataInputStream(ExtendedBlock b
       ) throws IOException;
 
   /**
-   * Returns the specified block's on-disk length (excluding metadata)
+   * Returns the specified block's on-disk length (excluding metadata).
    * @return   the specified block's on-disk length (excluding metadta)
    * @throws IOException on error
    */
-  public long getLength(ExtendedBlock b) throws IOException;
+  long getLength(ExtendedBlock b) throws IOException;
 
   /**
    * Get reference to the replica meta info in the replicasMap. 
@@ -265,47 +265,48 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return replica from the replicas map
    */
   @Deprecated
-  public Replica getReplica(String bpid, long blockId);
+  Replica getReplica(String bpid, long blockId);
 
   /**
    * @return replica meta information
    */
-  public String getReplicaString(String bpid, long blockId);
+  String getReplicaString(String bpid, long blockId);
 
   /**
    * @return the generation stamp stored with the block.
    */
-  public Block getStoredBlock(String bpid, long blkid) throws IOException;
-  
+  Block getStoredBlock(String bpid, long blkid) throws IOException;
+
   /**
-   * Returns an input stream at specified offset of the specified block
+   * Returns an input stream at specified offset of the specified block.
    * @param b block
    * @param seekOffset offset with in the block to seek to
    * @return an input stream to read the contents of the specified block,
    *  starting at the offset
    * @throws IOException
    */
-  public InputStream getBlockInputStream(ExtendedBlock b, long seekOffset)
+  InputStream getBlockInputStream(ExtendedBlock b, long seekOffset)
             throws IOException;
 
   /**
-   * Returns an input stream at specified offset of the specified block
+   * Returns an input stream at specified offset of the specified block.
    * The block is still in the tmp directory and is not finalized
    * @return an input stream to read the contents of the specified block,
    *  starting at the offset
    * @throws IOException
    */
-  public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
+  ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
       long ckoff) throws IOException;
 
   /**
    * Creates a temporary replica and returns the meta information of the replica
+   * .
    * 
    * @param b block
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  public ReplicaHandler createTemporary(StorageType storageType,
+  ReplicaHandler createTemporary(StorageType storageType,
       ExtendedBlock b) throws IOException;
 
   /**
@@ -315,11 +316,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  public ReplicaHandler createRbw(StorageType storageType,
+  ReplicaHandler createRbw(StorageType storageType,
       ExtendedBlock b, boolean allowLazyPersist) throws IOException;
 
   /**
-   * Recovers a RBW replica and returns the meta info of the replica
+   * Recovers a RBW replica and returns the meta info of the replica.
    * 
    * @param b block
    * @param newGS the new generation stamp for the replica
@@ -328,7 +329,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  public ReplicaHandler recoverRbw(ExtendedBlock b,
+  ReplicaHandler recoverRbw(ExtendedBlock b,
       long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException;
 
   /**
@@ -336,11 +337,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param temporary the temporary replica being converted
    * @return the result RBW
    */
-  public ReplicaInPipelineInterface convertTemporaryToRbw(
+  ReplicaInPipelineInterface convertTemporaryToRbw(
       ExtendedBlock temporary) throws IOException;
 
   /**
-   * Append to a finalized replica and returns the meta info of the replica
+   * Append to a finalized replica and returns the meta info of the replica.
    * 
    * @param b block
    * @param newGS the new generation stamp for the replica
@@ -348,12 +349,12 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meata info of the replica which is being written to
    * @throws IOException
    */
-  public ReplicaHandler append(ExtendedBlock b, long newGS,
+  ReplicaHandler append(ExtendedBlock b, long newGS,
       long expectedBlockLen) throws IOException;
 
   /**
-   * Recover a failed append to a finalized replica
-   * and returns the meta info of the replica
+   * Recover a failed append to a finalized replica and returns the meta
+   * info of the replica.
    * 
    * @param b block
    * @param newGS the new generation stamp for the replica
@@ -361,11 +362,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException
    */
-  public ReplicaHandler recoverAppend(
+  ReplicaHandler recoverAppend(
       ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException;
   
   /**
-   * Recover a failed pipeline close
+   * Recover a failed pipeline close.
    * It bumps the replica's generation stamp and finalize it if RBW replica
    * 
    * @param b block
@@ -374,7 +375,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the storage uuid of the replica.
    * @throws IOException
    */
-  public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
+  String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
       ) throws IOException;
   
   /**
@@ -386,21 +387,21 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * block is been finalized. For instance, the block resides on an HDFS volume
    * that has been removed.
    */
-  public void finalizeBlock(ExtendedBlock b) throws IOException;
+  void finalizeBlock(ExtendedBlock b) throws IOException;
 
   /**
    * Unfinalizes the block previously opened for writing using writeToBlock.
    * The temporary file associated with this block is deleted.
    * @throws IOException
    */
-  public void unfinalizeBlock(ExtendedBlock b) throws IOException;
+  void unfinalizeBlock(ExtendedBlock b) throws IOException;
 
   /**
    * Returns one block report per volume.
    * @param bpid Block Pool Id
    * @return - a map of DatanodeStorage to block report for the volume.
    */
-  public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
+  Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
 
   /**
    * Returns the cache report - the full list of cached block IDs of a
@@ -408,10 +409,10 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param   bpid Block Pool Id
    * @return  the cache report - the full list of cached block IDs.
    */
-  public List<Long> getCacheReport(String bpid);
+  List<Long> getCacheReport(String bpid);
 
   /** Does the dataset contain the block? */
-  public boolean contains(ExtendedBlock block);
+  boolean contains(ExtendedBlock block);
 
   /**
    * Check if a block is valid.
@@ -431,7 +432,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * 
    * @throws IOException                       May be thrown from the methods called. 
    */
-  public void checkBlock(ExtendedBlock b, long minLength, ReplicaState state)
+  void checkBlock(ExtendedBlock b, long minLength, ReplicaState state)
       throws ReplicaNotFoundException, UnexpectedReplicaStateException,
       FileNotFoundException, EOFException, IOException;
       
@@ -440,13 +441,13 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * Is the block valid?
    * @return - true if the specified block is valid
    */
-  public boolean isValidBlock(ExtendedBlock b);
+  boolean isValidBlock(ExtendedBlock b);
 
   /**
    * Is the block a valid RBW?
    * @return - true if the specified block is a valid RBW
    */
-  public boolean isValidRbw(ExtendedBlock b);
+  boolean isValidRbw(ExtendedBlock b);
 
   /**
    * Invalidates the specified blocks
@@ -454,21 +455,21 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param invalidBlks - the blocks to be invalidated
    * @throws IOException
    */
-  public void invalidate(String bpid, Block invalidBlks[]) throws IOException;
+  void invalidate(String bpid, Block invalidBlks[]) throws IOException;
 
   /**
    * Caches the specified blocks
    * @param bpid Block pool id
    * @param blockIds - block ids to cache
    */
-  public void cache(String bpid, long[] blockIds);
+  void cache(String bpid, long[] blockIds);
 
   /**
    * Uncaches the specified blocks
    * @param bpid Block pool id
    * @param blockIds - blocks ids to uncache
    */
-  public void uncache(String bpid, long[] blockIds);
+  void uncache(String bpid, long[] blockIds);
 
   /**
    * Determine if the specified block is cached.
@@ -476,18 +477,18 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param blockIds - block id
    * @return true if the block is cached
    */
-  public boolean isCached(String bpid, long blockId);
+  boolean isCached(String bpid, long blockId);
 
     /**
      * Check if all the data directories are healthy
      * @return A set of unhealthy data directories.
      */
-  public Set<File> checkDataDir();
+  Set<File> checkDataDir();
 
   /**
    * Shutdown the FSDataset
    */
-  public void shutdown();
+  void shutdown();
 
   /**
    * Sets the file pointer of the checksum stream so that the last checksum
@@ -497,7 +498,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param checksumSize number of bytes each checksum has
    * @throws IOException
    */
-  public void adjustCrcChannelPosition(ExtendedBlock b,
+  void adjustCrcChannelPosition(ExtendedBlock b,
       ReplicaOutputStreams outs, int checksumSize) throws IOException;
 
   /**
@@ -505,7 +506,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return true if more than the minimum number of valid volumes are left 
    * in the FSDataSet.
    */
-  public boolean hasEnoughResource();
+  boolean hasEnoughResource();
 
   /**
    * Get visible length of the specified replica.
@@ -517,14 +518,14 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return actual state of the replica on this data-node or 
    * null if data-node does not have the replica.
    */
-  public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock
+  ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock
       ) throws IOException;
 
   /**
    * Update replica's generation stamp and length and finalize it.
    * @return the ID of storage that stores the block
    */
-  public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
+  String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
       long recoveryId, long newBlockId, long newLength) throws IOException;
 
   /**
@@ -532,14 +533,14 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param bpid Block pool Id
    * @param conf Configuration
    */
-  public void addBlockPool(String bpid, Configuration conf) throws IOException;
-  
+  void addBlockPool(String bpid, Configuration conf) throws IOException;
+
   /**
    * Shutdown and remove the block pool from underlying storage.
    * @param bpid Block pool Id to be removed
    */
-  public void shutdownBlockPool(String bpid) ;
-  
+  void shutdownBlockPool(String bpid) ;
+
   /**
    * Deletes the block pool directories. If force is false, directories are 
    * deleted only if no block files exist for the block pool. If force 
@@ -551,12 +552,12 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    *        directory for the blockpool is deleted along with its contents.
    * @throws IOException
    */
-  public void deleteBlockPool(String bpid, boolean force) throws IOException;
-  
+  void deleteBlockPool(String bpid, boolean force) throws IOException;
+
   /**
    * Get {@link BlockLocalPathInfo} for the given block.
    */
-  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b
+  BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b
       ) throws IOException;
 
   /**
@@ -568,7 +569,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return metadata Metadata for the list of blocks
    * @throws IOException
    */
-  public HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid,
+  HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid,
       long[] blockIds) throws IOException;
 
   /**
@@ -576,51 +577,51 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * moved to a separate trash directory instead of being deleted immediately.
    * This can be useful for example during rolling upgrades.
    */
-  public void enableTrash(String bpid);
+  void enableTrash(String bpid);
 
   /**
    * Clear trash
    */
-  public void clearTrash(String bpid);
+  void clearTrash(String bpid);
 
   /**
    * @return true when trash is enabled
    */
-  public boolean trashEnabled(String bpid);
+  boolean trashEnabled(String bpid);
 
   /**
    * Create a marker file indicating that a rolling upgrade is in progress.
    */
-  public void setRollingUpgradeMarker(String bpid) throws IOException;
+  void setRollingUpgradeMarker(String bpid) throws IOException;
 
   /**
    * Delete the rolling upgrade marker file if it exists.
    * @param bpid
    */
-  public void clearRollingUpgradeMarker(String bpid) throws IOException;
+  void clearRollingUpgradeMarker(String bpid) throws IOException;
 
   /**
-   * submit a sync_file_range request to AsyncDiskService
+   * submit a sync_file_range request to AsyncDiskService.
    */
-  public void submitBackgroundSyncFileRangeRequest(final ExtendedBlock block,
+  void submitBackgroundSyncFileRangeRequest(final ExtendedBlock block,
       final FileDescriptor fd, final long offset, final long nbytes,
       final int flags);
 
   /**
    * Callback from RamDiskAsyncLazyPersistService upon async lazy persist task end
    */
-   public void onCompleteLazyPersist(String bpId, long blockId,
+  void onCompleteLazyPersist(String bpId, long blockId,
       long creationTime, File[] savedFiles, V targetVolume);
 
    /**
     * Callback from RamDiskAsyncLazyPersistService upon async lazy persist task fail
     */
-   public void onFailLazyPersist(String bpId, long blockId);
+   void onFailLazyPersist(String bpId, long blockId);
 
     /**
      * Move block from one storage to another storage
      */
-    public ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
+   ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
         StorageType targetStorageType) throws IOException;
 
   /**
@@ -629,15 +630,15 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    *
    * It is a no-op when dfs.datanode.block-pinning.enabled is set to false.
    */
-  public void setPinning(ExtendedBlock block) throws IOException;
+  void setPinning(ExtendedBlock block) throws IOException;
 
   /**
    * Check whether the block was pinned
    */
-  public boolean getPinning(ExtendedBlock block) throws IOException;
-  
+  boolean getPinning(ExtendedBlock block) throws IOException;
+
   /**
    * Confirm whether the block is deleting
    */
-  public boolean isDeletingBlock(String bpid, long blockId);
+  boolean isDeletingBlock(String bpid, long blockId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fe2e7c3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index d34022d..ee01924 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@ -39,38 +39,38 @@ public interface FsVolumeSpi {
   FsVolumeReference obtainReference() throws ClosedChannelException;
 
   /** @return the StorageUuid of the volume */
-  public String getStorageID();
+  String getStorageID();
 
   /** @return a list of block pools. */
-  public String[] getBlockPoolList();
+  String[] getBlockPoolList();
 
   /** @return the available storage space in bytes. */
-  public long getAvailable() throws IOException;
+  long getAvailable() throws IOException;
 
   /** @return the base path to the volume */
-  public String getBasePath();
+  String getBasePath();
 
   /** @return the path to the volume */
-  public String getPath(String bpid) throws IOException;
+  String getPath(String bpid) throws IOException;
 
   /** @return the directory for the finalized blocks in the block pool. */
-  public File getFinalizedDir(String bpid) throws IOException;
+  File getFinalizedDir(String bpid) throws IOException;
   
-  public StorageType getStorageType();
+  StorageType getStorageType();
+
+  /** Returns true if the volume is NOT backed by persistent storage. */
+  boolean isTransientStorage();
 
   /**
    * Reserve disk space for an RBW block so a writer does not run out of
    * space before the block is full.
    */
-  public void reserveSpaceForRbw(long bytesToReserve);
+  void reserveSpaceForRbw(long bytesToReserve);
 
   /**
    * Release disk space previously reserved for RBW block.
    */
-  public void releaseReservedSpace(long bytesToRelease);
-
-  /** Returns true if the volume is NOT backed by persistent storage. */
-  public boolean isTransientStorage();
+  void releaseReservedSpace(long bytesToRelease);
 
   /**
    * Release reserved memory for an RBW block written to transient storage
@@ -78,7 +78,7 @@ public interface FsVolumeSpi {
    * bytesToRelease will be rounded down to the OS page size since locked
    * memory reservation must always be a multiple of the page size.
    */
-  public void releaseLockedMemory(long bytesToRelease);
+  void releaseLockedMemory(long bytesToRelease);
 
   /**
    * BlockIterator will return ExtendedBlock entries from a block pool in
@@ -90,7 +90,7 @@ public interface FsVolumeSpi {
    *
    * Closing the iterator does not save it.  You must call save to save it.
    */
-  public interface BlockIterator extends Closeable {
+  interface BlockIterator extends Closeable {
     /**
      * Get the next block.<p/>
      *
@@ -107,17 +107,17 @@ public interface FsVolumeSpi {
      *                         this volume.  In this case, EOF will be set on
      *                         the iterator.
      */
-    public ExtendedBlock nextBlock() throws IOException;
+    ExtendedBlock nextBlock() throws IOException;
 
     /**
      * Returns true if we got to the end of the block pool.
      */
-    public boolean atEnd();
+    boolean atEnd();
 
     /**
      * Repositions the iterator at the beginning of the block pool.
      */
-    public void rewind();
+    void rewind();
 
     /**
      * Save this block iterator to the underlying volume.
@@ -127,7 +127,7 @@ public interface FsVolumeSpi {
      * @throws IOException   If there was an error when saving the block
      *                         iterator.
      */
-    public void save() throws IOException;
+    void save() throws IOException;
 
     /**
      * Set the maximum staleness of entries that we will return.<p/>
@@ -138,25 +138,25 @@ public interface FsVolumeSpi {
      * to 0, consumers of this API must handle race conditions where block
      * disappear before they can be processed.
      */
-    public void setMaxStalenessMs(long maxStalenessMs);
+    void setMaxStalenessMs(long maxStalenessMs);
 
     /**
      * Get the wall-clock time, measured in milliseconds since the Epoch,
      * when this iterator was created.
      */
-    public long getIterStartMs();
+    long getIterStartMs();
 
     /**
      * Get the wall-clock time, measured in milliseconds since the Epoch,
      * when this iterator was last saved.  Returns iterStartMs if the
      * iterator was never saved.
      */
-    public long getLastSavedMs();
+    long getLastSavedMs();
 
     /**
      * Get the id of the block pool which this iterator traverses.
      */
-    public String getBlockPoolId();
+    String getBlockPoolId();
   }
 
   /**
@@ -168,7 +168,7 @@ public interface FsVolumeSpi {
    *
    * @return                 The new block iterator.
    */
-  public BlockIterator newBlockIterator(String bpid, String name);
+  BlockIterator newBlockIterator(String bpid, String name);
 
   /**
    * Load a saved block iterator.
@@ -180,11 +180,10 @@ public interface FsVolumeSpi {
    * @throws IOException     If there was an IO error loading the saved
    *                           block iterator.
    */
-  public BlockIterator loadBlockIterator(String bpid, String name)
-      throws IOException;
+  BlockIterator loadBlockIterator(String bpid, String name) throws IOException;
 
   /**
    * Get the FSDatasetSpi which this volume is a part of.
    */
-  public FsDatasetSpi getDataset();
+  FsDatasetSpi getDataset();
 }