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/13 06:24:33 UTC

[GitHub] [helix] narendly opened a new pull request #892: Add integration tests for Helix Java APIs

narendly opened a new pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892
 
 
   ### Issues
   
   - [x] My PR addresses the following Helix issues and references them in the PR description:
   
   Fixes #891
   
   ### Description
   
   - [x] Here are some details about my PR, including screenshots of any UI changes:
   
   This commit adds a comprehensive integration test for Helix Java APIs. All Helix Java APIs are tested using regular resource rebalancing and task framework.
   
   There are a few minor changes to Helix Java API code. Those will be a separate PR for easier reviewing. Note that this PR contains all of those changes.
   
   ### Tests
   
   - [x] The following tests are written for this issue:
   
   TestMultiZkEnvironment
   
   - [x] The following is the result of the "mvn test" command on the appropriate module:
   
   (Copy & paste the result of "mvn test")
   
   ### Commits
   
   - [x] My commits all reference appropriate Apache Helix GitHub issues in their subject lines. 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"
   
   ### Documentation (Optional)
   
   - [ ] In case of new functionality, my PR adds documentation in the following wiki page:
   
   (Link the GitHub wiki you added)
   
   ### Code Quality
   
   - [x] My diff has been formatted using helix-style.xml 
   (helix-style-intellij.xml if IntelliJ IDE is used)

----------------------------------------------------------------
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] narendly commented on a change in pull request #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892#discussion_r393417930
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/multizk/TestMultiZkEnvironment.java
 ##########
 @@ -0,0 +1,448 @@
+package org.apache.helix.integration.multizk;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+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 com.google.common.collect.ImmutableList;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.InstanceType;
+import org.apache.helix.SystemPropertyKeys;
+import org.apache.helix.TestHelper;
+import org.apache.helix.api.config.RebalanceConfig;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.integration.task.MockTask;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKUtil;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.store.HelixPropertyStore;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * TestMultiZkEnvironment spins up multiple in-memory ZooKeepers with a pre-configured
+ * cluster-Zk realm routing information.
+ * This test verifies that all Helix Java APIs work as expected.
+ */
+public class TestMultiZkEnvironment {
+  private static final int NUM_ZK = 3;
+  private static final Map<String, ZkServer> ZK_SERVER_MAP = new HashMap<>();
+  private static final Map<String, HelixZkClient> ZK_CLIENT_MAP = new HashMap<>();
+  private static final Map<String, ClusterControllerManager> MOCK_CONTROLLERS = new HashMap<>();
+  private static final Set<MockParticipantManager> MOCK_PARTICIPANTS = new HashSet<>();
+  private static final List<String> CLUSTER_LIST =
+      ImmutableList.of("CLUSTER_1", "CLUSTER_2", "CLUSTER_3");
+
+  private MockMetadataStoreDirectoryServer _msds;
+  private static final Map<String, Collection<String>> _rawRoutingData = new HashMap<>();
+  private RealmAwareZkClient _zkClient;
+  private HelixAdmin _zkHelixAdmin;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    // Create 3 in-memory zookeepers and routing mapping
+    final String zkPrefix = "localhost:";
+    final int zkStartPort = 8777;
+
+    for (int i = 0; i < NUM_ZK; i++) {
+      String zkAddress = zkPrefix + (zkStartPort + i);
+      ZK_SERVER_MAP.put(zkAddress, TestHelper.startZkServer(zkAddress));
+      ZK_CLIENT_MAP.put(zkAddress, DedicatedZkClientFactory.getInstance()
+          .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
+              new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer())));
+
+      // One cluster per ZkServer created
+      _rawRoutingData.put(zkAddress, Collections.singletonList("/" + CLUSTER_LIST.get(i)));
+    }
+
+    // Create a Mock MSDS
+    final String msdsHostName = "localhost";
+    final int msdsPort = 11117;
+    final String msdsNamespace = "multiZkTest";
+    _msds = new MockMetadataStoreDirectoryServer(msdsHostName, msdsPort, msdsNamespace,
+        _rawRoutingData);
+    _msds.startServer();
+
+    // Turn on multiZk mode in System config
+    System.setProperty(SystemPropertyKeys.MULTI_ZK_ENABLED, "true");
+    // MSDS endpoint: http://localhost:11117/admin/v2/namespaces/multiZkTest
+    System.setProperty(MetadataStoreRoutingConstants.MSDS_SERVER_ENDPOINT_KEY,
 
 Review comment:
   Should be removed.

----------------------------------------------------------------
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] narendly merged pull request #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
narendly merged pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892
 
 
   

----------------------------------------------------------------
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] pkuwm commented on a change in pull request #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892#discussion_r393397958
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/multizk/TestMultiZkEnvironment.java
 ##########
 @@ -0,0 +1,448 @@
+package org.apache.helix.integration.multizk;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+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 com.google.common.collect.ImmutableList;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.InstanceType;
+import org.apache.helix.SystemPropertyKeys;
+import org.apache.helix.TestHelper;
+import org.apache.helix.api.config.RebalanceConfig;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.integration.task.MockTask;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKUtil;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.store.HelixPropertyStore;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * TestMultiZkEnvironment spins up multiple in-memory ZooKeepers with a pre-configured
+ * cluster-Zk realm routing information.
+ * This test verifies that all Helix Java APIs work as expected.
+ */
+public class TestMultiZkEnvironment {
 
 Review comment:
   Nit, "This test verifies that all Helix Java APIs work as expected." Looking at the comments and the class name, I am not sure if the test class name is accurate (it seems to the scope of MultiZkEnvironment is bigger), or something like `TestMultiZkHelixJavaApis`? Up to you.

----------------------------------------------------------------
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] narendly commented on a change in pull request #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892#discussion_r393421432
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/multizk/TestMultiZkEnvironment.java
 ##########
 @@ -0,0 +1,448 @@
+package org.apache.helix.integration.multizk;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+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 com.google.common.collect.ImmutableList;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.InstanceType;
+import org.apache.helix.SystemPropertyKeys;
+import org.apache.helix.TestHelper;
+import org.apache.helix.api.config.RebalanceConfig;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.integration.task.MockTask;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKUtil;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.store.HelixPropertyStore;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * TestMultiZkEnvironment spins up multiple in-memory ZooKeepers with a pre-configured
+ * cluster-Zk realm routing information.
+ * This test verifies that all Helix Java APIs work as expected.
+ */
+public class TestMultiZkEnvironment {
+  private static final int NUM_ZK = 3;
+  private static final Map<String, ZkServer> ZK_SERVER_MAP = new HashMap<>();
+  private static final Map<String, HelixZkClient> ZK_CLIENT_MAP = new HashMap<>();
+  private static final Map<String, ClusterControllerManager> MOCK_CONTROLLERS = new HashMap<>();
+  private static final Set<MockParticipantManager> MOCK_PARTICIPANTS = new HashSet<>();
+  private static final List<String> CLUSTER_LIST =
+      ImmutableList.of("CLUSTER_1", "CLUSTER_2", "CLUSTER_3");
+
+  private MockMetadataStoreDirectoryServer _msds;
+  private static final Map<String, Collection<String>> _rawRoutingData = new HashMap<>();
+  private RealmAwareZkClient _zkClient;
+  private HelixAdmin _zkHelixAdmin;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    // Create 3 in-memory zookeepers and routing mapping
+    final String zkPrefix = "localhost:";
+    final int zkStartPort = 8777;
+
+    for (int i = 0; i < NUM_ZK; i++) {
+      String zkAddress = zkPrefix + (zkStartPort + i);
+      ZK_SERVER_MAP.put(zkAddress, TestHelper.startZkServer(zkAddress));
+      ZK_CLIENT_MAP.put(zkAddress, DedicatedZkClientFactory.getInstance()
+          .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
+              new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer())));
+
+      // One cluster per ZkServer created
+      _rawRoutingData.put(zkAddress, Collections.singletonList("/" + CLUSTER_LIST.get(i)));
+    }
+
+    // Create a Mock MSDS
+    final String msdsHostName = "localhost";
+    final int msdsPort = 11117;
+    final String msdsNamespace = "multiZkTest";
+    _msds = new MockMetadataStoreDirectoryServer(msdsHostName, msdsPort, msdsNamespace,
+        _rawRoutingData);
+    _msds.startServer();
+
+    // Turn on multiZk mode in System config
+    System.setProperty(SystemPropertyKeys.MULTI_ZK_ENABLED, "true");
+    // MSDS endpoint: http://localhost:11117/admin/v2/namespaces/multiZkTest
+    System.setProperty(MetadataStoreRoutingConstants.MSDS_SERVER_ENDPOINT_KEY,
+        "http://" + msdsHostName + ":" + msdsPort + "/admin/v2/namespaces/" + msdsNamespace);
+
+    // Create a FederatedZkClient for admin work
+    _zkClient =
+        new FederatedZkClient(new RealmAwareZkClient.RealmAwareZkConnectionConfig.Builder().build(),
+            new RealmAwareZkClient.RealmAwareZkClientConfig());
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    // Kill all mock controllers and participants
+    MOCK_CONTROLLERS.values().forEach(ClusterControllerManager::syncStop);
+    MOCK_PARTICIPANTS.forEach(MockParticipantManager::syncStop);
+
+    // Tear down all clusters
+    CLUSTER_LIST.forEach(cluster -> TestHelper.dropCluster(cluster, _zkClient));
+
+    // Verify that all clusters are gone in each zookeeper
+    Assert.assertTrue(TestHelper.verify(() -> {
+      for (Map.Entry<String, HelixZkClient> zkClientEntry : ZK_CLIENT_MAP.entrySet()) {
+        List<String> children = zkClientEntry.getValue().getChildren("/");
+        if (children.stream().anyMatch(CLUSTER_LIST::contains)) {
+          return false;
+        }
+      }
+      return true;
+    }, TestHelper.WAIT_DURATION));
+
+    // Tear down zookeepers
+    ZK_SERVER_MAP.forEach((zkAddress, zkServer) -> zkServer.shutdown());
+
+    // Stop MockMSDS
+    _msds.stopServer();
+
+    // Turn off multiZK mode in System config
+    System.clearProperty(SystemPropertyKeys.MULTI_ZK_ENABLED);
 
 Review comment:
   Again, I can wrap it in try-finally but each test is responsible for controlling its own System property configs. So a well-written test with a proper beforeClass or a setup method shouldn't be affected. But I agree - wrapping it in a try-finally is not a bad idea so I will add it.

----------------------------------------------------------------
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] pkuwm commented on issue #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
pkuwm commented on issue #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892#issuecomment-599831758
 
 
   @narendly Can you also rebase this PR?

----------------------------------------------------------------
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] pkuwm commented on a change in pull request #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892#discussion_r393396233
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/multizk/TestMultiZkEnvironment.java
 ##########
 @@ -0,0 +1,448 @@
+package org.apache.helix.integration.multizk;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+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 com.google.common.collect.ImmutableList;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.InstanceType;
+import org.apache.helix.SystemPropertyKeys;
+import org.apache.helix.TestHelper;
+import org.apache.helix.api.config.RebalanceConfig;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.integration.task.MockTask;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKUtil;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.store.HelixPropertyStore;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * TestMultiZkEnvironment spins up multiple in-memory ZooKeepers with a pre-configured
+ * cluster-Zk realm routing information.
+ * This test verifies that all Helix Java APIs work as expected.
+ */
+public class TestMultiZkEnvironment {
+  private static final int NUM_ZK = 3;
+  private static final Map<String, ZkServer> ZK_SERVER_MAP = new HashMap<>();
+  private static final Map<String, HelixZkClient> ZK_CLIENT_MAP = new HashMap<>();
+  private static final Map<String, ClusterControllerManager> MOCK_CONTROLLERS = new HashMap<>();
+  private static final Set<MockParticipantManager> MOCK_PARTICIPANTS = new HashSet<>();
+  private static final List<String> CLUSTER_LIST =
+      ImmutableList.of("CLUSTER_1", "CLUSTER_2", "CLUSTER_3");
+
+  private MockMetadataStoreDirectoryServer _msds;
+  private static final Map<String, Collection<String>> _rawRoutingData = new HashMap<>();
+  private RealmAwareZkClient _zkClient;
+  private HelixAdmin _zkHelixAdmin;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    // Create 3 in-memory zookeepers and routing mapping
+    final String zkPrefix = "localhost:";
+    final int zkStartPort = 8777;
+
+    for (int i = 0; i < NUM_ZK; i++) {
+      String zkAddress = zkPrefix + (zkStartPort + i);
+      ZK_SERVER_MAP.put(zkAddress, TestHelper.startZkServer(zkAddress));
+      ZK_CLIENT_MAP.put(zkAddress, DedicatedZkClientFactory.getInstance()
+          .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
+              new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer())));
+
+      // One cluster per ZkServer created
+      _rawRoutingData.put(zkAddress, Collections.singletonList("/" + CLUSTER_LIST.get(i)));
+    }
+
+    // Create a Mock MSDS
+    final String msdsHostName = "localhost";
+    final int msdsPort = 11117;
+    final String msdsNamespace = "multiZkTest";
+    _msds = new MockMetadataStoreDirectoryServer(msdsHostName, msdsPort, msdsNamespace,
+        _rawRoutingData);
+    _msds.startServer();
+
+    // Turn on multiZk mode in System config
+    System.setProperty(SystemPropertyKeys.MULTI_ZK_ENABLED, "true");
+    // MSDS endpoint: http://localhost:11117/admin/v2/namespaces/multiZkTest
+    System.setProperty(MetadataStoreRoutingConstants.MSDS_SERVER_ENDPOINT_KEY,
+        "http://" + msdsHostName + ":" + msdsPort + "/admin/v2/namespaces/" + msdsNamespace);
+
+    // Create a FederatedZkClient for admin work
+    _zkClient =
+        new FederatedZkClient(new RealmAwareZkClient.RealmAwareZkConnectionConfig.Builder().build(),
+            new RealmAwareZkClient.RealmAwareZkClientConfig());
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    // Kill all mock controllers and participants
+    MOCK_CONTROLLERS.values().forEach(ClusterControllerManager::syncStop);
+    MOCK_PARTICIPANTS.forEach(MockParticipantManager::syncStop);
+
+    // Tear down all clusters
+    CLUSTER_LIST.forEach(cluster -> TestHelper.dropCluster(cluster, _zkClient));
+
+    // Verify that all clusters are gone in each zookeeper
+    Assert.assertTrue(TestHelper.verify(() -> {
+      for (Map.Entry<String, HelixZkClient> zkClientEntry : ZK_CLIENT_MAP.entrySet()) {
+        List<String> children = zkClientEntry.getValue().getChildren("/");
+        if (children.stream().anyMatch(CLUSTER_LIST::contains)) {
+          return false;
+        }
+      }
+      return true;
+    }, TestHelper.WAIT_DURATION));
+
+    // Tear down zookeepers
+    ZK_SERVER_MAP.forEach((zkAddress, zkServer) -> zkServer.shutdown());
+
+    // Stop MockMSDS
+    _msds.stopServer();
+
+    // Turn off multiZK mode in System config
+    System.clearProperty(SystemPropertyKeys.MULTI_ZK_ENABLED);
 
 Review comment:
   Just a corner case, if there is exception before clearProperty(), this property will be kept and may affect other tests? Usually what I do is using try...finally.

----------------------------------------------------------------
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] pkuwm commented on a change in pull request #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892#discussion_r393395522
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/multizk/TestMultiZkEnvironment.java
 ##########
 @@ -0,0 +1,448 @@
+package org.apache.helix.integration.multizk;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+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 com.google.common.collect.ImmutableList;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.InstanceType;
+import org.apache.helix.SystemPropertyKeys;
+import org.apache.helix.TestHelper;
+import org.apache.helix.api.config.RebalanceConfig;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.integration.task.MockTask;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKUtil;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.store.HelixPropertyStore;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * TestMultiZkEnvironment spins up multiple in-memory ZooKeepers with a pre-configured
+ * cluster-Zk realm routing information.
+ * This test verifies that all Helix Java APIs work as expected.
+ */
+public class TestMultiZkEnvironment {
+  private static final int NUM_ZK = 3;
+  private static final Map<String, ZkServer> ZK_SERVER_MAP = new HashMap<>();
+  private static final Map<String, HelixZkClient> ZK_CLIENT_MAP = new HashMap<>();
+  private static final Map<String, ClusterControllerManager> MOCK_CONTROLLERS = new HashMap<>();
+  private static final Set<MockParticipantManager> MOCK_PARTICIPANTS = new HashSet<>();
+  private static final List<String> CLUSTER_LIST =
+      ImmutableList.of("CLUSTER_1", "CLUSTER_2", "CLUSTER_3");
+
+  private MockMetadataStoreDirectoryServer _msds;
+  private static final Map<String, Collection<String>> _rawRoutingData = new HashMap<>();
+  private RealmAwareZkClient _zkClient;
+  private HelixAdmin _zkHelixAdmin;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    // Create 3 in-memory zookeepers and routing mapping
+    final String zkPrefix = "localhost:";
+    final int zkStartPort = 8777;
+
+    for (int i = 0; i < NUM_ZK; i++) {
+      String zkAddress = zkPrefix + (zkStartPort + i);
+      ZK_SERVER_MAP.put(zkAddress, TestHelper.startZkServer(zkAddress));
+      ZK_CLIENT_MAP.put(zkAddress, DedicatedZkClientFactory.getInstance()
+          .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
+              new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer())));
+
+      // One cluster per ZkServer created
+      _rawRoutingData.put(zkAddress, Collections.singletonList("/" + CLUSTER_LIST.get(i)));
+    }
+
+    // Create a Mock MSDS
+    final String msdsHostName = "localhost";
+    final int msdsPort = 11117;
+    final String msdsNamespace = "multiZkTest";
+    _msds = new MockMetadataStoreDirectoryServer(msdsHostName, msdsPort, msdsNamespace,
+        _rawRoutingData);
+    _msds.startServer();
+
+    // Turn on multiZk mode in System config
+    System.setProperty(SystemPropertyKeys.MULTI_ZK_ENABLED, "true");
+    // MSDS endpoint: http://localhost:11117/admin/v2/namespaces/multiZkTest
+    System.setProperty(MetadataStoreRoutingConstants.MSDS_SERVER_ENDPOINT_KEY,
 
 Review comment:
   Do we still need to keep this property after class?

----------------------------------------------------------------
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] narendly commented on a change in pull request #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892#discussion_r393838931
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/multizk/TestMultiZkEnvironment.java
 ##########
 @@ -0,0 +1,448 @@
+package org.apache.helix.integration.multizk;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+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 com.google.common.collect.ImmutableList;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.InstanceType;
+import org.apache.helix.SystemPropertyKeys;
+import org.apache.helix.TestHelper;
+import org.apache.helix.api.config.RebalanceConfig;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.integration.task.MockTask;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKUtil;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.store.HelixPropertyStore;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * TestMultiZkEnvironment spins up multiple in-memory ZooKeepers with a pre-configured
+ * cluster-Zk realm routing information.
+ * This test verifies that all Helix Java APIs work as expected.
+ */
+public class TestMultiZkEnvironment {
 
 Review comment:
   I updated the name as you suggested. 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] pkuwm commented on a change in pull request #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892#discussion_r393394402
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/multizk/TestMultiZkEnvironment.java
 ##########
 @@ -0,0 +1,448 @@
+package org.apache.helix.integration.multizk;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+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 com.google.common.collect.ImmutableList;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.InstanceType;
+import org.apache.helix.SystemPropertyKeys;
+import org.apache.helix.TestHelper;
+import org.apache.helix.api.config.RebalanceConfig;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.integration.task.MockTask;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKUtil;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.store.HelixPropertyStore;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * TestMultiZkEnvironment spins up multiple in-memory ZooKeepers with a pre-configured
+ * cluster-Zk realm routing information.
+ * This test verifies that all Helix Java APIs work as expected.
+ */
+public class TestMultiZkEnvironment {
+  private static final int NUM_ZK = 3;
+  private static final Map<String, ZkServer> ZK_SERVER_MAP = new HashMap<>();
+  private static final Map<String, HelixZkClient> ZK_CLIENT_MAP = new HashMap<>();
+  private static final Map<String, ClusterControllerManager> MOCK_CONTROLLERS = new HashMap<>();
+  private static final Set<MockParticipantManager> MOCK_PARTICIPANTS = new HashSet<>();
+  private static final List<String> CLUSTER_LIST =
+      ImmutableList.of("CLUSTER_1", "CLUSTER_2", "CLUSTER_3");
+
+  private MockMetadataStoreDirectoryServer _msds;
+  private static final Map<String, Collection<String>> _rawRoutingData = new HashMap<>();
+  private RealmAwareZkClient _zkClient;
+  private HelixAdmin _zkHelixAdmin;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    // Create 3 in-memory zookeepers and routing mapping
+    final String zkPrefix = "localhost:";
+    final int zkStartPort = 8777;
+
+    for (int i = 0; i < NUM_ZK; i++) {
+      String zkAddress = zkPrefix + (zkStartPort + i);
+      ZK_SERVER_MAP.put(zkAddress, TestHelper.startZkServer(zkAddress));
+      ZK_CLIENT_MAP.put(zkAddress, DedicatedZkClientFactory.getInstance()
+          .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
+              new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer())));
+
+      // One cluster per ZkServer created
+      _rawRoutingData.put(zkAddress, Collections.singletonList("/" + CLUSTER_LIST.get(i)));
+    }
+
+    // Create a Mock MSDS
+    final String msdsHostName = "localhost";
+    final int msdsPort = 11117;
+    final String msdsNamespace = "multiZkTest";
+    _msds = new MockMetadataStoreDirectoryServer(msdsHostName, msdsPort, msdsNamespace,
+        _rawRoutingData);
+    _msds.startServer();
+
+    // Turn on multiZk mode in System config
+    System.setProperty(SystemPropertyKeys.MULTI_ZK_ENABLED, "true");
 
 Review comment:
   Maybe it is more accurate for us to back up this system property and recover it after the test? If we simply clear the property after class, there may be a case that another test relying on this property. If the property is cleared, the test may fail. I understand the property should be set and cleared in each test class/method. But just in case. 

----------------------------------------------------------------
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] narendly commented on a change in pull request #892: Add integration tests for Helix Java APIs

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #892: Add integration tests for Helix Java APIs
URL: https://github.com/apache/helix/pull/892#discussion_r393417836
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/integration/multizk/TestMultiZkEnvironment.java
 ##########
 @@ -0,0 +1,448 @@
+package org.apache.helix.integration.multizk;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+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 com.google.common.collect.ImmutableList;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.InstanceType;
+import org.apache.helix.SystemPropertyKeys;
+import org.apache.helix.TestHelper;
+import org.apache.helix.api.config.RebalanceConfig;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.integration.task.MockTask;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKUtil;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.store.HelixPropertyStore;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * TestMultiZkEnvironment spins up multiple in-memory ZooKeepers with a pre-configured
+ * cluster-Zk realm routing information.
+ * This test verifies that all Helix Java APIs work as expected.
+ */
+public class TestMultiZkEnvironment {
+  private static final int NUM_ZK = 3;
+  private static final Map<String, ZkServer> ZK_SERVER_MAP = new HashMap<>();
+  private static final Map<String, HelixZkClient> ZK_CLIENT_MAP = new HashMap<>();
+  private static final Map<String, ClusterControllerManager> MOCK_CONTROLLERS = new HashMap<>();
+  private static final Set<MockParticipantManager> MOCK_PARTICIPANTS = new HashSet<>();
+  private static final List<String> CLUSTER_LIST =
+      ImmutableList.of("CLUSTER_1", "CLUSTER_2", "CLUSTER_3");
+
+  private MockMetadataStoreDirectoryServer _msds;
+  private static final Map<String, Collection<String>> _rawRoutingData = new HashMap<>();
+  private RealmAwareZkClient _zkClient;
+  private HelixAdmin _zkHelixAdmin;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    // Create 3 in-memory zookeepers and routing mapping
+    final String zkPrefix = "localhost:";
+    final int zkStartPort = 8777;
+
+    for (int i = 0; i < NUM_ZK; i++) {
+      String zkAddress = zkPrefix + (zkStartPort + i);
+      ZK_SERVER_MAP.put(zkAddress, TestHelper.startZkServer(zkAddress));
+      ZK_CLIENT_MAP.put(zkAddress, DedicatedZkClientFactory.getInstance()
+          .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
+              new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer())));
+
+      // One cluster per ZkServer created
+      _rawRoutingData.put(zkAddress, Collections.singletonList("/" + CLUSTER_LIST.get(i)));
+    }
+
+    // Create a Mock MSDS
+    final String msdsHostName = "localhost";
+    final int msdsPort = 11117;
+    final String msdsNamespace = "multiZkTest";
+    _msds = new MockMetadataStoreDirectoryServer(msdsHostName, msdsPort, msdsNamespace,
+        _rawRoutingData);
+    _msds.startServer();
+
+    // Turn on multiZk mode in System config
+    System.setProperty(SystemPropertyKeys.MULTI_ZK_ENABLED, "true");
 
 Review comment:
   As you mentioned, system configs should be reset for each test but I could certainly add what youre suggesting :)

----------------------------------------------------------------
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