|
@@ -81,13 +81,10 @@ import org.apache.hadoop.util.Lists;
|
|
|
import org.json.simple.JSONObject;
|
|
|
import org.json.simple.parser.ContainerFactory;
|
|
|
import org.json.simple.parser.JSONParser;
|
|
|
-import org.junit.Assert;
|
|
|
-import org.junit.Assume;
|
|
|
-import org.junit.Test;
|
|
|
-import org.junit.runner.RunWith;
|
|
|
-import org.junit.runners.Parameterized;
|
|
|
import org.eclipse.jetty.server.Server;
|
|
|
import org.eclipse.jetty.webapp.WebAppContext;
|
|
|
+import org.junit.jupiter.params.ParameterizedTest;
|
|
|
+import org.junit.jupiter.params.provider.MethodSource;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.FileOutputStream;
|
|
@@ -107,13 +104,15 @@ import java.util.Map;
|
|
|
import java.util.regex.Pattern;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
|
|
|
-import static org.junit.Assert.assertArrayEquals;
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertFalse;
|
|
|
-import static org.junit.Assert.assertNotNull;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertFalse;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertNotNull;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertNull;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertTrue;
|
|
|
+import static org.junit.jupiter.api.Assertions.fail;
|
|
|
+import static org.junit.jupiter.api.Assumptions.assumeFalse;
|
|
|
|
|
|
-@RunWith(value = Parameterized.class)
|
|
|
public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
protected abstract Path getProxiedFSTestDir();
|
|
|
|
|
@@ -251,12 +250,12 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
testCreate(path, true);
|
|
|
try {
|
|
|
testCreate(path, false);
|
|
|
- Assert.fail("the create should have failed because the file exists " +
|
|
|
- "and override is FALSE");
|
|
|
+ fail("the create should have failed because the file exists " +
|
|
|
+ "and override is FALSE");
|
|
|
} catch (IOException ex) {
|
|
|
System.out.println("#");
|
|
|
} catch (Exception ex) {
|
|
|
- Assert.fail(ex.toString());
|
|
|
+ fail(ex.toString());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -299,7 +298,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
final int newLength = blockSize;
|
|
|
|
|
|
boolean isReady = fs.truncate(file, newLength);
|
|
|
- assertTrue("Recovery is not expected.", isReady);
|
|
|
+ assertTrue(isReady, "Recovery is not expected.");
|
|
|
|
|
|
FileStatus fileStatus = fs.getFileStatus(file);
|
|
|
assertEquals(fileStatus.getLen(), newLength);
|
|
@@ -312,8 +311,8 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
|
|
|
private void assertPathCapabilityForTruncate(Path file) throws Exception {
|
|
|
FileSystem fs = this.getHttpFSFileSystem();
|
|
|
- assertTrue("HttpFS/WebHdfs/SWebHdfs support truncate",
|
|
|
- fs.hasPathCapability(file, CommonPathCapabilities.FS_TRUNCATE));
|
|
|
+ assertTrue(fs.hasPathCapability(file, CommonPathCapabilities.FS_TRUNCATE),
|
|
|
+ "HttpFS/WebHdfs/SWebHdfs support truncate");
|
|
|
fs.close();
|
|
|
}
|
|
|
|
|
@@ -371,10 +370,10 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
assertFalse(fs.exists(foo));
|
|
|
try {
|
|
|
hoopFs.delete(new Path(bar.toUri().getPath()), false);
|
|
|
- Assert.fail();
|
|
|
+ fail();
|
|
|
} catch (IOException ex) {
|
|
|
} catch (Exception ex) {
|
|
|
- Assert.fail();
|
|
|
+ fail();
|
|
|
}
|
|
|
assertTrue(fs.exists(bar));
|
|
|
assertTrue(hoopFs.delete(new Path(bar.toUri().getPath()), true));
|
|
@@ -467,10 +466,10 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
|
|
|
// The full path should be the path to the file. See HDFS-12139
|
|
|
FileStatus[] statl = fs.listStatus(path);
|
|
|
- Assert.assertEquals(1, statl.length);
|
|
|
- Assert.assertEquals(status2.getPath(), statl[0].getPath());
|
|
|
- Assert.assertEquals(statl[0].getPath().getName(), path.getName());
|
|
|
- Assert.assertEquals(stati[0].getPath(), statl[0].getPath());
|
|
|
+ assertEquals(1, statl.length);
|
|
|
+ assertEquals(status2.getPath(), statl[0].getPath());
|
|
|
+ assertEquals(statl[0].getPath().getName(), path.getName());
|
|
|
+ assertEquals(stati[0].getPath(), statl[0].getPath());
|
|
|
}
|
|
|
|
|
|
private void testFileStatusAttr() throws Exception {
|
|
@@ -483,18 +482,18 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Get the FileSystem instance that's being tested
|
|
|
FileSystem fs = this.getHttpFSFileSystem();
|
|
|
// Check FileStatus
|
|
|
- assertFalse("Snapshot should be disallowed by default",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertFalse(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "Snapshot should be disallowed by default");
|
|
|
// Allow snapshot
|
|
|
distributedFs.allowSnapshot(path);
|
|
|
// Check FileStatus
|
|
|
- assertTrue("Snapshot enabled bit is not set in FileStatus",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "Snapshot enabled bit is not set in FileStatus");
|
|
|
// Disallow snapshot
|
|
|
distributedFs.disallowSnapshot(path);
|
|
|
// Check FileStatus
|
|
|
- assertFalse("Snapshot enabled bit is not cleared in FileStatus",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertFalse(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "Snapshot enabled bit is not cleared in FileStatus");
|
|
|
// Cleanup
|
|
|
fs.delete(path, true);
|
|
|
fs.close();
|
|
@@ -529,7 +528,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// LocalFileSystem writes checksum files next to the data files, which
|
|
|
// show up when listing via LFS. This makes the listings not compare
|
|
|
// properly.
|
|
|
- Assume.assumeFalse(isLocalFS());
|
|
|
+ assumeFalse(isLocalFS());
|
|
|
|
|
|
FileSystem proxyFs = FileSystem.get(getProxiedFSConf());
|
|
|
Configuration conf = new Configuration();
|
|
@@ -555,13 +554,13 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
RemoteIterator<FileStatus> si = proxyFs.listStatusIterator(dir1);
|
|
|
FileStatus statusl = si.next();
|
|
|
FileStatus status = proxyFs.getFileStatus(file1);
|
|
|
- Assert.assertEquals(file1.getName(), statusl.getPath().getName());
|
|
|
- Assert.assertEquals(status.getPath(), statusl.getPath());
|
|
|
+ assertEquals(file1.getName(), statusl.getPath().getName());
|
|
|
+ assertEquals(status.getPath(), statusl.getPath());
|
|
|
|
|
|
si = proxyFs.listStatusIterator(file1);
|
|
|
statusl = si.next();
|
|
|
- Assert.assertEquals(file1.getName(), statusl.getPath().getName());
|
|
|
- Assert.assertEquals(status.getPath(), statusl.getPath());
|
|
|
+ assertEquals(file1.getName(), statusl.getPath().getName());
|
|
|
+ assertEquals(status.getPath(), statusl.getPath());
|
|
|
}
|
|
|
|
|
|
private void testWorkingdirectory() throws Exception {
|
|
@@ -845,7 +844,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
fs.setXAttr(path, name4, value4);
|
|
|
try {
|
|
|
fs.setXAttr(path, name5, value1);
|
|
|
- Assert.fail("Set xAttr with incorrect name format should fail.");
|
|
|
+ fail("Set xAttr with incorrect name format should fail.");
|
|
|
} catch (IOException e) {
|
|
|
} catch (IllegalArgumentException e) {
|
|
|
}
|
|
@@ -910,7 +909,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
final String name5 = "a1";
|
|
|
try {
|
|
|
value = fs.getXAttr(path, name5);
|
|
|
- Assert.fail("Get xAttr with incorrect name format should fail.");
|
|
|
+ fail("Get xAttr with incorrect name format should fail.");
|
|
|
} catch (IOException e) {
|
|
|
} catch (IllegalArgumentException e) {
|
|
|
}
|
|
@@ -961,7 +960,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
fs.removeXAttr(path, name4);
|
|
|
try {
|
|
|
fs.removeXAttr(path, name5);
|
|
|
- Assert.fail("Remove xAttr with incorrect name format should fail.");
|
|
|
+ fail("Remove xAttr with incorrect name format should fail.");
|
|
|
} catch (IOException e) {
|
|
|
} catch (IllegalArgumentException e) {
|
|
|
}
|
|
@@ -1151,7 +1150,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
}
|
|
|
|
|
|
private void testErasureCoding() throws Exception {
|
|
|
- Assume.assumeFalse("Assume its not a local FS!", isLocalFS());
|
|
|
+ assumeFalse(isLocalFS(), "Assume its not a local FS!");
|
|
|
FileSystem proxyFs = FileSystem.get(getProxiedFSConf());
|
|
|
FileSystem httpFS = getHttpFSFileSystem();
|
|
|
Path filePath = new Path(getProxiedFSTestDir(), "foo.txt");
|
|
@@ -1169,15 +1168,15 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
}
|
|
|
|
|
|
private void testStoragePolicy() throws Exception {
|
|
|
- Assume.assumeFalse("Assume its not a local FS", isLocalFS());
|
|
|
+ assumeFalse(isLocalFS(), "Assume its not a local FS");
|
|
|
FileSystem fs = FileSystem.get(getProxiedFSConf());
|
|
|
fs.mkdirs(getProxiedFSTestDir());
|
|
|
Path path = new Path(getProxiedFSTestDir(), "policy.txt");
|
|
|
FileSystem httpfs = getHttpFSFileSystem();
|
|
|
// test getAllStoragePolicies
|
|
|
- Assert.assertArrayEquals(
|
|
|
- "Policy array returned from the DFS and HttpFS should be equals",
|
|
|
- fs.getAllStoragePolicies().toArray(), httpfs.getAllStoragePolicies().toArray());
|
|
|
+ assertArrayEquals(fs.getAllStoragePolicies().toArray(),
|
|
|
+ httpfs.getAllStoragePolicies().toArray(),
|
|
|
+ "Policy array returned from the DFS and HttpFS should be equals");
|
|
|
|
|
|
// test get/set/unset policies
|
|
|
DFSTestUtil.createFile(fs, path, 0, (short) 1, 0L);
|
|
@@ -1189,22 +1188,15 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
BlockStoragePolicySpi dfsPolicy = fs.getStoragePolicy(path);
|
|
|
// get policy from webhdfs
|
|
|
BlockStoragePolicySpi httpFsPolicy = httpfs.getStoragePolicy(path);
|
|
|
- Assert
|
|
|
- .assertEquals(
|
|
|
- "Storage policy returned from the get API should"
|
|
|
- + " be same as set policy",
|
|
|
- HdfsConstants.COLD_STORAGE_POLICY_NAME.toString(),
|
|
|
- httpFsPolicy.getName());
|
|
|
- Assert.assertEquals(
|
|
|
- "Storage policy returned from the DFS and HttpFS should be equals",
|
|
|
- httpFsPolicy, dfsPolicy);
|
|
|
+ assertEquals(HdfsConstants.COLD_STORAGE_POLICY_NAME.toString(),
|
|
|
+ httpFsPolicy.getName(), "Storage policy returned from the get API should" +
|
|
|
+ " be same as set policy");
|
|
|
+ assertEquals(httpFsPolicy, dfsPolicy,
|
|
|
+ "Storage policy returned from the DFS and HttpFS should be equals");
|
|
|
// unset policy
|
|
|
httpfs.unsetStoragePolicy(path);
|
|
|
- Assert
|
|
|
- .assertEquals(
|
|
|
- "After unset storage policy, the get API shoudld"
|
|
|
- + " return the default policy",
|
|
|
- defaultdfsPolicy, httpfs.getStoragePolicy(path));
|
|
|
+ assertEquals(defaultdfsPolicy, httpfs.getStoragePolicy(path),
|
|
|
+ "After unset storage policy, the get API shoudld return the default policy");
|
|
|
fs.close();
|
|
|
}
|
|
|
|
|
@@ -1380,7 +1372,6 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Parameterized.Parameters
|
|
|
public static Collection operations() {
|
|
|
Object[][] ops = new Object[Operation.values().length][];
|
|
|
for (int i = 0; i < Operation.values().length; i++) {
|
|
@@ -1393,24 +1384,28 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
|
|
|
private Operation operation;
|
|
|
|
|
|
- public BaseTestHttpFSWith(Operation operation) {
|
|
|
- this.operation = operation;
|
|
|
+ public void initBaseTestHttpFSWith(Operation pOperation) {
|
|
|
+ this.operation = pOperation;
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @MethodSource("operations")
|
|
|
+ @ParameterizedTest
|
|
|
@TestDir
|
|
|
@TestJetty
|
|
|
@TestHdfs
|
|
|
- public void testOperation() throws Exception {
|
|
|
+ public void testOperation(Operation pOperation) throws Exception {
|
|
|
+ initBaseTestHttpFSWith(pOperation);
|
|
|
createHttpFSServer();
|
|
|
operation(operation);
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @MethodSource("operations")
|
|
|
+ @ParameterizedTest
|
|
|
@TestDir
|
|
|
@TestJetty
|
|
|
@TestHdfs
|
|
|
- public void testOperationDoAs() throws Exception {
|
|
|
+ public void testOperationDoAs(Operation pOperation) throws Exception {
|
|
|
+ initBaseTestHttpFSWith(pOperation);
|
|
|
createHttpFSServer();
|
|
|
UserGroupInformation ugi = UserGroupInformation.createProxyUser(HadoopUsersConfTestHelper.getHadoopUsers()[0],
|
|
|
UserGroupInformation.getCurrentUser());
|
|
@@ -1436,16 +1431,15 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
}
|
|
|
Path snapshotsDir = new Path("/tmp/tmp-snap-test/.snapshot");
|
|
|
FileStatus[] snapshotItems = fs.listStatus(snapshotsDir);
|
|
|
- assertTrue("Should have exactly one snapshot.",
|
|
|
- snapshotItems.length == 1);
|
|
|
+ assertTrue(snapshotItems.length == 1,
|
|
|
+ "Should have exactly one snapshot.");
|
|
|
String resultingSnapName = snapshotItems[0].getPath().getName();
|
|
|
if (snapshotName == null) {
|
|
|
- assertTrue("Snapshot auto generated name not matching pattern",
|
|
|
- Pattern.matches("(s)(\\d{8})(-)(\\d{6})(\\.)(\\d{3})",
|
|
|
- resultingSnapName));
|
|
|
+ assertTrue(Pattern.matches("(s)(\\d{8})(-)(\\d{6})(\\.)(\\d{3})",
|
|
|
+ resultingSnapName), "Snapshot auto generated name not matching pattern");
|
|
|
} else {
|
|
|
- assertTrue("Snapshot name is not same as passed name.",
|
|
|
- snapshotName.equals(resultingSnapName));
|
|
|
+ assertTrue(snapshotName.equals(resultingSnapName),
|
|
|
+ "Snapshot name is not same as passed name.");
|
|
|
}
|
|
|
cleanSnapshotTests(snapshottablePath, resultingSnapName);
|
|
|
}
|
|
@@ -1495,11 +1489,11 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
"snap-new-name");
|
|
|
Path snapshotsDir = new Path("/tmp/tmp-snap-test/.snapshot");
|
|
|
FileStatus[] snapshotItems = fs.listStatus(snapshotsDir);
|
|
|
- assertTrue("Should have exactly one snapshot.",
|
|
|
- snapshotItems.length == 1);
|
|
|
+ assertTrue(snapshotItems.length == 1,
|
|
|
+ "Should have exactly one snapshot.");
|
|
|
String resultingSnapName = snapshotItems[0].getPath().getName();
|
|
|
- assertTrue("Snapshot name is not same as passed name.",
|
|
|
- "snap-new-name".equals(resultingSnapName));
|
|
|
+ assertTrue("snap-new-name".equals(resultingSnapName),
|
|
|
+ "Snapshot name is not same as passed name.");
|
|
|
cleanSnapshotTests(snapshottablePath, resultingSnapName);
|
|
|
}
|
|
|
}
|
|
@@ -1513,12 +1507,12 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
fs.createSnapshot(snapshottablePath, "snap-to-delete");
|
|
|
Path snapshotsDir = new Path("/tmp/tmp-snap-test/.snapshot");
|
|
|
FileStatus[] snapshotItems = fs.listStatus(snapshotsDir);
|
|
|
- assertTrue("Should have exactly one snapshot.",
|
|
|
- snapshotItems.length == 1);
|
|
|
+ assertTrue(snapshotItems.length == 1,
|
|
|
+ "Should have exactly one snapshot.");
|
|
|
fs.deleteSnapshot(snapshottablePath, "snap-to-delete");
|
|
|
snapshotItems = fs.listStatus(snapshotsDir);
|
|
|
- assertTrue("There should be no snapshot anymore.",
|
|
|
- snapshotItems.length == 0);
|
|
|
+ assertTrue(snapshotItems.length == 0,
|
|
|
+ "There should be no snapshot anymore.");
|
|
|
fs.delete(snapshottablePath, true);
|
|
|
}
|
|
|
}
|
|
@@ -1531,8 +1525,8 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Get the FileSystem instance that's being tested
|
|
|
FileSystem fs = this.getHttpFSFileSystem();
|
|
|
// Check FileStatus
|
|
|
- assertFalse("Snapshot should be disallowed by default",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertFalse(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "Snapshot should be disallowed by default");
|
|
|
// Allow snapshot
|
|
|
if (fs instanceof HttpFSFileSystem) {
|
|
|
HttpFSFileSystem httpFS = (HttpFSFileSystem) fs;
|
|
@@ -1541,12 +1535,12 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
webHdfsFileSystem.allowSnapshot(path);
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() +
|
|
|
+ fail(fs.getClass().getSimpleName() +
|
|
|
" doesn't support allowSnapshot");
|
|
|
}
|
|
|
// Check FileStatus
|
|
|
- assertTrue("allowSnapshot failed",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "allowSnapshot failed");
|
|
|
// Cleanup
|
|
|
fs.delete(path, true);
|
|
|
}
|
|
@@ -1560,8 +1554,8 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Get the FileSystem instance that's being tested
|
|
|
FileSystem fs = this.getHttpFSFileSystem();
|
|
|
// Check FileStatus
|
|
|
- assertTrue("Snapshot should be allowed by DFS",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "Snapshot should be allowed by DFS");
|
|
|
// Disallow snapshot
|
|
|
if (fs instanceof HttpFSFileSystem) {
|
|
|
HttpFSFileSystem httpFS = (HttpFSFileSystem) fs;
|
|
@@ -1570,12 +1564,12 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
webHdfsFileSystem.disallowSnapshot(path);
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() +
|
|
|
+ fail(fs.getClass().getSimpleName() +
|
|
|
" doesn't support disallowSnapshot");
|
|
|
}
|
|
|
// Check FileStatus
|
|
|
- assertFalse("disallowSnapshot failed",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertFalse(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "disallowSnapshot failed");
|
|
|
// Cleanup
|
|
|
fs.delete(path, true);
|
|
|
}
|
|
@@ -1589,8 +1583,8 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Get the FileSystem instance that's being tested
|
|
|
FileSystem fs = this.getHttpFSFileSystem();
|
|
|
// Check FileStatus
|
|
|
- assertTrue("Snapshot should be allowed by DFS",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "Snapshot should be allowed by DFS");
|
|
|
// Create some snapshots
|
|
|
fs.createSnapshot(path, "snap-01");
|
|
|
fs.createSnapshot(path, "snap-02");
|
|
@@ -1613,17 +1607,17 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Expect SnapshotException
|
|
|
}
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() +
|
|
|
+ fail(fs.getClass().getSimpleName() +
|
|
|
" doesn't support disallowSnapshot");
|
|
|
}
|
|
|
if (disallowSuccess) {
|
|
|
- Assert.fail("disallowSnapshot doesn't throw SnapshotException when "
|
|
|
+ fail("disallowSnapshot doesn't throw SnapshotException when "
|
|
|
+ "disallowing snapshot on a directory with at least one snapshot");
|
|
|
}
|
|
|
// Check FileStatus, should still be enabled since
|
|
|
// disallow snapshot should fail
|
|
|
- assertTrue("disallowSnapshot should not have succeeded",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "disallowSnapshot should not have succeeded");
|
|
|
// Cleanup
|
|
|
fs.deleteSnapshot(path, "snap-02");
|
|
|
fs.deleteSnapshot(path, "snap-01");
|
|
@@ -1639,7 +1633,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Get the FileSystem instance that's being tested
|
|
|
FileSystem fs = this.getHttpFSFileSystem();
|
|
|
// Check FileStatus
|
|
|
- Assert.assertTrue(fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
// Create a file and take a snapshot
|
|
|
Path file1 = new Path(path, "file1");
|
|
|
testCreate(file1, false);
|
|
@@ -1659,13 +1653,13 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
diffReport = webHdfsFileSystem.getSnapshotDiffReport(path, "snap1", "snap2");
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff");
|
|
|
+ fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff");
|
|
|
}
|
|
|
// Verify result with DFS
|
|
|
DistributedFileSystem dfs =
|
|
|
(DistributedFileSystem) FileSystem.get(path.toUri(), this.getProxiedFSConf());
|
|
|
SnapshotDiffReport dfsDiffReport = dfs.getSnapshotDiffReport(path, "snap1", "snap2");
|
|
|
- Assert.assertEquals(diffReport.toString(), dfsDiffReport.toString());
|
|
|
+ assertEquals(diffReport.toString(), dfsDiffReport.toString());
|
|
|
} finally {
|
|
|
// Cleanup
|
|
|
fs.deleteSnapshot(path, "snap2");
|
|
@@ -1686,7 +1680,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
webHdfsFileSystem.getSnapshotDiffReport(path, oldsnapshotname,
|
|
|
snapshotname);
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() +
|
|
|
+ fail(fs.getClass().getSimpleName() +
|
|
|
" doesn't support getSnapshotDiff");
|
|
|
}
|
|
|
} catch (SnapshotException|IllegalArgumentException|RemoteException e) {
|
|
@@ -1694,12 +1688,12 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// or RemoteException(IllegalArgumentException)
|
|
|
if (e instanceof RemoteException) {
|
|
|
// Check RemoteException class name, should be IllegalArgumentException
|
|
|
- Assert.assertEquals(((RemoteException) e).getClassName()
|
|
|
+ assertEquals(((RemoteException) e).getClassName()
|
|
|
.compareTo(java.lang.IllegalArgumentException.class.getName()), 0);
|
|
|
}
|
|
|
return;
|
|
|
}
|
|
|
- Assert.fail("getSnapshotDiff illegal param didn't throw Exception");
|
|
|
+ fail("getSnapshotDiff illegal param didn't throw Exception");
|
|
|
}
|
|
|
|
|
|
private void testGetSnapshotDiffIllegalParam() throws Exception {
|
|
@@ -1710,9 +1704,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Get the FileSystem instance that's being tested
|
|
|
FileSystem fs = this.getHttpFSFileSystem();
|
|
|
// Check FileStatus
|
|
|
- assertTrue("Snapshot should be allowed by DFS",
|
|
|
- fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
- Assert.assertTrue(fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled(),
|
|
|
+ "Snapshot should be allowed by DFS");
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
// Get snapshot diff
|
|
|
testGetSnapshotDiffIllegalParamCase(fs, path, "", "");
|
|
|
testGetSnapshotDiffIllegalParamCase(fs, path, "snap1", "");
|
|
@@ -1734,12 +1728,12 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
sds = webHdfsFileSystem.getSnapshottableDirectoryList();
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() +
|
|
|
+ fail(fs.getClass().getSimpleName() +
|
|
|
" doesn't support getSnapshottableDirListing");
|
|
|
}
|
|
|
// Verify result with DFS
|
|
|
SnapshottableDirectoryStatus[] dfssds = dfs.getSnapshottableDirListing();
|
|
|
- Assert.assertEquals(JsonUtil.toJsonString(sds),
|
|
|
+ assertEquals(JsonUtil.toJsonString(sds),
|
|
|
JsonUtil.toJsonString(dfssds));
|
|
|
}
|
|
|
|
|
@@ -1751,7 +1745,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Get the FileSystem instance that's being tested
|
|
|
FileSystem fs = this.getHttpFSFileSystem();
|
|
|
// Check FileStatus
|
|
|
- Assert.assertTrue(fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
// Create a file and take a snapshot
|
|
|
Path file1 = new Path(path, "file1");
|
|
|
testCreate(file1, false);
|
|
@@ -1769,7 +1763,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
snapshotStatus = webHdfsFileSystem.getSnapshotListing(path);
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() +
|
|
|
+ fail(fs.getClass().getSimpleName() +
|
|
|
" doesn't support getSnapshotDiff");
|
|
|
}
|
|
|
// Verify result with DFS
|
|
@@ -1777,7 +1771,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
FileSystem.get(path.toUri(), this.getProxiedFSConf());
|
|
|
SnapshotStatus[] dfsStatus =
|
|
|
dfs.getSnapshotListing(path);
|
|
|
- Assert.assertEquals(JsonUtil.toJsonString(snapshotStatus),
|
|
|
+ assertEquals(JsonUtil.toJsonString(snapshotStatus),
|
|
|
JsonUtil.toJsonString(dfsStatus));
|
|
|
// Cleanup
|
|
|
fs.deleteSnapshot(path, "snap2");
|
|
@@ -1797,12 +1791,12 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Verify response when there is no snapshottable directory
|
|
|
verifyGetSnapshottableDirListing(fs, dfs);
|
|
|
createSnapshotTestsPreconditions(path1);
|
|
|
- Assert.assertTrue(fs.getFileStatus(path1).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path1).isSnapshotEnabled());
|
|
|
// Verify response when there is one snapshottable directory
|
|
|
verifyGetSnapshottableDirListing(fs, dfs);
|
|
|
Path path2 = new Path("/tmp/tmp-snap-dirlist-test-2");
|
|
|
createSnapshotTestsPreconditions(path2);
|
|
|
- Assert.assertTrue(fs.getFileStatus(path2).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path2).isSnapshotEnabled());
|
|
|
// Verify response when there are two snapshottable directories
|
|
|
verifyGetSnapshottableDirListing(fs, dfs);
|
|
|
|
|
@@ -1829,7 +1823,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
FileSystem httpfs = getHttpFSFileSystem(conf);
|
|
|
if (!(httpfs instanceof WebHdfsFileSystem)
|
|
|
&& !(httpfs instanceof HttpFSFileSystem)) {
|
|
|
- Assert.fail(httpfs.getClass().getSimpleName() +
|
|
|
+ fail(httpfs.getClass().getSimpleName() +
|
|
|
" doesn't support custom user and group name pattern. "
|
|
|
+ "Only WebHdfsFileSystem and HttpFSFileSystem support it.");
|
|
|
}
|
|
@@ -1857,8 +1851,8 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
for (AclEntry aclEntry : httpfsAclStat.getEntries()) {
|
|
|
strEntries.add(aclEntry.toStringStable());
|
|
|
}
|
|
|
- Assert.assertTrue(strEntries.contains(aclUser));
|
|
|
- Assert.assertTrue(strEntries.contains(aclGroup));
|
|
|
+ assertTrue(strEntries.contains(aclUser));
|
|
|
+ assertTrue(strEntries.contains(aclGroup));
|
|
|
// Clean up
|
|
|
proxyFs.delete(new Path(dir), true);
|
|
|
}
|
|
@@ -1873,12 +1867,11 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
sds = webHdfsFileSystem.getServerDefaults();
|
|
|
} else {
|
|
|
- Assert.fail(
|
|
|
- fs.getClass().getSimpleName() + " doesn't support getServerDefaults");
|
|
|
+ fail(fs.getClass().getSimpleName() + " doesn't support getServerDefaults");
|
|
|
}
|
|
|
// Verify result with DFS
|
|
|
FsServerDefaults dfssds = dfs.getServerDefaults();
|
|
|
- Assert.assertEquals(JsonUtil.toJsonString(sds),
|
|
|
+ assertEquals(JsonUtil.toJsonString(sds),
|
|
|
JsonUtil.toJsonString(dfssds));
|
|
|
}
|
|
|
|
|
@@ -1916,7 +1909,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
webHdfsFileSystem.access(p1, FsAction.READ);
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() + " doesn't support access");
|
|
|
+ fail(fs.getClass().getSimpleName() + " doesn't support access");
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1942,7 +1935,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
assertEquals(ecPolicy, ecPolicy1);
|
|
|
httpFS.unsetErasureCodingPolicy(p1);
|
|
|
ecPolicy1 = httpFS.getErasureCodingPolicy(p1);
|
|
|
- Assert.assertNull(ecPolicy1);
|
|
|
+ assertNull(ecPolicy1);
|
|
|
} else if (fs instanceof WebHdfsFileSystem) {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs;
|
|
|
webHdfsFileSystem.setErasureCodingPolicy(p1, ecPolicyName);
|
|
@@ -1951,9 +1944,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
assertEquals(ecPolicy, ecPolicy1);
|
|
|
webHdfsFileSystem.unsetErasureCodingPolicy(p1);
|
|
|
ecPolicy1 = dfs.getErasureCodingPolicy(p1);
|
|
|
- Assert.assertNull(ecPolicy1);
|
|
|
+ assertNull(ecPolicy1);
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() + " doesn't support access");
|
|
|
+ fail(fs.getClass().getSimpleName() + " doesn't support access");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -1988,7 +1981,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
assertTrue(xAttrs
|
|
|
.containsKey(HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY));
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() + " doesn't support access");
|
|
|
+ fail(fs.getClass().getSimpleName() + " doesn't support access");
|
|
|
}
|
|
|
dfs.delete(path1, true);
|
|
|
}
|
|
@@ -2020,7 +2013,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
blockLocations = webHdfsFileSystem.getFileBlockLocations(testFile, 0, 1);
|
|
|
assertNotNull(blockLocations);
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() + " doesn't support access");
|
|
|
+ fail(fs.getClass().getSimpleName() + " doesn't support access");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -2033,7 +2026,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
// Get the FileSystem instance that's being tested
|
|
|
FileSystem fs = this.getHttpFSFileSystem();
|
|
|
// Check FileStatus
|
|
|
- Assert.assertTrue(fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
+ assertTrue(fs.getFileStatus(path).isSnapshotEnabled());
|
|
|
// Create a file and take a snapshot
|
|
|
Path file1 = new Path(path, "file1");
|
|
|
testCreate(file1, false);
|
|
@@ -2056,7 +2049,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
.getSnapshotDiffReportListing(path.toUri().getPath(), "snap1", "snap2", emptyBytes,
|
|
|
-1);
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff");
|
|
|
+ fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff");
|
|
|
}
|
|
|
// Verify result with DFS
|
|
|
DistributedFileSystem dfs =
|
|
@@ -2117,7 +2110,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
httpFs.close();
|
|
|
dfs.close();
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem.");
|
|
|
+ fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem.");
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2144,7 +2137,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) httpFs;
|
|
|
diffErasureCodingPolicies = webHdfsFileSystem.getAllErasureCodingPolicies();
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() +
|
|
|
+ fail(fs.getClass().getSimpleName() +
|
|
|
" is not of type HttpFSFileSystem or WebHdfsFileSystem");
|
|
|
}
|
|
|
|
|
@@ -2152,7 +2145,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
assertEquals(dfsAllErasureCodingPolicies.size(), diffErasureCodingPolicies.size());
|
|
|
assertTrue(dfsAllErasureCodingPolicies.containsAll(diffErasureCodingPolicies));
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem.");
|
|
|
+ fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem.");
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2194,13 +2187,13 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
Map<String, String> diffErasureCodingCodecs = diffErasureCodingCodecsRef.get();
|
|
|
|
|
|
//Validate testGetECCodecs are the same as DistributedFileSystem
|
|
|
- Assert.assertEquals(dfsErasureCodingCodecs.size(), diffErasureCodingCodecs.size());
|
|
|
+ assertEquals(dfsErasureCodingCodecs.size(), diffErasureCodingCodecs.size());
|
|
|
|
|
|
for (Map.Entry<String, String> entry : dfsErasureCodingCodecs.entrySet()) {
|
|
|
String key = entry.getKey();
|
|
|
String value = entry.getValue();
|
|
|
- Assert.assertTrue(diffErasureCodingCodecs.containsKey(key));
|
|
|
- Assert.assertEquals(value, diffErasureCodingCodecs.get(key));
|
|
|
+ assertTrue(diffErasureCodingCodecs.containsKey(key));
|
|
|
+ assertEquals(value, diffErasureCodingCodecs.get(key));
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2232,38 +2225,38 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) httpFs;
|
|
|
diffTrashRoots = webHdfsFileSystem.getTrashRoots(true);
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() +
|
|
|
+ fail(fs.getClass().getSimpleName() +
|
|
|
" is not of type HttpFSFileSystem or WebHdfsFileSystem");
|
|
|
}
|
|
|
|
|
|
// Validate getTrashRoots are the same as DistributedFileSystem
|
|
|
assertEquals(dfsTrashRoots.size(), diffTrashRoots.size());
|
|
|
} else {
|
|
|
- Assert.fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem.");
|
|
|
+ fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem.");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private void assertHttpFsReportListingWithDfsClient(SnapshotDiffReportListing diffReportListing,
|
|
|
SnapshotDiffReportListing dfsDiffReportListing) {
|
|
|
- Assert.assertEquals(diffReportListing.getCreateList().size(),
|
|
|
+ assertEquals(diffReportListing.getCreateList().size(),
|
|
|
dfsDiffReportListing.getCreateList().size());
|
|
|
- Assert.assertEquals(diffReportListing.getDeleteList().size(),
|
|
|
+ assertEquals(diffReportListing.getDeleteList().size(),
|
|
|
dfsDiffReportListing.getDeleteList().size());
|
|
|
- Assert.assertEquals(diffReportListing.getModifyList().size(),
|
|
|
+ assertEquals(diffReportListing.getModifyList().size(),
|
|
|
dfsDiffReportListing.getModifyList().size());
|
|
|
- Assert.assertEquals(diffReportListing.getIsFromEarlier(),
|
|
|
+ assertEquals(diffReportListing.getIsFromEarlier(),
|
|
|
dfsDiffReportListing.getIsFromEarlier());
|
|
|
- Assert.assertEquals(diffReportListing.getLastIndex(), dfsDiffReportListing.getLastIndex());
|
|
|
- Assert.assertEquals(DFSUtil.bytes2String(diffReportListing.getLastPath()),
|
|
|
+ assertEquals(diffReportListing.getLastIndex(), dfsDiffReportListing.getLastIndex());
|
|
|
+ assertEquals(DFSUtil.bytes2String(diffReportListing.getLastPath()),
|
|
|
DFSUtil.bytes2String(dfsDiffReportListing.getLastPath()));
|
|
|
int i = 0;
|
|
|
for (SnapshotDiffReportListing.DiffReportListingEntry entry : diffReportListing
|
|
|
.getCreateList()) {
|
|
|
SnapshotDiffReportListing.DiffReportListingEntry dfsDiffEntry =
|
|
|
dfsDiffReportListing.getCreateList().get(i);
|
|
|
- Assert.assertEquals(entry.getDirId(), dfsDiffEntry.getDirId());
|
|
|
- Assert.assertEquals(entry.getFileId(), dfsDiffEntry.getFileId());
|
|
|
- Assert.assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()),
|
|
|
+ assertEquals(entry.getDirId(), dfsDiffEntry.getDirId());
|
|
|
+ assertEquals(entry.getFileId(), dfsDiffEntry.getFileId());
|
|
|
+ assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()),
|
|
|
DFSUtilClient.byteArray2bytes(dfsDiffEntry.getSourcePath()));
|
|
|
i++;
|
|
|
}
|
|
@@ -2272,9 +2265,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
.getDeleteList()) {
|
|
|
SnapshotDiffReportListing.DiffReportListingEntry dfsDiffEntry =
|
|
|
dfsDiffReportListing.getDeleteList().get(i);
|
|
|
- Assert.assertEquals(entry.getDirId(), dfsDiffEntry.getDirId());
|
|
|
- Assert.assertEquals(entry.getFileId(), dfsDiffEntry.getFileId());
|
|
|
- Assert.assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()),
|
|
|
+ assertEquals(entry.getDirId(), dfsDiffEntry.getDirId());
|
|
|
+ assertEquals(entry.getFileId(), dfsDiffEntry.getFileId());
|
|
|
+ assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()),
|
|
|
DFSUtilClient.byteArray2bytes(dfsDiffEntry.getSourcePath()));
|
|
|
i++;
|
|
|
}
|
|
@@ -2283,9 +2276,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
|
|
|
.getModifyList()) {
|
|
|
SnapshotDiffReportListing.DiffReportListingEntry dfsDiffEntry =
|
|
|
dfsDiffReportListing.getModifyList().get(i);
|
|
|
- Assert.assertEquals(entry.getDirId(), dfsDiffEntry.getDirId());
|
|
|
- Assert.assertEquals(entry.getFileId(), dfsDiffEntry.getFileId());
|
|
|
- Assert.assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()),
|
|
|
+ assertEquals(entry.getDirId(), dfsDiffEntry.getDirId());
|
|
|
+ assertEquals(entry.getFileId(), dfsDiffEntry.getFileId());
|
|
|
+ assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()),
|
|
|
DFSUtilClient.byteArray2bytes(dfsDiffEntry.getSourcePath()));
|
|
|
i++;
|
|
|
}
|