目录
HA模式下,HDFS Active NameNode基于QJournalProtocol的protobuf RPC协议来和QuorumJournalNode进行通信。通信过程中,Active NameNode作为RPC客户端,客户端实现是QJournalProtocolTranslatorPB类,QuorumJournalNode作为RPC协议的服务器端,服务器端实现类是JournalNodeRpcServer。
在这里我必须实现向不太了解Paxos算法的读者讲清楚,什么叫做Quorum。对Paxos的简单理解,在有2N+1个服务器组成的集群,如果某一项操作得到了N+1台服务器的认可,就可以认为这个操作得到了整个集群的认可。我们用Quorum来代表基于Paxos协议的集群,比如,由3台QuorumJournalNode服务器组成的一个Quorum。
HDFS NameNode端存放了两种类型的文件,Image文件和EditLog文件。Image文件存放的是某一个时刻HDFS的内存文件镜像,即文件(File)和HDFS 块(Block)之间的对应关系。为了节省存储空间,Image采用了压缩存储方式,因此具有非常好的读取效率,但是,Image文件不可以进行修改和追加之类的操作,只是用来在NameNode启动的时候重构文件和块关系的镜像。Edit文件是某个时间段的写操作的集合,客户端对HDFS的每一个操作都会以追加的方式写入到EditLog文件。当HDFS NameNode启动的时候,会通过读取Image文件构造内存的基本映像,但是这个映像只是在生成该Image文件时候的文件映像,即checkpoint,此时还需要读取EditLog文件并将EditLog文件中的操作依次进行重新执行(replay)来复原crash以前的内存映像从而实现文件映像的完整恢复。
因此,我们可以看到NameNode存放的fsimage文件和edit文件有以下几种:
edits_0000000001005577851-0000000001005579475 //edits 文件,存放的edit操作从1005577851起,到1005579475止
edits_inprogress_0000000001005579476 //当前正在编辑的edits文件,从1005579476开始的
fsimage_0000000001005534360 //checkpoint文件,存放了从我们HDFS系统上线(格式化)到1005534360 这个操作的全体镜像
fsimage_0000000001005534360.md5 //checkpoint文件的md5校验址
由于Active NameNode对edit log文件具有写权限,而Standby NameNode只有读权限,因此,Active和Standby NameNode 的editlog的生命周期和状态转换是不同的,FSEditLog.State
这个枚举类型定义了FSEditLog的所有可能状态:
private enum State {
UNINITIALIZED,//edit log构造之前的状态
BETWEEN_LOG_SEGMENTS,//当前的log完成了写操作正在被关闭,同时下一个log还没有被创建完成,这个事件非常短
IN_SEGMENT,//预示这个log可以被写入
OPEN_FOR_READING,//如果NameNode是standby状态,则整个standby状态下都是OPEN_FOR_READING
CLOSED;
}
Active NameNode所控制的edit log有比较复杂的状态转换,从打开到写入数据到最后关闭,然后打开一个新的edit log文件,都会发生状态转换,并且,行为发生前都会对当前状态进行校验以确认当前状态正常,这个我们可用阅读上图中标明的导致状态转换发生的方法调用的代码。
而StandBy NameNode只对远程的edit log文件具有读权限,因此,当Standby NameNode启动完成以后,它的FSEditLog对象的当前状态就一直处于State.OPEN_FOR_READING状态。
当使用ZKFC进行HDFS的主从切换和HA管理,那么在我们启动NameNode进程的时候,两个NameNode都进入Standby状态,无法接收写操作,只可以读取本地的FSImage文件以及EditLog文件来构成自己的内存文件镜像,我们可以通过跟踪以下代码获取验证:
启动NameNode的时候,JVM的入口方式是NameNode.main()
:
NameNode的main方法入口
public static void main(String argv[]) throws Exception {
//略
NameNode namenode = createNameNode(argv, null);根据用户启动NameNode时传入的参数,创建NameNode对象
} catch (Throwable e) {
//略
}
}
创建一个NameNode节点
public static NameNode createNameNode(String argv[], Configuration conf)
throws IOException {
LOG.info("createNameNode " + Arrays.asList(argv));
switch (startOpt) {//根据用户传入的启动参数,进行不同的启动方式,比如--upgrade,--format等等参数
case FORMAT: { //略 }
case RECOVER: { //略 }
default: {//默认启动模式,不加任何额外参数,正常启动
DefaultMetricsSystem.initialize("NameNode");//metrics的创建,用于jmx监控,与运行逻辑无关
return new NameNode(conf);
}
}
}
NameNode的构造函数
public NameNode(Configuration conf) throws IOException {
this(conf, NamenodeRole.NAMENODE);
}
protected NameNode(Configuration conf, NamenodeRole role)
throws IOException {
//略
this.haEnabled = HAUtil.isHAEnabled(conf, nsId);
state = createHAState(getStartupOption(conf));//获取当前是Active或者Standby状态,具体代码看下文
this.allowStaleStandbyReads = HAUtil.shouldAllowStandbyReads(conf);
this.haContext = createHAContext();
try {
initializeGenericKeys(conf, nsId, namenodeId);
initialize(conf);//初始化方法,会在这里构造FSNamesystem,并加载image和edit文件
try {
haContext.writeLock();
state.prepareToEnterState(haContext);
state.enterState(haContext);//进入Active状态,则是ActiveState,否则,是StandbyState
} finally {
haContext.writeUnlock();
}
} catch (IOException e) {
//异常处理
}
this.started.set(true);
}
//根据用户设置的启动参数,确定启动以后的初始状态,如果是正常启动,则全部直接进入Standby状态
protected HAState createHAState(StartupOption startOpt) {
if (!haEnabled || startOpt == StartupOption.UPGRADE
|| startOpt == StartupOption.UPGRADEONLY) {
return ACTIVE_STATE;//如果没有打开HA,或者启动模式为UPGRADE,或UPGRADEONLY或者upgradeOnly,则直接进入Active状态(显然,非HA模式下只有一个NameNode,这个NameNode自然就是Active NameNode)
} else {
return STANDBY_STATE;//正常启动,先进入standby状态,随后的状态切换被ZKFC的becomeStandby等接口决定
}
}
在上文NameNode构造方法中,state.enterState(haContext);
会根据当前的状态(启动时都是StandBy),调用FSNamesystem.startStandbyServices()
或者SNamesystem.startActiveServices()
。
HA模式下两台NameNode启动以后都进入StandBy状态,随后,两个NameNode开始通过HAServiceProtocol.proto文件定义的RPC协议,接受ZKFailoverController的管理,ZKFailoverController会从当前的NameNode中选择一台NameNode作为Active NameNode,通过transitionToActive
命令对应的NameNode转换成Active,或者通过transitionToStandby
命令对应的NameNode转换成StandBy状态。
如果是Active NameNode,在成为Active状态以后,通过FSImage对象维护了内存镜像,同时,FSEditLog对象负责将HDFS用户的任何读写操作写入到EditLog,这里的写包括了两个目标:
注意,只有HA模式下允许通过dfs.namenode.shared.edits.dir
配置JournalNode的地址。非HA模式下,不允许配置JournalNode地址,否则启动时抛出异常:
Invalid configuration: a shared edits dir must not be specified if HA is not enabled.
在我的另外一篇博客《HDFS使用Backup Node、Checkpoint Node以及Standby Namenode进行checkpoint的机制详解》讲到,使用Backup Node和Checkpoint Node的时候,集群必须以非HA模式启动。
NameNode启动的时候(无论是Active还是StandBy),会通过调用FSNamesystem.loadFromDisk()
构造FSImage对象,同时构造FSEditLog对象:
static FSNamesystem loadFromDisk(Configuration conf) throws IOException {
checkConfiguration(conf);
FSImage fsImage = new FSImage(conf,
FSNamesystem.getNamespaceDirs(conf),//dfs.namenode.name.dir所配置的存放img和EditLog文件的目录
FSNamesystem.getNamespaceEditsDirs(conf));//通过dfs.namenode.shared.edits.dir和dfs.namenode.edits.dir配置的本地和远程目录
FSNamesystem namesystem = new FSNamesystem(conf, fsImage, false);
StartupOption startOpt = NameNode.getStartupOption(conf);
if (startOpt == StartupOption.RECOVER) {
namesystem.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
}
FSImage构造方法,初始化NNStorage用来管理Image和EditLog目录,同时构造FSEditLog对象,用来对Edit文件进行管理,再一次强调,这里的Edit文件,包括了NameNode的本地EditLog文件和存放在远程的JournalNode上的EditLog文件。
protected FSImage(Configuration conf,
Collection imageDirs,
List editsDirs)
throws IOException {
//NameNode端的存储,包括Img文件和Editlog文件的存储和管理都交给NNStorage对象进行管理
storage = new NNStorage(conf, imageDirs, editsDirs);
//略
this.editLog = new FSEditLog(conf, storage, editsDirs);//editsDirs包含了本地的editsLog文件目录和远程的有3个以上的JournalNode组成的Quorum
}
editsDirs中存放了远程的多台JournalNode组成的Quorum的URI,同时包括本地存放EditLog文件的目录。当NameNode是Active NameNode,会调用initJournalsForWrite,初始化对edit log的写操作。如果是Standby,是没有对EditLog的写权限的。
//在FSNamesystem.startActiveServices中被调用,只有ActiveNameNode才有写权限
public synchronized void initJournalsForWrite() {
initJournals(this.editsDirs);
state = State.BETWEEN_LOG_SEGMENTS;
}
/**
* 比如,如果配置成qjournal://10.120.117.102:8485;10.120.117.103:8485;10.120.117.104:8485/datahdfsmaster,则这整个是一个Journal
* 初始化Journal,对于QJM而言,创建一个JournalAndStream对象,交给JournalSet进行管理,这个JournalAndStream包含了对应的Journalmanager
* @param dirs
*/
private synchronized void initJournals(List dirs) {
int minimumRedundantJournals = conf.getInt(
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY,
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT);
synchronized(journalSetLock) {
journalSet = new JournalSet(minimumRedundantJournals);
for (URI u : dirs) {
//required edits dir必须是edits dir 的一个子集。查看getRequiredNamespaceEditsDirs,
//可以看到shared dir都加进来了,因此,shared dir都会是required
boolean required = FSNamesystem.getRequiredNamespaceEditsDirs(conf)
.contains(u);
if (u.getScheme().equals(NNStorage.LOCAL_URI_SCHEME)) {//这是一个本地的editLog的dir
StorageDirectory sd = storage.getStorageDirectory(u);
if (sd != null) {
journalSet.add(new FileJournalManager(conf, sd, storage),
required, sharedEditsDirs.contains(u));
}
} else {
//这是一个远程的JournalNode的URI,如果配置成qjournal://10.120.117.102:8485,则schema为qjournal
//根据hdfs-default.xml的配置qjournal这个schema对应的默认类是org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager
//并且,每一个URI(qjournal://10.120.117.102:848510.120.117.103:8485;10.120.117.104:8485/datahdfsmaster是一个URI)对应一个QuorumJournalManager
journalSet.add(createJournal(u), required,
sharedEditsDirs.contains(u));
}
}
}
在initJournals()
方法中,遍历editsDir中的所有item,构造JournalManager的实现类,FSEditLog上的操作,正是通过这些JournalManager的实现类具体实现的。
对本地EditLog文件、对远程的JournalNode上的EditLog文件和我在我的另外一篇博客《HDFS使用Backup Node、Checkpoint Node以及Standby Namenode进行checkpoint的机制详解》中讲到的Backup Node,它们各自使用不同的JournalManager接口的实现:
- 如果是本地的EditLog,则负责进行管理的JournalManager实现类是FileJournalManager
,它负责管理的stream会负责把edit操作写入到本地文件;
- 如果是基于JournalNode实现edit log共享,则负责管理的JournalManager实现类是QuorumJournalManager
,QuorumJournalManager管理等stream会负责将edit操作通过RPC 写到远程的JournalNode;
- 如果我们使用Backup Node,这时候,Backup Node在启动的时候,会向Active NameNode注册自己,最终会通过调用FSEditLog.registerBackupNode()
,把指向Backup Node的stream也加入到JournalSet的管理之中,这时候的JournalManager的实现类是BackupJournalManager
。这样,会通过RPC协议写入到Backup Node中。
通过initJournals()
,完成了这些JournalManager的构造并纳入到FSEditLog的管理。以后的各种操作,比如开始一个segment文件、向segment文件写入一个edit操作、关闭一个segment文件等,FSEditLog会遍历这些JournalManager并以此调用对应的接口。
完成了journals的初始化,在写操作到来之前,还需要通过调用FSEditLog.openForWrite()
方法,开始打开一个segment文件,这个方法中国年调用startLogSegment()
来创建一个写segment的句柄,即我们在文章开头提到的处于in-progress状态的edit文件的写句柄,如果我们使用JournalNode,那么打开文件、关闭文件的操作都会通过RPC的方式告知JournalNode ,因此JournalNode端会创建相同的处于in-progress状态的edit文件。
为了保证每一个操作成功的transaction(比如文件的增删改等)都能够被准确地、无遗漏的记录下来,HDFS将对应的操作记录到edit log中,是在操作真正执行以前进行的,即edit log其实是一个Write-Ahead-Log:
/**
* Start writing to the log segment with the given txid.
* Transitions from BETWEEN_LOG_SEGMENTS state to IN_LOG_SEGMENT state.
* 开始对segment进行写入操作。这个在openForWrite中被调用,并且最终调用journalSet.startLogSegment
* 方法
* FSEdit.startLogSegment() -> JournalSet.startLogSegment() -> JournalAndStream.startLogSegment()
*/
synchronized void startLogSegment(final long segmentTxId,
boolean writeHeaderTxn) throws IOException {
//略
//检查当前segmentId是否小于新的segmentId,curSegmentTxId表示当前正在写的segment文件的最小的id
Preconditions.checkState(segmentTxId > curSegmentTxId,
"Cannot start writing to log segment " + segmentTxId +
" when previous log segment started at " + curSegmentTxId);
//查看openForWrite方法,可以看到新的segment文件的Id必须是 txid + 1,即上一个transactionId+1
Preconditions.checkArgument(segmentTxId == txid + 1,
"Cannot start log segment at txid %s when next expected " +
"txid is %s", segmentTxId, txid + 1);
numTransactions = totalTimeTransactions = numTransactionsBatchedInSync = 0;
// TODO no need to link this back to storage anymore!
// See HDFS-2174.
storage.attemptRestoreRemovedStorage();
try {
//JournalSetOutputStream
editLogStream = journalSet.startLogSegment(segmentTxId,
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
} catch (IOException ex) {
..异常 ,略
}
//设置当前in_progress
curSegmentTxId = segmentTxId;
state = State.IN_SEGMENT;
if (writeHeaderTxn) {
logEdit(LogSegmentOp.getInstance(cache.get(),
FSEditLogOpCodes.OP_START_LOG_SEGMENT));
logSync();
}
}
通过调用startLogSegment()
,获取了一个类型为EditLogOutputStream
的editLogStream,这个封装类并不是针对某一个文件的write stream,而是封装了对本地edit log和远程JournalNode的写操作,及通过调用editLogStream.write(),会将对应的数据同时写入到远程的JournalNode和本地的edit log。上文讲过,处于写状态的EditLog文件的后缀中含有inprogress后缀,如:edits_inprogress_0000000000678039832
。
当HDFS上有对应的增删改操作,会同时写到本地和远程的JournalNode。写本地文件自不用说,是通过FileJournalManager管理等,对于写远程JournalNode,是通过QuorumJournalManager管理的,并且,由于至少3台JournalNode才能够组成一个Journal Quorum,因此,每次QuorumJournalManager将一个操作向远程发送,其实都是依次发送远程的每一个JournalNode。
这里需要特别强调,通过跟踪FSImage构造函数查看FSEditLog的构造函数可以看到,editsDir中存放的本地edit log和远程的QuorumJournalNode的URI,一定是QuorumJournalNode的URI在本地的edit log之前,可见,HDFS中的任何写操作通过FSEditLog记录 edit操作时,都是先将对应操作写入到远程的JournalNode,然后才写入本地的edit log文件,从而保证本地的edit log中记录的每一个操作,都必定是成功保存在远程的JournalNode(shared edit dir)上的操作。
每一个HDFS操作,都会调用FSEditLog提供了log*()方法(如logDelete()
、logTruncate()
、logConcat()
、logRename()方法),用来将对应的HDFS操作记录到本地和远程的edit log文件中。我们以客户端删除一个HDFS文件为例,看看如何通过logDelete()
记录这次delete操作并保存到edit log 文件中:
/**
* Add delete file record to edit log
*/
void logDelete(String src, long timestamp, boolean toLogRpcIds) {
// 创建封装了删除操作的对象
DeleteOp op = DeleteOp.getInstance(cache.get())
.setPath(src)
.setTimestamp(timestamp);
logRpcIds(op, toLogRpcIds);
logEdit(op);//调用logEdit,将对应的操作写入edit log
}
所有的操作如增删改查,它们调用对应的FSEditLog.log*()方法的过程类似,都是将操作封装为DeleteOp对象,然后调用FSEditLog.logEdit()
来将对应的操作写入到edit log:
/**
* Write an operation to the edit log. Do not sync to persistent
* store yet.
*/
void logEdit(final FSEditLogOp op) {
synchronized (this) {//一个FSEditLog对象的logEdit方法不可以多个线程同时调用
//略
//事务开始,txit自增1
long start = beginTransaction();//设置myTransactionId这个Threadlocal变量
op.setTransactionId(txid);//为EditLog中的每一个Edit设置一个独立的id
try {
//调用JournalSet.JournalSetOutputStream.write方法
editLogStream.write(op);
} catch (IOException ex) {
// All journals failed, it is handled in logSync.
} finally {
op.reset();
}
endTransaction(start);
// check if it is time to schedule an automatic sync
if (!shouldForceSync()) {
return;
}
isAutoSyncScheduled = true;//准备进行同步操作,因此将isAutoSyncScheduled设置为true,避免其它线程在自己正在sync的时候进行写操作。
}
// sync buffered edit log entries to persistent store
//跳出同步快,logSync方法不用放在同步块中,因为logSync只是synch 的过程需要同步,但是flush的过程不需要同步。
logSync();
}
上文讲过,文件写入到本地edit log使用的是FileJournalManager
,文件写入到远程的JournalNode,使用的QuorumJournalManager
。QuorumJournalManager
使用QuorumOutputStream
这个类封装了对远程journal的写操作,而写本地文件的FileJournalManager
则使用EditLogFileOutputStream
来实现对本地edit log文件的写操作。
我们从后面的讲解可以看到,QuorumOutputStream
最终使用RPC远程调用将数据发送到远程的JournalNode,而EditLogFileOutputStream
由于是写本地文件,比较简单,就是使用文件读写流来对本地的edit log文件进行读写操作。
从logEdit()
方法可以看到,将一个操作写入到EditLog,分为两步,通过editLogStream.write(op)
进行写操作和调用logSync()
进行同步操作,下面详细讲解其过程:
因此,一次写操作,实际上是直接写入缓存(即内存),只有到指定时机,才会将缓存flush到文件。在这里,我们就必须介绍双缓存(double buffer)了,双缓存使得缓存在进行flush的时候不需要阻塞写操作。
EditsDoubleBuffer(代码如下)是双缓存的具体实现类。它内部封装了两个不同的buffer,一个负责作为当前写缓存接收数据(bufCurrent),另外一个buffer不接受写操作,而是存放即将flush到远程的数据(bufReady):
private TxnBuffer bufCurrent; //当前正在进行数据写入的buf
private TxnBuffer bufReady; //当前正在进行数据输出的buf
private final int initBufferSize;
public EditsDoubleBuffer(int defaultBufferSize) {
initBufferSize = defaultBufferSize;
bufCurrent = new TxnBuffer(initBufferSize);
bufReady = new TxnBuffer(initBufferSize);
}
在进行flush操作前,会调用sestReadyToFlush()
,将写缓存bufCurrent和待刷新缓存bufReady调换,这样,写操作就开始往新的缓存中写数据,而flush操作就可以将bufReady中的数据进行flush操作,write操作和flush分别操作不同缓存,互不干扰,并行进行:
//缓存交换
public void setReadyToFlush() {
assert isFlushed() : "previous data not flushed yet";//确保当前的ready buffer中的数据已经被flush
TxnBuffer tmp = bufReady;
bufReady = bufCurrent;
bufCurrent = tmp;
}
交换缓存前,通过isFlushed()
,确保当前的bufReady中的数据全部已经完成了flush操作,避免数据完成flush,文件就被后面的写操作覆盖掉了。
由于是双缓存,因此,写和刷新可以同时进行。我们从logEdit()
方法的实现也可以看到,只有editLogStream.write(op);
方法在synchronized()
代码块中,因为一次只能有一个线程操作bufCurrent
进行写操作,而logSync()
则在synchronized()
代码块之外,没有进行同步,这样,假如Thread-1完成了一次写操作,发现已经可以进行flush了,于是调用logSync()
进行flush()
,在Thread-1进行flush的同时,Thread-2同步在进行写,互不干扰。当然,我们看logSynch()
的代码可以发现,并非整个logSync()
都不同步,其中调用setReadyToFlush()
的部分也是同步的,这可以理解,因为在进行缓存交换的过程中,不允许有写操作。幸好缓存交换过程极快,并不影响效率。
我们看一下HDFS如何通过双缓存方式,实现任何时候写入和刷新可以完全并行进行。我们以写文件到JournalNode使用的QuorumJournalManager为例说明,上面说过,QuorumJournalManager使用QuorumOutputStream这个类实现了EditLogOutputStream接口,负责将写操作写入远程的JournalNode:
class QuorumOutputStream extends EditLogOutputStream {
private final AsyncLoggerSet loggers;//使用AsyncLoggerSet来负责将新的editlog发送到远程JournalNode
private EditsDoubleBuffer buf;//双缓存技术,可以让一个缓存的数据被flush out的同时,另外一个缓存的数据正在被写入,然后交换角色。
private final long segmentTxId;//这是整个segment文件的Id,在FSEditLog.startLogSegment()中被设置
private final int writeTimeoutMs;
public QuorumOutputStream(AsyncLoggerSet loggers,
long txId, int outputBufferCapacity,
int writeTimeoutMs) throws IOException {
super();
this.buf = new EditsDoubleBuffer(outputBufferCapacity);
this.loggers = loggers;
this.segmentTxId = txId;
this.writeTimeoutMs = writeTimeoutMs;
}
QuorumOutputStream.flushAndSync()
是对EditLogOutputStream.flushAndSync()
接口方法的具体实现,即负责将缓存中的数据发送给远程的JournalNode:
protected void flushAndSync(boolean durable) throws IOException {
int numReadyBytes = buf.countReadyBytes();// 已经处于ready状态的buf的大小
if (numReadyBytes > 0) {
int numReadyTxns = buf.countReadyTxns();// readyBuffer中的操作数量大小
long firstTxToFlush = buf.getFirstReadyTxId();//这个Buffer中第一个OP的transactionId
DataOutputBuffer bufToSend = new DataOutputBuffer(numReadyBytes);
buf.flushTo(bufToSend);//将缓存在内存中的数据放入DataOutputBuffer,准备发送,实际上将数据拷贝了一份出来
assert bufToSend.getLength() == numReadyBytes;
byte[] data = bufToSend.getData();
assert data.length == bufToSend.getLength();
//通过AsyncLogger,将这些edit发送到远程,异步方式,返回一个获取结果的类似Future的句柄
QuorumCall qcall = loggers.sendEdits(
segmentTxId, firstTxToFlush,
numReadyTxns, data);
loggers.waitForWriteQuorum(qcall, writeTimeoutMs, "sendEdits");//同步等待返回结果,由于是paxos协议,因此只需要等待大多数节点返回成功,就可以认为成功返回
// Since we successfully wrote this batch, let the loggers know. Any future
// RPCs will thus let the loggers know of the most recent transaction, even
// if a logger has fallen behind.
loggers.setCommittedTxId(firstTxToFlush + numReadyTxns - 1);
}
}
flushAndSync()
负责将缓存的待发送的数据拷贝一份出来,然后调用AsyncLoggerSet.sendEdits()
,将数据发送给远程JournalNode,注意,这个发送也不是同步的,而是一个类似Future的操作,返回了一个获取结果的句柄,然后通过waitForWriteQuorum()
来同步等待响应,直到确认数据发送成功。需要先异步发送,再同步等待结果,这是因为远程 接收数据的RPC服务器是由多台JournalNode组成的Quorum,因此需要并行发送以提高吞吐量。AsyncLoggerSet封装和管理了对远程Quorum集群的操作,我们一起来看AsyncLoggerSet的具体实现。
由于JournalNode是基于paxos算法的实现,因此至少需要3台机器来组成一个Quorum集群。AsyncLoggerSet抽象了这样一组Quorum。AsyncLoggerSet管理了多个AsyncLogger,每个AsyncLogger对象和远程的Quorum集群的QuorumNode一一对应,负责和这个QuorumJournalNode进行RPC通信。
例如,如果我们在hdfs-site.xml中这样配置QuorumJournalNode:
<property>
<name>dfs.namenode.shared.edits.dirname>
<value>qjournal://127.0.0.1:8485;127.0.0.2:8485;127.0.0.3:8485/datahdfsmastervalue>
property>
那么qjournal://127.0.0.1:8485;127.0.0.2:8485;127.0.0.3:8485/datahdfsmaster
被用来构造一个URI对象,这个URI对象是由三个JournalNode节点构成的一个Quorum Cluster,由AsyncLoggerSet负责管理这个URI即这个Quorum Cluster,而URI中的每个唯一节点由一个AsyncLogger对象来负责。按照HDFS的规范,我们可以为dfs.namenode.shared.edits.dir
配置项定义多个URI,以逗号分隔:比如:
<property>
<name>dfs.namenode.shared.edits.dirname>
<value>qjournal://127.0.0.1:8485;127.0.0.2:8485;127.0.0.3:8485/datahdfsmaster,qjournal://127.0.0.4:8485;127.0.0.5:8485;127.0.0.6:8485/datahdfsmastervalue>
property>
QuorumJournalNode的多URI配置是互为副本,但是,在大多数情况下,我们都只配置一组URI,由3台以上的QuorumJournalNode组成一个Quorum。
AsyncLoggerSet 维护了AsyncLogger对象的一个集合,每一个AsyncLogger维护了到某个JournalNode之间的RPC连接,基于这个RPC将对应的edit操作发送给远程的JournalNode,因此AsyncLoggerSet相当于是维护了对一个JournalNode集群的写操作。AsyncLogger其实是一个接口,定义了一个以异步方式发送日志到远程的操作集合,例如:
- startLogSegment():告诉远程RPC Server开始写一个edit log文件;
- sendEdits():将一系列的数据通过RPC发送到远程;
- finalizeLogSegment():通知远程关闭一个edit log文件;
我们从QuorumJournalManager的构造方法中可以看到,默认AsyncLogger的实现类是IPCLoggerChannel:
public class IPCLoggerChannel implements AsyncLogger {
private final Configuration conf;
protected final InetSocketAddress addr;
private QJournalProtocol proxy;//RPC客户端
IPCLoggerChannel封装了一个RPC的客户端,关于Hadoop RPC的相关解析,大家可以看我另外两篇博客《Hadoop 基于protobuf 的RPC的客户端实现原理》和《Hadoop 基于protobuf 的RPC的服务器端实现原理》,分别详细讲解了Hadoop基于google protobuf RPC 协议的远程过程调用的原理,本文中不再详细解释。总之,到了IPCLoggerChannel这一层,edit log就通过RPC发送到了远程的JournalNode。
在这里,这个RPC协议的名称是QJournalProtocol,协议文件定义在QJournalProtocol.proto中,RPC客户端是NameNode,RPC服务器端是JournalNode。在服务端即JournalNode端,对应的服务端实现是QJournalProtocolServerSideTranslatorPB类,其实是调用JournalNodeRpcServer来实现具体方法的调用。比如,NameNode结束一个旧的segment文件、开始一个新的segment文件,会调用startLogSegment()方法,通过基于QJournalProtocol协议的RPC,会调用这个协议的startLogSegment接口,最终在JournalNode端会引发QJournalProtocolServerSideTranslatorPB.startLogSegment()
方法的调用:
public StartLogSegmentResponseProto startLogSegment(RpcController controller,
StartLogSegmentRequestProto req) throws ServiceException {
try {
int layoutVersion = req.hasLayoutVersion() ? req.getLayoutVersion()
: NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
impl.startLogSegment(convert(req.getReqInfo()), req.getTxid(),
layoutVersion);//impl的实现是JournalNodeRpcServer对象
} catch (IOException e) {
throw new ServiceException(e);
}
return VOID_START_LOG_SEGMENT_RESPONSE;
}
沿着这个方法,会最终实现JournalNode对这个startLogSegment()接口的处理以及将响应返回给NameNode。
对于Active NameNode,QJournalProtocol协议主要会使用这个协议将相应的EditLog发送给远程的JournalNode,而对于Standby NameNode,则使用QJournalProtocol协议向远程的JournalNode询问可以进行同步拉取到本地的EditLog的文件列表,随后,Standby NameNode会根据这些文件列表,直接创建对应的Http Stream, 用来把这些文件同步到本地。我们在下文中会讲到这个过程。
其实,如果我们对HDFS的NameNode、JournalNode以后我在另外一篇文章会讲到的Backup Node这些租金按,当他们在作为RPC Server的角色时,其代码层次结构都是一样的,尤其是NameNode,它是作为多种RPC协议的服务端角色,每个协议在服务端都有一个.*ServerSideTranslatorPB
类对应这个协议的服务端调用入口,同时,每一个角色会有.*RpcServer
类,这个类负责真正实现了这个组件的各种协议的服务端调用实现并向调用者返回结果,每个协议的调用入口.*ServerSideTranslatorPB
都是直接调用.*RpcServer
对应方法并返回结果的:
比如:
NameNode作为服务端,其实涉及到用于处理用户请求的ClientProtocol、用户处理DataNode请求的DataNodeProtocol、用户处理Standby NameNode请求的NameNodeProtocol等等协议,他们的UML架构图如下:
从图上可以看到,NameNode同时作为多个RPC协议(ClientProtocol、DataNodeProtocol和NameNodeProtocol等)的客户端,每个协议都有一个对应的服务端的调用入口(*ProtocolServiceSideTranslatorPB),但是其实全部都是调用NameNodeRpcServer这个类似于公共工具类的类来具体实现的;
在比如,JournalNode是QJournalProtocol协议的服务端,上文已经讲过,这个协议是NameNode作为客户端发送edit 操作的接口,它的UML类图是:
JournalNode作为服务端,只有一个RPC协议QJournalProtocol,因此UML结构简单,但是代码层次与NameNode相似;
还有Backup Node(在我的另外一篇博客《HDFS使用Backup Node、Checkpoint Node以及Standby Namenode进行checkpoint的机制详解》中会讲到)作为JournalProtocol协议的服务端实现,这个协议主要是NameNode作为客户端发送edit操作的RPC接口:
同样,Backup Node也遵循一样的代码层次结构。
关于JournalNode作为服务端,收到QJournalProtocol协议的startLogSegment()
、sendEdits()
等等接口请求以后如何处理,即如何打开文件、保存这个写操作日志等,本文不进行叙述。
在这里,我们可以看到非常HDFS各个组件在RPC层面的代码结构非常规整。
上文中,我们看到QuorumOutputStream.flush()
方法最终调用AsyncLoggerSet.sendEdits()
将数据发送到远程的多个JournalNode:
public QuorumCall<AsyncLogger, Void> sendEdits(
long segmentTxId, long firstTxnId, int numTxns, byte[] data) {
Map<AsyncLogger, ListenableFuture<Void>> calls = Maps.newHashMap();
for (AsyncLogger logger : loggers) {
//默认的AsyncLogger实现类是IPCLoggerChannel,通过调用AsyncLogger.Factory().create进行构造
ListenableFuture<Void> future =
logger.sendEdits(segmentTxId, firstTxnId, numTxns, data);
calls.put(logger, future);
}
return QuorumCall.create(calls);
}
AsyncLoggerSet.sendEdits()
会调用自己管理的所有AsyncLogger的sendEdits()
,即,将数据发送到Quorum 集群的每一个节点。并且,把各自调用的回调句柄(类似java Future的异步调用句柄)保存到Map
中并作为返回值返回,用来给上层调用者获取结果并判断这组消息是否发送成功。由于是将消息同时发送给多个JournalNode,因此,是否发送成功,需要基于paxos协议的规定进行判断,QuorumCall就是对基于paxos协议的JournalNode集群的返回结果的封装,我们下文介绍。
QuorumCall是一个工具类,提供了对基于paxos算法的多节点的异步调用的封装。它的两个方法,提供了以下两个核心逻辑:
1. 向Quorum中的所有节点发送请求并返回回调句柄
当用户向远程的每个QuorumJournalNode发送了异步请求,对每一个QuorumJournalNode会拿到一个对应的回调句柄,QuorumCall管理这些回调句柄,create()方法为这些发送的请求创建了回调,当请求收到响应,回调方法会被自动调用,从下面的代码可以看到,这些回调方法主要是根据响应判断是成功响应还是异常响应:
/**
* 将这些对Quorum的异步调用纳入到QuorumCall 对象的管理之内。如果异步调用发生了返回结果,则会调用addException
* 或者addResult方法来记录结果,调用者通过调用waitFor来获取结果
* @param calls
* @return
*/
static QuorumCall create(Map> calls) {
final QuorumCall qr = new QuorumCall();
for (final Entry> e : calls.entrySet()) {
Preconditions.checkArgument(e.getValue() != null,
"null future for key: " + e.getKey());
Futures.addCallback(e.getValue(), new FutureCallback() {
@Override
public void onFailure(Throwable t) {
qr.addException(e.getKey(), t);
}
@Override
public void onSuccess(RESULT res) {
qr.addResult(e.getKey(), res);
}
});
}
return qr;
}
调用者在发起了异步远程调用以后,通过调用waitFor()
方法来等待返回结果。从参数中可以看到,通过设置minResponses、minSuccesses等等,可以对返回条件进行规定,最直观的情况,基于paxos协议,我们规定,在由2N+1个QuorumJournalManager组成的Quorum中,我们需要等待至少N+1个节点返回正确结果才认为请求发送成功:
/**
* 等待远程的quorum返回足够多的结果,即在Paxos协议认为写成功才返回
* @param minResponses 如果响应数量大于该值,就可以立刻返回,不论这些响应是成功还是失败,否则,继续等待直到超时
* @param minSuccesses 如果成功响应的个数大于该值,就可以立刻返回,否则继续等待直到超时
* @param maxExceptions 如果收到的异常的个数大于该值,就立刻返回
* @param millis 最长等待时间
*/
public synchronized void waitFor(
int minResponses, int minSuccesses, int maxExceptions,
int millis, String operationName)
throws InterruptedException, TimeoutException {
long st = Time.monotonicNow();
long nextLogTime = st + (long)(millis * WAIT_PROGRESS_INFO_THRESHOLD);
long et = st + millis;
while (true) {
//循环等待,直到满足返回条件
}
}
QuorumOutputStream.flushAndSync()
通过创建调用请求,将这些请求逐个发送给所有的QuorumNode,然后,通过调用QuorumCall.create()
,将句柄交给通过QuorumCall进行代理,然后,调用QuorumCall.waitFor()
,等待响应。
我们看看上文说的QuorumOutputStream.flushAndSync()
-> AsyncLoggerSet.waitForWriteQuorum()
来等待返回结果,就是调用QuorumCall.waitFor()
方法,可以看一下AsyncLoggerSet.waitForWriteQuorum()
对QuorumCall.waitFor()
的调用:
Map waitForWriteQuorum(QuorumCall q,
int timeoutMs, String operationName) throws IOException {
int majority = getMajoritySize();//获取大多数的数量
try {
q.waitFor(
loggers.size(), // either all respond
majority, // or we get a majority successes
majority, // or we get a majority failures,
timeoutMs, operationName);
} catch (InterruptedException e) {
方法中通过getMajoritySize()
来获取一个大多数的度量值:
/**
* @return the number of nodes which are required to obtain a quorum.
*/
int getMajoritySize() {
return loggers.size() / 2 + 1;
}
从getMajoritySize()
可以看到,大多数的定义,是根据当前journalnode的数量多1/2再加1,结合上文中对QuorumCall.waitFor()
的参数解释,我们可以看到,对Journal Quorum发送请求的时候,等待和判断成功或失败的标准:
Standby NameNode 使用EditLogTailer线程来负责向远程的QuorumJournalNode读取新的EditLog文件。在Standby NameNode启动的时候,构造了EditLogTailer对象,并启动了线程EditLogTailerThread来不断和远程的QuorumJournalNode通信以拉取新的log segment文件。
Standby NameNode启动的时候初始化并启动EditLogTailer线程:
void startStandbyServices(final Configuration conf) throws IOException {
if (!getFSImage().editLog.isOpenForRead()) {
// During startup, we're already open for read.
/**在NameNode启动的时候,会先进入Standby状态,然后开始读取EditLog文件
* 在 NameNode 启动的时候会进行数据恢复,首先把 FSImage 文件加载到内存中形成文件系统镜像,
* 然后再把 EditLog 之中 FsImage 的结束事务 id 之后的 EditLog 回放到这个文件系统镜像上
*/
getFSImage().editLog.initSharedJournalsForRead();
}
...略
editLogTailer = new EditLogTailer(this, conf);//创建一个editLogTailer线程,用来从远程的QJM服务器上拉取editlog
editLogTailer.start();
...略
}
EditLogTailer的工作代码:
@Override
public void run() {
//略
doWork();
//略
}
private void doWork() {
while (shouldRun) {
try {
//如果已经超过指定时间没有从远程进行过doTailEdits操作,即很久没有拉取过非空的segment
//并且,在上一次调用triggerActiveLogRoll到现在已经运行过doTailEdits,则调用一次triggerActiveLogRoll用来告知远程的Active NameNode
//进行一次roll操作,NameNode roll操作会告知QJM也进行roll操作,这样,Standby Namenode就可以拉取到这个segment
if (tooLongSinceLastLoad() &&
lastRollTriggerTxId < lastLoadedTxnId) {
triggerActiveLogRoll();
}
//略
doTailEdits();//开始拉取远程的segment文件
//略
}
//开始获取文件列表,并开始读取这些文件
void doTailEdits() throws IOException, InterruptedException {
try {
//略
FSImage image = namesystem.getFSImage();
long lastTxnId = image.getLastAppliedTxId();//Standby NameNode当前已经有的最大的TxId
Collection streams;
try {
//从lastTxnId + 1开始拉取segment
streams = editLog.selectInputStreams(lastTxnId + 1, 0, null, false);//false代表不允许获取处于in-progress状态的文件
//略
editsLoaded = image.loadEdits(streams, namesystem);
lastLoadedTxnId = image.getLastAppliedTxId();//记录同步过来的最后一个transactionId,代表当前Standby NameNode的同步位置
} finally {
namesystem.writeUnlock();
}
}
通过QJournalProtocol协议的getEditLogManifest()
接口,StandBy NameNode与远程的每一个JournalNode通信,获取了它们各自的edit log的清单。我们从doTailEdits()
方法调用editLog.selectInputStreams(lastTxnId + 1, 0, null, false)
的方式可以看到,最后一个参数false,代表是否允许请求处于in-progress状态的文件,false说明StandBy NameNode不会请求还处在读取和打开状态的edit log文件,只会请求已经完成读写并关闭的edit log文件。由此可见,StandBy NameNode将文件同步到本地,并非实时也并非准实时,而是存在着文件粒度的延迟的。但是,只要Active NameNode和JournalManager之间基于TCP协议的文件传输能够实现准实时,这些操作就不会丢失。
在完成了对文件清单的获取,下一步,就是向JournalNode读取清单中的文件了。下文将详细讲解,StandBy NameNode如何通过HTTP协议,读取远程的JournalNode中的文件到本地。
QuorumJournalManager.selectInputStreams()
就是完成创建对远程的edit log文件的HTTP读取流,供调用者进行文件读取。这个stream的建立,即对本地文件的读取流,或者对远程文件的基于http的读取流,是通过public static EditLogInputStream fromUrl(URLConnectionFactory connectionFactory, URL url, long startTxId, long endTxId, boolean inProgress)
方法创建的。我们跟踪这个stream的创建过程,不难发现其实底层就是通过java原生的java.net.HttpURLConnection
创建的连接,基于这个连接读取的文件。
很显然,有可能,由于这些stream来自多个远程的QuorumJournalNode,有可能存在多个stream对应的文件的startTxId相同,或者startTxId 和endTxId存在交叠,因此,在返回这些读取流的时候,会对他们进行排序、去重,同时,将具有相同txId的stream封装为冗余(redundant)stream,即互为备份,一个不可用,就从另外一个读取;
关于stream的排序,定义了两个比较器,分别为:
EDIT_LOG_INPUT_STREAM_COMPARATOR
(大文件优先比较器)
static final public Comparator
EDIT_LOG_INPUT_STREAM_COMPARATOR = new Comparator() {
@Override
public int compare(EditLogInputStream a, EditLogInputStream b) {
return ComparisonChain.start().
compare(a.getFirstTxId(), b.getFirstTxId()).
compare(b.getLastTxId(), a.getLastTxId()).
result();
}
};
这个排序器的作用:两个edit log文件,如果endTxId相同,那么startTxId越小越靠前,相反,如果startTxId相同,那么endTxId越大越靠前,这样做的目的是为了解决文件之间的范围重叠,比如,如果文件A的startTxId-endTxId范围完全覆盖了另外一个文件,即完全包含了另外一个文件,那我们当然使用范围更大的文件了,这样可以保证txId不会发生断裂,断裂,就中有一段儿范围的txId发生丢失。
以及LOCAL_LOG_PREFERENCE_COMPARATOR
(本地文件优先比较器)
private static final Comparator
LOCAL_LOG_PREFERENCE_COMPARATOR = new Comparator() {
@Override
public int compare(EditLogInputStream elis1, EditLogInputStream elis2) {
// we want local logs to be ordered earlier in the collection, and true
// is considered larger than false, so we want to invert the booleans here
return ComparisonChain.start().compare(!elis1.isLocalLog(),
!elis2.isLocalLog()).result();
}
};
这个排序器的作用:如果某个edit log与另外一个edit log的startTxId和endTxId相同,那么本地文件的优先级要高于远程文件,毕竟,从本地文件获取的成本要低很多。
从QuorumJournalManager.selectInputStreams()
方法跟踪,我们就可以看到如何通过以上两个比较器排序、去重,从而获得一个有序的、去重读、且具有冗余备份功能的多文件读取流:
获取远程每个节点和本地的所有edit log segment的文件清单
对于每一个QuorumJournalNode节点
取出该节点返回的文件清单
对于文件清单中的每一个文件
创建该文件的读取流
将该文件流添加到allStreams中 //注意,allStreams是使用比较器EDIT_LOG_INPUT_STREAM_COMPARATOR(大文件优先比较器)创建的一个PriorityQueue,因此allStreams中的文件按照startTxId从小到大排列,并且保证endTxId从大到小排列
//获得了所有节点的所有文件的stream,存放在allStreams中,并且是根据EDIT_LOG_INPUT_STREAM_COMPARATOR比较器有序的
//开始进行文件校验与合并(JournalSet.chainAndMakeRedundantStreams方法)
对这些文件流进行归并和转换,将具有相同startTxId的文件合并为一个冗余代理对象RedundantEditLogInputStream
chainAndMakeRedundantStreams()
方法对已经排序的stream集合转换成具有冗余备份功能的stream集合,基本做法,是对于那些startTxId相同的N(N>=1)个文件的文件流,转换成一个RedundantEditLogInputStream对象,这个对象相当于将这一个或者多个stream抽象成了一个stream,这个stream具有冗余备份功能,即,默认都是从第一个stream读取,如果第一个stream损坏,就可以从第二个stream读取,只要有一个stream成功,就成功返回。从封装的角度讲,这个故障转移的过程肯定是对调用者隐藏的。同时,RedundantEditLogInputStream对象代理的一个或者多个具有相同startTxId的stream,是按照LOCAL_LOG_PREFERENCE_COMPARATOR
(本地文件优先比较器)进行排序的,即优先从本地读取数据。
这个InputStream基于Http协议,由此可见,Standby NameNode将远程的JournalNode上的EditLog文件共享到本地,是基于Http进行的。而我们上文讲到,Active NameNode将本地的EditLog操作发送到远程的JournalNode是基于RPC进行的。并且,可以看到,它们还有一个重要区别,那就是Active NameNode将EditLog发送到远程是实时发送的,这种设计,一方面是因为实时发送可以尽量减少主备切换发生的时候的数据丢失,同时,由于是实时发送,基于HTTP的数据传输的数据传输率(有效数据占总体网络流量的比例)比较低,而基于Java NIO的hadoop RPC由于走的是TCP通道,具有很高的数据传输率,能够很好支持多次、小容量的数据传输特点。而Standby NameNode从JournalNode拉取数据,则不需要如此高的实时性,因为,如果Active NameNode挂掉,只要数据实时发送给JournalNode,那么这些数据都会到达Standby NameNode,实时性低只会让Standby NameNode接管整个集群的时间稍有延迟。
在完成了基于HTTP协议的文件读取流,就可以通过该stream直接开始读取文件了,读取文件的目的,就是顺序读取stream中的op操作,将这些操作加载到内存,从而使得自己的文件管理镜像始终与Active NameNode一致(当然,必须存在延迟的)。这里必须注意,Standby NameNode是不会存取edit log文件d到磁盘的,仅仅只是将这些文件在内存中重演。因为edit log文件的存取和备份已经由JournalNode完成了,即edit log文件具备了多个副本,因此Standby NameNode没有必要再进行备份。但是,Standby NameNode会进行checkpoint操作进而生成fsimage文件,这在我的另外一篇博客《HDFS使用Backup Node、Checkpoint Node以及Standby Namenode进行checkpoint的机制详解》中会详细讲解。因为这些操作都是Active NameNode保存在内存并且以管理者的身份将这些操作转换成命令发送给DataNode,而StandBy NameNode也需要模仿这个加载过程,只不过不需要像远程发送命令而已。因此,我把StandBy NameNode将edit log一个一个加载到内存中的过程叫做重演。这样,当Active NameNode出现问题,需要进行主备切换,只需要请求到延迟到部分,就可以接替Active NameNode管理整个系统,从而完成主备切换了,具体逻辑我来详细讲解;
回到上文中的StandBy NameNode通过EditLogTailer线程负责不断请求远程文件,EditLogTailer.doTailEdit()
方法中在通过调用
streams = editLog.selectInputStreams(lastTxnId + 1, 0, null, false);//false代表不允许获取处于in-progress状态的文件
来获取了文件读取流,然后,通过
editsLoaded = image.loadEdits(streams, namesystem);
读取远程文件中的操作并在自己内存中进行重演,我们来看FSImage.loadEdits()
方法的具体实现:
private long loadEdits(Iterable editStreams,
FSNamesystem target, StartupOption startOpt, MetaRecoveryContext recovery)
throws IOException {
long prevLastAppliedTxId = lastAppliedTxId;
try {
FSEditLogLoader loader = new FSEditLogLoader(target, lastAppliedTxId);//FSEditLogLoader负责对当前这一轮的多个stream进行读取并加载到内存
// Load latest edits
//遍历所有的edtiStream,将这个stream对应的EditLog文件读入本地
//每一个editIn对象是一个RedundantEditLogInputStream对象。可以搜索在standby namenode日志搜‘ expecting start txid’
for (EditLogInputStream editIn : editStreams) {
try {
//loader会不断从远程读取新的op,同时从op中提取txId来更新自己的txId,从而让StandbyNamenode的txid逐渐向前递增
loader.loadFSEdits(editIn, lastAppliedTxId + 1, startOpt, recovery);
} finally {
// Update lastAppliedTxId even in case of error, since some ops may
// have been successfully applied before the error.
lastAppliedTxId = loader.getLastAppliedTxId();//从loader中获取当前已经load过来的最新的txId,更新到FSImage对象
}
// If we are in recovery mode, we may have skipped over some txids.
if (editIn.getLastTxId() != HdfsConstants.INVALID_TXID) {
lastAppliedTxId = editIn.getLastTxId();
}
}
} finally {
FSEditLog.closeAllStreams(editStreams);//关闭所有读取流
}
return lastAppliedTxId - prevLastAppliedTxId;
}
loadEdit(
)会创建一个FSEditLogLoader对象,用来读取这些stream中的一个一个的操作并加载到内存,同时,用lastAppliedTxId记录当前已经成功读取并加载的位置。这样,当EditLogTailer的下一轮运行开始的时候,就从这个lastAppliedTxId开始请求edit log文件,因为这个位置之前的操作已经完成了同步和加载。
我们重点来看FSEditLogLoader如何从stream中读取操作然后在内存中重演的。
long loadEditRecords(EditLogInputStream in, boolean closeOnExit,
long expectedStartingTxId, StartupOption startOpt,
MetaRecoveryContext recovery) throws IOException {
FSDirectory fsDir = fsNamesys.dir;
fsNamesys.writeLock();//由于是可重入锁,因此允许同一线程的多次加锁
fsDir.writeLock();
long expectedTxId = expectedStartingTxId;
....
try {
while (true) {
try {
FSEditLogOp op;
try {
op = in.readOp();//读取一个op操作
...
} catch (Throwable e) {
//如果处于recovery模式下发生异常,则为用户提供恢复选择
MetaRecoveryContext.editLogLoaderPrompt(
"We failed to read txId " + expectedTxId,
recovery, "skipping the bad section in the log");
in.resync();//正常的StandBy NameNode同步发生异常,跳过这个操作
continue;
}
try {
long inodeId = applyEditLogOp(op, fsDir, startOpt,
in.getVersion(true), lastInodeId);//重要!!将这个op在内存中进行重演
....
}
if (op.hasTransactionId()) {
lastAppliedTxId = op.getTransactionId(); //更新lastAppliedTxId,代表当前已经同步过来的TxId
expectedTxId = lastAppliedTxId + 1;//下一个期望获得的txid值
} else {
expectedTxId = lastAppliedTxId = expectedStartingTxId;
.....
} catch (Exception e) { }
}
} finally {
...
fsDir.writeUnlock();//edit log对应的目录解锁
fsNamesys.writeUnlock();//namesystem解锁
}
return numEdits;
}
loadEditRecords()
就是负责逐个读取操作,通过调用applyEditLogOp()进行重演:
private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
StartupOption startOpt, int logVersion, long lastInodeId) throws IOException {
...
switch (op.opCode) {
case OP_ADD:
....;
break;
case OP_CLOSE:
....;
break;
case OP_APPEND:
....;
break;
....
....
default:
throw new IOException("Invalid operation read " + op.opCode);
}
return inodeId;
}
applyEditLogOp()
方法的主题结构非常清晰,通过一个switch-case
结构体判断op类型,然后对op执行对应的操作。这些定义为一个枚举类,叫做FSEditLogOpCodes,大家有兴趣可以自行参考。具体地,某一个类型的op,如何应用到内存,这属于NameNode的文件和块关系管理等部分内容,我会另起博客详细讲解。
这就是我们HA模式下的HDFS中的edit操作、edit log文件在Active NameNode、QuorumJournalManager和Standby NameNode之间的操作、传输过程。从Active NameNode到QuorumJournalManager通过RPC连接实时进行,采用PUSH的方式,而从QuorumJournalManager到Standby NameNode则采用基于HTTP方式的文件拷贝,属于PULL的方式。了解edit操作的传输备份过程,有助于我们理解HDFS的完备的备份恢复机制,这些是HDFS稳定性的保证,同时,只有了解了备份恢复机制,了解NameNode、JournalNode之间的通信、配合、数据传输,我们才有能力在HDFS出现异常的时候,理解堆栈,找到原因,及时解决问题。