|
@@ -17,50 +17,66 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
-import java.net.SocketTimeoutException;
|
|
|
-import org.apache.hadoop.io.IOUtils;
|
|
|
-import org.apache.hadoop.io.Writable;
|
|
|
-import org.apache.hadoop.io.LongWritable;
|
|
|
+import static org.mockito.Matchers.anyLong;
|
|
|
+import static org.mockito.Matchers.anyString;
|
|
|
+import static org.mockito.Mockito.doAnswer;
|
|
|
+import static org.mockito.Mockito.spy;
|
|
|
+
|
|
|
+import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
-import java.net.InetSocketAddress;
|
|
|
import java.io.InputStream;
|
|
|
import java.io.OutputStream;
|
|
|
+import java.net.InetSocketAddress;
|
|
|
+import java.net.SocketTimeoutException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
-import java.net.SocketTimeoutException;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
+
|
|
|
+import junit.framework.Assert;
|
|
|
+import junit.framework.TestCase;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.fs.*;
|
|
|
+import org.apache.hadoop.fs.ContentSummary;
|
|
|
+import org.apache.hadoop.fs.FileChecksum;
|
|
|
+import org.apache.hadoop.fs.FileStatus;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.DFSClient.DFSInputStream;
|
|
|
-import org.apache.hadoop.hdfs.protocol.*;
|
|
|
-import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
|
|
-
|
|
|
-import org.apache.hadoop.hdfs.server.common.*;
|
|
|
-import org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
|
|
+import org.apache.hadoop.hdfs.protocol.FSConstants;
|
|
|
+import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
|
|
|
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
|
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
-import org.apache.hadoop.io.*;
|
|
|
-import org.apache.hadoop.ipc.RemoteException;
|
|
|
-import org.apache.hadoop.security.AccessControlException;
|
|
|
-import org.apache.hadoop.security.UserGroupInformation;
|
|
|
-import org.apache.hadoop.security.token.Token;
|
|
|
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
-import org.apache.hadoop.ipc.Client;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
+import org.apache.hadoop.io.LongWritable;
|
|
|
+import org.apache.hadoop.io.Text;
|
|
|
+import org.apache.hadoop.io.Writable;
|
|
|
+import org.apache.hadoop.io.retry.RetryPolicies.MultipleLinearRandomRetry;
|
|
|
import org.apache.hadoop.ipc.RPC;
|
|
|
+import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
-
|
|
|
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
|
-
|
|
|
-import junit.framework.TestCase;
|
|
|
-import static org.mockito.Mockito.*;
|
|
|
-import org.mockito.stubbing.Answer;
|
|
|
+import org.apache.hadoop.security.AccessControlException;
|
|
|
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
+import org.apache.hadoop.security.token.Token;
|
|
|
+import org.apache.log4j.Level;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
+import org.mockito.stubbing.Answer;
|
|
|
|
|
|
/**
|
|
|
* These tests make sure that DFSClient retries fetching data from DFS
|
|
@@ -509,4 +525,148 @@ public class TestDFSClientRetries extends TestCase {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /** Test client retry with namenode restarting. */
|
|
|
+ public void testNamenodeRestart() throws Exception {
|
|
|
+ ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL);
|
|
|
+
|
|
|
+ final List<Exception> exceptions = new ArrayList<Exception>();
|
|
|
+
|
|
|
+ final Path dir = new Path("/testNamenodeRestart");
|
|
|
+
|
|
|
+ final Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_ENABLED_KEY, true);
|
|
|
+
|
|
|
+ final short numDatanodes = 3;
|
|
|
+ final MiniDFSCluster cluster = new MiniDFSCluster(
|
|
|
+ conf, numDatanodes, true, null);
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ //create a file
|
|
|
+ final DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
|
|
|
+ final long length = 1L << 20;
|
|
|
+ final Path file1 = new Path(dir, "foo");
|
|
|
+ DFSTestUtil.createFile(dfs, file1, length, numDatanodes, 20120406L);
|
|
|
+
|
|
|
+ //get file status
|
|
|
+ final FileStatus s1 = dfs.getFileStatus(file1);
|
|
|
+ assertEquals(length, s1.getLen());
|
|
|
+
|
|
|
+ //shutdown namenode
|
|
|
+ cluster.shutdownNameNode();
|
|
|
+
|
|
|
+ //namenode is down, create another file in a thread
|
|
|
+ final Path file3 = new Path(dir, "file");
|
|
|
+ final Thread thread = new Thread(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ //it should retry till namenode is up.
|
|
|
+ final FileSystem fs = AppendTestUtil.createHdfsWithDifferentUsername(conf);
|
|
|
+ DFSTestUtil.createFile(fs, file3, length, numDatanodes, 20120406L);
|
|
|
+ } catch (Exception e) {
|
|
|
+ exceptions.add(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
+ thread.start();
|
|
|
+
|
|
|
+ //restart namenode in a new thread
|
|
|
+ new Thread(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ //sleep, restart, and then wait active
|
|
|
+ TimeUnit.SECONDS.sleep(30);
|
|
|
+ cluster.restartNameNode(false, false);
|
|
|
+ cluster.waitActive();
|
|
|
+ } catch (Exception e) {
|
|
|
+ exceptions.add(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }).start();
|
|
|
+
|
|
|
+ //namenode is down, it should retry until namenode is up again.
|
|
|
+ final FileStatus s2 = dfs.getFileStatus(file1);
|
|
|
+ assertEquals(s1, s2);
|
|
|
+
|
|
|
+ //check file1 and file3
|
|
|
+ thread.join();
|
|
|
+ assertEquals(dfs.getFileChecksum(file1), dfs.getFileChecksum(file3));
|
|
|
+
|
|
|
+ //enter safe mode
|
|
|
+ dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
|
|
+
|
|
|
+ //leave safe mode in a new thread
|
|
|
+ new Thread(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ //sleep and then leave safe mode
|
|
|
+ TimeUnit.SECONDS.sleep(30);
|
|
|
+ dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
|
|
|
+ } catch (Exception e) {
|
|
|
+ exceptions.add(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }).start();
|
|
|
+
|
|
|
+ //namenode is in safe mode, create should retry until it leaves safe mode.
|
|
|
+ final Path file2 = new Path(dir, "bar");
|
|
|
+ DFSTestUtil.createFile(dfs, file2, length, numDatanodes, 20120406L);
|
|
|
+ assertEquals(dfs.getFileChecksum(file1), dfs.getFileChecksum(file2));
|
|
|
+
|
|
|
+ //make sure it won't retry on exceptions like FileNotFoundException
|
|
|
+ final Path nonExisting = new Path(dir, "nonExisting");
|
|
|
+ LOG.info("setPermission: " + nonExisting);
|
|
|
+ try {
|
|
|
+ dfs.setPermission(nonExisting, new FsPermission((short)0));
|
|
|
+ fail();
|
|
|
+ } catch(FileNotFoundException fnfe) {
|
|
|
+ LOG.info("GOOD!", fnfe);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!exceptions.isEmpty()) {
|
|
|
+ LOG.error("There are " + exceptions.size() + " exception(s):");
|
|
|
+ for(int i = 0; i < exceptions.size(); i++) {
|
|
|
+ LOG.error("Exception " + i, exceptions.get(i));
|
|
|
+ }
|
|
|
+ fail();
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public void testMultipleLinearRandomRetry() {
|
|
|
+ parseMultipleLinearRandomRetry(null, "");
|
|
|
+ parseMultipleLinearRandomRetry(null, "11");
|
|
|
+ parseMultipleLinearRandomRetry(null, "11,22,33");
|
|
|
+ parseMultipleLinearRandomRetry(null, "11,22,33,44,55");
|
|
|
+ parseMultipleLinearRandomRetry(null, "AA");
|
|
|
+ parseMultipleLinearRandomRetry(null, "11,AA");
|
|
|
+ parseMultipleLinearRandomRetry(null, "11,22,33,FF");
|
|
|
+ parseMultipleLinearRandomRetry(null, "11,-22");
|
|
|
+ parseMultipleLinearRandomRetry(null, "-11,22");
|
|
|
+
|
|
|
+ parseMultipleLinearRandomRetry("[22x11ms]",
|
|
|
+ "11,22");
|
|
|
+ parseMultipleLinearRandomRetry("[22x11ms, 44x33ms]",
|
|
|
+ "11,22,33,44");
|
|
|
+ parseMultipleLinearRandomRetry("[22x11ms, 44x33ms, 66x55ms]",
|
|
|
+ "11,22,33,44,55,66");
|
|
|
+ parseMultipleLinearRandomRetry("[22x11ms, 44x33ms, 66x55ms]",
|
|
|
+ " 11, 22, 33, 44, 55, 66 ");
|
|
|
+ }
|
|
|
+
|
|
|
+ static void parseMultipleLinearRandomRetry(String expected, String s) {
|
|
|
+ final MultipleLinearRandomRetry r = MultipleLinearRandomRetry.parseCommaSeparatedString(s);
|
|
|
+ LOG.info("input=" + s + ", parsed=" + r + ", expected=" + expected);
|
|
|
+ if (r == null) {
|
|
|
+ Assert.assertEquals(expected, null);
|
|
|
+ } else {
|
|
|
+ Assert.assertEquals("MultipleLinearRandomRetry" + expected, r.toString());
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|