|
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import java.io.*;
|
|
|
import java.nio.charset.StandardCharsets;
|
|
|
-import java.security.Permission;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
import java.text.SimpleDateFormat;
|
|
|
import java.util.ArrayList;
|
|
@@ -49,6 +48,7 @@ import org.apache.hadoop.fs.*;
|
|
|
import org.apache.hadoop.fs.permission.AclEntry;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
@@ -61,6 +61,7 @@ import org.apache.hadoop.io.SequenceFile;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.io.compress.BZip2Codec;
|
|
|
import org.apache.hadoop.io.compress.CompressionCodec;
|
|
|
+import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.net.ServerSocketUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
@@ -556,52 +557,24 @@ public class TestDFSShell {
|
|
|
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);
|
|
|
- }
|
|
|
- };
|
|
|
-
|
|
|
- //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) {}
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ 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 = Assert.assertThrows(RemoteException.class,
|
|
|
+ () -> client.create(dst.toString(), false));
|
|
|
+ LOG.info("GOOD", e);
|
|
|
+ Assert.assertEquals(e.getClassName(), AlreadyBeingCreatedException.class.getName());
|
|
|
}
|
|
|
- });
|
|
|
- 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"));
|