|
@@ -15,7 +15,7 @@
|
|
|
* See the License for the specific language governing permissions and
|
|
|
* limitations under the License.
|
|
|
*/
|
|
|
-package org.apache.hadoop.hdfs.qjournal;
|
|
|
+package org.apache.hadoop.hdfs.qjournal.server;
|
|
|
|
|
|
import com.google.common.base.Supplier;
|
|
|
import com.google.common.collect.Lists;
|
|
@@ -25,17 +25,21 @@ import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
|
|
|
+import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
|
|
|
import static org.apache.hadoop.hdfs.server.namenode.FileJournalManager
|
|
|
.getLogFile;
|
|
|
-
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
+import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
+import org.junit.rules.TestName;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
@@ -46,6 +50,7 @@ import java.util.Random;
|
|
|
* Unit test for Journal Node formatting upon re-installation and syncing.
|
|
|
*/
|
|
|
public class TestJournalNodeSync {
|
|
|
+ private Configuration conf;
|
|
|
private MiniQJMHACluster qjmhaCluster;
|
|
|
private MiniDFSCluster dfsCluster;
|
|
|
private MiniJournalCluster jCluster;
|
|
@@ -54,11 +59,18 @@ public class TestJournalNodeSync {
|
|
|
private int editsPerformed = 0;
|
|
|
private final String jid = "ns1";
|
|
|
|
|
|
+ @Rule
|
|
|
+ public TestName testName = new TestName();
|
|
|
+
|
|
|
@Before
|
|
|
public void setUpMiniCluster() throws IOException {
|
|
|
- final Configuration conf = new HdfsConfiguration();
|
|
|
+ conf = new HdfsConfiguration();
|
|
|
conf.setBoolean(DFSConfigKeys.DFS_JOURNALNODE_ENABLE_SYNC_KEY, true);
|
|
|
conf.setLong(DFSConfigKeys.DFS_JOURNALNODE_SYNC_INTERVAL_KEY, 1000L);
|
|
|
+ if (testName.getMethodName().equals(
|
|
|
+ "testSyncAfterJNdowntimeWithoutQJournalQueue")) {
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_QJOURNAL_QUEUE_SIZE_LIMIT_KEY, 0);
|
|
|
+ }
|
|
|
qjmhaCluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(2)
|
|
|
.build();
|
|
|
dfsCluster = qjmhaCluster.getDfsCluster();
|
|
@@ -214,6 +226,156 @@ public class TestJournalNodeSync {
|
|
|
GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
|
|
|
}
|
|
|
|
|
|
+ // Test JournalNode Sync when a JN id down while NN is actively writing
|
|
|
+ // logs and comes back up after some time.
|
|
|
+ @Test (timeout=300_000)
|
|
|
+ public void testSyncAfterJNdowntime() throws Exception {
|
|
|
+ File firstJournalDir = jCluster.getJournalDir(0, jid);
|
|
|
+ File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
|
|
|
+ .getCurrentDir();
|
|
|
+ File secondJournalDir = jCluster.getJournalDir(1, jid);
|
|
|
+ File secondJournalCurrentDir = new StorageDirectory(secondJournalDir)
|
|
|
+ .getCurrentDir();
|
|
|
+
|
|
|
+ long[] startTxIds = new long[10];
|
|
|
+
|
|
|
+ startTxIds[0] = generateEditLog();
|
|
|
+ startTxIds[1] = generateEditLog();
|
|
|
+
|
|
|
+ // Stop the first JN
|
|
|
+ jCluster.getJournalNode(0).stop(0);
|
|
|
+
|
|
|
+ // Roll some more edits while the first JN is down
|
|
|
+ for (int i = 2; i < 10; i++) {
|
|
|
+ startTxIds[i] = generateEditLog(5);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Re-start the first JN
|
|
|
+ jCluster.restartJournalNode(0);
|
|
|
+
|
|
|
+ // Roll an edit to update the committed tx id of the first JN
|
|
|
+ generateEditLog();
|
|
|
+
|
|
|
+ // List the edit logs rolled during JN down time.
|
|
|
+ List<File> missingLogs = Lists.newArrayList();
|
|
|
+ for (int i = 2; i < 10; i++) {
|
|
|
+ EditLogFile logFile = getLogFile(secondJournalCurrentDir, startTxIds[i],
|
|
|
+ false);
|
|
|
+ missingLogs.add(new File(firstJournalCurrentDir,
|
|
|
+ logFile.getFile().getName()));
|
|
|
+ }
|
|
|
+
|
|
|
+ // Check that JNSync downloaded the edit logs rolled during JN down time.
|
|
|
+ GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test JournalNode Sync when a JN id down while NN is actively writing
|
|
|
+ * logs and comes back up after some time with no edit log queueing.
|
|
|
+ * Queuing disabled during the cluster setup {@link #setUpMiniCluster()}
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test (timeout=300_000)
|
|
|
+ public void testSyncAfterJNdowntimeWithoutQJournalQueue() throws Exception{
|
|
|
+ // Queuing is disabled during the cluster setup {@link #setUpMiniCluster()}
|
|
|
+ File firstJournalDir = jCluster.getJournalDir(0, jid);
|
|
|
+ File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
|
|
|
+ .getCurrentDir();
|
|
|
+ File secondJournalDir = jCluster.getJournalDir(1, jid);
|
|
|
+ File secondJournalCurrentDir = new StorageDirectory(secondJournalDir)
|
|
|
+ .getCurrentDir();
|
|
|
+
|
|
|
+ long[] startTxIds = new long[10];
|
|
|
+
|
|
|
+ startTxIds[0] = generateEditLog();
|
|
|
+ startTxIds[1] = generateEditLog(2);
|
|
|
+
|
|
|
+ // Stop the first JN
|
|
|
+ jCluster.getJournalNode(0).stop(0);
|
|
|
+
|
|
|
+ // Roll some more edits while the first JN is down
|
|
|
+ for (int i = 2; i < 10; i++) {
|
|
|
+ startTxIds[i] = generateEditLog(5);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Re-start the first JN
|
|
|
+ jCluster.restartJournalNode(0);
|
|
|
+
|
|
|
+ // After JN restart and before rolling another edit, the missing edit
|
|
|
+ // logs will not by synced as the committed tx id of the JN will be
|
|
|
+ // less than the start tx id's of the missing edit logs and edit log queuing
|
|
|
+ // has been disabled.
|
|
|
+ // Roll an edit to update the committed tx id of the first JN
|
|
|
+ generateEditLog(2);
|
|
|
+
|
|
|
+ // List the edit logs rolled during JN down time.
|
|
|
+ List<File> missingLogs = Lists.newArrayList();
|
|
|
+ for (int i = 2; i < 10; i++) {
|
|
|
+ EditLogFile logFile = getLogFile(secondJournalCurrentDir, startTxIds[i],
|
|
|
+ false);
|
|
|
+ missingLogs.add(new File(firstJournalCurrentDir,
|
|
|
+ logFile.getFile().getName()));
|
|
|
+ }
|
|
|
+
|
|
|
+ // Check that JNSync downloaded the edit logs rolled during JN down time.
|
|
|
+ GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
|
|
|
+
|
|
|
+ // Check that all the missing edit logs have been downloaded via
|
|
|
+ // JournalNodeSyncer alone (as the edit log queueing has been disabled)
|
|
|
+ long numEditLogsSynced = jCluster.getJournalNode(0).getOrCreateJournal(jid)
|
|
|
+ .getMetrics().getNumEditLogsSynced().value();
|
|
|
+ Assert.assertTrue("Edit logs downloaded outside syncer. Expected 8 or " +
|
|
|
+ "more downloads, got " + numEditLogsSynced + " downloads instead",
|
|
|
+ numEditLogsSynced >= 8);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Test JournalNode Sync when a JN is formatted while NN is actively writing
|
|
|
+ // logs.
|
|
|
+ @Test (timeout=300_000)
|
|
|
+ public void testSyncAfterJNformat() throws Exception{
|
|
|
+ File firstJournalDir = jCluster.getJournalDir(0, jid);
|
|
|
+ File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
|
|
|
+ .getCurrentDir();
|
|
|
+ File secondJournalDir = jCluster.getJournalDir(1, jid);
|
|
|
+ File secondJournalCurrentDir = new StorageDirectory(secondJournalDir)
|
|
|
+ .getCurrentDir();
|
|
|
+
|
|
|
+ long[] startTxIds = new long[10];
|
|
|
+
|
|
|
+ startTxIds[0] = generateEditLog(1);
|
|
|
+ startTxIds[1] = generateEditLog(2);
|
|
|
+ startTxIds[2] = generateEditLog(4);
|
|
|
+ startTxIds[3] = generateEditLog(6);
|
|
|
+
|
|
|
+ Journal journal1 = jCluster.getJournalNode(0).getOrCreateJournal(jid);
|
|
|
+ NamespaceInfo nsInfo = journal1.getStorage().getNamespaceInfo();
|
|
|
+
|
|
|
+ // Delete contents of current directory of one JN
|
|
|
+ for (File file : firstJournalCurrentDir.listFiles()) {
|
|
|
+ file.delete();
|
|
|
+ }
|
|
|
+
|
|
|
+ // Format the JN
|
|
|
+ journal1.format(nsInfo);
|
|
|
+
|
|
|
+ // Roll some more edits
|
|
|
+ for (int i = 4; i < 10; i++) {
|
|
|
+ startTxIds[i] = generateEditLog(5);
|
|
|
+ }
|
|
|
+
|
|
|
+ // List the edit logs rolled during JN down time.
|
|
|
+ List<File> missingLogs = Lists.newArrayList();
|
|
|
+ for (int i = 0; i < 10; i++) {
|
|
|
+ EditLogFile logFile = getLogFile(secondJournalCurrentDir, startTxIds[i],
|
|
|
+ false);
|
|
|
+ missingLogs.add(new File(firstJournalCurrentDir,
|
|
|
+ logFile.getFile().getName()));
|
|
|
+ }
|
|
|
+
|
|
|
+ // Check that the formatted JN has all the edit logs.
|
|
|
+ GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
|
|
|
+ }
|
|
|
+
|
|
|
private File deleteEditLog(File currentDir, long startTxId)
|
|
|
throws IOException {
|
|
|
EditLogFile logFile = getLogFile(currentDir, startTxId);
|
|
@@ -242,8 +404,20 @@ public class TestJournalNodeSync {
|
|
|
* @return the startTxId of next segment after rolling edits.
|
|
|
*/
|
|
|
private long generateEditLog() throws IOException {
|
|
|
+ return generateEditLog(1);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Does specified number of edits and rolls the Edit Log.
|
|
|
+ *
|
|
|
+ * @param numEdits number of Edits to perform
|
|
|
+ * @return the startTxId of next segment after rolling edits.
|
|
|
+ */
|
|
|
+ private long generateEditLog(int numEdits) throws IOException {
|
|
|
long startTxId = namesystem.getFSImage().getEditLog().getLastWrittenTxId();
|
|
|
- Assert.assertTrue("Failed to do an edit", doAnEdit());
|
|
|
+ for (int i = 1; i <= numEdits; i++) {
|
|
|
+ Assert.assertTrue("Failed to do an edit", doAnEdit());
|
|
|
+ }
|
|
|
dfsCluster.getNameNode(0).getRpcServer().rollEditLog();
|
|
|
return startTxId;
|
|
|
}
|