|
@@ -18,20 +18,33 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
+import static org.hamcrest.CoreMatchers.containsString;
|
|
|
import static org.hamcrest.CoreMatchers.is;
|
|
|
+import static org.hamcrest.CoreMatchers.not;
|
|
|
import static org.junit.Assert.assertThat;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
+import org.apache.hadoop.ipc.AlignmentContext;
|
|
|
+import org.apache.hadoop.ipc.Client;
|
|
|
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.AfterClass;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.List;
|
|
|
+import org.mockito.invocation.InvocationOnMock;
|
|
|
+import org.mockito.stubbing.Answer;
|
|
|
+
|
|
|
|
|
|
/**
|
|
|
* Class is used to test server sending state alignment information to clients
|
|
@@ -91,7 +104,7 @@ public class TestStateAlignmentContext {
|
|
|
public void testStateTransferOnWrite() throws Exception {
|
|
|
long preWriteState = cluster.getNamesystem().getLastWrittenTransactionId();
|
|
|
DFSTestUtil.writeFile(dfs, new Path("/testFile1"), "abc");
|
|
|
- long clientState = dfs.dfs.lastSeenStateId;
|
|
|
+ long clientState = dfs.dfs.getAlignmentContext().getLastSeenStateId();
|
|
|
long postWriteState = cluster.getNamesystem().getLastWrittenTransactionId();
|
|
|
// Write(s) should have increased state. Check for greater than.
|
|
|
assertThat(clientState > preWriteState, is(true));
|
|
@@ -109,7 +122,8 @@ public class TestStateAlignmentContext {
|
|
|
long lastWrittenId = cluster.getNamesystem().getLastWrittenTransactionId();
|
|
|
DFSTestUtil.readFile(dfs, new Path("/testFile2"));
|
|
|
// Read should catch client up to last written state.
|
|
|
- assertThat(dfs.dfs.lastSeenStateId, is(lastWrittenId));
|
|
|
+ long clientState = dfs.dfs.getAlignmentContext().getLastSeenStateId();
|
|
|
+ assertThat(clientState, is(lastWrittenId));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -122,10 +136,86 @@ public class TestStateAlignmentContext {
|
|
|
long lastWrittenId = cluster.getNamesystem().getLastWrittenTransactionId();
|
|
|
try (DistributedFileSystem clearDfs =
|
|
|
(DistributedFileSystem) FileSystem.get(CONF)) {
|
|
|
- assertThat(clearDfs.dfs.lastSeenStateId, is(0L));
|
|
|
+ ClientGCIContext clientState = clearDfs.dfs.getAlignmentContext();
|
|
|
+ assertThat(clientState.getLastSeenStateId(), is(Long.MIN_VALUE));
|
|
|
DFSTestUtil.readFile(clearDfs, new Path("/testFile3"));
|
|
|
- assertThat(clearDfs.dfs.lastSeenStateId, is(lastWrittenId));
|
|
|
+ assertThat(clientState.getLastSeenStateId(), is(lastWrittenId));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * This test mocks an AlignmentContext and ensures that DFSClient
|
|
|
+ * writes its lastSeenStateId into RPC requests.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testClientSendsState() throws Exception {
|
|
|
+ AlignmentContext alignmentContext = dfs.dfs.getAlignmentContext();
|
|
|
+ AlignmentContext spiedAlignContext = Mockito.spy(alignmentContext);
|
|
|
+ Client.setAlignmentContext(spiedAlignContext);
|
|
|
+
|
|
|
+ // Collect RpcRequestHeaders for verification later.
|
|
|
+ final List<RpcHeaderProtos.RpcRequestHeaderProto.Builder> collectedHeaders =
|
|
|
+ new ArrayList<>();
|
|
|
+ Mockito.doAnswer(new Answer() {
|
|
|
+ @Override
|
|
|
+ public Object answer(InvocationOnMock a) throws Throwable {
|
|
|
+ Object[] arguments = a.getArguments();
|
|
|
+ RpcHeaderProtos.RpcRequestHeaderProto.Builder header =
|
|
|
+ (RpcHeaderProtos.RpcRequestHeaderProto.Builder) arguments[0];
|
|
|
+ collectedHeaders.add(header);
|
|
|
+ return a.callRealMethod();
|
|
|
+ }
|
|
|
+ }).when(spiedAlignContext).updateRequestState(Mockito.any());
|
|
|
+
|
|
|
+ DFSTestUtil.writeFile(dfs, new Path("/testFile4"), "shv");
|
|
|
+
|
|
|
+ // Ensure first header and last header have different state.
|
|
|
+ assertThat(collectedHeaders.size() > 1, is(true));
|
|
|
+ assertThat(collectedHeaders.get(0).getStateId(),
|
|
|
+ is(not(collectedHeaders.get(collectedHeaders.size() - 1))));
|
|
|
+
|
|
|
+ // Ensure collected RpcRequestHeaders are in increasing order.
|
|
|
+ long lastHeader = collectedHeaders.get(0).getStateId();
|
|
|
+ for(RpcHeaderProtos.RpcRequestHeaderProto.Builder header :
|
|
|
+ collectedHeaders.subList(1, collectedHeaders.size())) {
|
|
|
+ long currentHeader = header.getStateId();
|
|
|
+ assertThat(currentHeader >= lastHeader, is(true));
|
|
|
+ lastHeader = header.getStateId();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This test mocks an AlignmentContext to send stateIds greater than
|
|
|
+ * server's stateId in RPC requests.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testClientSendsGreaterState() throws Exception {
|
|
|
+ AlignmentContext alignmentContext = dfs.dfs.getAlignmentContext();
|
|
|
+ AlignmentContext spiedAlignContext = Mockito.spy(alignmentContext);
|
|
|
+ Client.setAlignmentContext(spiedAlignContext);
|
|
|
+
|
|
|
+ // Make every client call have a stateId > server's stateId.
|
|
|
+ Mockito.doAnswer(new Answer() {
|
|
|
+ @Override
|
|
|
+ public Object answer(InvocationOnMock a) throws Throwable {
|
|
|
+ Object[] arguments = a.getArguments();
|
|
|
+ RpcHeaderProtos.RpcRequestHeaderProto.Builder header =
|
|
|
+ (RpcHeaderProtos.RpcRequestHeaderProto.Builder) arguments[0];
|
|
|
+ try {
|
|
|
+ return a.callRealMethod();
|
|
|
+ } finally {
|
|
|
+ header.setStateId(Long.MAX_VALUE);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }).when(spiedAlignContext).updateRequestState(Mockito.any());
|
|
|
+
|
|
|
+ GenericTestUtils.LogCapturer logCapturer =
|
|
|
+ GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.LOG);
|
|
|
+ DFSTestUtil.writeFile(dfs, new Path("/testFile4"), "shv");
|
|
|
+ logCapturer.stopCapturing();
|
|
|
+
|
|
|
+ String output = logCapturer.getOutput();
|
|
|
+ assertThat(output, containsString("A client sent stateId: "));
|
|
|
+ }
|
|
|
+
|
|
|
}
|