HDFS租约与Flink StreamingFileSink
1 概述
- HDFS文件的特点是一次写多次读
- 并且不支持客户端的并行写操作,
- hdfs采用租约机制来保证对文件的互斥操作。
某个DFSClient打开文件流时,该DFSClient即为租约持有者(LeaseHolder),租约由NameNode发放。
租约就是在规定时间内拥有写文件权限的许可凭证,HDFS提供Lease机制来保证写入数据的一致性。也就是说,租约是HDFS给予客户端的一个写文件权限的临时许可证,无此许可者将不被允许操作此文件,实现了写文件者的唯一性。
HDFS租约特点:
- 每个实例用户持有一个租约
- 每个租约内部包含有一个租约持有者信息,还有此租约对应的文件ID列表,表示当前租约持有者可以写这些ID对应的文件。
- 每个租约内包含有一个最新近更新时间,最近更新时间将会决定此租约是否已过期。
过期的租约会导致租约持有者无法继续执行写数据到文件中,除非进行租约的更新。
2.1 客户端侧租约相关
说明:这里以Flink StreamingFileSink为例。
2.1 DFSClient和DFSOutputStream的创建
- StreamingFileSink的
initializeState
方法初始化会为本task实例创建管理Bucket的Buckets
实例,需要创建org.apache.flink.runtime.fs.hdfs.HadoopFileSystem
,此时就会初始化org.apache.hadoop.fs.FileSystem
。这里因为schema为hdfs所有是DistributedFileSystem
。 - 随后,在
DistributedFileSystem.initialize
根据我们的BasePath
创建DFSClient
- 当首条记录进入StreamingFileSink时,会使用
DFSClient.createWrappedOutputStream
打开一个inprogress文件用于写入。随后就需要打开一个HDFS文件写入流,最后会开始租约处理。
也就是说,StreamingFileSink运行过程中,一个Task实例一直使用一个FileSystem、DistributedFileSystem、DFSClient实例,只是会为每个写入的partFile开启不同的FSDataOutputStream
。
此时客户端侧调用栈如下 - 最后是调用客户端到NameNode的RPC接口进行文件create,在
DFSOutputStream.newStreamForCreate
有一个动态代理dfsClient.namenode
:
stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
new EnumSetWritable<>(flag), createParent, replication,
blockSize, SUPPORTED_CRYPTO_VERSIONS);
final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
flag, progress, checksum, favoredNodes);
out.start();
return out;
- org.apache.hadoop.hdfs.DFSClient proxy
proxyInfo = NameNodeProxiesClient.createProxyWithClientProtocol(conf,
nameNodeUri, nnFallbackToSimpleAuth);
this.namenode = proxyInfo.getProxy();
- 这里的动态代理InvocationHandler为
LossyRetryInvocationHandler
和其父类RetryInvocationHandler
,实际代理的类为实现自ClientProtocol
的ClientNamenodeProtocolTranslatorPB
该类被用来发送RPC请求给NameNode服务端,将请求参数翻译为ProtoBuf类型 - 接着进入HDFS服务端侧,请先看第三章服务端侧租约相关内容再看
2.2租约初始化
。
2.2 租约初始化和定时更新
2.2.1 待create请求结束返回后,调用DFSClient.beginFileLease
开启租约更新线程:
private void beginFileLease(final long inodeId, final DFSOutputStream out)
throws IOException {
synchronized (filesBeingWritten) {
// 记录当前文件inodeId对应的DFSOutputStream到filesBeingWritten
// 如果没有记录写入,就将当前时间保存到lastLeaseRenewal
putFileBeingWritten(inodeId, out);
// 1.根据hdfs namespace和当前用户名来创建key,并创建或获取Key唯一对应的LeaseRenewer
// 2.sleepPeriod = 1000,renewal=30000
// 3.将当前dfsClient记录到LeaseRenewer
// 4.put方法会在client依然运行,且初始或LeaseRenewer过期时
// 开启一个守护线程,该线程只要没有被中断就会一直循环:
// 5. 更新emptyTime为Long.MAX_VALUE,表示当前LeaseRenewer记录的dfsclients不为空
getLeaseRenewer().put(this);
}
}
2.2.2 租约更新线程
Lease.run
本线程只要没有被中断就会一直循环逻辑如下:
- 1 租约定期更新
如果循环时间累积到renewal
(默认30秒)以后,就会将本LeaseRenewer
关联的所有
DFSClient全部调用ClientProtocol.renewLease(clientName)
方法(注意是DFSClientName,也就是说只要task不重启,clientName不会改变)更新租约(前提是该DFSClient仍然运行且有文件在写(filesBeingWritten
不为空))
随后在 filesBeingWritten 更新lastLeaseRenewal
为当前时间。 - 2 如果更新租约发生IOException
- 2.1 如果当前时间减去
lastLeaseRenewal
已经超过1小时的HardLimit
未更新租约
此时会将本DFSClient的 filesBeingWritten 里的所有文件流强制abort
- 2.2 否则将异常抛出到上层处理
- 2.2.1 如果发生的是
SocketTimeoutException
,则会将当前 LeaseRenewer 线程过期并移除
并将所有关联的DFSClient的 filesBeingWritten里的所有文件流强制abort,随后本线程结束 - 2.2.2 如果是其他IOException,则仅进入下次循环重试
2.3 filesBeingWritten
- 作用
被用来记录本DFSClient所有写入的文件 inodeId和对应的DFSOutputStream - add
- beginFileLease(final long inodeId, final DFSOutputStream out) -> putFileBeingWritten
从filesBeingWritten增加单个文件
- remove
- endFileLease(final long inodeId)->removeFileBeingWritten
从filesBeingWritten移除单个文件 - closeAllFilesBeingWritten(final boolean abort)
- 更新某个DFSClient租约发生
IOException
且已经超过1小时的HardLimit
未更新租约时
该DFSClient closeAllFilesBeingWritten(true),从filesBeingWritten移除所有文件 - 更新某个DFSClient租约发生
SocketTimeoutException
且已经短于1小时的HardLimit
未更新租约时
当前LeaseRenewer关联的所有DFSClient调用closeAllFilesBeingWritten(true) - close
closeAllFilesBeingWritten(false),清空该DFSClient filesBeingWritten,正常关闭所有文件 - closeOutputStreams
由DistributedFileSystem
调用,closeAllFilesBeingWritten(false)
2.4 租约关闭
- 当StreamingFileSink发生因Checkpoint发生SnapShot时,会关闭
inprogressFile
写入流,调用栈如下:
- 如上图,StreamingFileSink关闭每个part-file时,都会调用
DFSClient.endFileLease
:
// 要关闭文件的inodeId
void endFileLease(final long inodeId) {
synchronized (filesBeingWritten) {
// 1.从本DFSClient的filesBeingWritten中删除本file
// 2.如果filesBeingWritten已经为空,就设lastLeaseRenewal = 0
removeFileBeingWritten(inodeId);
if (filesBeingWritten.isEmpty()) {
// 如果filesBeingWritten已经为空,从LeaseRenewer中移除本DFSClient
// 1. 如果LeaseRenewer关联的DFSClient已为空
// 1.1 如果租约更新线程已经未运行或LeaseRenewer已经过期则移除本LeaseRenewer
// 1.2 如果emptyTime == Long.MAX_VALUE,则更新为Time.monotonicNow()表示当前LeaseRenewer记录的dfsclients为空
// 2. 否则,将renewal更新为LeaseRenewer关联的当前所有DFSClient中HdfsTimeout最小值
getLeaseRenewer().closeClient(this);
}
}
}
- 注意,客户端侧的租约关闭其实是不会发送请求到NameNode的,而是在本地处理:
- 如果filesBeingWritten已经为空,从LeaseRenewer中移除本DFSClient。这就造成不会再为该DFSClient更新租约
- 如果未更新租约时间超过1小时,则会造成HDFS NameNode检测到
has expired hard limit
。
2.5 租约Recover
2.5.1 restoreInprogressFile
当StreamingFileSink发生Restore时,会调用buckets.initializeState
来恢复最后一次成功Checkpoint时的SnapShot中记录到的InprogressFile
和PendingFile
的状态。依次恢复Checkpoint中记录的每个BucketState:
- 第一步是
Bucket.restoreInProgressFile
来恢复最后一次成功checkPoint时的InProgressFile以继续写入
这里就会利用BucketState.getInProgressResumableFile
来获取HadoopFsRecoverable
。该类里面包含了targetFile
path、tempFile
path、记录到的写入位置offset
(用来恢复) HadoopRecoverableWriter.recover(HadoopFsRecoverable)
这里就会恢复Inprogress流,包装成BulkPartWriter
后可继续写入:
final RecoverableFsDataOutputStream stream = fsWriter.recover(resumable);
inProgressPart = partFileFactory.resumeFrom(
bucketId, stream, resumable, state.getInProgressFileCreationTime());
接下来继续看Recover流程,会调用new HadoopRecoverableFsDataOutputStream
HadoopRecoverableFsDataOutputStream.safelyTruncateFile(fs, tempFile, recoverable)
- 调用
waitUntilLeaseIsRevoked
,其中会调用DistributedFileSystem.recoverLease
来恢复目标文件租约(具体调用流程为DFSClient.recoverLease(tempFile)
->ClientProtocol.recoverLease(tempFile, clientName)
)。
因为要重命名的目标文件可能租约还属于上次出错的程序所有,所以必须先恢复文件租约。随后最多等待100_000L
毫秒看检查文件是否已经关闭。
- 调用
FileSystem.truncate
将目标文件截断为我们State中记录的大小。
- FSDataOutputStream out = fs.append(tempFile)
将目标InProgressFile append后得到FSDataOutputStream
继续写入。
2.5.2 commitRecoveredPendingFiles
依次恢复Checkpoint中记录的每个BucketState的第二步是commitRecoveredPendingFiles
,将最后一次成功checkPoint时的PendingFile提交。
对于该Bucket内的所有代表PendingFile的HadoopFsRecoverable
(对于StreamingFileSink来说,其实每个Checkpoint周期内每个Bucket内只有一个PendingFile或一个InProgressFile):
- 组装为
HadoopRecoverableFsDataOutputStream.HadoopFsCommitter(HadoopFsRecoverable)
组装为HadoopFsCommitter,准备在给定的Checkpoint上一致性地恢复一个可恢复的流,以进行finalizing和提交。这将发布目标PendingFile对读可见,该PendingFile内容是精确截止到创建HadoopFsRecoverable
之前写入的。 - 随后调用
HadoopRecoverableFsDataOutputStream.commitAfterRecovery
提交目标文件,对读可见。
且该方法对重复提交是幂等的。 - 如果HDFS上当前文件大小大于我们State里记录的文件大小,则先将该文件
HadoopRecoverableFsDataOutputStream.safelyTruncateFile
; - 最后调用
FileSystem.rename
将该文件重命名,去掉后缀转为类似part-0-2
的Finished状态文件,对读可见。
2.6 其他重要方法
2.6.1 LeaseRenewer.isRenewerExpired
该方法用来判断当前LeaseRenewer是否已经过期,判断过期标准是(当前时间减去上次本LeaseRenewer.dfsClients不为空的时间)已经超过60秒。
private synchronized boolean isRenewerExpired() {
return emptyTime != Long.MAX_VALUE
&& Time.monotonicNow() - emptyTime > gracePeriod;
}
如果过期,后果有几个:
- 租约更新线程
Lease.run
停止,造成当前LeaseRenewer从LeaseRenewer.Factory.renewers
中删除,下次开启租约时又需要重新创建 - endFileLease导致的
LeaseRenewer.closeClient
中,造成当前LeaseRenewer从LeaseRenewer.Factory.renewers
中删除,下次开启租约时又需要重新创建
3 HDFS服务端侧租约相关
3.1 Lease相关
3.1.1 LeaseManager
3.1.1.1 概述
在NameNode中运行,负责管理所有租约。
3.1.1.2 重要知识点
- Lease
就是文件租约实体类。 - private final SortedMap<String, Lease> leases = new TreeMap<>();
key为leaseHolderClientName,value为文件租约Lease。
也就是说一个Client对应一个Lease,但一个Lease内可有多个文件。 - private final TreeMap<Long, Lease> leasesById = new TreeMap<>();
key为文件INodeID,value为文件租约Lease。 - private final NavigableSet sortedLeases =
new TreeSet<>(
new Comparator<Lease>() {
@Override
public int compare(Lease o1, Lease o2) {
if (o1.getLastUpdate() != o2.getLastUpdate()) {
return Long.signum(o1.getLastUpdate() - o2.getLastUpdate());
} else {
return o1.holder.compareTo(o2.holder);
}
}
});
按租约更新时间由远到近排序Set
- 租约监控线程
用来检测租约是否超过限制,见这里 - 重要方法:
- addLease
添加租约
3.1.2 Lease(LeaseManager内部类)
3.1.2.1 概述
即文件租约实体类。
- 租约控制着单个客户端持有的所有锁。 对于每个客户端,都有一个相应的租约。
- 当客户端定期签入时,租约的时间戳将更新。
- 如果客户端死亡并允许其租约过期,则可以释放所有相应的锁。
3.1.2.2 重要属性
// 持有租约的clientName
private final String holder;
// 租约最后更新时间
private long lastUpdate;
// 本租约对应文件
private final HashSet<Long> files = new HashSet<>();
3.1.2.3 关于租约Recover
立刻撤销租约,使得文件能被强制正常关闭。租Recover约算法如下:
- NameNode搜索租约信息
- 对租约中的每个文件f,考虑f的最后一个Block b
- 获取所有包含b的DataNode列表
- 分配其中一个DataNode为主节点p
- p节点从NameNode获取新一代标记
- p节点从上述每个DataNode中获取b相关Block信息
- p计算最小的Block大小
- p使用新一代标记和最小Block长度来更新那些具有有效标记的数据节点
- p向NameNode确认更新结果
- NameNode更新Block信息
- NameNode从租约中移除 f,且待租约中所有文件都移除后租约也被移除
- NameNode提交更新到Edit Log。
3.1.2.4 其他重要方法
/** 更新当前时间 */
private void renew() {
this.lastUpdate = monotonicNow();
}
/**
* 判断当前租约最后更新时间是否超过hardLimit(60min)
* 如果超过,客户端没能及时更新租约时间戳,
* 则会认为该客户端已经失联,会自动关闭该文件,并Recover该租约
*
* 本方法由Monitor监控线程定时调用
*/
public boolean expiredHardLimit() {
return monotonicNow() - lastUpdate > hardLimit;
}
/**
* 判断当前租约最后更新时间是否超过softLimit(60s)
* 到期之前都可保证租约拥有者拥有唯一写租约对应文件权利
* 如果超过softLimit,客户端没能及时更新租约时间戳或关闭文件,则其他客户端可以竞争该租约
*/
public boolean expiredSoftLimit() {
return monotonicNow() - lastUpdate > softLimit;
}
/** Does this lease contain any path? */
boolean hasFiles() {return !files.isEmpty();}
boolean removeFile(long inodeId) {
return files.remove(inodeId);
}
3.1.3 Monitor(LeaseManager内部类)
是LeaseManager内部类,周期性的检查如果检测到租约超过HardLimit,则会触发LeaseRecover
机制,然后关闭文件。
在LeaseManager.startActiveServices
中,使用leaseManager.startMonitor()
开启租约Monitor:
void startMonitor() {
Preconditions.checkState(lmthread == null,
"Lease Monitor already running");
shouldRunMonitor = true;
// monitor是一个守护线程
lmthread = new Daemon(new Monitor());
// monitor开始
lmthread.start();
}
Monitor源码如下:
class Monitor implements Runnable {
final String name = getClass().getSimpleName();
/** Check leases periodically. */
@Override
public void run() {
// 只要Monitor和FSNamesystem还在运行,就不断循环检查租约
for(; shouldRunMonitor && fsnamesystem.isRunning(); ) {
boolean needSync = false;
try {
fsnamesystem.writeLockInterruptibly();
try {
if (!fsnamesystem.isInSafeMode()) {
// 检查租约,如果needSync,则代表需要同步此次reassignLease操作
needSync = checkLeases();
}
} finally {
fsnamesystem.writeUnlock("leaseManager");
if (needSync) {
// 同步此次reassignLease操作
fsnamesystem.getEditLog().logSync();
}
}
Thread.sleep(fsnamesystem.getLeaseRecheckIntervalMs());
} catch(InterruptedException ie) {
if (LOG.isDebugEnabled()) {
LOG.debug(name + " is interrupted", ie);
}
} catch(Throwable e) {
LOG.warn("Unexpected throwable: ", e);
}
}
}
}
3.1.4 重要方法
3.1.4.1 addLease
// holder为持有文件租约的clientName,inodeId为目标文件inodeId
synchronized Lease addLease(String holder, long inodeId) {
// 获取该leaseHolderClientName对应的Lease
Lease lease = getLease(holder);
if (lease == null) {
// 若还没有足额,就新建一个Lease
// 然后调用 renew 更新该Lease的 lastUpdate 为当前时间
lease = new Lease(holder);
// 然后记录该client的文件租约
leases.put(holder, lease);
// 租约放入按租约更新时间由远到近排序Set
sortedLeases.add(lease);
} else {
// 否则更新client对应的租约(更新lastUpdate)
renewLease(lease);
}
// 记录以文件inodeId为key对应的租约
leasesById.put(inodeId, lease);
// 将当前文件记录到对应的租约的文件集中
lease.files.add(inodeId);
return lease;
}
3.1.4.2 checkLeases
由monitor周期性调用。
synchronized boolean checkLeases() {
boolean needSync = false;
assert fsnamesystem.hasWriteLock();
// 当前毫秒级计时
long start = monotonicNow();
// 如果最旧的Lease最后更新时间已经超过了HardLimit
while(!sortedLeases.isEmpty() &&
sortedLeases.first().expiredHardLimit()
&& !isMaxLockHoldToReleaseLease(start)) {
// 说明最旧的文件租约已经超过HardLimit
Lease leaseToCheck = sortedLeases.first();
LOG.info(leaseToCheck + " has expired hard limit");
final List<Long> removing = new ArrayList<>();
// need to create a copy of the oldest lease files, because
// internalReleaseLease() removes files corresponding to empty files,
// i.e. it needs to modify the collection being iterated over
// causing ConcurrentModificationException
// 过期租约对应所有文件InodeId
Collection<Long> files = leaseToCheck.getFiles();
Long[] leaseINodeIds = files.toArray(new Long[files.size()]);
FSDirectory fsd = fsnamesystem.getFSDirectory();
String p = null;
// 获取当前LeaseManager内部holder 类似HDFS_NameNode_20200317 16:30:33.111格式
String newHolder = getInternalLeaseHolder();
// 遍历过期租约的所有文件Inode
for(Long id : leaseINodeIds) {
try {
INodesInPath iip = INodesInPath.fromINode(fsd.getInode(id));
// 目标文件全路径
p = iip.getPath();
// Sanity check to make sure the path is correct
if (!p.startsWith("/")) {
throw new IOException("Invalid path in the lease " + p);
}
final INodeFile lastINode = iip.getLastINode().asFile();
if (fsnamesystem.isFileDeleted(lastINode)) {
// 如果该文件已经被删除,则移除该文件租约
// 如果移除后该租约已无其他文件,则一同移除
removeLease(lastINode.getId());
continue;
}
boolean completed = false;
try {
// 根据文件状态,关闭或恢复文件
// 关于该方法可以参考`3.4租约Recover`
// 这里不同的是,holder不是另一个Client而是HDFS_NameNode
completed = fsnamesystem.internalReleaseLease(
leaseToCheck, p, iip, newHolder);
} catch (IOException e) {
LOG.warn("Cannot release the path " + p + " in the lease "
+ leaseToCheck + ". It will be retried.", e);
continue;
}
if (LOG.isDebugEnabled()) {
if (completed) {
LOG.debug("Lease recovery for inode " + id + " is complete. " +
"File closed.");
} else {
LOG.debug("Started block recovery " + p + " lease " + leaseToCheck);
}
}
// 如果文件开始恢复,则将needSync置为true,代表需要同步此次reassignLease操作
if (!needSync && !completed) {
needSync = true;
}
} catch (IOException e) {
LOG.warn("Removing lease with an invalid path: " + p + ","
+ leaseToCheck, e);
removing.add(id);
}
if (isMaxLockHoldToReleaseLease(start)) {
LOG.debug("Breaking out of checkLeases after " +
fsnamesystem.getMaxLockHoldToReleaseLeaseMs() + "ms.");
break;
}
}
for(Long id : removing) {
removeLease(leaseToCheck, id);
}
}
return needSync;
}
3.1.4.3 removeLease
// 如果该文件已经被删除,则移除该文件租约
// 如果移除后该租约已无其他文件,则一同移除
synchronized void removeLease(long inodeId) {
final Lease lease = leasesById.get(inodeId);
if (lease != null) {
removeLease(lease, inodeId);
}
}
private synchronized void removeLease(Lease lease, long inodeId) {
leasesById.remove(inodeId);
if (!lease.removeFile(inodeId)) {
if (LOG.isDebugEnabled()) {
LOG.debug("inode " + inodeId + " not found in lease.files (=" + lease
+ ")");
}
}
if (!lease.hasFiles()) {
leases.remove(lease.holder);
if (!sortedLeases.remove(lease)) {
LOG.error(lease + " not found in sortedLeases");
}
}
}
3.2 租约初始化
- NameNodeRpcServer.create
- FSNamesystem.startFile -> startFileInt
FSNamesystem中有一个leaseManager
实例 - FSDirWriteFileOp.startFile
fsn.leaseManager.addLease(
newNode.getFileUnderConstructionFeature().getClientName(),
newNode.getId());
- LeaseManager.renewLease
记录该新的文件租约,并调用renew
更新租约最后更新时间lastUpdate。
3.3 租约更新
租约更新是DFSClient实例级别,更新该clientName(即文件holder)对应的全部文件租约以及lastUpdate。
3.3.1 添加一个文件租约时
见LeaseManager.addLease
3.3.2 更新Client对应的租约
以下是HDFS NameNode侧流程
- NameNodeRpcServer.renewLease(holder)
- FSNamesystem.renewLease(holder)
- LeaseManager.addLease(holder)
synchronized void renewLease(String holder) {
// 更新本clientName对应的租约
renewLease(getLease(holder));
}
synchronized void renewLease(Lease lease) {
if (lease != null) {
// 在全局记录中更新本Lease
sortedLeases.remove(lease);
// 更新本Lease的 lastUpdate
lease.renew();
sortedLeases.add(lease);
}
}
3.4 租约Recover
对应客户端侧的 `ClientProtocol.recoverLease(tempFile, clientName)``。
关于HDFS Block状态,可查看org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState
以下是HDFS NameNode侧流程:
- NameNodeRpcServer.recoverLease(String src, String clientName)
- FSNamesystem.recoverLease(String src, String holder, String clientMachine)
- FSNamesystem.recoverLeaseInternal
- 如果目标文件租约和通过clientName查找到的租约相同直接抛出异常返回
- 否则继续判断,目标文件的原始Client对应的租约已经为空,直接抛出异常返回
- 否则调用
FSNamesystem.internalReleaseLease(oldClientLease, src, iip, newClientName)
该方法用来关闭或恢复文件。返回true时代表文件成功释放和关闭;false代表已经开始恢复过程,调用者应该调用logSync:
- 如果该文件所有Block都已经正常Complete,则调用
LeaseManager.removeLease(uc.getClientName(), pendingFile)
移除该客户端对该指定文件的租约。
随后关闭该文件,所有Block持久化,然后返回true
。 - 否则检查最后一个Block状态:
- COMPLETE
异常状态,返回false
- COMMITTED
判断倒数第一、第二个Block是否达到最小副本数:
- 如果达到就同样调用
FSNamesystem.removeLease
释放租约以及closeFile,最后返回true
- 未达到就抛异常,稍后再试
- UNDER_CONSTRUCTION(已分配write/append)/UNDER_RECOVERY(正在恢复,文件租约过期时最后一个Block可能不是COMPLETE需要恢复,即同步现有的副本到其他DataNode)
- 如果在所有DataNode都找不到该Block,则无需恢复租约,直接移除该Block然后关闭该文件。返回
true
。 - 否则恢复该Block,然后将该文件租约分配给新ClientName。最后调用
LeaseManager.renewLease
更新该租约。返回false
4 问题
4.1 租约超过HardLimit
日志显示:
org.apache.hadoop.hdfs.server.namenode.LeaseManager: [Lease. Holder: DFSCLient_NONMAPREDUCE_-555555555_80, pending creates: 6].has expired hard limit
这里代表该LeaseManager的Monitor线程定时调用checkLeases
方法时检测到DFSCLient_NONMAPREDUCE_-555555555_80
的租约已经超过HardLimit(1小时)未更新,触发Recovering过程,将租约对应的非Complete文件进行恢复和重分配租约给HDFS_NameNode
。而pending creates: 6
表示租约关联了6个文件。
每个文件Recovering时会输出类似以下日志:
org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Recovering [Lease. Holder: DFSCLient_NONMAPREDUCE_-555555555_80, pending creates: 6], src=/hive/xxx/student/20200317/.part-0-0.inprogress.1bef055e-28e0-4079-be67-06f8916a03d3
关闭或恢复该文件。
org.apache.hadoop.hdfs.StateChange.StateChange: BLOCK* internalReleaseLease: All existing blocks are COMPLETE, lease removed, file /hive/xxx/student/20200317/.part-0-0.inprogress.1bef055e-28e0-4079-be67-06f8916a03d3 closed.
表示该文件最后一个块处于COMMITTED
状态(Block已经提交到DataNode,但还没完成),且倒数第二、倒数第一个Block的副本数已经达到最小副本数。此时将文件转为COMPLETE,释放该文件租约、closeFile即可。