-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HDDS-1779. TestWatchForCommit tests are flaky. #1071
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
68f9369
446cb7d
04f2adf
420cca4
e8e074e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,155 @@ | ||
| /** | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with this | ||
| * work for additional information regarding copyright ownership. The ASF | ||
| * licenses this file to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance with the License. | ||
| * You may obtain a copy of the License at | ||
| * <p> | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * <p> | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
| * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the | ||
| * License for the specific language governing permissions and limitations under | ||
| * the License. | ||
| */ | ||
|
|
||
| package org.apache.hadoop.ozone.client.rpc; | ||
|
|
||
| import org.apache.hadoop.conf.StorageUnit; | ||
| import org.apache.hadoop.hdds.conf.OzoneConfiguration; | ||
| import org.apache.hadoop.hdds.protocol.proto.HddsProtos; | ||
| import org.apache.hadoop.hdds.scm.XceiverClientManager; | ||
| import org.apache.hadoop.hdds.scm.XceiverClientRatis; | ||
| import org.apache.hadoop.hdds.scm.XceiverClientReply; | ||
| import org.apache.hadoop.hdds.scm.XceiverClientSpi; | ||
| import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; | ||
| import org.apache.hadoop.hdds.scm.pipeline.Pipeline; | ||
| import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; | ||
| import org.apache.hadoop.ozone.MiniOzoneCluster; | ||
| import org.apache.hadoop.ozone.OzoneConfigKeys; | ||
| import org.apache.hadoop.ozone.client.ObjectStore; | ||
| import org.apache.hadoop.ozone.client.OzoneClient; | ||
| import org.apache.hadoop.ozone.client.OzoneClientFactory; | ||
| import org.apache.hadoop.ozone.container.ContainerTestHelper; | ||
| import org.apache.hadoop.test.GenericTestUtils; | ||
| import org.junit.Assert; | ||
| import org.junit.Test; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.UUID; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT; | ||
|
|
||
| // This class tests the 2 way | ||
| public class Test2WayCommitInRatis { | ||
|
|
||
| private MiniOzoneCluster cluster; | ||
| private OzoneClient client; | ||
| private ObjectStore objectStore; | ||
| private String volumeName; | ||
| private String bucketName; | ||
| private int chunkSize; | ||
| private int flushSize; | ||
| private int maxFlushSize; | ||
| private int blockSize; | ||
| private StorageContainerLocationProtocolClientSideTranslatorPB | ||
| storageContainerLocationClient; | ||
| private static String containerOwner = "OZONE"; | ||
|
|
||
| /** | ||
| * Create a MiniDFSCluster for testing. | ||
| * <p> | ||
| * Ozone is made active by setting OZONE_ENABLED = true | ||
| * | ||
| * @throws IOException | ||
| */ | ||
| private void startCluster(OzoneConfiguration conf) throws Exception { | ||
| chunkSize = 100; | ||
| flushSize = 2 * chunkSize; | ||
| maxFlushSize = 2 * flushSize; | ||
| blockSize = 2 * maxFlushSize; | ||
|
|
||
| conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS); | ||
| conf.setTimeDuration( | ||
| OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY, | ||
| 1, TimeUnit.SECONDS); | ||
|
|
||
| conf.setQuietMode(false); | ||
| cluster = MiniOzoneCluster.newBuilder(conf) | ||
| .setNumDatanodes(7) | ||
| .setBlockSize(blockSize) | ||
| .setChunkSize(chunkSize) | ||
| .setStreamBufferFlushSize(flushSize) | ||
| .setStreamBufferMaxSize(maxFlushSize) | ||
| .setStreamBufferSizeUnit(StorageUnit.BYTES) | ||
| .build(); | ||
| cluster.waitForClusterToBeReady(); | ||
| //the easiest way to create an open container is creating a key | ||
| client = OzoneClientFactory.getClient(conf); | ||
| objectStore = client.getObjectStore(); | ||
| volumeName = "watchforcommithandlingtest"; | ||
| bucketName = volumeName; | ||
| objectStore.createVolume(volumeName); | ||
| objectStore.getVolume(volumeName).createBucket(bucketName); | ||
| storageContainerLocationClient = cluster | ||
| .getStorageContainerLocationClient(); | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * Shutdown MiniDFSCluster. | ||
| */ | ||
| private void shutdown() { | ||
| if (cluster != null) { | ||
| cluster.shutdown(); | ||
| } | ||
| } | ||
|
|
||
|
|
||
| @Test | ||
| public void test2WayCommitForRetryfailure() throws Exception { | ||
| OzoneConfiguration conf = new OzoneConfiguration(); | ||
| conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 20, | ||
| TimeUnit.SECONDS); | ||
| conf.setInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY, 8); | ||
| startCluster(conf); | ||
| GenericTestUtils.LogCapturer logCapturer = | ||
| GenericTestUtils.LogCapturer.captureLogs(XceiverClientRatis.LOG); | ||
| XceiverClientManager clientManager = new XceiverClientManager(conf); | ||
|
|
||
| ContainerWithPipeline container1 = storageContainerLocationClient | ||
| .allocateContainer(HddsProtos.ReplicationType.RATIS, | ||
| HddsProtos.ReplicationFactor.THREE, containerOwner); | ||
| XceiverClientSpi xceiverClient = clientManager | ||
| .acquireClient(container1.getPipeline()); | ||
| Assert.assertEquals(1, xceiverClient.getRefcount()); | ||
| Assert.assertEquals(container1.getPipeline(), | ||
| xceiverClient.getPipeline()); | ||
| Pipeline pipeline = xceiverClient.getPipeline(); | ||
| XceiverClientRatis ratisClient = (XceiverClientRatis) xceiverClient; | ||
| XceiverClientReply reply = xceiverClient.sendCommandAsync( | ||
| ContainerTestHelper.getCreateContainerRequest( | ||
| container1.getContainerInfo().getContainerID(), | ||
| xceiverClient.getPipeline())); | ||
| reply.getResponse().get(); | ||
| Assert.assertEquals(3, ratisClient.getCommitInfoMap().size()); | ||
| cluster.shutdownHddsDatanode(pipeline.getNodes().get(0)); | ||
| reply = xceiverClient.sendCommandAsync(ContainerTestHelper | ||
| .getCloseContainer(pipeline, | ||
| container1.getContainerInfo().getContainerID())); | ||
| reply.getResponse().get(); | ||
| xceiverClient.watchForCommit(reply.getLogIndex(), 20000); | ||
|
|
||
| // commitInfo Map will be reduced to 2 here | ||
| Assert.assertEquals(2, ratisClient.getCommitInfoMap().size()); | ||
| clientManager.releaseClient(xceiverClient, false); | ||
| Assert.assertTrue(logCapturer.getOutput().contains("3 way commit failed")); | ||
| Assert | ||
| .assertTrue(logCapturer.getOutput().contains("Committed by majority")); | ||
| logCapturer.stopCapturing(); | ||
| shutdown(); | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -46,7 +46,9 @@ | |
| import java.io.OutputStream; | ||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
| import java.util.Random; | ||
| import java.util.UUID; | ||
| import java.util.concurrent.ExecutionException; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.TimeoutException; | ||
|
|
||
|
|
@@ -303,10 +305,14 @@ public void testWatchForCommitWithSmallerTimeoutValue() throws Exception { | |
| cluster.shutdownHddsDatanode(pipeline.getNodes().get(0)); | ||
| cluster.shutdownHddsDatanode(pipeline.getNodes().get(1)); | ||
| try { | ||
| // just watch for a lo index which in not updated in the commitInfo Map | ||
| xceiverClient.watchForCommit(index + 1, 3000); | ||
| // just watch for a log index which in not updated in the commitInfo Map | ||
| // as well as there is no logIndex generate in Ratis. | ||
| // The basic idea here is just to test if its throws an exception. | ||
| xceiverClient | ||
| .watchForCommit(index + new Random().nextInt(100) + 10, 3000); | ||
| Assert.fail("expected exception not thrown"); | ||
| } catch (Exception e) { | ||
| System.out.println("exception " + e); | ||
| Assert.assertTrue( | ||
| HddsClientUtils.checkForException(e) instanceof TimeoutException); | ||
| } | ||
|
|
@@ -343,61 +349,24 @@ public void testWatchForCommitForRetryfailure() throws Exception { | |
| cluster.shutdownHddsDatanode(pipeline.getNodes().get(1)); | ||
| // again write data with more than max buffer limit. This wi | ||
| try { | ||
| // just watch for a lo index which in not updated in the commitInfo Map | ||
| xceiverClient.watchForCommit(index + 1, 20000); | ||
| // just watch for a log index which in not updated in the commitInfo Map | ||
| // as well as there is no logIndex generate in Ratis. | ||
| // The basic idea here is just to test if its throws an exception. | ||
| xceiverClient | ||
| .watchForCommit(index + new Random().nextInt(100) + 10, 20000); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. instead of a Random increment, why not increment by a fixed number everytime - say 100 or 110? This applies to all the other modified test cases as well.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The idea here is to run the test each time with unique number so any possible hacks/errors get caught if any. |
||
| Assert.fail("expected exception not thrown"); | ||
| } catch (Exception e) { | ||
| Assert.assertTrue(HddsClientUtils | ||
| .checkForException(e) instanceof RaftRetryFailureException); | ||
| Assert.assertTrue(e instanceof ExecutionException); | ||
| // since the timeout value is quite long, the watch request will either | ||
| // fail with NotReplicated exceptio, RetryFailureException or | ||
| // RuntimeException | ||
| Assert.assertFalse(HddsClientUtils | ||
| .checkForException(e) instanceof TimeoutException); | ||
| } | ||
| clientManager.releaseClient(xceiverClient, false); | ||
| shutdown(); | ||
| } | ||
|
|
||
| @Test | ||
| public void test2WayCommitForRetryfailure() throws Exception { | ||
| OzoneConfiguration conf = new OzoneConfiguration(); | ||
| conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 20, | ||
| TimeUnit.SECONDS); | ||
| conf.setInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY, 8); | ||
| startCluster(conf); | ||
| GenericTestUtils.LogCapturer logCapturer = | ||
| GenericTestUtils.LogCapturer.captureLogs(XceiverClientRatis.LOG); | ||
| XceiverClientManager clientManager = new XceiverClientManager(conf); | ||
|
|
||
| ContainerWithPipeline container1 = storageContainerLocationClient | ||
| .allocateContainer(HddsProtos.ReplicationType.RATIS, | ||
| HddsProtos.ReplicationFactor.THREE, containerOwner); | ||
| XceiverClientSpi xceiverClient = clientManager | ||
| .acquireClient(container1.getPipeline()); | ||
| Assert.assertEquals(1, xceiverClient.getRefcount()); | ||
| Assert.assertEquals(container1.getPipeline(), | ||
| xceiverClient.getPipeline()); | ||
| Pipeline pipeline = xceiverClient.getPipeline(); | ||
| XceiverClientRatis ratisClient = (XceiverClientRatis) xceiverClient; | ||
| XceiverClientReply reply = xceiverClient.sendCommandAsync( | ||
| ContainerTestHelper.getCreateContainerRequest( | ||
| container1.getContainerInfo().getContainerID(), | ||
| xceiverClient.getPipeline())); | ||
| reply.getResponse().get(); | ||
| Assert.assertEquals(3, ratisClient.getCommitInfoMap().size()); | ||
| cluster.shutdownHddsDatanode(pipeline.getNodes().get(0)); | ||
| reply = xceiverClient.sendCommandAsync(ContainerTestHelper | ||
| .getCloseContainer(pipeline, | ||
| container1.getContainerInfo().getContainerID())); | ||
| reply.getResponse().get(); | ||
| xceiverClient.watchForCommit(reply.getLogIndex(), 20000); | ||
|
|
||
| // commitInfo Map will be reduced to 2 here | ||
| Assert.assertEquals(2, ratisClient.getCommitInfoMap().size()); | ||
| clientManager.releaseClient(xceiverClient, false); | ||
| Assert.assertTrue(logCapturer.getOutput().contains("3 way commit failed")); | ||
| Assert | ||
| .assertTrue(logCapturer.getOutput().contains("Committed by majority")); | ||
| logCapturer.stopCapturing(); | ||
| shutdown(); | ||
| } | ||
|
|
||
| @Test | ||
| public void test2WayCommitForTimeoutException() throws Exception { | ||
| OzoneConfiguration conf = new OzoneConfiguration(); | ||
|
|
@@ -477,8 +446,12 @@ public void testWatchForCommitForGroupMismatchException() throws Exception { | |
| pipelineList.add(pipeline); | ||
| ContainerTestHelper.waitForPipelineClose(pipelineList, cluster); | ||
| try { | ||
| // just watch for a lo index which in not updated in the commitInfo Map | ||
| xceiverClient.watchForCommit(reply.getLogIndex() + 1, 20000); | ||
| // just watch for a log index which in not updated in the commitInfo Map | ||
| // as well as there is no logIndex generate in Ratis. | ||
| // The basic idea here is just to test if its throws an exception. | ||
| xceiverClient | ||
| .watchForCommit(reply.getLogIndex() + new Random().nextInt(100) + 10, | ||
| 20000); | ||
| Assert.fail("Expected exception not thrown"); | ||
| } catch(Exception e) { | ||
| Assert.assertTrue(HddsClientUtils | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.