You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/10/21 22:36:33 UTC

[GitHub] [geode] demery-pivotal opened a new pull request #7025: Add unit tests for PartitionedRegion virtualPut

demery-pivotal opened a new pull request #7025:
URL: https://github.com/apache/geode/pull/7025


   Co-authored-by: Dale Emery <de...@vmware.com>
   Co-authored-by: Kirk Lund <kl...@apache.org>
   


-- 
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: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] kirklund commented on a change in pull request #7025: Add unit tests for PartitionedRegion virtualPut

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7025:
URL: https://github.com/apache/geode/pull/7025#discussion_r734819510



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProviderFactory.java
##########
@@ -0,0 +1,19 @@
+/*
+ * 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;
+
+public interface PRHARedundancyProviderFactory {

Review comment:
       Add `@FunctionalInterface` annotation.




-- 
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: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] demery-pivotal closed pull request #7025: Add unit tests for PartitionedRegion virtualPut

Posted by GitBox <gi...@apache.org>.
demery-pivotal closed pull request #7025:
URL: https://github.com/apache/geode/pull/7025


   


-- 
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: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] kirklund commented on a change in pull request #7025: Add unit tests for PartitionedRegion virtualPut

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7025:
URL: https://github.com/apache/geode/pull/7025#discussion_r734826694



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
##########
@@ -10157,4 +10039,5 @@ void notifyRegionCreated() {
   public boolean areRecoveriesInProgress() {
     return prStats.getRecoveriesInProgress() > 0;
   }
+

Review comment:
       Remove blank line from end of class?

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
##########
@@ -1041,7 +1085,8 @@ private synchronized boolean initPRInternals(InternalRegionArguments internalReg
     // If localMaxMemory is set to 0, do not initialize Data Store.
     final boolean storesData = this.localMaxMemory > 0;
     if (storesData) {
-      initializeDataStore(this.getAttributes());
+      this.getAttributes();

Review comment:
       Delete this

##########
File path: geode-junit/build.gradle
##########
@@ -42,6 +42,7 @@ dependencies {
   api('org.junit.jupiter:junit-jupiter-params')
   api('org.assertj:assertj-core')
   api('org.mockito:mockito-core')
+  api('org.mockito:mockito-junit-jupiter')

Review comment:
       Remove from geode-junit?

##########
File path: geode-junit/src/test/resources/expected-pom.xml
##########
@@ -77,6 +77,11 @@
       <artifactId>mockito-core</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-junit-jupiter</artifactId>
+      <scope>compile</scope>
+    </dependency>

Review comment:
       Remove from geode-junit?

##########
File path: geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionTest.java
##########
@@ -84,136 +83,58 @@
   private InternalCache cache;
   private InternalDistributedSystem system;
   private DistributionManager distributionManager;
-  private InternalResourceManager resourceManager;
-  private AttributesFactory attributesFactory;
-
   private PartitionedRegion partitionedRegion;
 
   @Rule
   public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(STRICT_STUBS);
 
   @Before
   public void setUp() {
-    system = mock(InternalDistributedSystem.class);
+    cache = mock(InternalCache.class);
     distributionManager = mock(DistributionManager.class);
+    system = mock(InternalDistributedSystem.class);
+
     InternalDistributedMember distributedMember = mock(InternalDistributedMember.class);
+    InternalDataView internalDataView = mock(InternalDataView.class);
     InternalResourceManager resourceManager = mock(InternalResourceManager.class);
+    PartitionedRegionStatsFactory partitionedRegionStatsFactory =
+        mock(PartitionedRegionStatsFactory.class);
+    RegionAdvisorFactory regionAdvisorFactory = mock(RegionAdvisorFactory.class);
+    SenderIdMonitorFactory senderIdMonitorFactory = mock(SenderIdMonitorFactory.class);
 
-    cache = mock(InternalCache.class);
-    attributesFactory = new AttributesFactory();
-    attributesFactory.setPartitionAttributes(
-        new PartitionAttributesFactory().setTotalNumBuckets(1).setRedundantCopies(1).create());
+    PartitionAttributes<?, ?> partitionAttributes =
+        new PartitionAttributesFactory<>().setTotalNumBuckets(1).setRedundantCopies(1).create();
+    AttributesFactory<?, ?> attributesFactory = new AttributesFactory<>();
+    attributesFactory.setPartitionAttributes(partitionAttributes);
 
-    when(cache.getDistributedSystem())
-        .thenReturn(system);
     when(cache.getInternalDistributedSystem())
         .thenReturn(system);
     when(cache.getInternalResourceManager())
         .thenReturn(resourceManager);
-    when(distributionManager.getId())
-        .thenReturn(distributedMember);
-    when(system.createAtomicStatistics(any(), any()))
-        .thenReturn(mock(Statistics.class));
+    when(partitionedRegionStatsFactory.create(any()))
+        .thenReturn(mock(PartitionedRegionStats.class));
     when(system.getClock())
         .thenReturn(mock(DSClock.class));
     when(system.getDistributedMember())
         .thenReturn(distributedMember);
     when(system.getDistributionManager())
         .thenReturn(distributionManager);
 
-    partitionedRegion = new PartitionedRegion("regionName", attributesFactory.create(), null,
-        cache, mock(InternalRegionArguments.class), disabledClock(),
-        ColocationLoggerFactory.create());
-  }
-
-  private Object[] cacheLoaderAndWriter() {
-    CacheLoader mockLoader = mock(CacheLoader.class);
-    CacheWriter mockWriter = mock(CacheWriter.class);
-    return new Object[] {
-        new Object[] {mockLoader, null},
-        new Object[] {null, mockWriter},
-        new Object[] {mockLoader, mockWriter},
-        new Object[] {null, null}
-    };
-  }
-
-  @Test
-  @Parameters(method = "cacheLoaderAndWriter")
-  @TestCaseName("{method}(CacheLoader={0}, CacheWriter={1})")
-  public void verifyPRConfigUpdatedAfterLoaderUpdate(CacheLoader cacheLoader,
-      CacheWriter cacheWriter) {
-    // ARRANGE
-    PartitionRegionConfig partitionRegionConfig = mock(PartitionRegionConfig.class);
-    Region<String, PartitionRegionConfig> partitionedRegionRoot = mock(LocalRegion.class);
-    PartitionedRegion.RegionLock regionLock = mock(PartitionedRegion.RegionLock.class);
-    PartitionedRegion spyPartitionedRegion = spy(partitionedRegion);
-    InternalDistributedMember ourMember = spyPartitionedRegion.getDistributionManager().getId();
-    InternalDistributedMember otherMember1 = mock(InternalDistributedMember.class);
-    InternalDistributedMember otherMember2 = mock(InternalDistributedMember.class);
-    Node ourNode = mock(Node.class, "ourNode");
-    Node otherNode1 = mock(Node.class, "otherNode1");
-    Node otherNode2 = mock(Node.class, "otherNode2");
-
-    when(otherNode1.getMemberId())
-        .thenReturn(otherMember1);
-    when(otherNode2.getMemberId())
-        .thenReturn(otherMember2);
-    when(ourNode.getMemberId())
-        .thenReturn(ourMember);
-    when(ourNode.isCacheLoaderAttached())
-        .thenReturn(cacheLoader != null);
-    when(ourNode.isCacheWriterAttached())
-        .thenReturn(cacheWriter != null);
-    when(partitionRegionConfig.getNodes())
-        .thenReturn(asSet(otherNode1, ourNode, otherNode2));
-    when(partitionedRegionRoot.get(spyPartitionedRegion.getRegionIdentifier()))
-        .thenReturn(partitionRegionConfig);
-    when(spyPartitionedRegion.getPRRoot())
-        .thenReturn(partitionedRegionRoot);
-
-    doReturn(cacheLoader)
-        .when(spyPartitionedRegion).basicGetLoader();
-    doReturn(cacheWriter)
-        .when(spyPartitionedRegion).basicGetWriter();
-    doReturn(regionLock)
-        .when(spyPartitionedRegion).getRegionLock();
-
-    // ACT
-    spyPartitionedRegion.updatePRNodeInformation();
-
-    // ASSERT
-    assertThat(partitionRegionConfig.getNodes())
-        .contains(ourNode);
-
-    Node verifyOurNode = null;
-    for (Node node : partitionRegionConfig.getNodes()) {
-      if (node.getMemberId().equals(ourMember)) {
-        verifyOurNode = node;
-      }
-    }
-    assertThat(verifyOurNode)
-        .withFailMessage("Failed to find " + ourMember + " in " + partitionRegionConfig.getNodes())
-        .isNotNull();
-
-    verify(partitionedRegionRoot)
-        .get(spyPartitionedRegion.getRegionIdentifier());
-    verify(partitionedRegionRoot)
-        .put(spyPartitionedRegion.getRegionIdentifier(), partitionRegionConfig);
-    verify(spyPartitionedRegion)
-        .updatePRConfig(partitionRegionConfig, false);
-
-    assertThat(verifyOurNode.isCacheLoaderAttached())
-        .isEqualTo(cacheLoader != null);
-    assertThat(verifyOurNode.isCacheWriterAttached())
-        .isEqualTo(cacheWriter != null);
+    partitionedRegion =
+        new PartitionedRegion("regionName", attributesFactory.create(), null, cache,
+            mock(InternalRegionArguments.class), disabledClock(), ColocationLoggerFactory.create(),
+            regionAdvisorFactory, internalDataView, null /* Node */, system,
+            partitionedRegionStatsFactory, senderIdMonitorFactory,
+            pr -> new PRHARedundancyProvider(pr, cache.getInternalResourceManager()),
+            pr -> new PartitionedRegionDataStore(pr, disabledClock()),
+            new DefaultPartitionMessageDistribution(), RetryTimeKeeper::new);

Review comment:
       Should `DefaultPartitionMessageDistribution` be a mock instead?

##########
File path: geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionCacheCallbackTest.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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;
+
+import static java.util.Arrays.asList;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.mockito.quality.Strictness.STRICT_STUBS;
+
+import java.util.HashSet;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+import org.apache.geode.annotations.internal.MakeNotStatic;
+import org.apache.geode.cache.AttributesFactory;
+import org.apache.geode.cache.CacheLoader;
+import org.apache.geode.cache.CacheWriter;
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.cache.PartitionAttributesFactory;
+import org.apache.geode.cache.Region;
+import org.apache.geode.distributed.internal.DSClock;
+import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.control.InternalResourceManager;
+import org.apache.geode.internal.cache.partitioned.DefaultPartitionMessageDistribution;
+import org.apache.geode.internal.cache.partitioned.RegionAdvisorFactory;
+import org.apache.geode.internal.cache.partitioned.RetryTimeKeeper;
+import org.apache.geode.internal.cache.partitioned.colocation.ColocationLoggerFactory;
+
+@RunWith(JUnitParamsRunner.class)

Review comment:
       Replace with `GeodeParamsRunner`.




-- 
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: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] kirklund commented on a change in pull request #7025: Add unit tests for PartitionedRegion virtualPut

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7025:
URL: https://github.com/apache/geode/pull/7025#discussion_r734818873



##########
File path: buildSrc/src/main/groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy
##########
@@ -162,6 +162,7 @@ class DependencyConstraints implements Plugin<Project> {
         api(group: 'org.jctools', name: 'jctools-core', version: '3.3.0')
         api(group: 'org.jgroups', name: 'jgroups', version: get('jgroups.version'))
         api(group: 'org.mockito', name: 'mockito-core', version: '3.12.4')
+        api(group: 'org.mockito', name: 'mockito-junit-jupiter', version: '3.11.2')

Review comment:
       Consider making this version the same as `mockito-core` which is `3.12.4`.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProviderFactory.java
##########
@@ -0,0 +1,19 @@
+/*
+ * 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;
+
+public interface PRHARedundancyProviderFactory {

Review comment:
       Add `FunctionalInterface` annotation.

##########
File path: geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionCacheCallbackTest.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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;
+
+import static java.util.Arrays.asList;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.mockito.quality.Strictness.STRICT_STUBS;
+
+import java.util.HashSet;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+import org.apache.geode.annotations.internal.MakeNotStatic;
+import org.apache.geode.cache.AttributesFactory;
+import org.apache.geode.cache.CacheLoader;
+import org.apache.geode.cache.CacheWriter;
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.cache.PartitionAttributesFactory;
+import org.apache.geode.cache.Region;
+import org.apache.geode.distributed.internal.DSClock;
+import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.control.InternalResourceManager;
+import org.apache.geode.internal.cache.partitioned.DefaultPartitionMessageDistribution;
+import org.apache.geode.internal.cache.partitioned.RegionAdvisorFactory;
+import org.apache.geode.internal.cache.partitioned.RetryTimeKeeper;
+import org.apache.geode.internal.cache.partitioned.colocation.ColocationLoggerFactory;
+
+@RunWith(JUnitParamsRunner.class)
+public class PartitionedRegionCacheCallbackTest {
+
+  // TODO: DHE Why is this static?

Review comment:
       Resolve TODO before merge.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStoreFactory.java
##########
@@ -0,0 +1,19 @@
+/*
+ * 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;
+
+public interface PartitionedRegionDataStoreFactory {

Review comment:
       Add `@FunctionInterface` annotation.

##########
File path: geode-core/src/main/java/org/apache/geode/cache/PartitionAttributes.java
##########
@@ -172,4 +172,6 @@
 
     return configAttributes;
   }
+
+  void setOffHeap(boolean offHeap);

Review comment:
       Remove from interface to fix binary incompatibility.




-- 
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: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org