|
@@ -22,18 +22,26 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.RemoteIterator;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
+import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
|
|
|
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
|
|
|
import org.apache.log4j.Level;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.EnumSet;
|
|
|
+import java.util.List;
|
|
|
import java.util.Random;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
|
|
|
/**
|
|
|
* Tests append on erasure coded file.
|
|
@@ -111,4 +119,31 @@ public class TestStripedFileAppend {
|
|
|
StripedFileTestUtil.getDefaultECPolicy(), totalSplit);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testAppendWithoutNewBlock() throws IOException {
|
|
|
+ Path file = new Path(dir, "testAppendWithoutNewBlock");
|
|
|
+
|
|
|
+ // Create file
|
|
|
+ FSDataOutputStream out = dfs.create(file);
|
|
|
+ out.write("testAppendWithoutNewBlock".getBytes());
|
|
|
+ out.close();
|
|
|
+
|
|
|
+ // Append file
|
|
|
+ try {
|
|
|
+ out = dfs.append(file, EnumSet.of(CreateFlag.APPEND), 4096, null);
|
|
|
+ out.write("testAppendWithoutNewBlock".getBytes());
|
|
|
+ fail("Should throw unsupported operation");
|
|
|
+ } catch (Exception e) {
|
|
|
+ assertTrue(e.getMessage()
|
|
|
+ .contains("Append on EC file without new block is not supported"));
|
|
|
+ }
|
|
|
+
|
|
|
+ List<OpenFilesType> types = new ArrayList<>();
|
|
|
+ types.add(OpenFilesType.ALL_OPEN_FILES);
|
|
|
+
|
|
|
+ RemoteIterator<OpenFileEntry> listOpenFiles = dfs
|
|
|
+ .listOpenFiles(EnumSet.copyOf(types), file.toString());
|
|
|
+ assertFalse("No file should be open after append failure",
|
|
|
+ listOpenFiles.hasNext());
|
|
|
+ }
|
|
|
}
|