001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hdfs.server.datanode.fsdataset;
019
020
021import java.io.File;
022import java.io.IOException;
023import java.io.InputStream;
024import java.util.List;
025import java.util.Map;
026
027import org.apache.hadoop.classification.InterfaceAudience;
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.hdfs.DFSConfigKeys;
030import org.apache.hadoop.hdfs.protocol.Block;
031import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
032import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
033import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
034import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
035import org.apache.hadoop.hdfs.server.datanode.DataNode;
036import org.apache.hadoop.hdfs.server.datanode.DataStorage;
037import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
038import org.apache.hadoop.hdfs.server.datanode.Replica;
039import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
040import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
041import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
042import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
043import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
044import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
045import org.apache.hadoop.hdfs.server.protocol.StorageReport;
046import org.apache.hadoop.util.DiskChecker.DiskErrorException;
047import org.apache.hadoop.util.ReflectionUtils;
048
049/**
050 * This is a service provider interface for the underlying storage that
051 * stores replicas for a data node.
052 * The default implementation stores replicas on local drives. 
053 */
054@InterfaceAudience.Private
055public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
056  /**
057   * A factory for creating {@link FsDatasetSpi} objects.
058   */
059  public static abstract class Factory<D extends FsDatasetSpi<?>> {
060    /** @return the configured factory. */
061    public static Factory<?> getFactory(Configuration conf) {
062      @SuppressWarnings("rawtypes")
063      final Class<? extends Factory> clazz = conf.getClass(
064          DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
065          FsDatasetFactory.class,
066          Factory.class);
067      return ReflectionUtils.newInstance(clazz, conf);
068    }
069
070    /** Create a new object. */
071    public abstract D newInstance(DataNode datanode, DataStorage storage,
072        Configuration conf) throws IOException;
073
074    /** Does the factory create simulated objects? */
075    public boolean isSimulated() {
076      return false;
077    }
078  }
079
080  /**
081   * Create rolling logs.
082   * 
083   * @param prefix the prefix of the log names.
084   * @return rolling logs
085   */
086  public RollingLogs createRollingLogs(String bpid, String prefix
087      ) throws IOException;
088
089  /** @return a list of volumes. */
090  public List<V> getVolumes();
091
092  /** @return one or more storage reports for attached volumes. */
093  public StorageReport[] getStorageReports(String bpid)
094      throws IOException;
095
096  /** @return the volume that contains a replica of the block. */
097  public V getVolume(ExtendedBlock b);
098
099  /** @return a volume information map (name => info). */
100  public Map<String, Object> getVolumeInfoMap();
101
102  /** @return a list of finalized blocks for the given block pool. */
103  public List<FinalizedReplica> getFinalizedBlocks(String bpid);
104
105  /**
106   * Check whether the in-memory block record matches the block on the disk,
107   * and, in case that they are not matched, update the record or mark it
108   * as corrupted.
109   */
110  public void checkAndUpdate(String bpid, long blockId, File diskFile,
111      File diskMetaFile, FsVolumeSpi vol);
112
113  /**
114   * @param b - the block
115   * @return a stream if the meta-data of the block exists;
116   *         otherwise, return null.
117   * @throws IOException
118   */
119  public LengthInputStream getMetaDataInputStream(ExtendedBlock b
120      ) throws IOException;
121
122  /**
123   * Returns the specified block's on-disk length (excluding metadata)
124   * @param b
125   * @return   the specified block's on-disk length (excluding metadta)
126   * @throws IOException
127   */
128  public long getLength(ExtendedBlock b) throws IOException;
129
130  /**
131   * Get reference to the replica meta info in the replicasMap. 
132   * To be called from methods that are synchronized on {@link FSDataset}
133   * @param blockId
134   * @return replica from the replicas map
135   */
136  @Deprecated
137  public Replica getReplica(String bpid, long blockId);
138
139  /**
140   * @return replica meta information
141   */
142  public String getReplicaString(String bpid, long blockId);
143
144  /**
145   * @return the generation stamp stored with the block.
146   */
147  public Block getStoredBlock(String bpid, long blkid) throws IOException;
148  
149  /**
150   * Returns an input stream at specified offset of the specified block
151   * @param b
152   * @param seekOffset
153   * @return an input stream to read the contents of the specified block,
154   *  starting at the offset
155   * @throws IOException
156   */
157  public InputStream getBlockInputStream(ExtendedBlock b, long seekOffset)
158            throws IOException;
159
160  /**
161   * Returns an input stream at specified offset of the specified block
162   * The block is still in the tmp directory and is not finalized
163   * @param b
164   * @param blkoff
165   * @param ckoff
166   * @return an input stream to read the contents of the specified block,
167   *  starting at the offset
168   * @throws IOException
169   */
170  public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
171      long ckoff) throws IOException;
172
173  /**
174   * Creates a temporary replica and returns the meta information of the replica
175   * 
176   * @param b block
177   * @return the meta info of the replica which is being written to
178   * @throws IOException if an error occurs
179   */
180  public ReplicaInPipelineInterface createTemporary(ExtendedBlock b
181      ) throws IOException;
182
183  /**
184   * Creates a RBW replica and returns the meta info of the replica
185   * 
186   * @param b block
187   * @return the meta info of the replica which is being written to
188   * @throws IOException if an error occurs
189   */
190  public ReplicaInPipelineInterface createRbw(ExtendedBlock b
191      ) throws IOException;
192
193  /**
194   * Recovers a RBW replica and returns the meta info of the replica
195   * 
196   * @param b block
197   * @param newGS the new generation stamp for the replica
198   * @param minBytesRcvd the minimum number of bytes that the replica could have
199   * @param maxBytesRcvd the maximum number of bytes that the replica could have
200   * @return the meta info of the replica which is being written to
201   * @throws IOException if an error occurs
202   */
203  public ReplicaInPipelineInterface recoverRbw(ExtendedBlock b, 
204      long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException;
205
206  /**
207   * Covert a temporary replica to a RBW.
208   * @param temporary the temporary replica being converted
209   * @return the result RBW
210   */
211  public ReplicaInPipelineInterface convertTemporaryToRbw(
212      ExtendedBlock temporary) throws IOException;
213
214  /**
215   * Append to a finalized replica and returns the meta info of the replica
216   * 
217   * @param b block
218   * @param newGS the new generation stamp for the replica
219   * @param expectedBlockLen the number of bytes the replica is expected to have
220   * @return the meata info of the replica which is being written to
221   * @throws IOException
222   */
223  public ReplicaInPipelineInterface append(ExtendedBlock b, long newGS,
224      long expectedBlockLen) throws IOException;
225
226  /**
227   * Recover a failed append to a finalized replica
228   * and returns the meta info of the replica
229   * 
230   * @param b block
231   * @param newGS the new generation stamp for the replica
232   * @param expectedBlockLen the number of bytes the replica is expected to have
233   * @return the meta info of the replica which is being written to
234   * @throws IOException
235   */
236  public ReplicaInPipelineInterface recoverAppend(ExtendedBlock b, long newGS,
237      long expectedBlockLen) throws IOException;
238  
239  /**
240   * Recover a failed pipeline close
241   * It bumps the replica's generation stamp and finalize it if RBW replica
242   * 
243   * @param b block
244   * @param newGS the new generation stamp for the replica
245   * @param expectedBlockLen the number of bytes the replica is expected to have
246   * @return the storage uuid of the replica.
247   * @throws IOException
248   */
249  public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
250      ) throws IOException;
251  
252  /**
253   * Finalizes the block previously opened for writing using writeToBlock.
254   * The block size is what is in the parameter b and it must match the amount
255   *  of data written
256   * @param b
257   * @throws IOException
258   */
259  public void finalizeBlock(ExtendedBlock b) throws IOException;
260
261  /**
262   * Unfinalizes the block previously opened for writing using writeToBlock.
263   * The temporary file associated with this block is deleted.
264   * @param b
265   * @throws IOException
266   */
267  public void unfinalizeBlock(ExtendedBlock b) throws IOException;
268
269  /**
270   * Returns one block report per volume.
271   * @param bpid Block Pool Id
272   * @return - a map of DatanodeStorage to block report for the volume.
273   */
274  public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
275
276  /**
277   * Returns the cache report - the full list of cached block IDs of a
278   * block pool.
279   * @param   bpid Block Pool Id
280   * @return  the cache report - the full list of cached block IDs.
281   */
282  public List<Long> getCacheReport(String bpid);
283
284  /** Does the dataset contain the block? */
285  public boolean contains(ExtendedBlock block);
286
287  /**
288   * Is the block valid?
289   * @param b
290   * @return - true if the specified block is valid
291   */
292  public boolean isValidBlock(ExtendedBlock b);
293
294  /**
295   * Is the block a valid RBW?
296   * @param b
297   * @return - true if the specified block is a valid RBW
298   */
299  public boolean isValidRbw(ExtendedBlock b);
300
301  /**
302   * Invalidates the specified blocks
303   * @param bpid Block pool Id
304   * @param invalidBlks - the blocks to be invalidated
305   * @throws IOException
306   */
307  public void invalidate(String bpid, Block invalidBlks[]) throws IOException;
308
309  /**
310   * Caches the specified blocks
311   * @param bpid Block pool id
312   * @param blockIds - block ids to cache
313   */
314  public void cache(String bpid, long[] blockIds);
315
316  /**
317   * Uncaches the specified blocks
318   * @param bpid Block pool id
319   * @param blockIds - blocks ids to uncache
320   */
321  public void uncache(String bpid, long[] blockIds);
322
323    /**
324     * Check if all the data directories are healthy
325     * @throws DiskErrorException
326     */
327  public void checkDataDir() throws DiskErrorException;
328
329  /**
330   * Shutdown the FSDataset
331   */
332  public void shutdown();
333
334  /**
335   * Sets the file pointer of the checksum stream so that the last checksum
336   * will be overwritten
337   * @param b block
338   * @param outs The streams for the data file and checksum file
339   * @param checksumSize number of bytes each checksum has
340   * @throws IOException
341   */
342  public void adjustCrcChannelPosition(ExtendedBlock b,
343      ReplicaOutputStreams outs, int checksumSize) throws IOException;
344
345  /**
346   * Checks how many valid storage volumes there are in the DataNode.
347   * @return true if more than the minimum number of valid volumes are left 
348   * in the FSDataSet.
349   */
350  public boolean hasEnoughResource();
351
352  /**
353   * Get visible length of the specified replica.
354   */
355  long getReplicaVisibleLength(final ExtendedBlock block) throws IOException;
356
357  /**
358   * Initialize a replica recovery.
359   * @return actual state of the replica on this data-node or 
360   * null if data-node does not have the replica.
361   */
362  public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock
363      ) throws IOException;
364
365  /**
366   * Update replica's generation stamp and length and finalize it.
367   * @return the ID of storage that stores the block
368   */
369  public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
370      long recoveryId, long newLength) throws IOException;
371
372  /**
373   * add new block pool ID
374   * @param bpid Block pool Id
375   * @param conf Configuration
376   */
377  public void addBlockPool(String bpid, Configuration conf) throws IOException;
378  
379  /**
380   * Shutdown and remove the block pool from underlying storage.
381   * @param bpid Block pool Id to be removed
382   */
383  public void shutdownBlockPool(String bpid) ;
384  
385  /**
386   * Deletes the block pool directories. If force is false, directories are 
387   * deleted only if no block files exist for the block pool. If force 
388   * is true entire directory for the blockpool is deleted along with its
389   * contents.
390   * @param bpid BlockPool Id to be deleted.
391   * @param force If force is false, directories are deleted only if no
392   *        block files exist for the block pool, otherwise entire 
393   *        directory for the blockpool is deleted along with its contents.
394   * @throws IOException
395   */
396  public void deleteBlockPool(String bpid, boolean force) throws IOException;
397  
398  /**
399   * Get {@link BlockLocalPathInfo} for the given block.
400   */
401  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b
402      ) throws IOException;
403
404  /**
405   * Get a {@link HdfsBlocksMetadata} corresponding to the list of blocks in 
406   * <code>blocks</code>.
407   * 
408   * @param blocks List of blocks for which to return metadata
409   * @return metadata Metadata for the list of blocks
410   * @throws IOException
411   */
412  public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
413      throws IOException;
414}
415