Skip to content

Commit f74bd2b

Browse files
committed
Refactor functions
1 parent cca89ed commit f74bd2b

File tree

1 file changed

+22
-27
lines changed

1 file changed

+22
-27
lines changed

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

+22-27
Original file line numberDiff line numberDiff line change
@@ -128,7 +128,7 @@ public void testWriteFailure(boolean failWithException) throws ExecutionExceptio
128128
when(trx.getStateMachineContext()).thenReturn(context);
129129

130130
setUpMockDispatcherReturn(failWithException);
131-
setUpMockRequestProto(context);
131+
setUpMockRequestProtoReturn(context, "Test Data", 1, 1);
132132

133133
AtomicReference<Throwable> throwable = new AtomicReference<>(null);
134134
Function<Throwable, ? extends Message> throwableSetter = getThrowableSetter(throwable);
@@ -141,13 +141,8 @@ public void testWriteFailure(boolean failWithException) throws ExecutionExceptio
141141
assertResults(failWithException, throwable);
142142

143143
// Writing data to another container(containerId 2) should also fail.
144-
when(context.getRequestProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
145-
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
146-
ContainerProtos.WriteChunkRequestProto.newBuilder().setData(ByteString.copyFromUtf8("Test Data"))
147-
.setBlockID(
148-
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(2).setLocalID(1).build()).build())
149-
.setContainerID(2)
150-
.setDatanodeUuid(UUID.randomUUID().toString()).build());
144+
setUpMockRequestProtoReturn(context, "Test Data", 2, 1);
145+
151146
stateMachine.write(entry, trx).exceptionally(throwableSetter).get();
152147
verify(dispatcher, times(0)).dispatch(any(ContainerProtos.ContainerCommandRequestProto.class),
153148
any(DispatcherContext.class));
@@ -167,22 +162,31 @@ public final void setUpMockDispatcherReturn(boolean failWithException) {
167162
}
168163
}
169164

170-
public final void setUpMockRequestProto(ContainerStateMachine.Context context) {
165+
public final void setUpMockRequestProtoReturn(ContainerStateMachine.Context context, String content,
166+
int containerId, int localId) {
171167
when(context.getRequestProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
172168
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
173-
ContainerProtos.WriteChunkRequestProto.newBuilder().setData(ByteString.copyFromUtf8("Test Data"))
169+
ContainerProtos.WriteChunkRequestProto.newBuilder().setData(ByteString.copyFromUtf8(content))
174170
.setBlockID(
175-
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(1).setLocalID(1).build()).build())
176-
.setContainerID(1)
171+
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(containerId).setLocalID(localId).build()).build())
172+
.setContainerID(containerId)
173+
.setDatanodeUuid(UUID.randomUUID().toString()).build());
174+
}
175+
176+
public final void setUpMockLogProtoReturn(ContainerStateMachine.Context context, int containerId, int localId) {
177+
when(context.getLogProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
178+
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
179+
ContainerProtos.WriteChunkRequestProto.newBuilder().setBlockID(
180+
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(containerId).setLocalID(localId).build()).build())
181+
.setContainerID(containerId)
177182
.setDatanodeUuid(UUID.randomUUID().toString()).build());
178183
}
179184

180185
public final Function<Throwable, ? extends Message> getThrowableSetter(AtomicReference<Throwable> throwable) {
181-
Function<Throwable, ? extends Message> throwableSetter = t -> {
186+
return t -> {
182187
throwable.set(t);
183188
return null;
184189
};
185-
return throwableSetter;
186190
}
187191

188192
public final void assertResults(boolean failWithException, AtomicReference<Throwable> throwable) {
@@ -209,12 +213,8 @@ public void testApplyTransactionFailure(boolean failWithException) throws Execut
209213

210214
setUpMockDispatcherReturn(failWithException);
211215
// Failing apply transaction on congtainer 1.
212-
when(context.getLogProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
213-
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
214-
ContainerProtos.WriteChunkRequestProto.newBuilder().setBlockID(
215-
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(1).setLocalID(1).build()).build())
216-
.setContainerID(1)
217-
.setDatanodeUuid(UUID.randomUUID().toString()).build());
216+
setUpMockLogProtoReturn(context, 1, 1);
217+
218218
AtomicReference<Throwable> throwable = new AtomicReference<>(null);
219219
Function<Throwable, ? extends Message> throwableSetter = getThrowableSetter(throwable);
220220
//apply transaction will fail because of runtime exception thrown by dispatcher, which marks the first
@@ -235,12 +235,7 @@ public void testApplyTransactionFailure(boolean failWithException) throws Execut
235235

236236
// Another apply transaction on a different container 2 shouldn't fail because the previous apply transaction
237237
// failure was only on container 1.
238-
when(context.getLogProto()).thenReturn(ContainerProtos.ContainerCommandRequestProto.newBuilder()
239-
.setCmdType(ContainerProtos.Type.WriteChunk).setWriteChunk(
240-
ContainerProtos.WriteChunkRequestProto.newBuilder().setBlockID(
241-
ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(2).setLocalID(1).build()).build())
242-
.setContainerID(2)
243-
.setDatanodeUuid(UUID.randomUUID().toString()).build());
238+
setUpMockLogProtoReturn(context, 2, 1);
244239

245240
reset(dispatcher);
246241
throwable.set(null);
@@ -277,7 +272,7 @@ public void testWriteTimout() throws Exception {
277272
return null;
278273
}).when(dispatcher).dispatch(any(), any());
279274

280-
setUpMockRequestProto(context);
275+
setUpMockRequestProtoReturn(context, "Test Data", 1, 1);
281276
AtomicReference<Throwable> throwable = new AtomicReference<>(null);
282277
Function<Throwable, ? extends Message> throwableSetter = getThrowableSetter(throwable);
283278
Field writeChunkWaitMaxNs = stateMachine.getClass().getDeclaredField("writeChunkWaitMaxNs");

0 commit comments

Comments
 (0)