You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by GitBox <gi...@apache.org> on 2020/03/05 21:06:54 UTC

[GitHub] [helix] alirezazamani opened a new pull request #869: Replace customized view cache with property cache

alirezazamani opened a new pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869
 
 
   ### Issues
   
   - [x] My PR addresses the following Helix issues and references them in the PR description:
   Fixes #868 
   
   ### Description
   - [x] Here are some details about my PR, including screenshots of any UI changes:
   In this PR, the custom implementation of customized view cache has been replaced with property cache implementation.
   
   ### Tests
   - [x] The following is the result of the "mvn test" command on the appropriate module:
   
   [INFO] Tests run: 914, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 3,492.45 s - in TestSuite
   [INFO]
   [INFO] Results:
   [INFO]
   [INFO] Tests run: 914, Failures: 0, Errors: 0, Skipped: 0
   [INFO]
   [INFO] ------------------------------------------------------------------------
   [INFO] BUILD SUCCESS
   [INFO] ------------------------------------------------------------------------
   [INFO] Total time:  58:17 min
   [INFO] Finished at: 2020-03-04T22:07:09-08:00
   [INFO] ------------------------------------------------------------------------### Commits
   
   - [x] My commits all reference appropriate Apache Helix GitHub issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Code Quality
   
   - [x] My diff has been formatted using helix-style.xml
   
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r397458124
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
 ##########
 @@ -308,19 +327,53 @@ public void testGetRoutingTableSnapshot() {
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeA");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeA");
+
     routingTableSnapshot =
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeB");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeB");
 
-    Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
-        routingTableProvider.getRoutingTableSnapshots();
-    Assert.assertEquals(routingTableSnapshots.size(), 2);
-    Assert.assertEquals(routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size(), 2);
+    // Make sure snapshot information is correct
+    // Check resources are in a correct state
+    boolean isRoutingTableUpdatedProperly = TestHelper.verify(() -> {
+      Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
+          routingTableProvider.getRoutingTableSnapshots();
+      RoutingTableSnapshot routingTableSnapshotTypeA =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeA");
+      RoutingTableSnapshot routingTableSnapshotTypeB =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeB");
+      String typeAp1h1 = "noState";
+      String typeAp1h2 = "noState";
+      String typeAp2h1 = "noState";
+      String typeAp3h2 = "noState";
+      String typeBp1h2 = "noState";
+      String typeBp1h4 = "noState";
+      try {
+        typeAp1h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h1");
+        typeAp1h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h2");
+        typeAp2h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p2").get("h1");
+        typeAp3h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p3").get("h2");
+        typeBp1h2 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h3");
+        typeBp1h4 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h4");
+      } catch (Exception e) {
+        // ok because RoutingTable has not been updated yet
+      }
+      return (routingTableSnapshots.size() == 2
+          && routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size() == 2
+          && typeAp1h1.equals("testState1") && typeAp1h2.equals("testState1")
+          && typeAp2h1.equals("testState2") && typeAp3h2.equals("testState3")
+          && typeBp1h2.equals("testState3") && typeBp1h4.equals("testState2"));
+    }, TestHelper.WAIT_DURATION);
+    Assert.assertTrue(isRoutingTableUpdatedProperly);
 
 Review comment:
   Please also add message in the assertion for easier understanding.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r400428231
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/common/caches/CustomizedViewCache.java
 ##########
 @@ -71,87 +77,18 @@ protected CustomizedViewCache(String clusterName, PropertyType propertyType, Str
    */
   public void refresh(HelixDataAccessor accessor) {
     long startTime = System.currentTimeMillis();
-    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
-    Set<PropertyKey> currentPropertyKeys = new HashSet<>();
-
-    List<String> resources = accessor.getChildNames(customizedViewsKey(keyBuilder));
-
-    for (String resource : resources) {
-      currentPropertyKeys.add(customizedViewKey(keyBuilder, resource));
-    }
-
-    Set<PropertyKey> cachedKeys = new HashSet<>();
-    Map<PropertyKey, CustomizedView> cachedCustomizedViewMap = Maps.newHashMap();
-    for (String resource : _customizedViewCache.keySet()) {
-      PropertyKey key = customizedViewKey(keyBuilder, resource);
-      cachedKeys.add(key);
-      cachedCustomizedViewMap.put(key, _customizedViewCache.get(resource));
-    }
-    cachedKeys.retainAll(currentPropertyKeys);
-
-    Set<PropertyKey> reloadKeys = new HashSet<>(currentPropertyKeys);
-    reloadKeys.removeAll(cachedKeys);
-
-    Map<PropertyKey, CustomizedView> updatedMap =
-        refreshProperties(accessor, reloadKeys, new ArrayList<>(cachedKeys),
-            cachedCustomizedViewMap, new HashSet<>());
-
-    Map<String, CustomizedView> newCustomizedViewMap = Maps.newHashMap();
-    for (CustomizedView customizedView : updatedMap.values()) {
-      newCustomizedViewMap.put(customizedView.getResourceName(), customizedView);
-    }
-
-    _customizedViewCache = new HashMap<>(newCustomizedViewMap);
-    _customizedViewMap = new HashMap<>(newCustomizedViewMap);
-
+    _customizedViewCache.refresh(accessor);
     long endTime = System.currentTimeMillis();
-    LOG.info("Refresh " + _customizedViewMap.size() + " CustomizedViews of type " + _customizedStateType
-        + " for cluster " + _clusterName + ", took " + (endTime - startTime) + " ms");
-  }
-
-  private PropertyKey customizedViewsKey(PropertyKey.Builder keyBuilder) {
-    PropertyKey customizedViewPropertyKey;
-    if (_propertyType.equals(PropertyType.CUSTOMIZEDVIEW)){
-      customizedViewPropertyKey = keyBuilder.customizedView(_customizedStateType);
-    } else {
-      throw new HelixException(
-          "Failed to refresh CustomizedViewCache, Wrong property type " + _propertyType + "!");
-    }
-    return customizedViewPropertyKey;
-  }
 
-  private PropertyKey customizedViewKey(PropertyKey.Builder keyBuilder, String resource) {
-    PropertyKey customizedViewPropertyKey;
-    if (_propertyType.equals(PropertyType.CUSTOMIZEDVIEW)) {
-      customizedViewPropertyKey = keyBuilder.customizedView(_customizedStateType, resource);
-    } else {
-      throw new HelixException(
-          "Failed to refresh CustomizedViewCache, Wrong property type " + _propertyType + "!");
-    }
-    return customizedViewPropertyKey;
+    LOG.info("Refresh " + _customizedViewCache.getPropertyMap().size() + " CustomizedViews of type " + _customizedStateType
 
 Review comment:
   This is the propertyCache log:
       LogUtil.logInfo(LOG, genEventInfo(),
           String.format("Refreshed %s property %s took %s ms. Selective: %s", _objMap.size(),
               _propertyDescription, System.currentTimeMillis() - start, _useSelectiveUpdate));
   
   I think it is overlapping. So there is no lead to log here.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani edited a comment on issue #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
alirezazamani edited a comment on issue #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#issuecomment-595448191
 
 
   This PR is depending on PR #834 and only the last commit is new compared to the other PR. Please do not review it until other one in merged.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani commented on issue #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
alirezazamani commented on issue #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#issuecomment-595448191
 
 
   This PR is depending on PR #834

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r398797422
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
 ##########
 @@ -308,19 +327,53 @@ public void testGetRoutingTableSnapshot() {
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeA");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeA");
+
     routingTableSnapshot =
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeB");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeB");
 
-    Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
-        routingTableProvider.getRoutingTableSnapshots();
-    Assert.assertEquals(routingTableSnapshots.size(), 2);
-    Assert.assertEquals(routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size(), 2);
+    // Make sure snapshot information is correct
+    // Check resources are in a correct state
+    boolean isRoutingTableUpdatedProperly = TestHelper.verify(() -> {
+      Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
+          routingTableProvider.getRoutingTableSnapshots();
+      RoutingTableSnapshot routingTableSnapshotTypeA =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeA");
+      RoutingTableSnapshot routingTableSnapshotTypeB =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeB");
+      String typeAp1h1 = "noState";
+      String typeAp1h2 = "noState";
+      String typeAp2h1 = "noState";
+      String typeAp3h2 = "noState";
+      String typeBp1h2 = "noState";
+      String typeBp1h4 = "noState";
+      try {
+        typeAp1h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h1");
+        typeAp1h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h2");
+        typeAp2h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p2").get("h1");
+        typeAp3h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p3").get("h2");
+        typeBp1h2 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h3");
+        typeBp1h4 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h4");
+      } catch (Exception e) {
+        // ok because RoutingTable has not been updated yet
 
 Review comment:
   nit, return false directly here.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r398797000
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/common/caches/CustomizedViewCache.java
 ##########
 @@ -19,32 +19,24 @@
  * under the License.
  */
 
-import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyType;
 import org.apache.helix.model.CustomizedView;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Maps;
 
 /**
  * Cache to hold all CustomizedView of a specific type.
  */
 public class CustomizedViewCache extends AbstractDataCache<CustomizedView> {
   private static final Logger LOG = LoggerFactory.getLogger(CustomizedViewCache.class.getName());
 
-  protected Map<String, CustomizedView> _customizedViewMap;
-  protected Map<String, CustomizedView> _customizedViewCache;
+  private final PropertyCache<CustomizedView> _customizedViewCache;
 
 Review comment:
   I agree. There should be no extra logic you need to add in addition, once you are using PropertyCache.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang merged pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
jiajunwang merged pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r398797864
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
 ##########
 @@ -308,19 +327,53 @@ public void testGetRoutingTableSnapshot() {
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeA");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeA");
+
     routingTableSnapshot =
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeB");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeB");
 
-    Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
-        routingTableProvider.getRoutingTableSnapshots();
-    Assert.assertEquals(routingTableSnapshots.size(), 2);
-    Assert.assertEquals(routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size(), 2);
+    // Make sure snapshot information is correct
+    // Check resources are in a correct state
+    boolean isRoutingTableUpdatedProperly = TestHelper.verify(() -> {
+      Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
+          routingTableProvider.getRoutingTableSnapshots();
+      RoutingTableSnapshot routingTableSnapshotTypeA =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeA");
+      RoutingTableSnapshot routingTableSnapshotTypeB =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeB");
+      String typeAp1h1 = "noState";
+      String typeAp1h2 = "noState";
+      String typeAp2h1 = "noState";
+      String typeAp3h2 = "noState";
+      String typeBp1h2 = "noState";
+      String typeBp1h4 = "noState";
+      try {
+        typeAp1h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h1");
+        typeAp1h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h2");
+        typeAp2h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p2").get("h1");
+        typeAp3h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p3").get("h2");
+        typeBp1h2 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h3");
+        typeBp1h4 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h4");
+      } catch (Exception e) {
+        // ok because RoutingTable has not been updated yet
+      }
+      return (routingTableSnapshots.size() == 2
+          && routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size() == 2
+          && typeAp1h1.equals("testState1") && typeAp1h2.equals("testState1")
+          && typeAp2h1.equals("testState2") && typeAp3h2.equals("testState3")
+          && typeBp1h2.equals("testState3") && typeBp1h4.equals("testState2"));
+    }, TestHelper.WAIT_DURATION);
 
 Review comment:
   Will 20 seconds be too long for this check?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r399838458
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
 ##########
 @@ -308,19 +327,53 @@ public void testGetRoutingTableSnapshot() {
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeA");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeA");
+
     routingTableSnapshot =
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeB");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeB");
 
-    Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
-        routingTableProvider.getRoutingTableSnapshots();
-    Assert.assertEquals(routingTableSnapshots.size(), 2);
-    Assert.assertEquals(routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size(), 2);
+    // Make sure snapshot information is correct
+    // Check resources are in a correct state
+    boolean isRoutingTableUpdatedProperly = TestHelper.verify(() -> {
+      Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
+          routingTableProvider.getRoutingTableSnapshots();
+      RoutingTableSnapshot routingTableSnapshotTypeA =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeA");
+      RoutingTableSnapshot routingTableSnapshotTypeB =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeB");
+      String typeAp1h1 = "noState";
+      String typeAp1h2 = "noState";
+      String typeAp2h1 = "noState";
+      String typeAp3h2 = "noState";
+      String typeBp1h2 = "noState";
+      String typeBp1h4 = "noState";
+      try {
+        typeAp1h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h1");
+        typeAp1h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h2");
+        typeAp2h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p2").get("h1");
+        typeAp3h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p3").get("h2");
+        typeBp1h2 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h3");
+        typeBp1h4 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h4");
+      } catch (Exception e) {
+        // ok because RoutingTable has not been updated yet
+      }
+      return (routingTableSnapshots.size() == 2
+          && routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size() == 2
+          && typeAp1h1.equals("testState1") && typeAp1h2.equals("testState1")
+          && typeAp2h1.equals("testState2") && typeAp3h2.equals("testState3")
+          && typeBp1h2.equals("testState3") && typeBp1h4.equals("testState2"));
+    }, TestHelper.WAIT_DURATION);
 
 Review comment:
   Added WAIT_DURATION of 5 second to this test and use it here.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r397458124
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
 ##########
 @@ -308,19 +327,53 @@ public void testGetRoutingTableSnapshot() {
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeA");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeA");
+
     routingTableSnapshot =
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeB");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeB");
 
-    Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
-        routingTableProvider.getRoutingTableSnapshots();
-    Assert.assertEquals(routingTableSnapshots.size(), 2);
-    Assert.assertEquals(routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size(), 2);
+    // Make sure snapshot information is correct
+    // Check resources are in a correct state
+    boolean isRoutingTableUpdatedProperly = TestHelper.verify(() -> {
+      Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
+          routingTableProvider.getRoutingTableSnapshots();
+      RoutingTableSnapshot routingTableSnapshotTypeA =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeA");
+      RoutingTableSnapshot routingTableSnapshotTypeB =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeB");
+      String typeAp1h1 = "noState";
+      String typeAp1h2 = "noState";
+      String typeAp2h1 = "noState";
+      String typeAp3h2 = "noState";
+      String typeBp1h2 = "noState";
+      String typeBp1h4 = "noState";
+      try {
+        typeAp1h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h1");
+        typeAp1h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h2");
+        typeAp2h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p2").get("h1");
+        typeAp3h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p3").get("h2");
+        typeBp1h2 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h3");
+        typeBp1h4 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h4");
+      } catch (Exception e) {
+        // ok because RoutingTable has not been updated yet
+      }
+      return (routingTableSnapshots.size() == 2
+          && routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size() == 2
+          && typeAp1h1.equals("testState1") && typeAp1h2.equals("testState1")
+          && typeAp2h1.equals("testState2") && typeAp3h2.equals("testState3")
+          && typeBp1h2.equals("testState3") && typeBp1h4.equals("testState2"));
+    }, TestHelper.WAIT_DURATION);
+    Assert.assertTrue(isRoutingTableUpdatedProperly);
 
 Review comment:
   Please also add message in the assertion for easier understanding.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on issue #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on issue #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#issuecomment-606186271
 
 
   Please resolve the minor comment. The code looks good to me.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani removed a comment on issue #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
alirezazamani removed a comment on issue #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#issuecomment-595448191
 
 
   This PR is depending on PR #834 and only the last commit is new compared to the other PR. Please do not review it until other one in merged.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r399837189
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/common/caches/CustomizedViewCache.java
 ##########
 @@ -19,32 +19,24 @@
  * under the License.
  */
 
-import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyType;
 import org.apache.helix.model.CustomizedView;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Maps;
 
 /**
  * Cache to hold all CustomizedView of a specific type.
  */
 public class CustomizedViewCache extends AbstractDataCache<CustomizedView> {
   private static final Logger LOG = LoggerFactory.getLogger(CustomizedViewCache.class.getName());
 
-  protected Map<String, CustomizedView> _customizedViewMap;
-  protected Map<String, CustomizedView> _customizedViewCache;
+  private final PropertyCache<CustomizedView> _customizedViewCache;
 
 Review comment:
   I chatted with Meng about this. Since both of our PRs has been implemented based on this file, I think it makes more sense to have separate file for customizedViewCache and have it implemented based on PropertyCache as discussed.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani commented on issue #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
alirezazamani commented on issue #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#issuecomment-606307317
 
 
   This PR is ready to be merged, approved by @jiajunwang 
   
   Final commit message:
   
   Replace the CustomizedView cache with Property cache
   
   In this commit, the custom implementation of customized view cache
   has been replaced with property cache implementation.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r399838436
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
 ##########
 @@ -308,19 +327,53 @@ public void testGetRoutingTableSnapshot() {
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeA");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeA");
+
     routingTableSnapshot =
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeB");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeB");
 
-    Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
-        routingTableProvider.getRoutingTableSnapshots();
-    Assert.assertEquals(routingTableSnapshots.size(), 2);
-    Assert.assertEquals(routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size(), 2);
+    // Make sure snapshot information is correct
+    // Check resources are in a correct state
+    boolean isRoutingTableUpdatedProperly = TestHelper.verify(() -> {
+      Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
+          routingTableProvider.getRoutingTableSnapshots();
+      RoutingTableSnapshot routingTableSnapshotTypeA =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeA");
+      RoutingTableSnapshot routingTableSnapshotTypeB =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeB");
+      String typeAp1h1 = "noState";
+      String typeAp1h2 = "noState";
+      String typeAp2h1 = "noState";
+      String typeAp3h2 = "noState";
+      String typeBp1h2 = "noState";
+      String typeBp1h4 = "noState";
+      try {
+        typeAp1h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h1");
+        typeAp1h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h2");
+        typeAp2h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p2").get("h1");
+        typeAp3h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p3").get("h2");
+        typeBp1h2 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h3");
+        typeBp1h4 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h4");
+      } catch (Exception e) {
+        // ok because RoutingTable has not been updated yet
 
 Review comment:
   Done.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r400556444
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/common/caches/CustomizedViewCache.java
 ##########
 @@ -71,87 +77,18 @@ protected CustomizedViewCache(String clusterName, PropertyType propertyType, Str
    */
   public void refresh(HelixDataAccessor accessor) {
     long startTime = System.currentTimeMillis();
-    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
-    Set<PropertyKey> currentPropertyKeys = new HashSet<>();
-
-    List<String> resources = accessor.getChildNames(customizedViewsKey(keyBuilder));
-
-    for (String resource : resources) {
-      currentPropertyKeys.add(customizedViewKey(keyBuilder, resource));
-    }
-
-    Set<PropertyKey> cachedKeys = new HashSet<>();
-    Map<PropertyKey, CustomizedView> cachedCustomizedViewMap = Maps.newHashMap();
-    for (String resource : _customizedViewCache.keySet()) {
-      PropertyKey key = customizedViewKey(keyBuilder, resource);
-      cachedKeys.add(key);
-      cachedCustomizedViewMap.put(key, _customizedViewCache.get(resource));
-    }
-    cachedKeys.retainAll(currentPropertyKeys);
-
-    Set<PropertyKey> reloadKeys = new HashSet<>(currentPropertyKeys);
-    reloadKeys.removeAll(cachedKeys);
-
-    Map<PropertyKey, CustomizedView> updatedMap =
-        refreshProperties(accessor, reloadKeys, new ArrayList<>(cachedKeys),
-            cachedCustomizedViewMap, new HashSet<>());
-
-    Map<String, CustomizedView> newCustomizedViewMap = Maps.newHashMap();
-    for (CustomizedView customizedView : updatedMap.values()) {
-      newCustomizedViewMap.put(customizedView.getResourceName(), customizedView);
-    }
-
-    _customizedViewCache = new HashMap<>(newCustomizedViewMap);
-    _customizedViewMap = new HashMap<>(newCustomizedViewMap);
-
+    _customizedViewCache.refresh(accessor);
     long endTime = System.currentTimeMillis();
-    LOG.info("Refresh " + _customizedViewMap.size() + " CustomizedViews of type " + _customizedStateType
-        + " for cluster " + _clusterName + ", took " + (endTime - startTime) + " ms");
-  }
-
-  private PropertyKey customizedViewsKey(PropertyKey.Builder keyBuilder) {
-    PropertyKey customizedViewPropertyKey;
-    if (_propertyType.equals(PropertyType.CUSTOMIZEDVIEW)){
-      customizedViewPropertyKey = keyBuilder.customizedView(_customizedStateType);
-    } else {
-      throw new HelixException(
-          "Failed to refresh CustomizedViewCache, Wrong property type " + _propertyType + "!");
-    }
-    return customizedViewPropertyKey;
-  }
 
-  private PropertyKey customizedViewKey(PropertyKey.Builder keyBuilder, String resource) {
-    PropertyKey customizedViewPropertyKey;
-    if (_propertyType.equals(PropertyType.CUSTOMIZEDVIEW)) {
-      customizedViewPropertyKey = keyBuilder.customizedView(_customizedStateType, resource);
-    } else {
-      throw new HelixException(
-          "Failed to refresh CustomizedViewCache, Wrong property type " + _propertyType + "!");
-    }
-    return customizedViewPropertyKey;
+    LOG.info("Refresh " + _customizedViewCache.getPropertyMap().size() + " CustomizedViews of type " + _customizedStateType
 
 Review comment:
   Done. Thanks.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
alirezazamani commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r399836939
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
 ##########
 @@ -308,19 +327,53 @@ public void testGetRoutingTableSnapshot() {
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeA");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeA");
+
     routingTableSnapshot =
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeB");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeB");
 
-    Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
-        routingTableProvider.getRoutingTableSnapshots();
-    Assert.assertEquals(routingTableSnapshots.size(), 2);
-    Assert.assertEquals(routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size(), 2);
+    // Make sure snapshot information is correct
+    // Check resources are in a correct state
+    boolean isRoutingTableUpdatedProperly = TestHelper.verify(() -> {
+      Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
+          routingTableProvider.getRoutingTableSnapshots();
+      RoutingTableSnapshot routingTableSnapshotTypeA =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeA");
+      RoutingTableSnapshot routingTableSnapshotTypeB =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeB");
+      String typeAp1h1 = "noState";
+      String typeAp1h2 = "noState";
+      String typeAp2h1 = "noState";
+      String typeAp3h2 = "noState";
+      String typeBp1h2 = "noState";
+      String typeBp1h4 = "noState";
+      try {
+        typeAp1h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h1");
+        typeAp1h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h2");
+        typeAp2h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p2").get("h1");
+        typeAp3h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p3").get("h2");
+        typeBp1h2 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h3");
+        typeBp1h4 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h4");
+      } catch (Exception e) {
+        // ok because RoutingTable has not been updated yet
+      }
+      return (routingTableSnapshots.size() == 2
+          && routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size() == 2
+          && typeAp1h1.equals("testState1") && typeAp1h2.equals("testState1")
+          && typeAp2h1.equals("testState2") && typeAp3h2.equals("testState3")
+          && typeBp1h2.equals("testState3") && typeBp1h4.equals("testState2"));
+    }, TestHelper.WAIT_DURATION);
+    Assert.assertTrue(isRoutingTableUpdatedProperly);
 
 Review comment:
   Done.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r397470089
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/common/caches/CustomizedViewCache.java
 ##########
 @@ -19,32 +19,24 @@
  * under the License.
  */
 
-import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyType;
 import org.apache.helix.model.CustomizedView;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Maps;
 
 /**
  * Cache to hold all CustomizedView of a specific type.
  */
 public class CustomizedViewCache extends AbstractDataCache<CustomizedView> {
   private static final Logger LOG = LoggerFactory.getLogger(CustomizedViewCache.class.getName());
 
-  protected Map<String, CustomizedView> _customizedViewMap;
-  protected Map<String, CustomizedView> _customizedViewCache;
+  private final PropertyCache<CustomizedView> _customizedViewCache;
 
 Review comment:
   Do we still need this CustomizedViewCache file? I thought this would be the same as ExternalViewCache, which is deprecated. And we use PropertyCache<CustomizedView> instead in the place we need it. Please confirm with @jiajunwang .

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #869: Replace customized view cache with property cache

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #869: Replace customized view cache with property cache
URL: https://github.com/apache/helix/pull/869#discussion_r397455323
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
 ##########
 @@ -308,19 +327,53 @@ public void testGetRoutingTableSnapshot() {
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeA");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeA");
+
     routingTableSnapshot =
         routingTableProvider.getRoutingTableSnapshot(PropertyType.CUSTOMIZEDVIEW, "typeB");
     Assert.assertEquals(routingTableSnapshot.getPropertyType(), PropertyType.CUSTOMIZEDVIEW);
     Assert.assertEquals(routingTableSnapshot.getCustomizedStateType(), "typeB");
 
-    Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
-        routingTableProvider.getRoutingTableSnapshots();
-    Assert.assertEquals(routingTableSnapshots.size(), 2);
-    Assert.assertEquals(routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size(), 2);
+    // Make sure snapshot information is correct
+    // Check resources are in a correct state
+    boolean isRoutingTableUpdatedProperly = TestHelper.verify(() -> {
+      Map<String, Map<String, RoutingTableSnapshot>> routingTableSnapshots =
+          routingTableProvider.getRoutingTableSnapshots();
+      RoutingTableSnapshot routingTableSnapshotTypeA =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeA");
+      RoutingTableSnapshot routingTableSnapshotTypeB =
+          routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).get("typeB");
+      String typeAp1h1 = "noState";
+      String typeAp1h2 = "noState";
+      String typeAp2h1 = "noState";
+      String typeAp3h2 = "noState";
+      String typeBp1h2 = "noState";
+      String typeBp1h4 = "noState";
+      try {
+        typeAp1h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h1");
+        typeAp1h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h2");
+        typeAp2h1 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p2").get("h1");
+        typeAp3h2 = routingTableSnapshotTypeA.getCustomizeViews().iterator().next()
+            .getStateMap("p3").get("h2");
+        typeBp1h2 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h3");
+        typeBp1h4 = routingTableSnapshotTypeB.getCustomizeViews().iterator().next()
+            .getStateMap("p1").get("h4");
+      } catch (Exception e) {
+        // ok because RoutingTable has not been updated yet
+      }
+      return (routingTableSnapshots.size() == 2
+          && routingTableSnapshots.get(PropertyType.CUSTOMIZEDVIEW.name()).size() == 2
+          && typeAp1h1.equals("testState1") && typeAp1h2.equals("testState1")
+          && typeAp2h1.equals("testState2") && typeAp3h2.equals("testState3")
+          && typeBp1h2.equals("testState3") && typeBp1h4.equals("testState2"));
+    }, TestHelper.WAIT_DURATION);
+    Assert.assertTrue(isRoutingTableUpdatedProperly);
 
 Review comment:
   Please also add message in the assertion for easy understanding. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org