Skip to content

Commit 07394a5

Browse files
committed
HDDS-12604. Reduce duplication in TestContainerStateMachine
1 parent c8e77f8 commit 07394a5

File tree

1 file changed

+45
-46
lines changed

1 file changed

+45
-46
lines changed

hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestContainerStateMachine.java

+45-46
Original file line numberDiff line numberDiff line change
@@ -129,6 +129,37 @@ public void testWriteFailure(boolean failWithException) throws ExecutionExceptio
129129
TransactionContext trx = mock(TransactionContext.class);
130130
ContainerStateMachine.Context context = mock(ContainerStateMachine.Context.class);
131131
when(trx.getStateMachineContext()).thenReturn(context);
132+
133+
setUpMockDispatcherReturn(failWithException);
134+
setUpMockRequestProto(context);
135+
136+
AtomicReference<Throwable> throwable = new AtomicReference<>(null);
137+
Function<Throwable, ? extends Message> throwableSetter = getThrowableSetter(throwable);
138+
139+
stateMachine.write(entry, trx).exceptionally(throwableSetter).get();
140+
verify(dispatcher, times(1)).dispatch(any(ContainerProtos.ContainerCommandRequestProto.class),
141+
any(DispatcherContext.class));
142+
reset(dispatcher);
143+
assertNotNull(throwable.get());
144+
assertResults(failWithException, throwable);
145+
146+
// Writing data to another container(containerId 2) should also fail.
147+
when(context.getRequestProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
148+
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
149+
ContainerProtos.WriteChunkRequestProto.newBuilder().setData(ByteString.copyFromUtf8("Test Data"))
150+
.setBlockID(
151+
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(2).setLocalID(1).build()).build())
152+
.setContainerID(2)
153+
.setDatanodeUuid(UUID.randomUUID().toString()).build());
154+
stateMachine.write(entryNext, trx).exceptionally(throwableSetter).get();
155+
verify(dispatcher, times(0)).dispatch(any(ContainerProtos.ContainerCommandRequestProto.class),
156+
any(DispatcherContext.class));
157+
assertInstanceOf(StorageContainerException.class, throwable.get());
158+
StorageContainerException sce = (StorageContainerException) throwable.get();
159+
assertEquals(ContainerProtos.Result.CONTAINER_UNHEALTHY, sce.getResult());
160+
}
161+
162+
public final void setUpMockDispatcherReturn(boolean failWithException) {
132163
if (failWithException) {
133164
when(dispatcher.dispatch(any(), any())).thenThrow(new RuntimeException());
134165
} else {
@@ -137,45 +168,34 @@ public void testWriteFailure(boolean failWithException) throws ExecutionExceptio
137168
.setResult(ContainerProtos.Result.CONTAINER_INTERNAL_ERROR)
138169
.build());
139170
}
171+
}
140172

173+
public final void setUpMockRequestProto(ContainerStateMachine.Context context) {
141174
when(context.getRequestProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
142175
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
143176
ContainerProtos.WriteChunkRequestProto.newBuilder().setData(ByteString.copyFromUtf8("Test Data"))
144177
.setBlockID(
145178
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(1).setLocalID(1).build()).build())
146179
.setContainerID(1)
147180
.setDatanodeUuid(UUID.randomUUID().toString()).build());
148-
AtomicReference<Throwable> throwable = new AtomicReference<>(null);
181+
}
182+
183+
public final Function<Throwable, ? extends Message> getThrowableSetter(AtomicReference<Throwable> throwable) {
149184
Function<Throwable, ? extends Message> throwableSetter = t -> {
150185
throwable.set(t);
151186
return null;
152187
};
153-
stateMachine.write(entry, trx).exceptionally(throwableSetter).get();
154-
verify(dispatcher, times(1)).dispatch(any(ContainerProtos.ContainerCommandRequestProto.class),
155-
any(DispatcherContext.class));
156-
reset(dispatcher);
157-
assertNotNull(throwable.get());
188+
return throwableSetter;
189+
}
190+
191+
public final void assertResults(boolean failWithException, AtomicReference<Throwable> throwable) {
158192
if (failWithException) {
159193
assertInstanceOf(RuntimeException.class, throwable.get());
160194
} else {
161195
assertInstanceOf(StorageContainerException.class, throwable.get());
162196
StorageContainerException sce = (StorageContainerException) throwable.get();
163197
assertEquals(ContainerProtos.Result.CONTAINER_INTERNAL_ERROR, sce.getResult());
164198
}
165-
// Writing data to another container(containerId 2) should also fail.
166-
when(context.getRequestProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
167-
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
168-
ContainerProtos.WriteChunkRequestProto.newBuilder().setData(ByteString.copyFromUtf8("Test Data"))
169-
.setBlockID(
170-
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(2).setLocalID(1).build()).build())
171-
.setContainerID(2)
172-
.setDatanodeUuid(UUID.randomUUID().toString()).build());
173-
stateMachine.write(entryNext, trx).exceptionally(throwableSetter).get();
174-
verify(dispatcher, times(0)).dispatch(any(ContainerProtos.ContainerCommandRequestProto.class),
175-
any(DispatcherContext.class));
176-
assertInstanceOf(StorageContainerException.class, throwable.get());
177-
StorageContainerException sce = (StorageContainerException) throwable.get();
178-
assertEquals(ContainerProtos.Result.CONTAINER_UNHEALTHY, sce.getResult());
179199
}
180200

181201
@ParameterizedTest
@@ -189,14 +209,8 @@ public void testApplyTransactionFailure(boolean failWithException) throws Execut
189209
ContainerStateMachine.Context context = mock(ContainerStateMachine.Context.class);
190210
when(trx.getLogEntry()).thenReturn(entry);
191211
when(trx.getStateMachineContext()).thenReturn(context);
192-
if (failWithException) {
193-
when(dispatcher.dispatch(any(), any())).thenThrow(new RuntimeException());
194-
} else {
195-
when(dispatcher.dispatch(any(), any())).thenReturn(ContainerProtos.ContainerCommandResponseProto
196-
.newBuilder().setCmdType(ContainerProtos.Type.WriteChunk)
197-
.setResult(ContainerProtos.Result.CONTAINER_INTERNAL_ERROR)
198-
.build());
199-
}
212+
213+
setUpMockDispatcherReturn(failWithException);
200214
// Failing apply transaction on congtainer 1.
201215
when(context.getLogProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
202216
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
@@ -205,24 +219,15 @@ public void testApplyTransactionFailure(boolean failWithException) throws Execut
205219
.setContainerID(1)
206220
.setDatanodeUuid(UUID.randomUUID().toString()).build());
207221
AtomicReference<Throwable> throwable = new AtomicReference<>(null);
208-
Function<Throwable, ? extends Message> throwableSetter = t -> {
209-
throwable.set(t);
210-
return null;
211-
};
222+
Function<Throwable, ? extends Message> throwableSetter = getThrowableSetter(throwable);
212223
//apply transaction will fail because of runtime exception thrown by dispatcher, which marks the first
213224
// failure on container 1.
214225
stateMachine.applyTransaction(trx).exceptionally(throwableSetter).get();
215226
verify(dispatcher, times(1)).dispatch(any(ContainerProtos.ContainerCommandRequestProto.class),
216227
any(DispatcherContext.class));
217228
reset(dispatcher);
218229
assertNotNull(throwable.get());
219-
if (failWithException) {
220-
assertInstanceOf(RuntimeException.class, throwable.get());
221-
} else {
222-
assertInstanceOf(StorageContainerException.class, throwable.get());
223-
StorageContainerException sce = (StorageContainerException) throwable.get();
224-
assertEquals(ContainerProtos.Result.CONTAINER_INTERNAL_ERROR, sce.getResult());
225-
}
230+
assertResults(failWithException, throwable);
226231
// Another apply transaction on same container 1 should fail because the previous apply transaction failed.
227232
stateMachine.applyTransaction(trx).exceptionally(throwableSetter).get();
228233
verify(dispatcher, times(0)).dispatch(any(ContainerProtos.ContainerCommandRequestProto.class),
@@ -275,13 +280,7 @@ public void testWriteTimout() throws Exception {
275280
return null;
276281
}).when(dispatcher).dispatch(any(), any());
277282

278-
when(context.getRequestProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
279-
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
280-
ContainerProtos.WriteChunkRequestProto.newBuilder().setData(ByteString.copyFromUtf8("Test Data"))
281-
.setBlockID(
282-
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(1).setLocalID(1).build()).build())
283-
.setContainerID(1)
284-
.setDatanodeUuid(UUID.randomUUID().toString()).build());
283+
setUpMockRequestProto(context);
285284
AtomicReference<Throwable> throwable = new AtomicReference<>(null);
286285
Function<Throwable, ? extends Message> throwableSetter = t -> {
287286
throwable.set(t);

0 commit comments

Comments
 (0)