|
@@ -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);
|
|
|
}
|