DonalEvans commented on a change in pull request #6909:
URL: https://github.com/apache/geode/pull/6909#discussion_r728344219



##########
File path: 
geode-core/src/test/java/org/apache/geode/internal/cache/PRHARedundancyProviderTest.java
##########
@@ -285,6 +285,25 @@ public void 
startTaskCompletesExceptionallyIfExceptionIsThrown(
     verify(providerStartupTask).completeExceptionally(exception);
   }
 
+  @Test
+  public void reportsScheduleCreateMissingBuckets() {

Review comment:
       This test would be better named 
"scheduleCreateMissingBucketsExecutesCreateMissingBucketsTask"

##########
File path: 
geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java
##########
@@ -58,4 +74,44 @@ protected void createMissingBuckets(PartitionedRegion 
region) {
       }
     }
   }
+
+  /**
+   * Wait for Colocation to complete. Wait all nodes to Register this 
PartitionedRegion.
+   */
+  protected boolean waitForColocationCompleted(PartitionedRegion 
partitionedRegion) {
+    int retryCount = 0;
+    int sleepInterval = 
PartitionedRegionHelper.DEFAULT_WAIT_PER_RETRY_ITERATION;
+
+    while (!ColocationHelper.isColocationComplete(partitionedRegion)
+        && (retryCount < MAX_NUMBER_INTERVALS)) {
+
+      // Didn't time out. Sleep a bit and then continue
+      boolean interrupted = Thread.interrupted();
+      try {
+        Thread.sleep(sleepInterval);

Review comment:
       It might be good to add some logging here (info level would probably be 
okay, since this isn't something that should be executed repeatedly in quick 
succession) so that a user can see how long they've been waiting for colocation 
to complete, and understand why they might see a thread stuck here for over 2 
minutes.

##########
File path: 
geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTaskTest.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.geode.internal.cache.partitioned;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+import org.mockito.quality.Strictness;
+
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.PRHARedundancyProvider;
+import org.apache.geode.internal.cache.PartitionRegionConfig;
+import org.apache.geode.internal.cache.PartitionedRegion;
+
+public class CreateMissingBucketsTaskTest {
+
+  private CreateMissingBucketsTask task;
+  private PRHARedundancyProvider prhaRedundancyProvider;
+
+  private PartitionedRegion partitionedRegion;
+  private PartitionRegionConfig partitionRegionConfig;
+  private DistributedRegion prRoot;
+
+  @Rule
+  public MockitoRule mockitoRule = 
MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
+
+
+  @Before
+  public void setup() {
+
+    prhaRedundancyProvider = mock(PRHARedundancyProvider.class);
+
+    task = new CreateMissingBucketsTask(prhaRedundancyProvider);
+
+    InternalCache cache = mock(InternalCache.class);
+
+    partitionRegionConfig = mock(PartitionRegionConfig.class);
+    partitionedRegion = mock(PartitionedRegion.class);
+    prRoot = mock(DistributedRegion.class);
+
+    when(cache.getRegion(anyString(), anyBoolean())).thenReturn(prRoot);
+    when(partitionedRegion.getCache()).thenReturn(cache);
+
+    when(prRoot.get(any())).thenReturn(partitionRegionConfig);
+
+  }
+
+  @Test
+  public void testWaitForColocationIsCompleted() {
+    when(partitionRegionConfig.isColocationComplete()).thenReturn(true);
+    assertThat(task.waitForColocationCompleted(partitionedRegion)).isTrue();
+  }
+
+  @Test
+  public void testWaitForColocationNotCompleted() {
+    when(partitionRegionConfig.isColocationComplete()).thenReturn(false);
+    assertThat(task.waitForColocationCompleted(partitionedRegion)).isFalse();
+  }
+
+  @Test
+  public void testCreateMissingBuckets() {
+    PartitionedRegion leaderRegion = mock(PartitionedRegion.class);
+    PartitionedRegion.getPrIdToPR().put(1, leaderRegion);
+
+    PartitionAttributes partitionAttributes = mock(PartitionAttributes.class);
+    
when(partitionedRegion.getPartitionAttributes()).thenReturn(partitionAttributes);
+    when(partitionAttributes.getColocatedWith()).thenReturn("region2");
+    when(partitionRegionConfig.getPRId()).thenReturn(1);
+
+    PartitionAttributes leaderPartitionAttributes = 
mock(PartitionAttributes.class);
+    
when(leaderRegion.getPartitionAttributes()).thenReturn(leaderPartitionAttributes);
+    when(leaderPartitionAttributes.getColocatedWith()).thenReturn(null);
+
+    task.createMissingBuckets(partitionedRegion);
+
+    verify(partitionedRegion).getTotalNumberOfBuckets();

Review comment:
       This test should be doing some better validation of the expected 
behaviour here. We don't really care that `createMissingBuckets()` causes the 
partition region to call `getTotalNumberOfBuckets()`, but rather that it 
results in a call to `createBucketAtomically()` for buckets whose redundancy 
doesn't match that of the parent region.
   
   It would be good to modify the test to have two buckets, one of which causes 
the check below to evaluate to true, and one that evaluates to false, and 
assert that we call `createBucketAtomically()` only for the one that evaluates 
to true.
   ```
   parentRegion.getRegionAdvisor().getBucketAdvisor(i).getBucketRedundancy() != 
region
             .getRegionAdvisor().getBucketAdvisor(i).getBucketRedundancy()
   ```

##########
File path: 
geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTaskTest.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.geode.internal.cache.partitioned;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+import org.mockito.quality.Strictness;
+
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.PRHARedundancyProvider;
+import org.apache.geode.internal.cache.PartitionRegionConfig;
+import org.apache.geode.internal.cache.PartitionedRegion;
+
+public class CreateMissingBucketsTaskTest {
+
+  private CreateMissingBucketsTask task;
+  private PRHARedundancyProvider prhaRedundancyProvider;
+
+  private PartitionedRegion partitionedRegion;
+  private PartitionRegionConfig partitionRegionConfig;
+  private DistributedRegion prRoot;
+
+  @Rule
+  public MockitoRule mockitoRule = 
MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
+
+
+  @Before
+  public void setup() {
+
+    prhaRedundancyProvider = mock(PRHARedundancyProvider.class);
+
+    task = new CreateMissingBucketsTask(prhaRedundancyProvider);
+
+    InternalCache cache = mock(InternalCache.class);
+
+    partitionRegionConfig = mock(PartitionRegionConfig.class);
+    partitionedRegion = mock(PartitionedRegion.class);
+    prRoot = mock(DistributedRegion.class);
+
+    when(cache.getRegion(anyString(), anyBoolean())).thenReturn(prRoot);
+    when(partitionedRegion.getCache()).thenReturn(cache);
+
+    when(prRoot.get(any())).thenReturn(partitionRegionConfig);
+
+  }
+
+  @Test
+  public void testWaitForColocationIsCompleted() {
+    when(partitionRegionConfig.isColocationComplete()).thenReturn(true);
+    assertThat(task.waitForColocationCompleted(partitionedRegion)).isTrue();
+  }
+
+  @Test
+  public void testWaitForColocationNotCompleted() {
+    when(partitionRegionConfig.isColocationComplete()).thenReturn(false);
+    assertThat(task.waitForColocationCompleted(partitionedRegion)).isFalse();
+  }
+
+  @Test
+  public void testCreateMissingBuckets() {
+    PartitionedRegion leaderRegion = mock(PartitionedRegion.class);
+    PartitionedRegion.getPrIdToPR().put(1, leaderRegion);
+
+    PartitionAttributes partitionAttributes = mock(PartitionAttributes.class);
+    
when(partitionedRegion.getPartitionAttributes()).thenReturn(partitionAttributes);
+    when(partitionAttributes.getColocatedWith()).thenReturn("region2");
+    when(partitionRegionConfig.getPRId()).thenReturn(1);
+
+    PartitionAttributes leaderPartitionAttributes = 
mock(PartitionAttributes.class);
+    
when(leaderRegion.getPartitionAttributes()).thenReturn(leaderPartitionAttributes);
+    when(leaderPartitionAttributes.getColocatedWith()).thenReturn(null);
+
+    task.createMissingBuckets(partitionedRegion);
+
+    verify(partitionedRegion).getTotalNumberOfBuckets();
+  }
+
+  @Test
+  public void testTaskRun() {
+
+    
when(prhaRedundancyProvider.getPartitionedRegion()).thenReturn(partitionedRegion);
+    when(partitionRegionConfig.isColocationComplete()).thenReturn(true);
+
+    PartitionedRegion leaderRegion = mock(PartitionedRegion.class);
+    PartitionedRegion.getPrIdToPR().put(1, leaderRegion);
+
+    PartitionAttributes partitionAttributes = mock(PartitionAttributes.class);
+    
when(partitionedRegion.getPartitionAttributes()).thenReturn(partitionAttributes);
+    when(partitionAttributes.getColocatedWith()).thenReturn("region2");
+    when(partitionRegionConfig.getPRId()).thenReturn(1);
+
+    PartitionAttributes leaderPartitionAttributes = 
mock(PartitionAttributes.class);
+    
when(leaderRegion.getPartitionAttributes()).thenReturn(leaderPartitionAttributes);
+    when(leaderPartitionAttributes.getColocatedWith()).thenReturn(null);
+
+    PartitionedRegion.RecoveryLock lock = 
mock(PartitionedRegion.RecoveryLock.class);
+
+    when(leaderRegion.getRecoveryLock()).thenReturn(lock);
+
+    task.run2();
+
+    verify(partitionedRegion).getTotalNumberOfBuckets();

Review comment:
       This test needs better validation of the expected behaviour, and some 
more test cases for different scenarios. There should be test cases that assert 
the following: 
    - That if `waitForColocationCompleted()` returns false, we do nothing (do 
not lock the recovery lock or call `createMissingBuckets()`
    - That if `waitForColocationCompleted()` returns true, we lock the recovery 
lock and call `createMissingBuckets()`, then unlock the recovery lock
    - That we both lock and unlock the recovery lock if an exception is thrown 
from `createMissingBuckets()`

##########
File path: 
geode-core/src/test/java/org/apache/geode/internal/cache/PRHARedundancyProviderTest.java
##########
@@ -285,6 +285,25 @@ public void 
startTaskCompletesExceptionallyIfExceptionIsThrown(
     verify(providerStartupTask).completeExceptionally(exception);
   }
 
+  @Test
+  public void reportsScheduleCreateMissingBuckets() {
+    CompletableFuture<Void> providerStartupTask = 
mock(CompletableFuture.class);
+    when(partitionedRegion.getColocatedWith()).thenReturn("leaderRegion");
+    when(partitionedRegion.getGemFireCache()).thenReturn(cache);
+    when(cache.getInternalResourceManager()).thenReturn(resourceManager);
+
+    ScheduledExecutorService executorService = 
mock(ScheduledExecutorService.class);
+    when(resourceManager.getExecutor()).thenReturn(executorService);
+
+    prHaRedundancyProvider = new PRHARedundancyProvider(partitionedRegion, 
resourceManager,
+        (a, b) -> mock(PersistentBucketRecoverer.class),
+        PRHARedundancyProviderTest::createRebalanceOp, providerStartupTask);
+
+    prHaRedundancyProvider.scheduleCreateMissingBuckets();
+
+    verify(executorService).execute(any());

Review comment:
       This verification can be strengthened to 
`verify(executorService).execute(any(CreateMissingBucketsTask.class));`

##########
File path: 
geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java
##########
@@ -58,4 +74,44 @@ protected void createMissingBuckets(PartitionedRegion 
region) {
       }
     }
   }
+
+  /**
+   * Wait for Colocation to complete. Wait all nodes to Register this 
PartitionedRegion.
+   */
+  protected boolean waitForColocationCompleted(PartitionedRegion 
partitionedRegion) {
+    int retryCount = 0;
+    int sleepInterval = 
PartitionedRegionHelper.DEFAULT_WAIT_PER_RETRY_ITERATION;
+
+    while (!ColocationHelper.isColocationComplete(partitionedRegion)
+        && (retryCount < MAX_NUMBER_INTERVALS)) {
+
+      // Didn't time out. Sleep a bit and then continue
+      boolean interrupted = Thread.interrupted();
+      try {
+        Thread.sleep(sleepInterval);
+      } catch (InterruptedException ignore) {
+        interrupted = true;
+      } finally {
+        if (interrupted) {
+          Thread.currentThread().interrupt();
+        }
+      }
+
+      retryCount++;
+      if (retryCount == SMALL_200MS_INTERVALS) {
+        sleepInterval = 2 * 
PartitionedRegionHelper.DEFAULT_WAIT_PER_RETRY_ITERATION;
+      } else if (retryCount == SMALL_500MS_INTERVALS) {
+        sleepInterval = 5 * 
PartitionedRegionHelper.DEFAULT_WAIT_PER_RETRY_ITERATION;
+      } else if (retryCount == MEDIUM_1SEC_INTERVALS) {
+        sleepInterval = 10 * 
PartitionedRegionHelper.DEFAULT_WAIT_PER_RETRY_ITERATION;
+      } else if (retryCount == MEDIUM_2SEC_INTERVALS) {
+        sleepInterval = 20 * 
PartitionedRegionHelper.DEFAULT_WAIT_PER_RETRY_ITERATION;
+      } else if (retryCount == LARGE_5SEC_INTERVALS) {
+        sleepInterval = 50 * 
PartitionedRegionHelper.DEFAULT_WAIT_PER_RETRY_ITERATION;
+      }
+
+    }
+    return ColocationHelper.isColocationComplete(partitionedRegion);

Review comment:
       It seems like we should be doing something other than just returning 
`false` here, in the event that we wait for the full timeout (over 2 minutes) 
and colocation is still not complete, since the `CreateMissingBucketsTask` will 
just silently return with no indication that it wasn't successful.

##########
File path: 
geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTaskTest.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.geode.internal.cache.partitioned;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+import org.mockito.quality.Strictness;
+
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.PRHARedundancyProvider;
+import org.apache.geode.internal.cache.PartitionRegionConfig;
+import org.apache.geode.internal.cache.PartitionedRegion;
+
+public class CreateMissingBucketsTaskTest {
+
+  private CreateMissingBucketsTask task;
+  private PRHARedundancyProvider prhaRedundancyProvider;
+
+  private PartitionedRegion partitionedRegion;
+  private PartitionRegionConfig partitionRegionConfig;
+  private DistributedRegion prRoot;
+
+  @Rule
+  public MockitoRule mockitoRule = 
MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
+
+
+  @Before
+  public void setup() {
+
+    prhaRedundancyProvider = mock(PRHARedundancyProvider.class);
+
+    task = new CreateMissingBucketsTask(prhaRedundancyProvider);
+
+    InternalCache cache = mock(InternalCache.class);
+
+    partitionRegionConfig = mock(PartitionRegionConfig.class);
+    partitionedRegion = mock(PartitionedRegion.class);
+    prRoot = mock(DistributedRegion.class);
+
+    when(cache.getRegion(anyString(), anyBoolean())).thenReturn(prRoot);
+    when(partitionedRegion.getCache()).thenReturn(cache);
+
+    when(prRoot.get(any())).thenReturn(partitionRegionConfig);
+
+  }
+
+  @Test
+  public void testWaitForColocationIsCompleted() {
+    when(partitionRegionConfig.isColocationComplete()).thenReturn(true);
+    assertThat(task.waitForColocationCompleted(partitionedRegion)).isTrue();
+  }
+
+  @Test
+  public void testWaitForColocationNotCompleted() {

Review comment:
       This test would be better named 
"waitForColocationCompletedReturnsFalseWhenColocationIsNotCompleteAfterTimeout".
 It might also be good to add an assertion around the number of calls to 
`isColocationComplete()` to show that we're not exceeding 
`MAX_NUMBER_INTERVALS`.

##########
File path: 
geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTaskTest.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.geode.internal.cache.partitioned;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+import org.mockito.quality.Strictness;
+
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.PRHARedundancyProvider;
+import org.apache.geode.internal.cache.PartitionRegionConfig;
+import org.apache.geode.internal.cache.PartitionedRegion;
+
+public class CreateMissingBucketsTaskTest {
+
+  private CreateMissingBucketsTask task;
+  private PRHARedundancyProvider prhaRedundancyProvider;
+
+  private PartitionedRegion partitionedRegion;
+  private PartitionRegionConfig partitionRegionConfig;
+  private DistributedRegion prRoot;
+
+  @Rule
+  public MockitoRule mockitoRule = 
MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
+
+
+  @Before
+  public void setup() {
+
+    prhaRedundancyProvider = mock(PRHARedundancyProvider.class);
+
+    task = new CreateMissingBucketsTask(prhaRedundancyProvider);
+
+    InternalCache cache = mock(InternalCache.class);
+
+    partitionRegionConfig = mock(PartitionRegionConfig.class);
+    partitionedRegion = mock(PartitionedRegion.class);
+    prRoot = mock(DistributedRegion.class);
+
+    when(cache.getRegion(anyString(), anyBoolean())).thenReturn(prRoot);
+    when(partitionedRegion.getCache()).thenReturn(cache);
+
+    when(prRoot.get(any())).thenReturn(partitionRegionConfig);
+
+  }
+
+  @Test
+  public void testWaitForColocationIsCompleted() {
+    when(partitionRegionConfig.isColocationComplete()).thenReturn(true);
+    assertThat(task.waitForColocationCompleted(partitionedRegion)).isTrue();
+  }
+
+  @Test
+  public void testWaitForColocationNotCompleted() {
+    when(partitionRegionConfig.isColocationComplete()).thenReturn(false);
+    assertThat(task.waitForColocationCompleted(partitionedRegion)).isFalse();
+  }
+
+  @Test
+  public void testCreateMissingBuckets() {

Review comment:
       This test would be better named 
"createMissingBucketsCreatesBucketsAtomically"

##########
File path: 
geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTaskTest.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.geode.internal.cache.partitioned;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+import org.mockito.quality.Strictness;
+
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.PRHARedundancyProvider;
+import org.apache.geode.internal.cache.PartitionRegionConfig;
+import org.apache.geode.internal.cache.PartitionedRegion;
+
+public class CreateMissingBucketsTaskTest {
+
+  private CreateMissingBucketsTask task;
+  private PRHARedundancyProvider prhaRedundancyProvider;
+
+  private PartitionedRegion partitionedRegion;
+  private PartitionRegionConfig partitionRegionConfig;
+  private DistributedRegion prRoot;
+
+  @Rule
+  public MockitoRule mockitoRule = 
MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
+
+
+  @Before
+  public void setup() {
+
+    prhaRedundancyProvider = mock(PRHARedundancyProvider.class);
+
+    task = new CreateMissingBucketsTask(prhaRedundancyProvider);
+
+    InternalCache cache = mock(InternalCache.class);
+
+    partitionRegionConfig = mock(PartitionRegionConfig.class);
+    partitionedRegion = mock(PartitionedRegion.class);
+    prRoot = mock(DistributedRegion.class);
+
+    when(cache.getRegion(anyString(), anyBoolean())).thenReturn(prRoot);
+    when(partitionedRegion.getCache()).thenReturn(cache);
+
+    when(prRoot.get(any())).thenReturn(partitionRegionConfig);
+
+  }
+
+  @Test
+  public void testWaitForColocationIsCompleted() {

Review comment:
       This test would be better named 
"waitForColocationCompletedReturnsTrueWhenColocationIsComplete". It would also 
be good to add a separate test where `isColocationComplete()` returns false 
initially, then true, to show that the class behaves as expected when 
colocation is not initially complete, but does complete before the timeout.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to