You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ka...@apache.org on 2014/04/14 19:01:34 UTC

git commit: [HELIX-427] Write a test for using preference lists as execution queues

Repository: helix
Updated Branches:
  refs/heads/master a47ff2e1a -> 04a6021ed


[HELIX-427] Write a test for using preference lists as execution queues


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/04a6021e
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/04a6021e
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/04a6021e

Branch: refs/heads/master
Commit: 04a6021ed1ef4732c5e95f16255c8c639e7ba22d
Parents: a47ff2e
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Thu Apr 10 18:47:13 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Mon Apr 14 10:01:26 2014 -0700

----------------------------------------------------------------------
 .../integration/TestPreferenceListAsQueue.java  | 487 +++++++++++++++++++
 1 file changed, 487 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/04a6021e/helix-core/src/test/java/org/apache/helix/integration/TestPreferenceListAsQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestPreferenceListAsQueue.java b/helix-core/src/test/java/org/apache/helix/integration/TestPreferenceListAsQueue.java
new file mode 100644
index 0000000..633d046
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestPreferenceListAsQueue.java
@@ -0,0 +1,487 @@
+package org.apache.helix.integration;
+
+/*
+ * 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.Arrays;
+import java.util.List;
+
+import org.I0Itec.zkclient.DataUpdater;
+import org.apache.helix.AccessOption;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.NotificationContext;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
+import org.apache.helix.model.ClusterConstraints.ConstraintType;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.IdealState.IdealStateProperty;
+import org.apache.helix.model.IdealState.RebalanceMode;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.model.builder.ConstraintItemBuilder;
+import org.apache.helix.participant.statemachine.StateModel;
+import org.apache.helix.participant.statemachine.StateModelFactory;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.log4j.Logger;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestPreferenceListAsQueue extends ZkUnitTestBase {
+  private static final Logger LOG = Logger.getLogger(TestPreferenceListAsQueue.class);
+  private static final int TRANSITION_TIME = 500;
+  private static final int WAIT_TIME = TRANSITION_TIME + (TRANSITION_TIME / 2);
+  private static final int PARALLELISM = 1;
+
+  private List<String> _instanceList;
+  private String _clusterName;
+  private String _stateModel;
+  private ClusterSetup _clusterSetup;
+  private HelixAdmin _admin;
+
+  @BeforeMethod
+  public void beforeMethod() {
+    _instanceList = Lists.newLinkedList();
+    _clusterSetup = new ClusterSetup(ZK_ADDR);
+    _admin = _clusterSetup.getClusterManagementTool();
+
+    // Create cluster
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    _clusterName = className + "_" + methodName;
+    _clusterSetup.addCluster(_clusterName, true);
+  }
+
+  /**
+   * This test ensures successful creation when the state model has OFFLINE --> deprioritized and
+   * a partition-level constraint enforces parallelism
+   * @throws Exception
+   */
+  @Test
+  public void testReprioritizedWithConstraint() throws Exception {
+    _stateModel = "OnlineOfflineReprioritized";
+
+    // Add a state model with the transition to ONLINE deprioritized
+    _clusterSetup.addStateModelDef(_clusterName, _stateModel,
+        createReprioritizedStateModelDef(_stateModel));
+
+    // Add a constraint of one transition per partition
+    ConstraintItemBuilder constraintItemBuilder = new ConstraintItemBuilder();
+    constraintItemBuilder
+        .addConstraintAttribute(ConstraintAttribute.MESSAGE_TYPE.toString(), "STATE_TRANSITION")
+        .addConstraintAttribute(ConstraintAttribute.PARTITION.toString(), ".*")
+        .addConstraintAttribute(ConstraintAttribute.CONSTRAINT_VALUE.toString(),
+            String.valueOf(PARALLELISM));
+    _admin.setConstraint(_clusterName, ConstraintType.MESSAGE_CONSTRAINT, "constraint_1",
+        constraintItemBuilder.build());
+    runTest();
+  }
+
+  /**
+   * This test directly embeds the parallelism per partition directly into the upper bound for the
+   * ONLINE state. This does not require the controller to support partition-level constraints.
+   * @throws Exception
+   */
+  @Test
+  public void testParallelismInStateModel() throws Exception {
+    _stateModel = "OnlineOfflineBounded";
+
+    // Add a state model with the parallelism implicit
+    _clusterSetup.addStateModelDef(_clusterName, _stateModel,
+        createEnforcedParallelismStateModelDef(_stateModel, PARALLELISM));
+    runTest();
+  }
+
+  private void runTest() throws Exception {
+    final int NUM_PARTITIONS = 1;
+    final int NUM_REPLICAS = 2;
+    final int NUM_INSTANCES = 2;
+    final String RESOURCE_NAME = "MyResource";
+
+    // Setup instances
+    String[] instanceInfoArray = {
+        "localhost_1", "localhost_2"
+    };
+    _clusterSetup.addInstancesToCluster(_clusterName, instanceInfoArray);
+
+    // Add resource
+    _clusterSetup.addResourceToCluster(_clusterName, RESOURCE_NAME, NUM_PARTITIONS, _stateModel,
+        RebalanceMode.SEMI_AUTO.toString());
+
+    // Update resource with empty preference lists
+    IdealState idealState = _admin.getResourceIdealState(_clusterName, RESOURCE_NAME);
+    for (int i = 0; i < NUM_PARTITIONS; i++) {
+      String partitionName = RESOURCE_NAME + "_" + i;
+      List<String> dummyPreferences = Lists.newArrayList();
+      for (int j = 0; j < NUM_REPLICAS; j++) {
+        // hack: need to have some dummy values in the preference list to pass validation
+        dummyPreferences.add("");
+      }
+      idealState.getRecord().setListField(partitionName, dummyPreferences);
+    }
+    idealState.setReplicas(String.valueOf(NUM_REPLICAS));
+    _admin.setResourceIdealState(_clusterName, RESOURCE_NAME, idealState);
+
+    // Start some instances
+    HelixManager[] participants = new HelixManager[NUM_INSTANCES];
+    for (int i = 0; i < NUM_INSTANCES; i++) {
+      participants[i] =
+          HelixManagerFactory.getZKHelixManager(_clusterName, instanceInfoArray[i],
+              InstanceType.PARTICIPANT, ZK_ADDR);
+      participants[i].getStateMachineEngine().registerStateModelFactory(_stateModel,
+          new PrefListTaskOnlineOfflineStateModelFactory());
+      participants[i].connect();
+    }
+
+    // Start the controller
+    HelixManager controller =
+        HelixManagerFactory.getZKHelixManager(_clusterName, null, InstanceType.CONTROLLER, ZK_ADDR);
+    controller.connect();
+
+    // This resource only has 1 partition
+    String partitionName = RESOURCE_NAME + "_" + 0;
+
+    // There should be no preference lists yet
+    Assert.assertTrue(preferenceListIsCorrect(_admin, _clusterName, RESOURCE_NAME, partitionName,
+        Arrays.asList("", "")));
+
+    // Add one instance
+    addInstanceToPreferences(participants[0].getHelixDataAccessor(),
+        participants[0].getInstanceName(), RESOURCE_NAME, Arrays.asList(partitionName));
+    Assert.assertTrue(preferenceListIsCorrect(_admin, _clusterName, RESOURCE_NAME, partitionName,
+        Arrays.asList("localhost_1", "")));
+
+    // Add a second instance immediately; the first one should still exist
+    addInstanceToPreferences(participants[1].getHelixDataAccessor(),
+        participants[1].getInstanceName(), RESOURCE_NAME, Arrays.asList(partitionName));
+    Assert.assertTrue(preferenceListIsCorrect(_admin, _clusterName, RESOURCE_NAME, partitionName,
+        Arrays.asList("localhost_1", "localhost_2")));
+
+    // Add the first instance again; it should already exist
+    addInstanceToPreferences(participants[0].getHelixDataAccessor(),
+        participants[0].getInstanceName(), RESOURCE_NAME, Arrays.asList(partitionName));
+    Assert.assertTrue(preferenceListIsCorrect(_admin, _clusterName, RESOURCE_NAME, partitionName,
+        Arrays.asList("localhost_1", "localhost_2")));
+
+    // Now wait for the first instance to be done
+    Thread.sleep(WAIT_TIME);
+    Assert.assertTrue(preferenceListIsCorrect(_admin, _clusterName, RESOURCE_NAME, partitionName,
+        Arrays.asList("localhost_2", "")));
+
+    // Add the first instance again; it should not exist
+    addInstanceToPreferences(participants[0].getHelixDataAccessor(),
+        participants[0].getInstanceName(), RESOURCE_NAME, Arrays.asList(partitionName));
+    Assert.assertTrue(preferenceListIsCorrect(_admin, _clusterName, RESOURCE_NAME, partitionName,
+        Arrays.asList("localhost_2", "localhost_1")));
+
+    // Now wait for the second instance to be done
+    Thread.sleep(WAIT_TIME);
+    Assert.assertTrue(preferenceListIsCorrect(_admin, _clusterName, RESOURCE_NAME, partitionName,
+        Arrays.asList("localhost_1", "")));
+
+    // Now wait for the first instance to be done again
+    Thread.sleep(WAIT_TIME);
+    Assert.assertTrue(preferenceListIsCorrect(_admin, _clusterName, RESOURCE_NAME, partitionName,
+        Arrays.asList("", "")));
+    Assert.assertEquals(_instanceList.size(), 0);
+
+    // Cleanup
+    controller.disconnect();
+    for (HelixManager participant : participants) {
+      participant.disconnect();
+    }
+  }
+
+  /**
+   * Create a modified version of OnlineOffline where the transition to ONLINE is given lowest
+   * priority
+   * @param stateModelName
+   * @return
+   */
+  private StateModelDefinition createReprioritizedStateModelDef(String stateModelName) {
+    StateModelDefinition.Builder builder =
+        new StateModelDefinition.Builder(stateModelName).addState("ONLINE", 1).addState("OFFLINE")
+            .addState("DROPPED").addState("ERROR").initialState("OFFLINE")
+            .addTransition("ERROR", "OFFLINE", 1).addTransition("ONLINE", "OFFLINE", 2)
+            .addTransition("OFFLINE", "DROPPED", 3).addTransition("OFFLINE", "ONLINE", 4)
+            .dynamicUpperBound("ONLINE", "R").upperBound("OFFLINE", -1).upperBound("DROPPED", -1)
+            .upperBound("ERROR", -1);
+    return builder.build();
+  }
+
+  /**
+   * Create a modified version of OnlineOffline where the parallelism is enforced by the upper bound
+   * of ONLINE
+   * @param stateModelName
+   * @param parallelism
+   * @return
+   */
+  private StateModelDefinition createEnforcedParallelismStateModelDef(String stateModelName,
+      int parallelism) {
+    StateModelDefinition.Builder builder =
+        new StateModelDefinition.Builder(stateModelName).addState("ONLINE", 1).addState("OFFLINE")
+            .addState("DROPPED").addState("ERROR").initialState("OFFLINE")
+            .addTransition("ERROR", "OFFLINE", 1).addTransition("ONLINE", "OFFLINE", 2)
+            .addTransition("OFFLINE", "DROPPED", 3).addTransition("OFFLINE", "ONLINE", 4)
+            .dynamicUpperBound("ONLINE", String.valueOf(PARALLELISM)).upperBound("OFFLINE", -1)
+            .upperBound("DROPPED", -1).upperBound("ERROR", -1);
+    return builder.build();
+  }
+
+  /**
+   * Check if the provided list matches the currently persisted preference list
+   * @param admin
+   * @param clusterName
+   * @param resourceName
+   * @param partitionName
+   * @param expectPreferenceList
+   * @return
+   */
+  private boolean preferenceListIsCorrect(HelixAdmin admin, String clusterName,
+      String resourceName, String partitionName, List<String> expectPreferenceList) {
+    IdealState idealState = admin.getResourceIdealState(clusterName, resourceName);
+    List<String> preferenceList = idealState.getPreferenceList(partitionName);
+    return expectPreferenceList.equals(preferenceList);
+  }
+
+  /**
+   * Update an ideal state so that partitions will have an instance removed from their preference
+   * lists
+   * @param accessor
+   * @param instanceName
+   * @param resourceName
+   * @param partitionName
+   */
+  private static void removeInstanceFromPreferences(HelixDataAccessor accessor,
+      final String instanceName, final String resourceName, final String partitionName) {
+    // Updater for ideal state
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    String idealStatePath = keyBuilder.idealStates(resourceName).getPath();
+    DataUpdater<ZNRecord> idealStateUpdater = new DataUpdater<ZNRecord>() {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
+        List<String> preferenceList = currentData.getListField(partitionName);
+        int numReplicas =
+            Integer.valueOf(currentData.getSimpleField(IdealStateProperty.REPLICAS.toString()));
+        currentData.setListField(partitionName,
+            removeInstanceFromPreferenceList(preferenceList, instanceName, numReplicas));
+        return currentData;
+      }
+    };
+
+    // Updater for instance config
+    String instanceConfigPath = keyBuilder.instanceConfig(instanceName).getPath();
+    DataUpdater<ZNRecord> instanceConfigUpdater = new DataUpdater<ZNRecord>() {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
+        // currentData.setBooleanField(InstanceConfigProperty.HELIX_ENABLED.toString(), false);
+        return currentData;
+      }
+    };
+    List<DataUpdater<ZNRecord>> updaters = Lists.newArrayList();
+    updaters.add(idealStateUpdater);
+    updaters.add(instanceConfigUpdater);
+    accessor.updateChildren(Arrays.asList(idealStatePath, instanceConfigPath), updaters,
+        AccessOption.PERSISTENT);
+  }
+
+  /**
+   * Update an ideal state so that partitions will have a new instance at the tails of their
+   * preference lists
+   * @param accessor
+   * @param instanceName
+   * @param resourceName
+   * @param partitions
+   */
+  private static void addInstanceToPreferences(HelixDataAccessor accessor,
+      final String instanceName, final String resourceName, final List<String> partitions) {
+    // Updater for ideal state
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    String idealStatePath = keyBuilder.idealStates(resourceName).getPath();
+    DataUpdater<ZNRecord> idealStateUpdater = new DataUpdater<ZNRecord>() {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
+        for (String partitionName : partitions) {
+          List<String> preferenceList = currentData.getListField(partitionName);
+          int numReplicas =
+              Integer.valueOf(currentData.getSimpleField(IdealStateProperty.REPLICAS.toString()));
+          currentData.setListField(partitionName,
+              addInstanceToPreferenceList(preferenceList, instanceName, numReplicas));
+        }
+        return currentData;
+      }
+    };
+
+    // Updater for instance config
+    String instanceConfigPath = keyBuilder.instanceConfig(instanceName).getPath();
+    DataUpdater<ZNRecord> instanceConfigUpdater = new DataUpdater<ZNRecord>() {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
+        // currentData.setBooleanField(InstanceConfigProperty.HELIX_ENABLED.toString(), true);
+        return currentData;
+      }
+    };
+
+    // Send update requests together
+    List<DataUpdater<ZNRecord>> updaters = Lists.newArrayList();
+    updaters.add(idealStateUpdater);
+    updaters.add(instanceConfigUpdater);
+    accessor.updateChildren(Arrays.asList(idealStatePath, instanceConfigPath), updaters,
+        AccessOption.PERSISTENT);
+  }
+
+  /**
+   * Add the instance to the preference list, removing padding as necessary
+   * @param existPreferences
+   * @param instanceName
+   * @param numReplicas
+   * @return A new preference list with the instance added to the end (if it did not already exist)
+   */
+  private static List<String> addInstanceToPreferenceList(List<String> existPreferences,
+      String instanceName, int numReplicas) {
+    if (existPreferences == null) {
+      existPreferences = Lists.newArrayList();
+    }
+    List<String> newPreferences = Lists.newArrayListWithCapacity(numReplicas);
+    for (String existInstance : existPreferences) {
+      if (!existInstance.isEmpty()) {
+        newPreferences.add(existInstance);
+      }
+    }
+    if (!newPreferences.contains(instanceName)) {
+      newPreferences.add(instanceName);
+    }
+    addDummiesToPreferenceList(newPreferences, numReplicas);
+    return newPreferences;
+  }
+
+  /**
+   * Remove an instance from a preference list, padding as necessary
+   * @param existPreferences
+   * @param instanceName
+   * @param numReplicas
+   * @return new preference list with the instance removed (if it existed)
+   */
+  private static List<String> removeInstanceFromPreferenceList(List<String> existPreferences,
+      String instanceName, int numReplicas) {
+    if (existPreferences == null) {
+      existPreferences = Lists.newArrayList();
+    }
+    List<String> newPreferences = Lists.newArrayListWithCapacity(numReplicas);
+    for (String existInstance : existPreferences) {
+      if (!existInstance.isEmpty() && !existInstance.equals(instanceName)) {
+        newPreferences.add(existInstance);
+      }
+    }
+    addDummiesToPreferenceList(newPreferences, numReplicas);
+    return newPreferences;
+  }
+
+  /**
+   * Pad a preference list with empty strings to have it reach numReplicas size
+   * @param preferences
+   * @param numReplicas
+   */
+  private static void addDummiesToPreferenceList(List<String> preferences, int numReplicas) {
+    int numRemaining = numReplicas - preferences.size();
+    numRemaining = (numRemaining > 0) ? numRemaining : 0;
+    for (int i = 0; i < numRemaining; i++) {
+      preferences.add("");
+    }
+  }
+
+  /**
+   * A state model whose callbacks correspond to when to run a task. When becoming online, the task
+   * should be run, and then the instance should be removed from the preference list for the task
+   * (padded by spaces). All other transitions are no-ops.
+   */
+  public class PrefListTaskOnlineOfflineStateModel extends StateModel {
+    /**
+     * Run the task. The parallelism of this is dictated by the constraints that are set.
+     * @param message
+     * @param context
+     * @throws InterruptedException
+     */
+    public void onBecomeOnlineFromOffline(final Message message, NotificationContext context)
+        throws InterruptedException {
+      // Do the work, and then finally remove the instance from the preference list for this
+      // partition
+      HelixManager manager = context.getManager();
+      LOG.info("START onBecomeOnlineFromOffline for " + message.getPartitionName() + " on "
+          + manager.getInstanceName());
+      int oldSize;
+      synchronized (_instanceList) {
+        oldSize = _instanceList.size();
+        _instanceList.add(manager.getInstanceName());
+      }
+      Assert.assertEquals(oldSize, 0); // ensure these transitions are fully synchronized
+
+      Thread.sleep(TRANSITION_TIME); // a sleep simulates work
+
+      // Need to disable in order to get the transition the next time
+      HelixDataAccessor accessor = manager.getHelixDataAccessor();
+      removeInstanceFromPreferences(accessor, manager.getInstanceName(), message.getResourceName(),
+          message.getPartitionName());
+      LOG.info("FINISH onBecomeOnlineFromOffline for " + message.getPartitionName() + " on "
+          + manager.getInstanceName());
+
+      int newSize;
+      synchronized (_instanceList) {
+        _instanceList.remove(_instanceList.size() - 1);
+        newSize = _instanceList.size();
+      }
+      Assert.assertEquals(newSize, oldSize); // ensure nothing came in during this time
+    }
+
+    public void onBecomeOfflineFromOnline(Message message, NotificationContext context) {
+      HelixManager manager = context.getManager();
+      LOG.info("onBecomeOfflineFromOnline for " + message.getPartitionName() + " on "
+          + manager.getInstanceName());
+    }
+
+    public void onBecomeDroppedFromOffline(Message message, NotificationContext context) {
+      HelixManager manager = context.getManager();
+      LOG.info("onBecomeDroppedFromOffline for " + message.getPartitionName() + " on "
+          + manager.getInstanceName());
+    }
+
+    public void onBecomeOfflineFromError(Message message, NotificationContext context) {
+      HelixManager manager = context.getManager();
+      LOG.info("onBecomeOfflineFromError for " + message.getPartitionName() + " on "
+          + manager.getInstanceName());
+    }
+  }
+
+  public class PrefListTaskOnlineOfflineStateModelFactory extends
+      StateModelFactory<PrefListTaskOnlineOfflineStateModel> {
+    @Override
+    public PrefListTaskOnlineOfflineStateModel createNewStateModel(String partitionName) {
+      return new PrefListTaskOnlineOfflineStateModel();
+    }
+  }
+}