Skip to content

Commit 4d837b0

Browse files
committed
Fixed syntax
1 parent 4a23d3a commit 4d837b0

File tree

5 files changed

+14
-14
lines changed

5 files changed

+14
-14
lines changed

hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,7 @@ final class DummyBlockInputStreamWithRetry
6363
try {
6464
BlockLocationInfo blockLocationInfo = mock(BlockLocationInfo.class);
6565
Pipeline mockPipeline = MockPipeline.createPipeline(1);
66-
doReturn(mockPipeline).when(blockLocationInfo.getPipeline());
66+
doReturn(mockPipeline).when(blockLocationInfo).getPipeline();
6767
return blockLocationInfo;
6868
} catch (IOException e) {
6969
throw new RuntimeException(e);

hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java

+7-7
Original file line numberDiff line numberDiff line change
@@ -297,18 +297,18 @@ void refreshesPipelineOnReadFailure(IOException ex) throws Exception {
297297
// GIVEN
298298
Pipeline pipeline = MockPipeline.createSingleNodePipeline();
299299
BlockLocationInfo blockLocationInfo = mock(BlockLocationInfo.class);
300-
doReturn(pipeline).when(blockLocationInfo.getPipeline());
300+
doReturn(pipeline).when(blockLocationInfo).getPipeline();
301301
Pipeline newPipeline = MockPipeline.createSingleNodePipeline();
302302
BlockLocationInfo newBlockLocationInfo = mock(BlockLocationInfo.class);
303303

304304
testRefreshesPipelineOnReadFailure(ex, blockLocationInfo,
305305
id -> newBlockLocationInfo);
306306

307-
doReturn(newPipeline).when(newBlockLocationInfo.getPipeline());
307+
doReturn(newPipeline).when(newBlockLocationInfo).getPipeline();
308308
testRefreshesPipelineOnReadFailure(ex, blockLocationInfo,
309309
id -> blockLocationInfo);
310310

311-
doReturn(null).when(newBlockLocationInfo.getPipeline());
311+
doReturn(null).when(newBlockLocationInfo).getPipeline();
312312
testRefreshesPipelineOnReadFailure(ex, blockLocationInfo,
313313
id -> newBlockLocationInfo);
314314
}
@@ -359,7 +359,7 @@ private static ChunkInputStream throwingChunkInputStream(IOException ex,
359359
if (succeedOnRetry) {
360360
stubbing.thenReturn(len);
361361
}
362-
doReturn((long) len).when(stream.getRemaining());
362+
doReturn((long) len).when(stream).getRemaining();
363363
return stream;
364364
}
365365

@@ -415,13 +415,13 @@ public void testRefreshOnReadFailureAfterUnbuffer(IOException ex)
415415
XceiverClientFactory clientFactory = mock(XceiverClientFactory.class);
416416
XceiverClientSpi client = mock(XceiverClientSpi.class);
417417
BlockLocationInfo blockLocationInfo = mock(BlockLocationInfo.class);
418-
doReturn(client).when(clientFactory.acquireClientForReadData(pipeline));
418+
doReturn(client).when(clientFactory).acquireClientForReadData(pipeline);
419419

420420
final int len = 200;
421421
final ChunkInputStream stream = throwingChunkInputStream(ex, len, true);
422422

423-
doReturn(blockLocationInfo).when(refreshFunction.apply(blockID));
424-
doReturn(newPipeline).when(blockLocationInfo.getPipeline());
423+
doReturn(blockLocationInfo).when(refreshFunction).apply(blockID);
424+
doReturn(newPipeline).when(blockLocationInfo).getPipeline();
425425

426426
OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
427427
clientConfig.setChecksumVerify(false);

hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -157,7 +157,7 @@ private BlockOutputStream createBlockOutputStream(BufferPool bufferPool)
157157
final Pipeline pipeline = MockPipeline.createRatisPipeline();
158158

159159
final XceiverClientManager xcm = mock(XceiverClientManager.class);
160-
doReturn(new MockXceiverClientSpi(pipeline)).when(xcm.acquireClient(any()));
160+
doReturn(new MockXceiverClientSpi(pipeline)).when(xcm).acquireClient(any());
161161

162162
OzoneClientConfig config = new OzoneClientConfig();
163163
config.setStreamBufferSize(4 * 1024 * 1024);
@@ -185,7 +185,7 @@ private BlockOutputStream createBlockOutputStream(BufferPool bufferPool)
185185
private ECBlockOutputStream createECBlockOutputStream(OzoneClientConfig clientConfig,
186186
ECReplicationConfig repConfig, BlockID blockID, Pipeline pipeline) throws IOException {
187187
final XceiverClientManager xcm = mock(XceiverClientManager.class);
188-
doReturn(new MockXceiverClientSpi(pipeline)).when(xcm.acquireClient(any()));
188+
doReturn(new MockXceiverClientSpi(pipeline)).when(xcm).acquireClient(any());
189189

190190
ContainerClientMetrics clientMetrics = ContainerClientMetrics.acquire();
191191
StreamBufferArgs streamBufferArgs =

hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestChunkInputStream.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -245,16 +245,16 @@ public void connectsToNewPipeline() throws Exception {
245245
Pipeline newPipeline = MockPipeline.createSingleNodePipeline();
246246

247247
Token<?> token = mock(Token.class);
248-
doReturn("oldToken").when(token.encodeToUrlString());
248+
doReturn("oldToken").when(token).encodeToUrlString();
249249
Token<?> newToken = mock(Token.class);
250-
doReturn("newToken").when(newToken.encodeToUrlString());
250+
doReturn("newToken").when(newToken).encodeToUrlString();
251251

252252
AtomicReference<Pipeline> pipelineRef = new AtomicReference<>(pipeline);
253253
AtomicReference<Token<?>> tokenRef = new AtomicReference<>(token);
254254

255255
XceiverClientFactory clientFactory = mock(XceiverClientFactory.class);
256256
XceiverClientSpi client = mock(XceiverClientSpi.class);
257-
doReturn(client).when(clientFactory.acquireClientForReadData(any()));
257+
doReturn(client).when(clientFactory).acquireClientForReadData(any());
258258
ArgumentCaptor<ContainerCommandRequestProto> requestCaptor =
259259
ArgumentCaptor.forClass(ContainerCommandRequestProto.class);
260260
when(client.getPipeline())

hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -60,7 +60,7 @@ public void testNonECGivesBlockInputStream() throws IOException {
6060
1024 * 1024 * 10);
6161
Pipeline pipeline = Mockito.spy(blockInfo.getPipeline());
6262
blockInfo.setPipeline(pipeline);
63-
doReturn(1).when(pipeline.getReplicaIndex(any(DatanodeDetails.class)));
63+
doReturn(1).when(pipeline).getReplicaIndex(any(DatanodeDetails.class));
6464
OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
6565
clientConfig.setChecksumVerify(true);
6666
BlockExtendedInputStream stream =

0 commit comments

Comments
 (0)