|
@@ -18,6 +18,14 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs.web;
|
|
|
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertNotEquals;
|
|
@@ -53,6 +61,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
+import org.apache.hadoop.fs.FsServerDefaults;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
@@ -74,7 +83,9 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|
|
import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
@@ -92,11 +103,13 @@ import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
+import org.apache.hadoop.util.DataChecksum;
|
|
|
import org.apache.log4j.Level;
|
|
|
import org.codehaus.jettison.json.JSONException;
|
|
|
import org.codehaus.jettison.json.JSONObject;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
import org.mockito.internal.util.reflection.Whitebox;
|
|
|
|
|
|
import static org.mockito.Matchers.any;
|
|
@@ -1318,4 +1331,125 @@ public class TestWebHDFS {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test fsserver defaults response from {@link DistributedFileSystem} and
|
|
|
+ * {@link WebHdfsFileSystem} are the same.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testFsserverDefaults() throws Exception {
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ final Configuration conf = WebHdfsTestUtil.createConf();
|
|
|
+ // Here we override all the default values so that we can verify that it
|
|
|
+ // doesn't pick up the default value.
|
|
|
+ long blockSize = 256*1024*1024;
|
|
|
+ int bytesPerChecksum = 256;
|
|
|
+ int writePacketSize = 128*1024;
|
|
|
+ int replicationFactor = 0;
|
|
|
+ int bufferSize = 1024;
|
|
|
+ boolean encryptDataTransfer = true;
|
|
|
+ long trashInterval = 1;
|
|
|
+ String checksumType = "CRC32";
|
|
|
+ // Setting policy to a special value 7 because BlockManager will
|
|
|
+ // create defaultSuite with policy id 7.
|
|
|
+ byte policyId = (byte) 7;
|
|
|
+
|
|
|
+ conf.setLong(DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
+ conf.setInt(DFS_BYTES_PER_CHECKSUM_KEY, bytesPerChecksum);
|
|
|
+ conf.setInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY, writePacketSize);
|
|
|
+ conf.setInt(DFS_REPLICATION_KEY, replicationFactor);
|
|
|
+ conf.setInt(IO_FILE_BUFFER_SIZE_KEY, bufferSize);
|
|
|
+ conf.setBoolean(DFS_ENCRYPT_DATA_TRANSFER_KEY, encryptDataTransfer);
|
|
|
+ conf.setLong(FS_TRASH_INTERVAL_KEY, trashInterval);
|
|
|
+ conf.set(DFS_CHECKSUM_TYPE_KEY, checksumType);
|
|
|
+ FsServerDefaults originalServerDefaults = new FsServerDefaults(blockSize,
|
|
|
+ bytesPerChecksum, writePacketSize, (short)replicationFactor,
|
|
|
+ bufferSize, encryptDataTransfer, trashInterval,
|
|
|
+ DataChecksum.Type.valueOf(checksumType), "", policyId);
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
|
|
|
+ final DistributedFileSystem dfs = cluster.getFileSystem();
|
|
|
+ final WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(
|
|
|
+ conf, WebHdfsConstants.WEBHDFS_SCHEME);
|
|
|
+ FsServerDefaults dfsServerDefaults = dfs.getServerDefaults();
|
|
|
+ FsServerDefaults webfsServerDefaults = webfs.getServerDefaults();
|
|
|
+ // Verify whether server defaults value that we override is equal to
|
|
|
+ // dfsServerDefaults.
|
|
|
+ compareFsServerDefaults(originalServerDefaults, dfsServerDefaults);
|
|
|
+ // Verify whether dfs serverdefaults is equal to
|
|
|
+ // webhdfsServerDefaults.
|
|
|
+ compareFsServerDefaults(dfsServerDefaults, webfsServerDefaults);
|
|
|
+ webfs.getServerDefaults();
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void compareFsServerDefaults(FsServerDefaults serverDefaults1,
|
|
|
+ FsServerDefaults serverDefaults2) throws Exception {
|
|
|
+ Assert.assertEquals("Block size is different",
|
|
|
+ serverDefaults1.getBlockSize(),
|
|
|
+ serverDefaults2.getBlockSize());
|
|
|
+ Assert.assertEquals("Bytes per checksum are different",
|
|
|
+ serverDefaults1.getBytesPerChecksum(),
|
|
|
+ serverDefaults2.getBytesPerChecksum());
|
|
|
+ Assert.assertEquals("Write packet size is different",
|
|
|
+ serverDefaults1.getWritePacketSize(),
|
|
|
+ serverDefaults2.getWritePacketSize());
|
|
|
+ Assert.assertEquals("Default replication is different",
|
|
|
+ serverDefaults1.getReplication(),
|
|
|
+ serverDefaults2.getReplication());
|
|
|
+ Assert.assertEquals("File buffer size are different",
|
|
|
+ serverDefaults1.getFileBufferSize(),
|
|
|
+ serverDefaults2.getFileBufferSize());
|
|
|
+ Assert.assertEquals("Encrypt data transfer key is different",
|
|
|
+ serverDefaults1.getEncryptDataTransfer(),
|
|
|
+ serverDefaults2.getEncryptDataTransfer());
|
|
|
+ Assert.assertEquals("Trash interval is different",
|
|
|
+ serverDefaults1.getTrashInterval(),
|
|
|
+ serverDefaults2.getTrashInterval());
|
|
|
+ Assert.assertEquals("Checksum type is different",
|
|
|
+ serverDefaults1.getChecksumType(),
|
|
|
+ serverDefaults2.getChecksumType());
|
|
|
+ Assert.assertEquals("Key provider uri is different",
|
|
|
+ serverDefaults1.getKeyProviderUri(),
|
|
|
+ serverDefaults2.getKeyProviderUri());
|
|
|
+ Assert.assertEquals("Default storage policy is different",
|
|
|
+ serverDefaults1.getDefaultStoragePolicyId(),
|
|
|
+ serverDefaults2.getDefaultStoragePolicyId());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests the case when client is upgraded to return {@link FsServerDefaults}
|
|
|
+ * but then namenode is not upgraded.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testFsserverDefaultsBackwardsCompatible() throws Exception {
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ final Configuration conf = WebHdfsTestUtil.createConf();
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
|
|
|
+ final WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(
|
|
|
+ conf, WebHdfsConstants.WEBHDFS_SCHEME);
|
|
|
+ NamenodeWebHdfsMethods.resetServerDefaultsResponse();
|
|
|
+ FSNamesystem fsnSpy =
|
|
|
+ NameNodeAdapter.spyOnNamesystem(cluster.getNameNode());
|
|
|
+ Mockito.when(fsnSpy.getServerDefaults()).
|
|
|
+ thenThrow(new UnsupportedOperationException());
|
|
|
+ try {
|
|
|
+ webfs.getServerDefaults();
|
|
|
+ Assert.fail("should have thrown UnSupportedOperationException.");
|
|
|
+ } catch (UnsupportedOperationException uoe) {
|
|
|
+ //Expected exception.
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|