szetszwo commented on code in PR #7566:
URL: https://github.com/apache/hadoop/pull/7566#discussion_r2061500341
##########
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java:
##########
@@ -574,28 +575,36 @@ public void run() {
};
//use SecurityManager to pause the copying of f1 and begin copying f2
- SecurityManager sm = System.getSecurityManager();
- System.out.println("SecurityManager = " + sm);
- System.setSecurityManager(new SecurityManager() {
- private boolean firstTime = true;
+ SecurityManager sm = null;
Review Comment:
Back in 2007, DFSClient won't create a file in NN before flushing the first
block. As a result, the client 2 below will succeed but client 1 will fail
with a file already created exception. We want s client 1 succeeds and client
2 fails since client 1 has started earlier.
1. Client 1 starts writing to a file but not yet flushes the first block.
2. After that client 2 starts writing to the same file and also flushes the
first block.
[HADOOP-999](https://issues.apache.org/jira/browse/HADOOP-999) was to
changing DFSClient to create the file in NN in the beginning of write. The
test was to use SecurityManager to pause client 1 in order to delay flushing
the first block.
Just check the existing test (before this PR), it actually not working since
it tries to search "FileUtil.copyContent" in the stack trace but the
`FileUtil.copyContent` method no longer exists, i.e. the search always fails.
I suggest simply changing the test as below
```diff
@@ -556,52 +586,25 @@ public void testPut() throws IOException {
final Path root = mkdir(dfs, new Path("/testPut"));
final Path dst = new Path(root, "dst");
- show("begin");
-
- final Thread copy2ndFileThread = new Thread() {
- @Override
- public void run() {
- try {
- show("copy local " + f2 + " to remote " + dst);
- dfs.copyFromLocalFile(false, false, new Path(f2.getPath()), dst);
- } catch (IOException ioe) {
- show("good " + StringUtils.stringifyException(ioe));
- return;
- }
- //should not be here, must got IOException
- assertTrue(false);
+ final String hello = "hello";
+ try (FSDataOutputStream out = dfs.create(dst, false)) {
+ // It should fail to create a new client writing to the same file.
+ try(DFSClient client = new DFSClient(dfs.getUri(), dfs.getConf())) {
+ final RemoteException e =
Assertions.assertThrows(RemoteException.class,
+ () -> client.create(dst.toString(), false));
+ LOG.info("GOOD", e);
+
Assertions.assertEquals(AlreadyBeingCreatedException.class.getName(),
e.getClassName());
}
- };
-
- //use SecurityManager to pause the copying of f1 and begin copying f2
- SecurityManager sm = System.getSecurityManager();
- System.out.println("SecurityManager = " + sm);
- System.setSecurityManager(new SecurityManager() {
- private boolean firstTime = true;
- @Override
- public void checkPermission(Permission perm) {
- if (firstTime) {
- Thread t = Thread.currentThread();
- if (!t.toString().contains("DataNode")) {
- String s = "" + Arrays.asList(t.getStackTrace());
- if (s.contains("FileUtil.copyContent")) {
- //pause at FileUtil.copyContent
-
- firstTime = false;
- copy2ndFileThread.start();
- try {Thread.sleep(5000);} catch (InterruptedException e) {}
- }
- }
- }
- }
- });
- show("copy local " + f1 + " to remote " + dst);
- dfs.copyFromLocalFile(false, false, new Path(f1.getPath()), dst);
- show("done");
+ // It should succeed to continue writing to the file.
+ out.writeUTF(hello);
+ }
- try {copy2ndFileThread.join();} catch (InterruptedException e) { }
- System.setSecurityManager(sm);
+ // Verify the file content.
+ try (FSDataInputStream in = dfs.open(dst)) {
+ final String read = in.readUTF();
+ assertEquals(hello, read);
+ }
// copy multiple files to destination directory
final Path destmultiple = mkdir(dfs, new Path(root, "putmultiple"));
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]