【前言】对于线上生产环境的HDFS,开启回收站功能是必不可少的。该功能类似于linux系统的回收站设计,HDFS会为每个用户创建一个专属的回收站目录(/user/${user.name}/.Trash),用户删除文件时,实际上是被移动到了回收站目录。用于预防当用户误删HDFS上的数据时,能够及时从回收站恢复这些数据(当然回收站是防不住删库跑路的)。
一. 参数介绍
<name>fs.trash.interval</name>
<value>0</value>
<description>Number of minutes after which the checkpoint gets deleted.
If zero, the trash feature is disabled.
This option may be configured both on the server and the client.
If trash is disabled server side then the client side configuration is checked.
If trash is enabled on the server side then the value configured on the server is used
and the client configuration value is ignored.
</description>
描述:单位(minute),回收站数据保存周期,默认值0,代表HDFS不开启回收站功能。
<name>fs.trash.checkpoint.interval</name>
<value>0</value>
<description>Number of minutes between trash checkpoints.
Should be smaller or equal to fs.trash.interval.
If zero, the value is set to the value of fs.trash.interval.
Every time the checkpointer runs it creates a new checkpoint out of current
and removes checkpoints created more than fs.trash.interval minutes ago.
</description>
描述:单位(minute),回收站数据判断是否需要清理的检查周期,默认值为0 (如果集群未自定义设置且fs.trash.interval
大于0,则 fs.trash.checkpoint.interval=${fs.trash.interval}
)
二. 工作原理
2.1 初始化
NameNode启动时会在后台启动一个emptier
守护线程,用于定时(NameNode重启周期清零)清理HDFS集群上每个用户下的回收站数据,定时周期为fs.trash.checkpoint.interval
。
源码路径:org.apache.hadoop.hdfs.server.namenode
private void startTrashEmptier(final Configuration conf) throws IOException {
long trashInterval =
conf.getLong(FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT);
if (trashInterval == 0) {
return;
} else if (trashInterval < 0) {
throw new IOException("Cannot start trash emptier with negative interval."
+ " Set " + FS_TRASH_INTERVAL_KEY + " to a positive value.");
}
// This may be called from the transitionToActive code path, in which
// case the current user is the administrator, not the NN. The trash
// emptier needs to run as the NN. See HDFS-3972.
FileSystem fs = SecurityUtil.doAsLoginUser(
new PrivilegedExceptionAction<FileSystem>() {
@Override
public FileSystem run() throws IOException {
return FileSystem.get(conf);
}
});
this.emptier = new Thread(new Trash(fs, conf).getEmptier(), "Trash Emptier");
this.emptier.setDaemon(true);
this.emptier.start();
}
调用Trash类初始化配置信息和垃圾回收策略。
源码路径:org.apache.hadoop.fs.Trash
public Trash(FileSystem fs, Configuration conf) throws IOException {
super(conf);
trashPolicy = TrashPolicy.getInstance(conf, fs, fs.getHomeDirectory());
}
HDFS为每个执行用户创建一个专属主目录/user/$USER/
,被删除的数据会移动到执行用户的主目录下。
源码路径:org.apache.hadoop.fs.FileSystem
/** Return the current user's home directory in this filesystem.
* The default implementation returns "/user/$USER/".
*/
public Path getHomeDirectory() {
return this.makeQualified(
new Path("/user/"+System.getProperty("user.name")));
}
通过反射创建TrashPolicy对象,垃圾回收策略可以用户自定义实现,通过参数fs.trash.classname
指定。系统默认使用TrashPolicyDefault.class
。
源码路径:org.apache.hadoop.fs.TrashPolicy
public static TrashPolicy getInstance(Configuration conf, FileSystem fs, Path home) {
Class<? extends TrashPolicy> trashClass = conf.getClass(
"fs.trash.classname", TrashPolicyDefault.class, TrashPolicy.class);
TrashPolicy trash = ReflectionUtils.newInstance(trashClass, conf);
trash.initialize(conf, fs, home); // initialize TrashPolicy
return trash;
}
2.2 启动定时线程
NameNode通过this.emptier.start()
方法启动线程,emptier线程周期性休眠后唤醒,执行删除垃圾数据trashPolicy.deleteCheckpoint()
和创建检查点trashPolicy.createCheckpoint()
操作。
源码路径:org.apache.hadoop.fs.TrashPolicy
@Override
public void run() {
if (emptierInterval == 0)
return; // trash disabled
long now = Time.now();
long end;
while (true) {
end = ceiling(now, emptierInterval);
try { // sleep for interval
Thread.sleep(end - now);
} catch (InterruptedException e) {
break; // exit on interrupt
}
try {
now = Time.now();
if (now >= end) {
FileStatus[] homes = null;
try {
homes = fs.listStatus(homesParent); // list all home dirs
} catch (IOException e) {
LOG.warn("Trash can't list homes: "+e+" Sleeping.");
continue;
}
for (FileStatus home : homes) { // dump each trash
if (!home.isDirectory())
continue;
try {
TrashPolicyDefault trash = new TrashPolicyDefault(
fs, home.getPath(), conf);
trash.deleteCheckpoint(); //删除垃圾数据
trash.createCheckpoint(); //创建检查点
} catch (IOException e) {
LOG.warn("Trash caught: "+e+". Skipping "+home.getPath()+".");
}
}
}
} catch (Exception e) {
LOG.warn("RuntimeException during Trash.Emptier.run(): ", e);
}
}
try {
fs.close();
} catch(IOException e) {
LOG.warn("Trash cannot close FileSystem: ", e);
}
}
2.3 删除垃圾数据
检查/user/${user.name}/.Trash/
(所有用户)下的第一级子目录,将目录名为格式yyMMddHHmmss
的目录转化为时间 time(跳过Current和无法解析的目录),如果符合条件(now - deletionInterval > time),则删除该目录 (deletionInterval = ${fs.trash.interval}
)。
回收站的默认清理机制粒度比较粗,只针对/user/${user.name}/.Trash/
下的第一级子目录.
public void deleteCheckpoint() throws IOException {
FileStatus[] dirs = null;
try {
dirs = fs.listStatus(trash); // scan trash sub-directories
} catch (FileNotFoundException fnfe) {
return;
}
long now = Time.now();
for (int i = 0; i < dirs.length; i++) {
Path path = dirs[i].getPath();
String dir = path.toUri().getPath();
String name = path.getName();
if (name.equals(CURRENT.getName())) // skip current
continue;
long time;
try {
time = getTimeFromCheckpoint(name); //将目录名转换为时间
} catch (ParseException e) {
LOG.warn("Unexpected item in trash: "+dir+". Ignoring.");
continue;
}
if ((now - deletionInterval) > time) {
if (fs.delete(path, true)) { //删除目录
LOG.info("Deleted trash checkpoint: "+dir);
} else {
LOG.warn("Couldn't delete checkpoint: "+dir+" Ignoring.");
}
}
}
}
2.4 创建检查点
如果/user/${user.name}/.Trash/
目录下存在Current
目录,则将该目录重命名为yyMMddHHmmss
(执行到该条代码的当前时间)。如果不存在Current
目录,则直接跳过。重命名后,新的删除数据写入时仍会创建Current目录。
public void createCheckpoint() throws IOException {
if (!fs.exists(current)) // no trash, no checkpoint
return;
Path checkpointBase;
synchronized (CHECKPOINT) {
checkpointBase = new Path(trash, CHECKPOINT.format(new Date()));
}
Path checkpoint = checkpointBase;
int attempt = 0;
while (true) {
try {
fs.rename(current, checkpoint, Rename.NONE); //重命名目录
break;
} catch (FileAlreadyExistsException e) {
if (++attempt > 1000) {
throw new IOException("Failed to checkpoint trash: "+checkpoint);
}
checkpoint = checkpointBase.suffix("-" + attempt);
}
}
LOG.info("Created trash checkpoint: "+checkpoint.toUri().getPath());
}
三. 反面案例
集群配置垃圾回收参数如下:
fs.trash.interval = 4320 //3天
fs.trash.checkpoint.interval = 0 //未自定义设置,`fs.trash.checkpoint.interval=${fs.trash.interval}
理想状态图.PNG
2018:11:27 08:00:00开始唤醒emptier线程,先执行deleteCheckpoint()方法,理想情况下应该是符合条件((now - deletionInterval) > time)。
now:大于181127080000,小于181127080010的某个时间点
deletionInterval:4320 minutes
time:181124080000
=> 符合条件,开始删除181124080000目录
而在现实操作中,往往会发生如下极端情况:
极端状态图.PNGnow:大于181127080000,小于181127080010的某个时间点
deletionInterval:4320 minutes
time:181124080033
=> 不符合条件,跳过执行createCheckpoint()方法
fs.trash.checkpoint.interval
默认不设置的情况下,会出现本来设置回收站数据保存3天,而实际上会保留接近9天的情况。
【备注】
- 线上生产环境如果设置参数
fs.trash.interval
大于1天,则必须同时设置fs.trash.checkpoint.interval
,建议参考设置为1天(数据会保留fs.trash.interval
+fs.trash.checkpoint.interval
); - 如果不自定义设置参数
fs.trash.checkpoint.interval
,也可以通过定时执行hadoop shell命令(-expunge)达到同样的效果;
四. expunge命令
用户可以通过手动执行hadoop shell命令清理过期检查点和创建新的检查点,功能同emptier线程的单次执行。
hdfs dfs -expunge
hadoop fs -expunge
源码路径:org.apache.hadoop.fs.shell
protected void processArguments(LinkedList<PathData> args)
throws IOException {
Trash trash = new Trash(getConf());
trash.expunge();
trash.checkpoint();
}
源码路径:org.apache.hadoop.fs.Trash
/** Delete old checkpoint(s). */
public void expunge() throws IOException {
trashPolicy.deleteCheckpoint();
}
五. 特别说明
- 在开启回收站功能的前提下,用户调用hadoop的rm命令会将删除文件放入回收站。如果用户添加参数skipTrash则不会放入回收站,直接删除不可找回。
hdfs dfs -rm -r -skipTrash /test
hadoop fs -rm -r -skipTrash /test
- 用户调用hdfs delete API执行删除文件操作时,不会将删除数据移动到回收站中,用户必须自己实现Transh类:
val hdfs = FileSystem.get(sc.hadoopConfiguration)
hdfs.delete(new Path("**"), true) //直接删除不可找回
val hfsTrash = new Trash(hdfs, sc.hadoopConfiguration)
hdfsTrash.moveToTrash(new Path("**")) //移动到回收站
网友评论