HDFS-13596. NN restart fails after RollingUpgrade from 2.x to 3.x. Contributed by Fei Hui.

(cherry picked from commit abc8fde4ca)
This commit is contained in:
Akira Ajisaka 2019-08-23 14:44:34 +09:00
parent a5690b29a7
commit b40aa71cf0
No known key found for this signature in database
GPG Key ID: C1EDBB9CA400FD50
13 changed files with 163 additions and 22 deletions

View File

@ -433,7 +433,7 @@ public class QuorumJournalManager implements JournalManager {
loggers.waitForWriteQuorum(q, startSegmentTimeoutMs,
"startLogSegment(" + txId + ")");
return new QuorumOutputStream(loggers, txId, outputBufferCapacity,
writeTxnsTimeoutMs);
writeTxnsTimeoutMs, layoutVersion);
}
@Override

View File

@ -36,17 +36,18 @@ class QuorumOutputStream extends EditLogOutputStream {
public QuorumOutputStream(AsyncLoggerSet loggers,
long txId, int outputBufferCapacity,
int writeTimeoutMs) throws IOException {
int writeTimeoutMs, int logVersion) throws IOException {
super();
this.buf = new EditsDoubleBuffer(outputBufferCapacity);
this.loggers = loggers;
this.segmentTxId = txId;
this.writeTimeoutMs = writeTimeoutMs;
setCurrentLogVersion(logVersion);
}
@Override
public void write(FSEditLogOp op) throws IOException {
buf.writeOp(op);
buf.writeOp(op, getCurrentLogVersion());
}
@Override

View File

@ -59,7 +59,7 @@ class BackupJournalManager implements JournalManager {
public EditLogOutputStream startLogSegment(long txId, int layoutVersion)
throws IOException {
EditLogBackupOutputStream stm = new EditLogBackupOutputStream(bnReg,
journalInfo);
journalInfo, layoutVersion);
stm.startLogSegment(txId);
return stm;
}

View File

@ -53,8 +53,8 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
private EditsDoubleBuffer doubleBuf;
EditLogBackupOutputStream(NamenodeRegistration bnReg, // backup node
JournalInfo journalInfo) // active name-node
throws IOException {
JournalInfo journalInfo, int logVersion) // active name-node
throws IOException {
super();
this.bnRegistration = bnReg;
this.journalInfo = journalInfo;
@ -70,11 +70,12 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
}
this.doubleBuf = new EditsDoubleBuffer(DEFAULT_BUFFER_SIZE);
this.out = new DataOutputBuffer(DEFAULT_BUFFER_SIZE);
setCurrentLogVersion(logVersion);
}
@Override // EditLogOutputStream
public void write(FSEditLogOp op) throws IOException {
doubleBuf.writeOp(op);
doubleBuf.writeOp(op, getCurrentLogVersion());
}
@Override
@ -89,6 +90,7 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
public void create(int layoutVersion) throws IOException {
assert doubleBuf.isFlushed() : "previous data is not flushed yet";
this.doubleBuf = new EditsDoubleBuffer(DEFAULT_BUFFER_SIZE);
setCurrentLogVersion(layoutVersion);
}
@Override // EditLogOutputStream

View File

@ -94,7 +94,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
@Override
public void write(FSEditLogOp op) throws IOException {
doubleBuf.writeOp(op);
doubleBuf.writeOp(op, getCurrentLogVersion());
}
/**
@ -120,6 +120,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
writeHeader(layoutVersion, doubleBuf.getCurrentBuf());
setReadyToFlush();
flush();
setCurrentLogVersion(layoutVersion);
}
/**

View File

@ -35,6 +35,8 @@ public abstract class EditLogOutputStream implements Closeable {
// these are statistics counters
private long numSync; // number of sync(s) to disk
private long totalTimeSync; // total time to sync
// The version of the current edit log
private int currentLogVersion;
public EditLogOutputStream() throws IOException {
numSync = totalTimeSync = 0;
@ -147,4 +149,18 @@ public abstract class EditLogOutputStream implements Closeable {
public String generateReport() {
return toString();
}
/**
* @return The version of the current edit log
*/
public int getCurrentLogVersion() {
return currentLogVersion;
}
/**
* @param logVersion The version of the current edit log
*/
public void setCurrentLogVersion(int logVersion) {
this.currentLogVersion = logVersion;
}
}

View File

@ -55,9 +55,9 @@ public class EditsDoubleBuffer {
bufReady = new TxnBuffer(initBufferSize);
}
public void writeOp(FSEditLogOp op) throws IOException {
bufCurrent.writeOp(op);
public void writeOp(FSEditLogOp op, int logVersion) throws IOException {
bufCurrent.writeOp(op, logVersion);
}
public void writeRaw(byte[] bytes, int offset, int length) throws IOException {
@ -148,13 +148,13 @@ public class EditsDoubleBuffer {
reset();
}
public void writeOp(FSEditLogOp op) throws IOException {
public void writeOp(FSEditLogOp op, int logVersion) throws IOException {
if (firstTxId == HdfsServerConstants.INVALID_TXID) {
firstTxId = op.txid;
} else {
assert op.txid > firstTxId;
}
writer.writeOp(op);
writer.writeOp(op, logVersion);
numTxns++;
}

View File

@ -155,7 +155,9 @@ public class FSEditLog implements LogsPurgeable {
//initialize
private JournalSet journalSet = null;
private EditLogOutputStream editLogStream = null;
@VisibleForTesting
EditLogOutputStream editLogStream = null;
// a monotonically increasing counter that represents transactionIds.
// All of the threads which update/increment txid are synchronized,

View File

@ -282,6 +282,11 @@ public abstract class FSEditLogOp {
public abstract void writeFields(DataOutputStream out)
throws IOException;
public void writeFields(DataOutputStream out, int logVersion)
throws IOException {
writeFields(out);
}
static interface BlockListUpdatingOp {
Block[] getBlocks();
String getPath();
@ -546,6 +551,12 @@ public abstract class FSEditLogOp {
@Override
public void writeFields(DataOutputStream out) throws IOException {
throw new IOException("Unsupported without logversion");
}
@Override
public void writeFields(DataOutputStream out, int logVersion)
throws IOException {
FSImageSerialization.writeLong(inodeId, out);
FSImageSerialization.writeString(path, out);
FSImageSerialization.writeShort(replication, out);
@ -564,7 +575,10 @@ public abstract class FSEditLogOp {
FSImageSerialization.writeString(clientMachine,out);
FSImageSerialization.writeBoolean(overwrite, out);
FSImageSerialization.writeByte(storagePolicyId, out);
FSImageSerialization.writeByte(erasureCodingPolicyId, out);
if (NameNodeLayoutVersion.supports(
NameNodeLayoutVersion.Feature.ERASURE_CODING, logVersion)) {
FSImageSerialization.writeByte(erasureCodingPolicyId, out);
}
// write clientId and callId
writeRpcIds(rpcClientId, rpcCallId, out);
}
@ -4854,16 +4868,18 @@ public abstract class FSEditLogOp {
* Write an operation to the output stream
*
* @param op The operation to write
* @param logVersion The version of edit log
* @throws IOException if an error occurs during writing.
*/
public void writeOp(FSEditLogOp op) throws IOException {
public void writeOp(FSEditLogOp op, int logVersion)
throws IOException {
int start = buf.getLength();
// write the op code first to make padding and terminator verification
// work
buf.writeByte(op.opCode.getOpCode());
buf.writeInt(0); // write 0 for the length first
buf.writeLong(op.txid);
op.writeFields(buf);
op.writeFields(buf, logVersion);
int end = buf.getLength();
// write the length back: content of the op + 4 bytes checksum - op_code

View File

@ -2444,6 +2444,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp
.getErasureCodingPolicy(this, ecPolicyName, iip);
if (ecPolicy != null && (!ecPolicy.isReplicationPolicy())) {
checkErasureCodingSupported("createWithEC");
if (blockSize < ecPolicy.getCellSize()) {
throw new IOException("Specified block size (" + blockSize
+ ") is less than the cell size (" + ecPolicy.getCellSize()
@ -7485,6 +7486,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
UnresolvedLinkException, SafeModeException, AccessControlException {
final String operationName = "setErasureCodingPolicy";
checkOperation(OperationCategory.WRITE);
checkErasureCodingSupported(operationName);
FileStatus resultingStat = null;
final FSPermissionChecker pc = getPermissionChecker();
boolean success = false;
@ -7517,6 +7519,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final String operationName = "addErasureCodingPolicies";
List<String> addECPolicyNames = new ArrayList<>(policies.length);
checkOperation(OperationCategory.WRITE);
checkErasureCodingSupported(operationName);
List<AddErasureCodingPolicyResponse> responses =
new ArrayList<>(policies.length);
boolean success = false;
@ -7558,6 +7561,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final boolean logRetryCache) throws IOException {
final String operationName = "removeErasureCodingPolicy";
checkOperation(OperationCategory.WRITE);
checkErasureCodingSupported(operationName);
boolean success = false;
writeLock();
try {
@ -7588,6 +7592,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final boolean logRetryCache) throws IOException {
final String operationName = "enableErasureCodingPolicy";
checkOperation(OperationCategory.WRITE);
checkErasureCodingSupported(operationName);
boolean success = false;
writeLock();
try {
@ -7619,6 +7624,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final boolean logRetryCache) throws IOException {
final String operationName = "disableErasureCodingPolicy";
checkOperation(OperationCategory.WRITE);
checkErasureCodingSupported(operationName);
boolean success = false;
LOG.info("Disable the erasure coding policy " + ecPolicyName);
writeLock();
@ -7652,6 +7658,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
UnresolvedLinkException, SafeModeException, AccessControlException {
final String operationName = "unsetErasureCodingPolicy";
checkOperation(OperationCategory.WRITE);
checkErasureCodingSupported(operationName);
FileStatus resultingStat = null;
final FSPermissionChecker pc = getPermissionChecker();
boolean success = false;
@ -7679,6 +7686,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final String operationName = "getErasureCodingPolicy";
boolean success = false;
checkOperation(OperationCategory.READ);
checkErasureCodingSupported(operationName);
final FSPermissionChecker pc = getPermissionChecker();
readLock();
try {
@ -7700,6 +7708,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final String operationName = "getErasureCodingPolicies";
boolean success = false;
checkOperation(OperationCategory.READ);
checkErasureCodingSupported(operationName);
readLock();
try {
checkOperation(OperationCategory.READ);
@ -7720,6 +7729,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final String operationName = "getErasureCodingCodecs";
boolean success = false;
checkOperation(OperationCategory.READ);
checkErasureCodingSupported(operationName);
readLock();
try {
checkOperation(OperationCategory.READ);
@ -8105,5 +8115,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return "disableRestoreFailedStorage";
}
}
/**
* Check whether operation is supported.
* @param operationName the name of operation.
* @throws UnsupportedActionException throws UAE if not supported.
*/
public void checkErasureCodingSupported(String operationName)
throws UnsupportedActionException {
if (!NameNodeLayoutVersion.supports(
NameNodeLayoutVersion.Feature.ERASURE_CODING,
getEffectiveLayoutVersion())) {
throw new UnsupportedActionException(operationName + " not supported.");
}
}
}

View File

@ -56,7 +56,7 @@ public abstract class QJMTestUtil {
for (long txid = startTxn; txid < startTxn + numTxns; txid++) {
FSEditLogOp op = NameNodeAdapter.createMkdirOp("tx " + txid);
op.setTransactionId(txid);
writer.writeOp(op);
writer.writeOp(op, FAKE_NSINFO.getLayoutVersion());
}
return Arrays.copyOf(buf.getData(), buf.getLength());
@ -73,7 +73,7 @@ public abstract class QJMTestUtil {
for (long txid = startTxId; txid < startTxId + numTxns; txid++) {
FSEditLogOp op = new TestEditLog.GarbageMkdirOp();
op.setTransactionId(txid);
writer.writeOp(op);
writer.writeOp(op, FAKE_NSINFO.getLayoutVersion());
}
return Arrays.copyOf(buf.getData(), buf.getLength());
}

View File

@ -64,13 +64,16 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSInotifyEventInputStream;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@ -1706,4 +1709,77 @@ public class TestEditLog {
LogManager.getRootLogger().removeAppender(appender);
}
}
/**
* Test edits can be writen and read without ErasureCoding supported.
*/
@Test
public void testEditLogWithoutErasureCodingSupported()
throws IOException {
Configuration conf = getConf();
MiniDFSCluster cluster = null;
// ERASURECODING not supported
int logVersion = -61;
assertFalse(NameNodeLayoutVersion.supports(
NameNodeLayoutVersion.Feature.ERASURE_CODING, logVersion));
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster.waitActive();
final FSNamesystem namesystem = cluster.getNamesystem();
FSImage fsimage = namesystem.getFSImage();
FileSystem fileSys = cluster.getFileSystem();
final FSEditLog editLog = fsimage.getEditLog();
editLog.editLogStream.setCurrentLogVersion(logVersion);
// Write new version edit log
long txid = editLog.rollEditLog(logVersion);
String testDir = "/test";
String testFile = "testfile_001";
String testFilePath = testDir + "/" + testFile;
fileSys.mkdirs(new Path(testDir), new FsPermission("755"));
// Create a file
Path p = new Path(testFilePath);
DFSTestUtil.createFile(fileSys, p, 0, (short) 1, 1);
long blkId = 1;
long blkNumBytes = 1024;
long timestamp = 1426222918;
// Add a block to the file
BlockInfoContiguous blockInfo =
new BlockInfoContiguous(
new Block(blkId, blkNumBytes, timestamp),
(short)1);
INodeFile file
= (INodeFile)namesystem.getFSDirectory().getINode(testFilePath);
file.addBlock(blockInfo);
file.toUnderConstruction("testClient", "testMachine");
// Write edit log
editLog.logAddBlock(testFilePath, file);
editLog.rollEditLog(logVersion);
// Read edit log
Collection<EditLogInputStream> editStreams
= editLog.selectInputStreams(txid, txid + 1);
EditLogInputStream inputStream = null;
for (EditLogInputStream s : editStreams) {
if (s.getFirstTxId() == txid) {
inputStream = s;
break;
}
}
assertNotNull(inputStream);
int readLogVersion = inputStream.getVersion(false);
assertEquals(logVersion, readLogVersion);
FSEditLogLoader loader = new FSEditLogLoader(namesystem, 0);
long records = loader.loadFSEdits(inputStream, txid);
assertTrue(records > 0);
editLog.close();
cluster.shutdown();
}
}

View File

@ -87,6 +87,9 @@ public class TestEditsDoubleBuffer {
@Test
public void testDumpEdits() throws IOException {
final int defaultBufferSize = 256;
final int fakeLogVersion =
NameNodeLayoutVersion.Feature.ROLLING_UPGRADE
.getInfo().getLayoutVersion();
EditsDoubleBuffer buffer = new EditsDoubleBuffer(defaultBufferSize);
FSEditLogOp.OpInstanceCache cache = new FSEditLogOp.OpInstanceCache();
@ -98,7 +101,7 @@ public class TestEditsDoubleBuffer {
.setPath(src)
.setReplication(replication);
op.setTransactionId(1);
buffer.writeOp(op);
buffer.writeOp(op, fakeLogVersion);
src = "/testdumpedits2";
@ -107,13 +110,13 @@ public class TestEditsDoubleBuffer {
.setPath(src)
.setTimestamp(0);
op2.setTransactionId(2);
buffer.writeOp(op2);
buffer.writeOp(op2, fakeLogVersion);
FSEditLogOp.AllocateBlockIdOp op3 =
FSEditLogOp.AllocateBlockIdOp.getInstance(cache.get())
.setBlockId(0);
op3.setTransactionId(3);
buffer.writeOp(op3);
buffer.writeOp(op3, fakeLogVersion);
GenericTestUtils.LogCapturer logs =
GenericTestUtils.LogCapturer.captureLogs(EditsDoubleBuffer.LOG);