|
@@ -20,6 +20,10 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.net.URI;
|
|
|
+import java.text.DateFormat;
|
|
|
+import java.text.SimpleDateFormat;
|
|
|
+import java.util.Date;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.Map;
|
|
@@ -29,11 +33,16 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|
|
import java.util.concurrent.ThreadFactory;
|
|
|
import java.util.concurrent.ThreadPoolExecutor;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.regex.Matcher;
|
|
|
+import java.util.regex.Pattern;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
|
|
@@ -42,6 +51,8 @@ import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.nativeio.NativeIOException;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.server.common.Storage.STORAGE_DIR_CURRENT;
|
|
|
+
|
|
|
/**
|
|
|
* This class is a container of multiple thread pools, each for a volume,
|
|
|
* so that we can schedule async disk operations easily.
|
|
@@ -65,7 +76,17 @@ class FsDatasetAsyncDiskService {
|
|
|
// ThreadPool maximum pool size
|
|
|
private static final int MAXIMUM_THREADS_PER_VOLUME = 4;
|
|
|
// ThreadPool keep-alive time for threads over core pool size
|
|
|
- private static final long THREADS_KEEP_ALIVE_SECONDS = 60;
|
|
|
+ private static final long THREADS_KEEP_ALIVE_SECONDS = 60;
|
|
|
+
|
|
|
+ private static final String BLOCK_POOL_ID_PATTERN_BASE =
|
|
|
+ Pattern.quote(File.separator) +
|
|
|
+ "BP-\\d+-\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}-\\d+" +
|
|
|
+ Pattern.quote(File.separator);
|
|
|
+
|
|
|
+ private static final Pattern BLOCK_POOL_CURRENT_PATH_PATTERN =
|
|
|
+ Pattern.compile("^(.*)(" + BLOCK_POOL_ID_PATTERN_BASE + ")(" +
|
|
|
+ STORAGE_DIR_CURRENT + ")(.*)$");
|
|
|
+
|
|
|
|
|
|
private final DataNode datanode;
|
|
|
private final FsDatasetImpl fsdatasetImpl;
|
|
@@ -76,6 +97,8 @@ class FsDatasetAsyncDiskService {
|
|
|
= new HashMap<String, Set<Long>>();
|
|
|
private static final int MAX_DELETED_BLOCKS = 64;
|
|
|
private int numDeletedBlocks = 0;
|
|
|
+ private final Configuration conf;
|
|
|
+ private final boolean replicaTrashEnabled;
|
|
|
|
|
|
/**
|
|
|
* Create a AsyncDiskServices with a set of volumes (specified by their
|
|
@@ -84,10 +107,15 @@ class FsDatasetAsyncDiskService {
|
|
|
* The AsyncDiskServices uses one ThreadPool per volume to do the async
|
|
|
* disk operations.
|
|
|
*/
|
|
|
- FsDatasetAsyncDiskService(DataNode datanode, FsDatasetImpl fsdatasetImpl) {
|
|
|
+ FsDatasetAsyncDiskService(DataNode datanode, FsDatasetImpl fsdatasetImpl,
|
|
|
+ Configuration configuration) {
|
|
|
this.datanode = datanode;
|
|
|
this.fsdatasetImpl = fsdatasetImpl;
|
|
|
this.threadGroup = new ThreadGroup(getClass().getSimpleName());
|
|
|
+ this.conf = configuration;
|
|
|
+ this.replicaTrashEnabled = conf.getBoolean(DFSConfigKeys
|
|
|
+ .DFS_DATANODE_ENABLE_REPLICA_TRASH_KEY, DFSConfigKeys
|
|
|
+ .DFS_DATANODE_ENABLE_REPLICA_TRASH_DEFAULT);
|
|
|
}
|
|
|
|
|
|
private void addExecutorForVolume(final FsVolumeImpl volume) {
|
|
@@ -277,6 +305,49 @@ class FsDatasetAsyncDiskService {
|
|
|
(replicaToDelete.deleteMetadata() || !replicaToDelete.metadataExists());
|
|
|
}
|
|
|
|
|
|
+ private boolean moveFilesToReplicaTrash() {
|
|
|
+
|
|
|
+ DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd-HH");
|
|
|
+ Date date = new Date();
|
|
|
+
|
|
|
+ URI blockURI = replicaToDelete.getBlockURI();
|
|
|
+
|
|
|
+ String replicaTrashBaseDir;
|
|
|
+ File blockFile = new File(blockURI);
|
|
|
+ Matcher matcher = BLOCK_POOL_CURRENT_PATH_PATTERN.matcher(blockFile
|
|
|
+ .getParent());
|
|
|
+ replicaTrashBaseDir = matcher.replaceFirst("$1$2" + DataStorage
|
|
|
+ .STORAGE_DIR_REPLICA_TRASH);
|
|
|
+
|
|
|
+ File replicaTrashDir = new File(replicaTrashBaseDir + File
|
|
|
+ .separator + dateFormat.format(date));
|
|
|
+
|
|
|
+ try {
|
|
|
+ volume.getFileIoProvider().mkdirsWithExistsCheck(
|
|
|
+ volume, replicaTrashDir);
|
|
|
+ } catch (IOException e) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Moving files " + replicaToDelete.getBlockURI() + " and " +
|
|
|
+ replicaToDelete.getMetadataURI() + " to replica-trash.");
|
|
|
+ }
|
|
|
+
|
|
|
+ final String blockName = replicaToDelete.getBlockName();
|
|
|
+ final long genstamp = replicaToDelete.getGenerationStamp();
|
|
|
+ File newBlockFile = new File(replicaTrashDir, blockName);
|
|
|
+ File newMetaFile = new File(replicaTrashDir,
|
|
|
+ DatanodeUtil.getMetaName(blockName, genstamp));
|
|
|
+ try {
|
|
|
+ return (replicaToDelete.renameData(newBlockFile.toURI()) &&
|
|
|
+ replicaToDelete.renameMeta(newMetaFile.toURI()));
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Error moving files to trash: " + replicaToDelete, e);
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
private boolean moveFiles() {
|
|
|
if (trashDirectory == null) {
|
|
|
LOG.error("Trash dir for replica " + replicaToDelete + " is null");
|
|
@@ -316,11 +387,18 @@ class FsDatasetAsyncDiskService {
|
|
|
final long metaLength = replicaToDelete.getMetadataLength();
|
|
|
boolean result;
|
|
|
|
|
|
- result = (trashDirectory == null) ? deleteFiles() : moveFiles();
|
|
|
+ if (trashDirectory != null) {
|
|
|
+ result = moveFiles();
|
|
|
+ } else if (replicaTrashEnabled) {
|
|
|
+ result = moveFilesToReplicaTrash();
|
|
|
+ } else {
|
|
|
+ result = deleteFiles();
|
|
|
+ }
|
|
|
|
|
|
if (!result) {
|
|
|
LOG.warn("Unexpected error trying to "
|
|
|
- + (trashDirectory == null ? "delete" : "move")
|
|
|
+ + ((trashDirectory == null || !replicaTrashEnabled) ?
|
|
|
+ "delete" : "move")
|
|
|
+ " block " + block.getBlockPoolId() + " " + block.getLocalBlock()
|
|
|
+ " at file " + replicaToDelete.getBlockURI() + ". Ignored.");
|
|
|
} else {
|