|
@@ -17,6 +17,7 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.nfs.nfs3;
|
|
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
@@ -26,8 +27,7 @@ import java.nio.ByteBuffer;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.concurrent.ConcurrentNavigableMap;
|
|
|
|
|
|
-import junit.framework.Assert;
|
|
|
-
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
@@ -41,6 +41,7 @@ import org.apache.hadoop.nfs.nfs3.IdUserGroup;
|
|
|
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
|
|
import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
|
|
|
import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
|
|
|
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
|
|
|
import org.apache.hadoop.nfs.nfs3.request.CREATE3Request;
|
|
|
import org.apache.hadoop.nfs.nfs3.request.READ3Request;
|
|
|
import org.apache.hadoop.nfs.nfs3.request.SetAttr3;
|
|
@@ -49,6 +50,8 @@ import org.apache.hadoop.nfs.nfs3.response.CREATE3Response;
|
|
|
import org.apache.hadoop.nfs.nfs3.response.READ3Response;
|
|
|
import org.apache.hadoop.oncrpc.XDR;
|
|
|
import org.apache.hadoop.oncrpc.security.SecurityHandler;
|
|
|
+import org.jboss.netty.channel.Channel;
|
|
|
+import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.Mockito;
|
|
|
|
|
@@ -140,32 +143,33 @@ public class TestWrites {
|
|
|
|
|
|
// Test inactive open file context
|
|
|
ctx.setActiveStatusForTest(false);
|
|
|
- ret = ctx.checkCommit(dfsClient, 0, null, 1, attr);
|
|
|
+ Channel ch = Mockito.mock(Channel.class);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 0, ch, 1, attr, false);
|
|
|
Assert.assertTrue(ret == COMMIT_STATUS.COMMIT_INACTIVE_CTX);
|
|
|
|
|
|
ctx.getPendingWritesForTest().put(new OffsetRange(5, 10),
|
|
|
new WriteCtx(null, 0, 0, 0, null, null, null, 0, false, null));
|
|
|
- ret = ctx.checkCommit(dfsClient, 0, null, 1, attr);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 0, ch, 1, attr, false);
|
|
|
Assert.assertTrue(ret == COMMIT_STATUS.COMMIT_INACTIVE_WITH_PENDING_WRITE);
|
|
|
|
|
|
// Test request with non zero commit offset
|
|
|
ctx.setActiveStatusForTest(true);
|
|
|
Mockito.when(fos.getPos()).thenReturn((long) 10);
|
|
|
- COMMIT_STATUS status = ctx.checkCommitInternal(5, null, 1, attr);
|
|
|
+ COMMIT_STATUS status = ctx.checkCommitInternal(5, null, 1, attr, false);
|
|
|
Assert.assertTrue(status == COMMIT_STATUS.COMMIT_DO_SYNC);
|
|
|
// Do_SYNC state will be updated to FINISHED after data sync
|
|
|
- ret = ctx.checkCommit(dfsClient, 5, null, 1, attr);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 5, ch, 1, attr, false);
|
|
|
Assert.assertTrue(ret == COMMIT_STATUS.COMMIT_FINISHED);
|
|
|
|
|
|
- status = ctx.checkCommitInternal(10, null, 1, attr);
|
|
|
+ status = ctx.checkCommitInternal(10, ch, 1, attr, false);
|
|
|
Assert.assertTrue(status == COMMIT_STATUS.COMMIT_DO_SYNC);
|
|
|
- ret = ctx.checkCommit(dfsClient, 10, null, 1, attr);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 10, ch, 1, attr, false);
|
|
|
Assert.assertTrue(ret == COMMIT_STATUS.COMMIT_FINISHED);
|
|
|
|
|
|
ConcurrentNavigableMap<Long, CommitCtx> commits = ctx
|
|
|
.getPendingCommitsForTest();
|
|
|
Assert.assertTrue(commits.size() == 0);
|
|
|
- ret = ctx.checkCommit(dfsClient, 11, null, 1, attr);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 11, ch, 1, attr, false);
|
|
|
Assert.assertTrue(ret == COMMIT_STATUS.COMMIT_WAIT);
|
|
|
Assert.assertTrue(commits.size() == 1);
|
|
|
long key = commits.firstKey();
|
|
@@ -174,7 +178,7 @@ public class TestWrites {
|
|
|
// Test request with zero commit offset
|
|
|
commits.remove(new Long(11));
|
|
|
// There is one pending write [5,10]
|
|
|
- ret = ctx.checkCommit(dfsClient, 0, null, 1, attr);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 0, ch, 1, attr, false);
|
|
|
Assert.assertTrue(ret == COMMIT_STATUS.COMMIT_WAIT);
|
|
|
Assert.assertTrue(commits.size() == 1);
|
|
|
key = commits.firstKey();
|
|
@@ -182,10 +186,79 @@ public class TestWrites {
|
|
|
|
|
|
// Empty pending writes
|
|
|
ctx.getPendingWritesForTest().remove(new OffsetRange(5, 10));
|
|
|
- ret = ctx.checkCommit(dfsClient, 0, null, 1, attr);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 0, ch, 1, attr, false);
|
|
|
Assert.assertTrue(ret == COMMIT_STATUS.COMMIT_FINISHED);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ // Validate all the commit check return codes OpenFileCtx.COMMIT_STATUS, which
|
|
|
+ // includes COMMIT_FINISHED, COMMIT_WAIT, COMMIT_INACTIVE_CTX,
|
|
|
+ // COMMIT_INACTIVE_WITH_PENDING_WRITE, COMMIT_ERROR, and COMMIT_DO_SYNC.
|
|
|
+ public void testCheckCommitFromRead() throws IOException {
|
|
|
+ DFSClient dfsClient = Mockito.mock(DFSClient.class);
|
|
|
+ Nfs3FileAttributes attr = new Nfs3FileAttributes();
|
|
|
+ HdfsDataOutputStream fos = Mockito.mock(HdfsDataOutputStream.class);
|
|
|
+ Mockito.when(fos.getPos()).thenReturn((long) 0);
|
|
|
+
|
|
|
+ OpenFileCtx ctx = new OpenFileCtx(fos, attr, "/dumpFilePath", dfsClient,
|
|
|
+ new IdUserGroup());
|
|
|
+
|
|
|
+ FileHandle h = new FileHandle(1); // fake handle for "/dumpFilePath"
|
|
|
+ COMMIT_STATUS ret;
|
|
|
+ WriteManager wm = new WriteManager(new IdUserGroup(), new Configuration());
|
|
|
+ assertTrue(wm.addOpenFileStream(h, ctx));
|
|
|
+
|
|
|
+ // Test inactive open file context
|
|
|
+ ctx.setActiveStatusForTest(false);
|
|
|
+ Channel ch = Mockito.mock(Channel.class);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 0, ch, 1, attr, true);
|
|
|
+ assertEquals( COMMIT_STATUS.COMMIT_INACTIVE_CTX, ret);
|
|
|
+ assertEquals(Nfs3Status.NFS3_OK, wm.commitBeforeRead(dfsClient, h, 0));
|
|
|
+
|
|
|
+ ctx.getPendingWritesForTest().put(new OffsetRange(5, 10),
|
|
|
+ new WriteCtx(null, 0, 0, 0, null, null, null, 0, false, null));
|
|
|
+ ret = ctx.checkCommit(dfsClient, 0, ch, 1, attr, true);
|
|
|
+ assertEquals(COMMIT_STATUS.COMMIT_INACTIVE_WITH_PENDING_WRITE, ret);
|
|
|
+ assertEquals(Nfs3Status.NFS3ERR_IO, wm.commitBeforeRead(dfsClient, h, 0));
|
|
|
+
|
|
|
+ // Test request with non zero commit offset
|
|
|
+ ctx.setActiveStatusForTest(true);
|
|
|
+ Mockito.when(fos.getPos()).thenReturn((long) 10);
|
|
|
+ COMMIT_STATUS status = ctx.checkCommitInternal(5, ch, 1, attr, false);
|
|
|
+ assertEquals(COMMIT_STATUS.COMMIT_DO_SYNC, status);
|
|
|
+ // Do_SYNC state will be updated to FINISHED after data sync
|
|
|
+ ret = ctx.checkCommit(dfsClient, 5, ch, 1, attr, true);
|
|
|
+ assertEquals(COMMIT_STATUS.COMMIT_FINISHED, ret);
|
|
|
+ assertEquals(Nfs3Status.NFS3_OK, wm.commitBeforeRead(dfsClient, h, 5));
|
|
|
+
|
|
|
+ status = ctx.checkCommitInternal(10, ch, 1, attr, true);
|
|
|
+ assertTrue(status == COMMIT_STATUS.COMMIT_DO_SYNC);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 10, ch, 1, attr, true);
|
|
|
+ assertEquals(COMMIT_STATUS.COMMIT_FINISHED, ret);
|
|
|
+ assertEquals(Nfs3Status.NFS3_OK, wm.commitBeforeRead(dfsClient, h, 10));
|
|
|
+
|
|
|
+ ConcurrentNavigableMap<Long, CommitCtx> commits = ctx
|
|
|
+ .getPendingCommitsForTest();
|
|
|
+ assertTrue(commits.size() == 0);
|
|
|
+ ret = ctx.checkCommit(dfsClient, 11, ch, 1, attr, true);
|
|
|
+ assertEquals(COMMIT_STATUS.COMMIT_WAIT, ret);
|
|
|
+ assertEquals(0, commits.size()); // commit triggered by read doesn't wait
|
|
|
+ assertEquals(Nfs3Status.NFS3ERR_JUKEBOX, wm.commitBeforeRead(dfsClient, h, 11));
|
|
|
+
|
|
|
+ // Test request with zero commit offset
|
|
|
+ // There is one pending write [5,10]
|
|
|
+ ret = ctx.checkCommit(dfsClient, 0, ch, 1, attr, true);
|
|
|
+ assertEquals(COMMIT_STATUS.COMMIT_WAIT, ret);
|
|
|
+ assertEquals(0, commits.size());
|
|
|
+ assertEquals(Nfs3Status.NFS3ERR_JUKEBOX, wm.commitBeforeRead(dfsClient, h, 0));
|
|
|
+
|
|
|
+ // Empty pending writes
|
|
|
+ ctx.getPendingWritesForTest().remove(new OffsetRange(5, 10));
|
|
|
+ ret = ctx.checkCommit(dfsClient, 0, ch, 1, attr, true);
|
|
|
+ assertEquals(COMMIT_STATUS.COMMIT_FINISHED, ret);
|
|
|
+ assertEquals(Nfs3Status.NFS3_OK, wm.commitBeforeRead(dfsClient, h, 0));
|
|
|
+ }
|
|
|
+
|
|
|
private void waitWrite(RpcProgramNfs3 nfsd, FileHandle handle, int maxWaitTime)
|
|
|
throws InterruptedException {
|
|
|
int waitedTime = 0;
|