diff --git a/.github/workflows/Helix-PR-CI.yml b/.github/workflows/Helix-PR-CI.yml index d86c8d8af2..8ed1fb9ade 100644 --- a/.github/workflows/Helix-PR-CI.yml +++ b/.github/workflows/Helix-PR-CI.yml @@ -1,7 +1,7 @@ name: Helix PR CI on: pull_request: - branches: [ master, metaclient, ApplicationClusterManager] # TODO: remove side branch + branches: [ master, metaclient, ApplicationClusterManager, helix-gateway-service] # TODO: remove side branch paths-ignore: - '.github/**' - 'helix-front/**' diff --git a/bump-snapshot.sh b/bump-snapshot.sh index 9a664ae8b7..3cf1d9ad1c 100755 --- a/bump-snapshot.sh +++ b/bump-snapshot.sh @@ -45,6 +45,7 @@ mv metrics-common/metrics-common-$version-SNAPSHOT.ivy metrics-common/metrics-co mv zookeeper-api/zookeeper-api-$version-SNAPSHOT.ivy zookeeper-api/zookeeper-api-$new_version-SNAPSHOT.ivy mv helix-view-aggregator/helix-view-aggregator-$version-SNAPSHOT.ivy helix-view-aggregator/helix-view-aggregator-$new_version-SNAPSHOT.ivy mv meta-client/meta-client-$version-SNAPSHOT.ivy meta-client/meta-client-$new_version-SNAPSHOT.ivy +mv helix-gateway/helix-gateway-$version-SNAPSHOT.ivy helix-gateway/helix-gateway-$new_version-SNAPSHOT.ivy find . -type f -name '*.ivy' -exec sed -i "s/$version/$new_version/g" {} \; diff --git a/bump-up.sh b/bump-up.sh index e3d7db6731..8abffea91e 100755 --- a/bump-up.sh +++ b/bump-up.sh @@ -74,7 +74,7 @@ echo "bump up: $current_version -> $new_version" update_pom_version "pom.xml" $current_version for module in "metrics-common" "metadata-store-directory-common" "zookeeper-api" "helix-common" "helix-core" \ - "helix-rest" "helix-lock" "helix-view-aggregator" "helix-agent" "meta-client"; do + "helix-rest" "helix-lock" "helix-view-aggregator" "helix-agent" "meta-client" "helix-gateway"; do update_ivy $module update_pom_version $module/pom.xml $current_version done diff --git a/helix-core/src/main/java/org/apache/helix/controller/common/CapacityNode.java b/helix-core/src/main/java/org/apache/helix/controller/common/CapacityNode.java index fa5068e132..bd49040ffd 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/common/CapacityNode.java +++ b/helix-core/src/main/java/org/apache/helix/controller/common/CapacityNode.java @@ -21,23 +21,55 @@ import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; +import org.apache.helix.controller.rebalancer.topology.Topology; +import org.apache.helix.model.ClusterConfig; +import org.apache.helix.model.ClusterTopologyConfig; +import org.apache.helix.model.InstanceConfig; + /** * A Node is an entity that can serve capacity recording purpose. It has a capacity and knowledge * of partitions assigned to it, so it can decide if it can receive additional partitions. */ -public class CapacityNode { +public class CapacityNode implements Comparable { private int _currentlyAssigned; private int _capacity; - private final String _id; + private final String _instanceName; + private final String _logicaId; + private final String _faultZone; private final Map> _partitionMap; - public CapacityNode(String id) { - _partitionMap = new HashMap<>(); - _currentlyAssigned = 0; - this._id = id; + /** + * Constructor used for non-topology-aware use case + * @param instanceName The instance name of this node + * @param capacity The capacity of this node + */ + public CapacityNode(String instanceName, int capacity) { + this(instanceName, null, null, null); + this._capacity = capacity; + } + + /** + * Constructor used for topology-aware use case + * @param instanceName The instance name of this node + * @param clusterConfig The cluster config for current helix cluster + * @param clusterTopologyConfig The cluster topology config for current helix cluster + * @param instanceConfig The instance config for current instance + */ + public CapacityNode(String instanceName, ClusterConfig clusterConfig, + ClusterTopologyConfig clusterTopologyConfig, InstanceConfig instanceConfig) { + this._instanceName = instanceName; + this._logicaId = clusterTopologyConfig != null ? instanceConfig.getLogicalId( + clusterTopologyConfig.getEndNodeType()) : instanceName; + this._faultZone = + clusterConfig != null ? computeFaultZone(clusterConfig, instanceConfig) : null; + this._partitionMap = new HashMap<>(); + this._capacity = + clusterConfig != null ? clusterConfig.getGlobalMaxPartitionAllowedPerInstance() : 0; + this._currentlyAssigned = 0; } /** @@ -52,11 +84,25 @@ public boolean canAdd(String resource, String partition) { && _partitionMap.get(resource).contains(partition))) { return false; } + + // Add the partition to the resource's set of partitions in this node _partitionMap.computeIfAbsent(resource, k -> new HashSet<>()).add(partition); _currentlyAssigned++; return true; } + /** + * Checks if a specific resource + partition is assigned to this node. + * + * @param resource the name of the resource + * @param partition the partition + * @return {@code true} if the resource + partition is assigned to this node, {@code false} otherwise + */ + public boolean hasPartition(String resource, String partition) { + Set partitions = _partitionMap.get(resource); + return partitions != null && partitions.contains(partition); + } + /** * Set the capacity of this node * @param capacity The capacity to set @@ -66,11 +112,27 @@ public void setCapacity(int capacity) { } /** - * Get the ID of this node - * @return The ID of this node + * Get the instance name of this node + * @return The instance name of this node */ - public String getId() { - return _id; + public String getInstanceName() { + return _instanceName; + } + + /** + * Get the logical id of this node + * @return The logical id of this node + */ + public String getLogicalId() { + return _logicaId; + } + + /** + * Get the fault zone of this node + * @return The fault zone of this node + */ + public String getFaultZone() { + return _faultZone; } /** @@ -84,8 +146,40 @@ public int getCurrentlyAssigned() { @Override public String toString() { StringBuilder sb = new StringBuilder(); - sb.append("##########\nname=").append(_id).append("\nassigned:").append(_currentlyAssigned) - .append("\ncapacity:").append(_capacity); + sb.append("##########\nname=").append(_instanceName).append("\nassigned:") + .append(_currentlyAssigned).append("\ncapacity:").append(_capacity).append("\nlogicalId:") + .append(_logicaId).append("\nfaultZone:").append(_faultZone); return sb.toString(); } + + @Override + public int compareTo(CapacityNode o) { + if (_logicaId != null) { + return _logicaId.compareTo(o.getLogicalId()); + } + return _instanceName.compareTo(o.getInstanceName()); + } + + /** + * Computes the fault zone id based on the domain and fault zone type when topology is enabled. + * For example, when + * the domain is "zone=2, instance=testInstance" and the fault zone type is "zone", this function + * returns "2". + * If cannot find the fault zone type, this function leaves the fault zone id as the instance name. + * TODO: change the return value to logical id when no fault zone type found. Also do the same for + * waged rebalancer in helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/AssignableNode.java + */ + private String computeFaultZone(ClusterConfig clusterConfig, InstanceConfig instanceConfig) { + LinkedHashMap instanceTopologyMap = + Topology.computeInstanceTopologyMap(clusterConfig, instanceConfig.getInstanceName(), + instanceConfig, true /*earlyQuitTillFaultZone*/); + + StringBuilder faultZoneStringBuilder = new StringBuilder(); + for (Map.Entry entry : instanceTopologyMap.entrySet()) { + faultZoneStringBuilder.append(entry.getValue()); + faultZoneStringBuilder.append('/'); + } + faultZoneStringBuilder.setLength(faultZoneStringBuilder.length() - 1); + return faultZoneStringBuilder.toString(); + } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/dataproviders/ResourceControllerDataProvider.java b/helix-core/src/main/java/org/apache/helix/controller/dataproviders/ResourceControllerDataProvider.java index b006a4f1e0..50088e69c1 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/dataproviders/ResourceControllerDataProvider.java +++ b/helix-core/src/main/java/org/apache/helix/controller/dataproviders/ResourceControllerDataProvider.java @@ -41,16 +41,20 @@ import org.apache.helix.controller.LogUtil; import org.apache.helix.controller.common.CapacityNode; import org.apache.helix.controller.pipeline.Pipeline; -import org.apache.helix.controller.rebalancer.strategy.GreedyRebalanceStrategy; +import org.apache.helix.controller.rebalancer.strategy.StickyRebalanceStrategy; import org.apache.helix.controller.rebalancer.waged.WagedInstanceCapacity; import org.apache.helix.controller.rebalancer.waged.WagedResourceWeightsProvider; +import org.apache.helix.controller.stages.CurrentStateOutput; import org.apache.helix.controller.stages.MissingTopStateRecord; +import org.apache.helix.model.ClusterConfig; +import org.apache.helix.model.ClusterTopologyConfig; import org.apache.helix.model.CustomizedState; import org.apache.helix.model.CustomizedStateConfig; import org.apache.helix.model.CustomizedView; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.Partition; import org.apache.helix.model.ResourceAssignment; import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.slf4j.Logger; @@ -73,6 +77,9 @@ public class ResourceControllerDataProvider extends BaseControllerDataProvider { // a map from customized state type to customized view cache private final Map _customizedViewCacheMap; + // maintain a cache of ideal state (preference list + best possible assignment) which will be managed ondemand in rebalancer + private final Map _ondemandIdealStateCache; + // maintain a cache of bestPossible assignment across pipeline runs // TODO: this is only for customRebalancer, remove it and merge it with _idealMappingCache. private Map _resourceAssignmentCache; @@ -149,6 +156,7 @@ public String getObjName(ExternalView obj) { _refreshedChangeTypes = ConcurrentHashMap.newKeySet(); _customizedStateCache = new CustomizedStateCache(this, _aggregationEnabledTypes); _customizedViewCacheMap = new HashMap<>(); + _ondemandIdealStateCache = new HashMap<>(); } public synchronized void refresh(HelixDataAccessor accessor) { @@ -182,16 +190,17 @@ public synchronized void refresh(HelixDataAccessor accessor) { refreshStablePartitionList(getIdealStates()); refreshDisabledInstancesForAllPartitionsSet(); - if (getClusterConfig().getGlobalMaxPartitionAllowedPerInstance() != -1) { - buildSimpleCapacityMap(getClusterConfig().getGlobalMaxPartitionAllowedPerInstance()); + if (getClusterConfig() != null + && getClusterConfig().getGlobalMaxPartitionAllowedPerInstance() != -1) { + buildSimpleCapacityMap(); // Remove all cached IdealState because it is a global computation cannot partially be // performed for some resources. The computation is simple as well not taking too much resource // to recompute the assignments. - Set cachedGreedyIdealStates = _idealMappingCache.values().stream().filter( + Set cachedStickyIdealStates = _idealMappingCache.values().stream().filter( record -> record.getSimpleField(IdealState.IdealStateProperty.REBALANCE_STRATEGY.name()) - .equals(GreedyRebalanceStrategy.class.getName())).map(ZNRecord::getId) + .equals(StickyRebalanceStrategy.class.getName())).map(ZNRecord::getId) .collect(Collectors.toSet()); - _idealMappingCache.keySet().removeAll(cachedGreedyIdealStates); + _idealMappingCache.keySet().removeAll(cachedStickyIdealStates); } LogUtil.logInfo(logger, getClusterEventId(), String.format( @@ -388,6 +397,28 @@ public Map> getLastTopStateLocationMap() { return _lastTopStateLocationMap; } + /** + * Get cached ideal state (preference list + best possible assignment) for a resource + * @param resource + * @return + */ + public ZNRecord getCachedOndemandIdealState(String resource) { + return _ondemandIdealStateCache.get(resource); + } + + /** + * Cache ideal state (preference list + best possible assignment) for a resource + * @param resource + * @return + */ + public void setCachedOndemandIdealState(String resource, ZNRecord idealState) { + _ondemandIdealStateCache.put(resource, idealState); + } + + public void clearCachedOndemandIdealStates() { + _ondemandIdealStateCache.clear(); + } + /** * Get cached resourceAssignment (bestPossible mapping) for a resource * @param resource @@ -552,11 +583,16 @@ public WagedInstanceCapacity getWagedInstanceCapacity() { return _wagedInstanceCapacity; } - private void buildSimpleCapacityMap(int globalMaxPartitionAllowedPerInstance) { + private void buildSimpleCapacityMap() { + ClusterConfig clusterConfig = getClusterConfig(); + ClusterTopologyConfig clusterTopologyConfig = + ClusterTopologyConfig.createFromClusterConfig(clusterConfig); + Map instanceConfigMap = getAssignableInstanceConfigMap(); _simpleCapacitySet = new HashSet<>(); - for (String instance : getEnabledLiveInstances()) { - CapacityNode capacityNode = new CapacityNode(instance); - capacityNode.setCapacity(globalMaxPartitionAllowedPerInstance); + for (String instanceName : getAssignableInstances()) { + CapacityNode capacityNode = + new CapacityNode(instanceName, clusterConfig, clusterTopologyConfig, + instanceConfigMap.getOrDefault(instanceName, new InstanceConfig(instanceName))); _simpleCapacitySet.add(capacityNode); } } @@ -565,6 +601,35 @@ public Set getSimpleCapacitySet() { return _simpleCapacitySet; } + public void populateSimpleCapacitySetUsage(final Set resourceNameSet, + final CurrentStateOutput currentStateOutput) { + // Convert the assignableNodes to map for quick lookup + Map simpleCapacityMap = new HashMap<>(); + for (CapacityNode node : _simpleCapacitySet) { + simpleCapacityMap.put(node.getInstanceName(), node); + } + for (String resourceName : resourceNameSet) { + // Process current state mapping + populateCapacityNodeUsageFromStateMap(resourceName, simpleCapacityMap, + currentStateOutput.getCurrentStateMap(resourceName)); + // Process pending state mapping + populateCapacityNodeUsageFromStateMap(resourceName, simpleCapacityMap, + currentStateOutput.getPendingMessageMap(resourceName)); + } + } + + private void populateCapacityNodeUsageFromStateMap(String resourceName, + Map simpleCapacityMap, Map> stateMap) { + for (Map.Entry> entry : stateMap.entrySet()) { + for (String instanceName : entry.getValue().keySet()) { + CapacityNode node = simpleCapacityMap.get(instanceName); + if (node != null) { + node.canAdd(resourceName, entry.getKey().getPartitionName()); + } + } + } + } + private void refreshDisabledInstancesForAllPartitionsSet() { _disabledInstancesForAllPartitionsSet.clear(); Collection allConfigs = getInstanceConfigMap().values(); diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/ConditionBasedRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/ConditionBasedRebalancer.java new file mode 100644 index 0000000000..6e68033c49 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/ConditionBasedRebalancer.java @@ -0,0 +1,215 @@ +package org.apache.helix.controller.rebalancer; + +/* + * 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.ArrayList; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.helix.HelixException; +import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; +import org.apache.helix.controller.rebalancer.condition.RebalanceCondition; +import org.apache.helix.controller.stages.CurrentStateOutput; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Partition; +import org.apache.helix.model.Resource; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.zookeeper.datamodel.ZNRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@code ConditionBasedRebalancer} class extends the {@link AbstractRebalancer} and + * perform the rebalance operation based on specific list of conditions defined by the + * {@link RebalanceCondition} interface. + */ +public class ConditionBasedRebalancer extends AbstractRebalancer { + private static final Logger LOG = LoggerFactory.getLogger(ConditionBasedRebalancer.class); + private final List _rebalanceConditions; + + public ConditionBasedRebalancer() { + this._rebalanceConditions = new ArrayList<>(); + } + + public ConditionBasedRebalancer(List rebalanceConditions) { + this._rebalanceConditions = rebalanceConditions; + } + + /** + * Compute new Ideal State iff all conditions are met, otherwise just return from cached Ideal State + * + * @param resourceName the name of the resource for which to compute the new ideal state. + * @param currentIdealState the current {@link IdealState} of the resource. + * @param currentStateOutput the current state output, containing the actual states of the + * partitions. + * @param clusterData the {@link ResourceControllerDataProvider} instance providing + * additional data required for the computation. + * @return the newly computed {@link IdealState} for the resource. + */ + @Override + public IdealState computeNewIdealState(String resourceName, IdealState currentIdealState, + CurrentStateOutput currentStateOutput, ResourceControllerDataProvider clusterData) { + ZNRecord cachedIdealState = clusterData.getCachedOndemandIdealState(resourceName); + // If previous placement list exists in cache && all condition met -> return cached value + if (cachedIdealState != null && cachedIdealState.getListFields() != null + && !cachedIdealState.getListFields().isEmpty() && !this._rebalanceConditions.stream() + .allMatch(condition -> condition.shouldPerformRebalance(clusterData))) { + return new IdealState(cachedIdealState); + } + + LOG.info("Computing IdealState for " + resourceName); + + List partitions = getStablePartitionList(clusterData, currentIdealState); + String stateModelName = currentIdealState.getStateModelDefRef(); + StateModelDefinition stateModelDef = clusterData.getStateModelDef(stateModelName); + if (stateModelDef == null) { + LOG.error("State Model Definition null for resource: " + resourceName); + throw new HelixException("State Model Definition null for resource: " + resourceName); + } + Map assignableLiveInstance = clusterData.getAssignableLiveInstances(); + int replicas = currentIdealState.getReplicaCount(assignableLiveInstance.size()); + + LinkedHashMap stateCountMap = + stateModelDef.getStateCountMap(assignableLiveInstance.size(), replicas); + Set assignableLiveNodes = new HashSet<>(assignableLiveInstance.keySet()); + Set assignableNodes = new HashSet<>(clusterData.getAssignableInstances()); + assignableNodes.removeAll(clusterData.getDisabledInstances()); + assignableLiveNodes.retainAll(assignableNodes); + + Map> currentMapping = + currentMapping(currentStateOutput, resourceName, partitions, stateCountMap); + + // If there are nodes tagged with resource name, use only those nodes + Set taggedNodes = new HashSet(); + Set taggedLiveNodes = new HashSet(); + if (currentIdealState.getInstanceGroupTag() != null) { + for (String instanceName : assignableNodes) { + if (clusterData.getAssignableInstanceConfigMap().get(instanceName) + .containsTag(currentIdealState.getInstanceGroupTag())) { + taggedNodes.add(instanceName); + if (assignableLiveInstance.containsKey(instanceName)) { + taggedLiveNodes.add(instanceName); + } + } + } + if (!taggedLiveNodes.isEmpty()) { + // live nodes exist that have this tag + if (LOG.isInfoEnabled()) { + LOG.info( + "found the following participants with tag " + currentIdealState.getInstanceGroupTag() + + " for " + resourceName + ": " + taggedLiveNodes); + } + } else if (taggedNodes.isEmpty()) { + // no live nodes and no configured nodes have this tag + LOG.warn("Resource " + resourceName + " has tag " + currentIdealState.getInstanceGroupTag() + + " but no configured participants have this tag"); + } else { + // configured nodes have this tag, but no live nodes have this tag + LOG.warn("Resource " + resourceName + " has tag " + currentIdealState.getInstanceGroupTag() + + " but no live participants have this tag"); + } + assignableNodes = new HashSet<>(taggedNodes); + assignableLiveNodes = new HashSet<>(taggedLiveNodes); + } + + // sort node lists to ensure consistent preferred assignments + List assignableNodesList = + assignableNodes.stream().sorted().collect(Collectors.toList()); + List assignableLiveNodesList = + assignableLiveNodes.stream().sorted().collect(Collectors.toList()); + + int maxPartition = currentIdealState.getMaxPartitionsPerInstance(); + _rebalanceStrategy = + getRebalanceStrategy(currentIdealState.getRebalanceStrategy(), partitions, resourceName, + stateCountMap, maxPartition); + ZNRecord newMapping = + _rebalanceStrategy.computePartitionAssignment(assignableNodesList, assignableLiveNodesList, + currentMapping, clusterData); + + if (LOG.isDebugEnabled()) { + LOG.debug("currentMapping: {}", currentMapping); + LOG.debug("stateCountMap: {}", stateCountMap); + LOG.debug("assignableLiveNodes: {}", assignableLiveNodes); + LOG.debug("assignableNodes: {}", assignableNodes); + LOG.debug("maxPartition: {}", maxPartition); + LOG.debug("newMapping: {}", newMapping); + } + + IdealState newIdealState = new IdealState(resourceName); + newIdealState.getRecord().setSimpleFields(currentIdealState.getRecord().getSimpleFields()); + newIdealState.setRebalanceMode(IdealState.RebalanceMode.FULL_AUTO); + newIdealState.getRecord().setListFields(newMapping.getListFields()); + + clusterData.setCachedOndemandIdealState(resourceName, newIdealState.getRecord()); + + return newIdealState; + } + + /** + * Compute new assignment iff all conditions are met, otherwise just return from cached assignment + * + * @param cache the {@link ResourceControllerDataProvider} instance providing + * metadata and state information about the cluster. + * @param idealState the {@link IdealState} representing the current ideal state. + * @param resource the {@link Resource} for which to compute the best possible partition + * state. + * @param currentStateOutput the {@link CurrentStateOutput} containing the current states of the + * partitions. + * @return the {@link ResourceAssignment} representing the best possible state assignment for the + * partitions of the resource. + */ + @Override + public ResourceAssignment computeBestPossiblePartitionState(ResourceControllerDataProvider cache, + IdealState idealState, Resource resource, CurrentStateOutput currentStateOutput) { + ZNRecord cachedIdealState = cache.getCachedOndemandIdealState(resource.getResourceName()); + // If previous assignment map exists in cache && all condition met -> return cached value + if (cachedIdealState.getMapFields() != null && !cachedIdealState.getMapFields().isEmpty() + && !this._rebalanceConditions.stream() + .allMatch(condition -> condition.shouldPerformRebalance(cache))) { + ResourceAssignment partitionMapping = new ResourceAssignment(resource.getResourceName()); + for (Partition partition : resource.getPartitions()) { + partitionMapping.addReplicaMap(partition, + cachedIdealState.getMapFields().get(partition.getPartitionName())); + } + return partitionMapping; + } + + LOG.info("Computing BestPossibleMapping for " + resource.getResourceName()); + + // TODO: Change the logic to apply different assignment strategy + ResourceAssignment assignment = + super.computeBestPossiblePartitionState(cache, idealState, resource, currentStateOutput); + // Cache the assignment so no need to recompute the result next time + cachedIdealState.setMapFields(assignment.getRecord().getMapFields()); + cache.setCachedOndemandIdealState(resource.getResourceName(), cachedIdealState); + + if (LOG.isDebugEnabled()) { + LOG.debug("Processed resource: {}", resource.getResourceName()); + LOG.debug("Final Mapping of resource : {}", assignment); + } + return assignment; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/ConfigChangeBasedCondition.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/ConfigChangeBasedCondition.java new file mode 100644 index 0000000000..9441ed1b90 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/ConfigChangeBasedCondition.java @@ -0,0 +1,30 @@ +package org.apache.helix.controller.rebalancer.condition; + +/* + * 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 org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; + +public class ConfigChangeBasedCondition implements RebalanceCondition { + @Override + public boolean shouldPerformRebalance(ResourceControllerDataProvider cache) { + // TODO: implement the condition check for config change + return false; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/RebalanceCondition.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/RebalanceCondition.java new file mode 100644 index 0000000000..cf04f00f4c --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/RebalanceCondition.java @@ -0,0 +1,38 @@ +package org.apache.helix.controller.rebalancer.condition; + +/* + * 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 org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; + +/** + * The {@code RebalanceCondition} interface defines a condition under which a rebalance operation + * should be performed. Implementations of this interface provide specific criteria to determine + * whether a rebalance is necessary based on the current state of the system. + */ +public interface RebalanceCondition { + /** + * Determines whether a rebalance should be performed based on the provided + * {@link ResourceControllerDataProvider} cache data. + * + * @param cache the {@code ResourceControllerDataProvider} cached data of the resources being managed. + * @return {@code true} if the rebalance should be performed, {@code false} otherwise. + */ + boolean shouldPerformRebalance(ResourceControllerDataProvider cache); +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/RebalanceConditionsBuilder.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/RebalanceConditionsBuilder.java new file mode 100644 index 0000000000..c62b9d7d22 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/RebalanceConditionsBuilder.java @@ -0,0 +1,41 @@ +package org.apache.helix.controller.rebalancer.condition; + +/* + * 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.ArrayList; +import java.util.List; + +public class RebalanceConditionsBuilder { + private final List _rebalanceConditions = new ArrayList<>(); + + public RebalanceConditionsBuilder withConfigChangeBasedCondition() { + _rebalanceConditions.add(new ConfigChangeBasedCondition()); + return this; + } + + public RebalanceConditionsBuilder withTopologyChangeBasedCondition() { + _rebalanceConditions.add(new TopologyChangeBasedCondition()); + return this; + } + + public List build() { + return _rebalanceConditions; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/TopologyChangeBasedCondition.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/TopologyChangeBasedCondition.java new file mode 100644 index 0000000000..82e9398a8b --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/condition/TopologyChangeBasedCondition.java @@ -0,0 +1,30 @@ +package org.apache.helix.controller.rebalancer.condition; + +/* + * 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 org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; + +public class TopologyChangeBasedCondition implements RebalanceCondition { + @Override + public boolean shouldPerformRebalance(ResourceControllerDataProvider cache) { + // TODO: implement the condition check for topology change + return false; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/GreedyRebalanceStrategy.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/GreedyRebalanceStrategy.java deleted file mode 100644 index 60580c40a4..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/GreedyRebalanceStrategy.java +++ /dev/null @@ -1,103 +0,0 @@ -package org.apache.helix.controller.rebalancer.strategy; - -/* - * 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.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.helix.controller.common.CapacityNode; -import org.apache.helix.zookeeper.datamodel.ZNRecord; -import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class GreedyRebalanceStrategy implements RebalanceStrategy { - private static Logger logger = LoggerFactory.getLogger(GreedyRebalanceStrategy.class); - private String _resourceName; - private List _partitions; - private LinkedHashMap _states; - - public GreedyRebalanceStrategy() { - } - - @Override - public void init(String resourceName, final List partitions, - final LinkedHashMap states, int maximumPerNode) { - _resourceName = resourceName; - _partitions = partitions; - _states = states; - } - - @Override - public ZNRecord computePartitionAssignment(final List allNodes, final List liveNodes, - final Map> currentMapping, ResourceControllerDataProvider clusterData) { - int numReplicas = countStateReplicas(); - ZNRecord znRecord = new ZNRecord(_resourceName); - if (liveNodes.size() == 0) { - return znRecord; - } - - if (clusterData.getSimpleCapacitySet() == null) { - logger.warn("No capacity set for resource: " + _resourceName); - return znRecord; - } - - // Sort the assignable nodes by id - List assignableNodes = new ArrayList<>(clusterData.getSimpleCapacitySet()); - Collections.sort(assignableNodes, Comparator.comparing(CapacityNode::getId)); - - // Assign partitions to node by order. - for (int i = 0, index = 0; i < _partitions.size(); i++) { - int startIndex = index; - List preferenceList = new ArrayList<>(); - for (int j = 0; j < numReplicas; j++) { - while (index - startIndex < assignableNodes.size()) { - CapacityNode node = assignableNodes.get(index++ % assignableNodes.size()); - if (node.canAdd(_resourceName, _partitions.get(i))) { - preferenceList.add(node.getId()); - break; - } - } - - if (index - startIndex >= assignableNodes.size()) { - // If the all nodes have been tried out, then no node can be assigned. - logger.warn("No enough assignable nodes for resource: " + _resourceName); - } - } - znRecord.setListField(_partitions.get(i), preferenceList); - } - - return znRecord; - } - - private int countStateReplicas() { - int total = 0; - for (Integer count : _states.values()) { - total += count; - } - return total; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/StickyRebalanceStrategy.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/StickyRebalanceStrategy.java new file mode 100644 index 0000000000..0471f128ea --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/StickyRebalanceStrategy.java @@ -0,0 +1,199 @@ +package org.apache.helix.controller.rebalancer.strategy; + +/* + * 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.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.helix.controller.common.CapacityNode; +import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; +import org.apache.helix.zookeeper.datamodel.ZNRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class StickyRebalanceStrategy implements RebalanceStrategy { + private static final Logger logger = LoggerFactory.getLogger(StickyRebalanceStrategy.class); + private String _resourceName; + private List _partitions; + private LinkedHashMap _states; + private int _statesReplicaCount; + + public StickyRebalanceStrategy() { + } + + @Override + public void init(String resourceName, final List partitions, + final LinkedHashMap states, int maximumPerNode) { + _resourceName = resourceName; + _partitions = partitions; + _states = states; + if (_states != null) { + _statesReplicaCount = _states.values().stream().mapToInt(Integer::intValue).sum(); + } + } + + @Override + public ZNRecord computePartitionAssignment(final List allNodes, + final List liveNodes, final Map> currentMapping, + ResourceControllerDataProvider clusterData) { + ZNRecord znRecord = new ZNRecord(_resourceName); + if (liveNodes.isEmpty()) { + return znRecord; + } + + if (clusterData.getSimpleCapacitySet() == null) { + logger.warn("No capacity set for resource: {}", _resourceName); + return znRecord; + } + + // Filter out the nodes if not in the liveNodes parameter + // Note the liveNodes parameter here might be processed within the rebalancer, e.g. filter based on tags + Set assignableNodeSet = new HashSet<>(clusterData.getSimpleCapacitySet()); + Set liveNodesSet = new HashSet<>(liveNodes); + assignableNodeSet.removeIf(n -> !liveNodesSet.contains(n.getInstanceName())); + + // Convert the assignableNodes to map for quick lookup + Map assignableNodeMap = assignableNodeSet.stream() + .collect(Collectors.toMap(CapacityNode::getInstanceName, node -> node)); + + // Populate valid state map given current mapping + Map> stateMap = + populateValidAssignmentMapFromCurrentMapping(currentMapping, assignableNodeMap); + + if (logger.isDebugEnabled()) { + logger.debug("currentMapping: {}", currentMapping); + logger.debug("stateMap: {}", stateMap); + } + + // Sort the assignable nodes by id + List assignableNodeList = assignableNodeSet.stream().sorted() + .collect(Collectors.toList()); + + // Assign partitions to node by order. + for (int i = 0, index = 0; i < _partitions.size(); i++) { + int startIndex = index; + Map currentFaultZoneCountMap = new HashMap<>(); + int remainingReplica = _statesReplicaCount; + if (stateMap.containsKey(_partitions.get(i))) { + Set existingReplicas = stateMap.get(_partitions.get(i)); + remainingReplica = remainingReplica - existingReplicas.size(); + for (String instanceName : existingReplicas) { + String faultZone = assignableNodeMap.get(instanceName).getFaultZone(); + currentFaultZoneCountMap.put(faultZone, + currentFaultZoneCountMap.getOrDefault(faultZone, 0) + 1); + } + } + for (int j = 0; j < remainingReplica; j++) { + while (index - startIndex < assignableNodeList.size()) { + CapacityNode node = assignableNodeList.get(index++ % assignableNodeList.size()); + if (this.canAdd(node, _partitions.get(i), currentFaultZoneCountMap)) { + stateMap.computeIfAbsent(_partitions.get(i), m -> new HashSet<>()) + .add(node.getInstanceName()); + if (node.getFaultZone() != null) { + currentFaultZoneCountMap.put(node.getFaultZone(), + currentFaultZoneCountMap.getOrDefault(node.getFaultZone(), 0) + 1); + } + break; + } + } + + if (index - startIndex >= assignableNodeList.size()) { + // If the all nodes have been tried out, then no node can be assigned. + logger.warn("No enough assignable nodes for resource: {}", _resourceName); + } + } + } + for (Map.Entry> entry : stateMap.entrySet()) { + znRecord.setListField(entry.getKey(), new ArrayList<>(entry.getValue())); + } + if (logger.isDebugEnabled()) { + logger.debug("znRecord: {}", znRecord); + } + + return znRecord; + } + + /** + * Populates a valid state map from the current mapping, filtering out invalid nodes. + * + * @param currentMapping the current mapping of partitions to node states + * @param assignableNodeMap the map of instance name -> nodes that can be assigned + * @return a map of partitions to valid nodes + */ + private Map> populateValidAssignmentMapFromCurrentMapping( + final Map> currentMapping, + final Map assignableNodeMap) { + Map> validAssignmentMap = new HashMap<>(); + if (currentMapping != null) { + for (Map.Entry> entry : currentMapping.entrySet()) { + String partition = entry.getKey(); + Map currentNodeStateMap = new HashMap<>(entry.getValue()); + // Filter out invalid node assignment + currentNodeStateMap.entrySet() + .removeIf(e -> !isValidNodeAssignment(partition, e.getKey(), assignableNodeMap)); + + validAssignmentMap.put(partition, new HashSet<>(currentNodeStateMap.keySet())); + } + } + return validAssignmentMap; + } + + /** + * Checks if a node assignment is valid for a given partition. + * + * @param partition the partition to be assigned + * @param nodeId the ID of the node to be checked + * @param assignableNodeMap the map of node IDs to CapacityNode objects + * @return true if the node is valid for the assignment, false otherwise + */ + private boolean isValidNodeAssignment(final String partition, final String nodeId, + final Map assignableNodeMap) { + CapacityNode node = assignableNodeMap.get(nodeId); + // Return valid when following conditions match: + // 1. Node is in assignableNodeMap + // 2. Node hold current partition or we can assign current partition to the node + return node != null && (node.hasPartition(_resourceName, partition) || node.canAdd( + _resourceName, partition)); + } + + /** + * Checks if it's valid to assign the partition to node + * + * @param node node to assign partition + * @param partition partition name + * @param currentFaultZoneCountMap the map of fault zones -> count + * @return true if it's valid to assign the partition to node, false otherwise + */ + protected boolean canAdd(CapacityNode node, String partition, + Map currentFaultZoneCountMap) { + // Valid assignment when following conditions match: + // 1. Replica is not within the same fault zones of other replicas + // 2. Node has capacity to hold the replica + return !currentFaultZoneCountMap.containsKey(node.getFaultZone()) && node.canAdd(_resourceName, + partition); + } +} + diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java index 2814e4062f..79325bea26 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java @@ -134,6 +134,15 @@ public void process(ClusterEvent event) throws Exception { handleResourceCapacityCalculation(event, (ResourceControllerDataProvider) cache, currentStateOutput); } + + // Populate the capacity for simple CapacityNode + if (cache.getClusterConfig() != null + && cache.getClusterConfig().getGlobalMaxPartitionAllowedPerInstance() != -1 + && cache instanceof ResourceControllerDataProvider) { + final ResourceControllerDataProvider dataProvider = (ResourceControllerDataProvider) cache; + dataProvider.populateSimpleCapacitySetUsage(resourceToRebalance.keySet(), + currentStateExcludingUnknown); + } } // update all pending messages to CurrentStateOutput. diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java index 2044dbcd79..20910c8915 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java @@ -394,7 +394,7 @@ public static synchronized void carryOverPreviousCurrentState(HelixDataAccessor continue; } - // If the the current state is related to tasks, there is no need to carry it over to new session. + // If the current state is related to tasks, there is no need to carry it over to new session. // Note: this check is not necessary due to TaskCurrentStates, but keep it for backwards compatibility if (stateModelDefRef.equals(TaskConstants.STATE_MODEL_NAME)) { continue; diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java index a77acae35a..cd9b009872 100644 --- a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java +++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java @@ -71,7 +71,7 @@ public enum ClusterConfigProperty { // The following concerns maintenance mode MAX_PARTITIONS_PER_INSTANCE, // The maximum number of partitions that an instance can serve in this cluster. - // This only works for GreedyRebalanceStrategy. + // This only works for StickyRebalanceStrategy. // TODO: if we want to support this for other rebalancers, we need to implement that logic GLOBAL_MAX_PARTITIONS_ALLOWED_PER_INSTANCE, // The following two include offline AND disabled instances diff --git a/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java b/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java index 278cdfc7ac..50ab9c93a5 100644 --- a/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java +++ b/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java @@ -19,7 +19,6 @@ * under the License. */ -import com.google.common.base.Preconditions; import java.io.IOException; import java.lang.management.ManagementFactory; import java.lang.reflect.Method; @@ -33,6 +32,7 @@ import javax.management.MBeanServerConnection; import javax.management.ObjectName; +import com.google.common.base.Preconditions; import org.apache.helix.BaseDataAccessor; import org.apache.helix.ConfigAccessor; import org.apache.helix.HelixAdmin; @@ -50,6 +50,7 @@ import org.apache.helix.controller.pipeline.Pipeline; import org.apache.helix.controller.pipeline.Stage; import org.apache.helix.controller.pipeline.StageContext; +import org.apache.helix.controller.rebalancer.ConditionBasedRebalancer; import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer; import org.apache.helix.controller.rebalancer.strategy.AutoRebalanceStrategy; import org.apache.helix.controller.rebalancer.waged.WagedRebalancer; @@ -385,6 +386,13 @@ protected IdealState createResourceWithWagedRebalance(String clusterName, String -1, WagedRebalancer.class.getName(), null); } + protected IdealState createResourceWithConditionBasedRebalance(String clusterName, String db, + String stateModel, int numPartition, int replica, int minActiveReplica, + String rebalanceStrategy) { + return createResource(clusterName, db, stateModel, numPartition, replica, minActiveReplica, -1, + ConditionBasedRebalancer.class.getName(), rebalanceStrategy); + } + private IdealState createResource(String clusterName, String db, String stateModel, int numPartition, int replica, int minActiveReplica, long delay, String rebalancerClassName, String rebalanceStrategy) { diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/TestGreedyRebalanceStrategy.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/TestGreedyRebalanceStrategy.java deleted file mode 100644 index d90e16136b..0000000000 --- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/TestGreedyRebalanceStrategy.java +++ /dev/null @@ -1,85 +0,0 @@ -package org.apache.helix.controller.rebalancer; - -/* - * 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.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; - -import org.apache.helix.controller.common.CapacityNode; -import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; -import org.apache.helix.controller.rebalancer.strategy.GreedyRebalanceStrategy; -import org.apache.helix.model.ClusterConfig; -import org.apache.helix.zookeeper.datamodel.ZNRecord; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.Test; - -import static org.mockito.Mockito.when; - -public class TestGreedyRebalanceStrategy { - private static final String TEST_CLUSTER_NAME = "TestCluster"; - private static final String TEST_RESOURCE_PREFIX = "TestResource_"; - - @Test - public void testAssignmentWithGlobalPartitionLimit() { - - ResourceControllerDataProvider clusterDataCache = - Mockito.mock(ResourceControllerDataProvider.class); - LinkedHashMap states = new LinkedHashMap(2); - states.put("OFFLINE", 0); - states.put("ONLINE", 1); - - Set capacityNodeSet = new HashSet<>(); - for (int i = 0; i < 5; i++) { - CapacityNode capacityNode = new CapacityNode("Node-" + i); - capacityNode.setCapacity(1); - capacityNodeSet.add(capacityNode); - } - - List liveNodes = - capacityNodeSet.stream().map(CapacityNode::getId).collect(Collectors.toList()); - - List partitions = new ArrayList<>(); - for (int i = 0; i < 3; i++) { - partitions.add(TEST_RESOURCE_PREFIX + "0_" + i); - } - when(clusterDataCache.getSimpleCapacitySet()).thenReturn(capacityNodeSet); - - GreedyRebalanceStrategy greedyRebalanceStrategy = new GreedyRebalanceStrategy(); - greedyRebalanceStrategy.init(TEST_RESOURCE_PREFIX + 0, partitions, states, 1); - greedyRebalanceStrategy.computePartitionAssignment(null, liveNodes, null, clusterDataCache); - - partitions = new ArrayList<>(); - for (int i = 0; i < 2; i++) { - partitions.add(TEST_RESOURCE_PREFIX + "1_" + i); - } - greedyRebalanceStrategy = new GreedyRebalanceStrategy(); - greedyRebalanceStrategy.init(TEST_RESOURCE_PREFIX + 1, partitions, states, 1); - greedyRebalanceStrategy.computePartitionAssignment(null, liveNodes, null, clusterDataCache); - - Assert.assertEquals( - capacityNodeSet.stream().filter(node -> node.getCurrentlyAssigned() != 1).count(), 0); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/TestStickyRebalanceStrategy.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/TestStickyRebalanceStrategy.java new file mode 100644 index 0000000000..acd30c2c76 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/TestStickyRebalanceStrategy.java @@ -0,0 +1,211 @@ +package org.apache.helix.controller.rebalancer; + +/* + * 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.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.helix.controller.common.CapacityNode; +import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; +import org.apache.helix.controller.rebalancer.strategy.StickyRebalanceStrategy; +import org.apache.helix.zookeeper.datamodel.ZNRecord; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.Test; + +import static org.mockito.Mockito.when; + +public class TestStickyRebalanceStrategy { + private static final String TEST_CLUSTER_NAME = "TestCluster"; + private static final String TEST_RESOURCE_PREFIX = "TestResource_"; + + @Test + public void testAssignmentWithGlobalPartitionLimit() { + + ResourceControllerDataProvider clusterDataCache = + Mockito.mock(ResourceControllerDataProvider.class); + LinkedHashMap states = new LinkedHashMap(2); + states.put("OFFLINE", 0); + states.put("ONLINE", 1); + + Set capacityNodeSet = new HashSet<>(); + for (int i = 0; i < 5; i++) { + CapacityNode capacityNode = new CapacityNode("Node-" + i, 1); + capacityNodeSet.add(capacityNode); + } + + List liveNodes = + capacityNodeSet.stream().map(CapacityNode::getInstanceName).collect(Collectors.toList()); + + List partitions = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + partitions.add(TEST_RESOURCE_PREFIX + "0_" + i); + } + when(clusterDataCache.getSimpleCapacitySet()).thenReturn(capacityNodeSet); + + StickyRebalanceStrategy greedyRebalanceStrategy = new StickyRebalanceStrategy(); + greedyRebalanceStrategy.init(TEST_RESOURCE_PREFIX + 0, partitions, states, 1); + greedyRebalanceStrategy.computePartitionAssignment(null, liveNodes, null, clusterDataCache); + + partitions = new ArrayList<>(); + for (int i = 0; i < 2; i++) { + partitions.add(TEST_RESOURCE_PREFIX + "1_" + i); + } + greedyRebalanceStrategy = new StickyRebalanceStrategy(); + greedyRebalanceStrategy.init(TEST_RESOURCE_PREFIX + 1, partitions, states, 1); + greedyRebalanceStrategy.computePartitionAssignment(null, liveNodes, null, clusterDataCache); + + Assert.assertEquals( + capacityNodeSet.stream().filter(node -> node.getCurrentlyAssigned() != 1).count(), 0); + } + + @Test + public void testStickyAssignment() { + final int nReplicas = 4; + final int nPartitions = 4; + final int nNode = 16; + + ResourceControllerDataProvider clusterDataCache = + Mockito.mock(ResourceControllerDataProvider.class); + LinkedHashMap states = new LinkedHashMap(2); + states.put("OFFLINE", 0); + states.put("ONLINE", nReplicas); + + Set capacityNodeSet = new HashSet<>(); + for (int i = 0; i < nNode; i++) { + CapacityNode capacityNode = new CapacityNode("Node-" + i, 1); + capacityNodeSet.add(capacityNode); + } + + List liveNodes = + capacityNodeSet.stream().map(CapacityNode::getInstanceName).collect(Collectors.toList()); + + List partitions = new ArrayList<>(); + for (int i = 0; i < nPartitions; i++) { + partitions.add(TEST_RESOURCE_PREFIX + i); + } + when(clusterDataCache.getSimpleCapacitySet()).thenReturn(capacityNodeSet); + + // Populate previous assignment with currentMapping + Map> currentMapping = new HashMap<>(); + currentMapping.put(TEST_RESOURCE_PREFIX + "0", new HashMap<>()); + currentMapping.get(TEST_RESOURCE_PREFIX + "0").put("Node-0", "ONLINE"); + currentMapping.get(TEST_RESOURCE_PREFIX + "0").put("Node-2", "ONLINE"); + currentMapping.get(TEST_RESOURCE_PREFIX + "0").put("Node-4", "ONLINE"); + currentMapping.get(TEST_RESOURCE_PREFIX + "0").put("Node-6", "ONLINE"); + currentMapping.put(TEST_RESOURCE_PREFIX + "2", new HashMap<>()); + currentMapping.get(TEST_RESOURCE_PREFIX + "2").put("Node-1", "ONLINE"); + currentMapping.get(TEST_RESOURCE_PREFIX + "2").put("Node-5", "ONLINE"); + currentMapping.get(TEST_RESOURCE_PREFIX + "2").put("Node-8", "ONLINE"); + + StickyRebalanceStrategy greedyRebalanceStrategy = new StickyRebalanceStrategy(); + greedyRebalanceStrategy.init(TEST_RESOURCE_PREFIX + 0, partitions, states, 1); + ZNRecord shardAssignment = + greedyRebalanceStrategy.computePartitionAssignment(null, liveNodes, currentMapping, + clusterDataCache); + + // Assert the existing assignment won't be changed + Assert.assertEquals(currentMapping.get(TEST_RESOURCE_PREFIX + "0").keySet(), + new HashSet<>(shardAssignment.getListField(TEST_RESOURCE_PREFIX + "0"))); + Assert.assertTrue(shardAssignment.getListField(TEST_RESOURCE_PREFIX + "2") + .containsAll(currentMapping.get(TEST_RESOURCE_PREFIX + "2").keySet())); + } + + @Test + public void testStickyAssignmentMultipleTimes() { + final int nReplicas = 4; + final int nPartitions = 4; + final int nNode = 12; + + ResourceControllerDataProvider clusterDataCache = + Mockito.mock(ResourceControllerDataProvider.class); + LinkedHashMap states = new LinkedHashMap(2); + states.put("OFFLINE", 0); + states.put("ONLINE", nReplicas); + + Set capacityNodeSet = new HashSet<>(); + for (int i = 0; i < nNode; i++) { + CapacityNode capacityNode = new CapacityNode("Node-" + i, 1); + capacityNodeSet.add(capacityNode); + } + + List liveNodes = + capacityNodeSet.stream().map(CapacityNode::getInstanceName).collect(Collectors.toList()); + + List partitions = new ArrayList<>(); + for (int i = 0; i < nPartitions; i++) { + partitions.add(TEST_RESOURCE_PREFIX + i); + } + when(clusterDataCache.getSimpleCapacitySet()).thenReturn(capacityNodeSet); + + StickyRebalanceStrategy stickyRebalanceStrategy = new StickyRebalanceStrategy(); + stickyRebalanceStrategy.init(TEST_RESOURCE_PREFIX + 0, partitions, states, 1); + // First round assignment computation: + // 1. Without previous assignment (currentMapping is null) + // 2. Without enough assignable nodes + ZNRecord firstRoundShardAssignment = + stickyRebalanceStrategy.computePartitionAssignment(null, liveNodes, null, clusterDataCache); + + // Assert only 3 partitions are fulfilled with assignment + Assert.assertEquals(firstRoundShardAssignment.getListFields().entrySet().stream() + .filter(e -> e.getValue().size() == nReplicas).count(), 3); + + // Assign 4 more nodes which is used in second round assignment computation + for (int i = nNode; i < nNode + 4; i++) { + CapacityNode capacityNode = new CapacityNode("Node-" + i, 1); + capacityNodeSet.add(capacityNode); + } + + liveNodes = + capacityNodeSet.stream().map(CapacityNode::getInstanceName).collect(Collectors.toList()); + + // Populate previous assignment (currentMapping) with first round assignment computation result + Map> currentMapping = new HashMap<>(); + firstRoundShardAssignment.getListFields().entrySet().stream() + .filter(e -> e.getValue().size() == nReplicas).forEach(e -> { + currentMapping.put(e.getKey(), new HashMap<>()); + for (String nodeId : e.getValue()) { + currentMapping.get(e.getKey()).put(nodeId, "ONLINE"); + } + }); + + // Second round assignment computation: + // 1. With previous assignment (currentMapping) + // 2. With enough assignable nodes + ZNRecord secondRoundShardAssignment = + stickyRebalanceStrategy.computePartitionAssignment(null, liveNodes, currentMapping, + clusterDataCache); + + // Assert all partitions have been assigned with enough replica + Assert.assertEquals(secondRoundShardAssignment.getListFields().entrySet().stream() + .filter(e -> e.getValue().size() == nReplicas).count(), nPartitions); + // For previously existing assignment, assert there is no assignment change + currentMapping.forEach((partition, nodeMapping) -> { + Assert.assertEquals(nodeMapping.keySet(), + new HashSet<>(secondRoundShardAssignment.getListField(partition))); + }); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestStickyRebalanceStrategy.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestStickyRebalanceStrategy.java new file mode 100644 index 0000000000..04860d6927 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestStickyRebalanceStrategy.java @@ -0,0 +1,444 @@ +package org.apache.helix.integration.rebalancer; + +/* + * 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.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.ConfigAccessor; +import org.apache.helix.TestHelper; +import org.apache.helix.common.ZkTestBase; +import org.apache.helix.controller.rebalancer.strategy.StickyRebalanceStrategy; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; +import org.apache.helix.model.BuiltInStateModelDefinitions; +import org.apache.helix.model.ClusterConfig; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier; +import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class TestStickyRebalanceStrategy extends ZkTestBase { + static final int NUM_NODE = 18; + static final int ADDITIONAL_NUM_NODE = 2; + protected static final int START_PORT = 12918; + protected static final int PARTITIONS = 2; + protected static final int REPLICAS = 3; + protected final String CLASS_NAME = getShortClassName(); + protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME; + + protected ClusterControllerManager _controller; + protected List _participants = new ArrayList<>(); + protected List _additionalParticipants = new ArrayList<>(); + protected Map _instanceNameZoneMap = new HashMap<>(); + protected int _minActiveReplica = 0; + protected ZkHelixClusterVerifier _clusterVerifier; + protected List _testDBs = new ArrayList<>(); + protected ConfigAccessor _configAccessor; + protected String[] TestStateModels = + {BuiltInStateModelDefinitions.MasterSlave.name(), BuiltInStateModelDefinitions.LeaderStandby.name(), BuiltInStateModelDefinitions.OnlineOffline.name()}; + + @BeforeClass + public void beforeClass() throws Exception { + System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); + _configAccessor = new ConfigAccessor(_gZkClient); + + _gSetupTool.addCluster(CLUSTER_NAME, true); + + for (int i = 0; i < NUM_NODE; i++) { + _participants.addAll(addInstance("" + START_PORT + i, "zone-" + i % REPLICAS, true)); + } + + for (int i = NUM_NODE; i < NUM_NODE + ADDITIONAL_NUM_NODE; i++) { + _additionalParticipants.addAll( + addInstance("" + START_PORT + i, "zone-" + i % REPLICAS, false)); + } + + // start controller + String controllerName = CONTROLLER_PREFIX + "_0"; + _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName); + _controller.syncStart(); + + _clusterVerifier = + new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkClient(_gZkClient) + .setWaitTillVerify(TestHelper.DEFAULT_REBALANCE_PROCESSING_WAIT_TIME).build(); + + enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true); + } + + @AfterClass + public void afterClass() throws Exception { + if (_clusterVerifier != null) { + _clusterVerifier.close(); + } + /* + shutdown order: 1) disconnect the controller 2) disconnect participants + */ + _controller.syncStop(); + for (MockParticipantManager participant : _participants) { + participant.syncStop(); + } + deleteCluster(CLUSTER_NAME); + System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); + } + + @BeforeMethod + public void beforeTest() { + // Restart any participants that has been disconnected from last test + for (int i = 0; i < _participants.size(); i++) { + if (!_participants.get(i).isConnected()) { + _participants.set(i, new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, + _participants.get(i).getInstanceName())); + _participants.get(i).syncStart(); + } + } + + // Stop any additional participants that has been added from last test + for (MockParticipantManager additionalParticipant : _additionalParticipants) { + if (additionalParticipant.isConnected()) { + additionalParticipant.syncStop(); + } + } + } + + @AfterMethod + public void afterTest() throws InterruptedException { + // delete all DBs create in last test + for (String db : _testDBs) { + _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, db); + } + _testDBs.clear(); + _clusterVerifier.verifyByPolling(); + } + + @Test + public void testNoSameZoneAssignment() throws Exception { + setTopologyAwareAndGlobalMaxPartitionAllowedPerInstanceInCluster(CLUSTER_NAME, 1); + Map externalViews = createTestDBs(); + for (ExternalView ev : externalViews.values()) { + Map> assignments = ev.getRecord().getMapFields(); + Assert.assertNotNull(assignments); + Assert.assertEquals(assignments.size(), PARTITIONS); + for (Map assignmentMap : assignments.values()) { + Assert.assertEquals(assignmentMap.keySet().size(), REPLICAS); + Set zoneSet = new HashSet<>(); + for (String instanceName : assignmentMap.keySet()) { + zoneSet.add(_instanceNameZoneMap.get(instanceName)); + } + Assert.assertEquals(zoneSet.size(), REPLICAS); + } + } + } + @Test + public void testFirstTimeAssignmentWithNoInitialLiveNodes() throws Exception { + setTopologyAwareAndGlobalMaxPartitionAllowedPerInstanceInCluster(CLUSTER_NAME, 1); + // Shut down all the nodes + for (int i = 0; i < NUM_NODE; i++) { + _participants.get(i).syncStop(); + } + // Create resource + Map externalViewsBefore = createTestDBs(); + // Start all the nodes + for (int i = 0; i < NUM_NODE; i++) { + _participants.set(i, new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, + _participants.get(i).getInstanceName())); + _participants.get(i).syncStart(); + } + Assert.assertTrue(_clusterVerifier.verifyByPolling()); + + Map externalViewsAfter = new HashMap<>(); + for (String db : _testDBs) { + ExternalView ev = + _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db); + externalViewsAfter.put(db, ev); + } + validateAllPartitionAssigned(externalViewsAfter); + } + + @Test + public void testNoPartitionMovementWithNewInstanceAdd() throws Exception { + setTopologyAwareAndGlobalMaxPartitionAllowedPerInstanceInCluster(CLUSTER_NAME, 1); + Map externalViewsBefore = createTestDBs(); + + // Start more new instances + for (int i = 0; i < _additionalParticipants.size(); i++) { + _additionalParticipants.set(i, new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, + _additionalParticipants.get(i).getInstanceName())); + _additionalParticipants.get(i).syncStart(); + } + Assert.assertTrue(_clusterVerifier.verifyByPolling()); + // All partition assignment should remain the same + for (String db : _testDBs) { + ExternalView ev = + _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db); + IdealState is = + _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db); + validateNoPartitionMove(is, externalViewsBefore.get(db), ev); + } + } + + @Test + public void testNoPartitionMovementWithInstanceDown() throws Exception { + setTopologyAwareAndGlobalMaxPartitionAllowedPerInstanceInCluster(CLUSTER_NAME, 1); + Map externalViewsBefore = createTestDBs(); + + // Shut down 2 instances + _participants.get(0).syncStop(); + _participants.get(_participants.size() - 1).syncStop(); + Assert.assertTrue(_clusterVerifier.verifyByPolling()); + // No movement for previous remaining assignment + Map externalViewsAfter = new HashMap<>(); + Map idealStates = new HashMap<>(); + for (String db : _testDBs) { + ExternalView ev = + _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db); + IdealState is = + _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db); + externalViewsAfter.put(db, ev); + idealStates.put(db, is); + } + validateNoPartitionMoveWithDiffCount(idealStates, externalViewsBefore, externalViewsAfter, 2); + } + + @Test + public void testNoPartitionMovementWithInstanceRestart() throws Exception { + setTopologyAwareAndGlobalMaxPartitionAllowedPerInstanceInCluster(CLUSTER_NAME, 1); + // Create resource + Map externalViewsBefore = createTestDBs(); + // Shut down half of the nodes + for (int i = 0; i < _participants.size(); i++) { + if (i % 2 == 0) { + _participants.get(i).syncStop(); + } + } + Assert.assertTrue(_clusterVerifier.verifyByPolling()); + + Map externalViewsAfter = new HashMap<>(); + Map idealStates = new HashMap<>(); + for (String db : _testDBs) { + ExternalView ev = + _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db); + IdealState is = + _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db); + externalViewsAfter.put(db, ev); + idealStates.put(db, is); + } + validateNoPartitionMoveWithDiffCount(idealStates, externalViewsBefore, externalViewsAfter, + NUM_NODE / 2); + + // Start all the nodes + for (int i = 0; i < _participants.size(); i++) { + if (!_participants.get(i).isConnected()) { + _participants.set(i, new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, + _participants.get(i).getInstanceName())); + _participants.get(i).syncStart(); + } + } + Assert.assertTrue(_clusterVerifier.verifyByPolling()); + + for (String db : _testDBs) { + ExternalView ev = + _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db); + IdealState is = + _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db); + validateNoPartitionMove(is, externalViewsBefore.get(db), ev); + } + } + + @Test + public void testFirstTimeAssignmentWithStackingPlacement() throws Exception { + setTopologyAwareAndGlobalMaxPartitionAllowedPerInstanceInCluster(CLUSTER_NAME, 2); + Map externalViewsBefore = createTestDBs(); + validateAllPartitionAssigned(externalViewsBefore); + } + + @Test + public void testNoPartitionMovementWithNewInstanceAddWithStackingPlacement() throws Exception { + setTopologyAwareAndGlobalMaxPartitionAllowedPerInstanceInCluster(CLUSTER_NAME, 2); + Map externalViewsBefore = createTestDBs(); + + // Start more new instances + for (int i = 0; i < _additionalParticipants.size(); i++) { + _additionalParticipants.set(i, new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, + _additionalParticipants.get(i).getInstanceName())); + _additionalParticipants.get(i).syncStart(); + } + Assert.assertTrue(_clusterVerifier.verifyByPolling()); + // All partition assignment should remain the same + for (String db : _testDBs) { + ExternalView ev = + _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db); + IdealState is = + _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db); + validateNoPartitionMove(is, externalViewsBefore.get(db), ev); + } + } + + @Test + public void testNoPartitionMovementWithInstanceDownWithStackingPlacement() throws Exception { + setTopologyAwareAndGlobalMaxPartitionAllowedPerInstanceInCluster(CLUSTER_NAME, 2); + // Shut down half of the nodes given we allow stacking placement + for (int i = 0; i < NUM_NODE / 2; i++) { + _participants.get(i).syncStop(); + } + Map externalViewsBefore = createTestDBs(); + + // Shut down 2 instances + _participants.get(_participants.size() - 1).syncStop(); + _participants.get(_participants.size() - 2).syncStop(); + Assert.assertTrue(_clusterVerifier.verifyByPolling()); + // No movement for previous remaining assignment + Map externalViewsAfter = new HashMap<>(); + Map idealStates = new HashMap<>(); + for (String db : _testDBs) { + ExternalView ev = + _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db); + IdealState is = + _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db); + externalViewsAfter.put(db, ev); + idealStates.put(db, is); + } + validateNoPartitionMoveWithDiffCount(idealStates, externalViewsBefore, externalViewsAfter, 4); + } + + // create test DBs, wait it converged and return externalviews + protected Map createTestDBs() throws InterruptedException { + Map externalViews = new HashMap(); + int i = 0; + for (String stateModel : TestStateModels) { + String db = "Test-DB-" + i++; + createResourceWithConditionBasedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, REPLICAS, + _minActiveReplica, StickyRebalanceStrategy.class.getName()); + _testDBs.add(db); + } + Assert.assertTrue(_clusterVerifier.verifyByPolling()); + for (String db : _testDBs) { + ExternalView ev = + _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db); + externalViews.put(db, ev); + } + return externalViews; + } + + protected void validateNoPartitionMove(IdealState is, ExternalView evBefore, + ExternalView evAfter) { + for (String partition : is.getPartitionSet()) { + Map assignmentsBefore = evBefore.getRecord().getMapField(partition); + Map assignmentsAfter = evAfter.getRecord().getMapField(partition); + + Set instancesBefore = new HashSet<>(assignmentsBefore.keySet()); + Set instancesAfter = new HashSet<>(assignmentsAfter.keySet()); + + Assert.assertEquals(instancesBefore, instancesAfter, + String.format("%s has been moved to new instances, before: %s, after: %s", partition, + assignmentsBefore, assignmentsAfter)); + } + } + + protected void validateNoPartitionMoveWithDiffCount(Map idealStates, + Map externalViewsBefore, Map externalViewsAfter, + int diffCount) { + for (Map.Entry entry : idealStates.entrySet()) { + String resourceName = entry.getKey(); + IdealState is = entry.getValue(); + for (String partition : is.getPartitionSet()) { + Map assignmentsBefore = + externalViewsBefore.get(resourceName).getRecord().getMapField(partition); + Map assignmentsAfter = + externalViewsAfter.get(resourceName).getRecord().getMapField(partition); + + Set instancesBefore = new HashSet<>(assignmentsBefore.keySet()); + Set instancesAfter = new HashSet<>(assignmentsAfter.keySet()); + + if (instancesBefore.size() == instancesAfter.size()) { + Assert.assertEquals(instancesBefore, instancesAfter, + String.format("%s has been moved to new instances, before: %s, after: %s", partition, + assignmentsBefore, assignmentsAfter)); + } else { + Assert.assertTrue(instancesBefore.containsAll(instancesAfter), + String.format("%s has been moved to new instances, before: %s, after: %s", partition, + assignmentsBefore, assignmentsAfter)); + diffCount = diffCount - (instancesBefore.size() - instancesAfter.size()); + } + } + } + Assert.assertEquals(diffCount, 0, + String.format("Partition movement detected, before: %s, after: %s", externalViewsBefore, + externalViewsAfter)); + } + + private void validateAllPartitionAssigned(Map externalViewsBefore) { + for (ExternalView ev : externalViewsBefore.values()) { + Map> assignments = ev.getRecord().getMapFields(); + Assert.assertNotNull(assignments); + Assert.assertEquals(assignments.size(), PARTITIONS); + for (Map assignmentMap : assignments.values()) { + Assert.assertEquals(assignmentMap.keySet().size(), REPLICAS); + } + } + } + + private void setTopologyAwareAndGlobalMaxPartitionAllowedPerInstanceInCluster(String clusterName, + int maxPartitionAllowed) { + ClusterConfig clusterConfig = _configAccessor.getClusterConfig(clusterName); + clusterConfig.setTopology("/zone/host/logicalId"); + clusterConfig.setFaultZoneType("zone"); + clusterConfig.setTopologyAwareEnabled(true); + clusterConfig.setGlobalMaxPartitionAllowedPerInstance(maxPartitionAllowed); + _configAccessor.setClusterConfig(clusterName, clusterConfig); + } + + private List addInstance(String instanceNameSuffix, String zone, + boolean enabled) { + List participants = new ArrayList<>(); + String storageNodeName = PARTICIPANT_PREFIX + "_" + instanceNameSuffix; + _instanceNameZoneMap.put(storageNodeName, zone); + _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName); + + String domain = + String.format("zone=%s,host=%s,logicalId=%s", zone, storageNodeName, instanceNameSuffix); + InstanceConfig instanceConfig = + _configAccessor.getInstanceConfig(CLUSTER_NAME, storageNodeName); + instanceConfig.setDomain(domain); + _gSetupTool.getClusterManagementTool() + .setInstanceConfig(CLUSTER_NAME, storageNodeName, instanceConfig); + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, storageNodeName); + if (enabled) { + // start dummy participant + participant.syncStart(); + } + participants.add(participant); + + return participants; + } +} diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestGreedyRebalanceWithGlobalPerInstancePartitionLimit.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestStickyRebalanceWithGlobalPerInstancePartitionLimit.java similarity index 77% rename from helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestGreedyRebalanceWithGlobalPerInstancePartitionLimit.java rename to helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestStickyRebalanceWithGlobalPerInstancePartitionLimit.java index 6b675a8a35..e0d160b78e 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestGreedyRebalanceWithGlobalPerInstancePartitionLimit.java +++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestStickyRebalanceWithGlobalPerInstancePartitionLimit.java @@ -1,5 +1,24 @@ package org.apache.helix.integration.rebalancer; +/* + * 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.Date; import java.util.HashMap; import java.util.Map; @@ -15,7 +34,7 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -public class TestGreedyRebalanceWithGlobalPerInstancePartitionLimit extends TaskTestBase { +public class TestStickyRebalanceWithGlobalPerInstancePartitionLimit extends TaskTestBase { @BeforeClass public void beforeClass() throws Exception { @@ -48,14 +67,14 @@ public void testGreedyRebalanceWithGlobalPerInstancePartitionLimit() throws Inte IdealState idealState = _gSetupTool.getClusterManagementTool() .getResourceIdealState(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB); idealState.setRebalanceStrategy( - "org.apache.helix.controller.rebalancer.strategy.GreedyRebalanceStrategy"); + "org.apache.helix.controller.rebalancer.strategy.StickyRebalanceStrategy"); _gSetupTool.getClusterManagementTool() .setResourceIdealState(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, idealState); Assert.assertTrue(_clusterVerifier.verifyByPolling()); _gSetupTool.getClusterManagementTool().addResource(CLUSTER_NAME, "NewDB", 2, "OnlineOffline", IdealState.RebalanceMode.FULL_AUTO.name(), - "org.apache.helix.controller.rebalancer.strategy.GreedyRebalanceStrategy"); + "org.apache.helix.controller.rebalancer.strategy.StickyRebalanceStrategy"); _gSetupTool.getClusterManagementTool().rebalance(CLUSTER_NAME, "NewDB", 1); Assert.assertTrue(_clusterVerifier.verifyByPolling()); diff --git a/helix-gateway/LICENSE b/helix-gateway/LICENSE new file mode 100644 index 0000000000..d78ae52e78 --- /dev/null +++ b/helix-gateway/LICENSE @@ -0,0 +1,270 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. + + + +For xstream: + +Copyright (c) 2003-2006, Joe Walnes +Copyright (c) 2006-2009, 2011 XStream Committers +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this list of +conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, this list of +conditions and the following disclaimer in the documentation and/or other materials provided +with the distribution. + +3. Neither the name of XStream nor the names of its contributors may be used to endorse +or promote products derived from this software without specific prior written +permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES +OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT +SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED +TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR +BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY +WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH +DAMAGE. + +for jline: + +Copyright (c) 2002-2006, Marc Prud'hommeaux +All rights reserved. + +Redistribution and use in source and binary forms, with or +without modification, are permitted provided that the following +conditions are met: + +Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with +the distribution. + +Neither the name of JLine nor the names of its contributors +may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, +BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY +AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO +EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, +OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED +AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING +IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED +OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/helix-gateway/NOTICE b/helix-gateway/NOTICE new file mode 100644 index 0000000000..ff5a745feb --- /dev/null +++ b/helix-gateway/NOTICE @@ -0,0 +1,37 @@ +Apache Helix +Copyright 2014 The Apache Software Foundation + + +I. Included Software + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). +Licensed under the Apache License 2.0. + +This product includes software developed at +Codehaus (http://www.codehaus.org/). +Licensed under the BSD License. + +This product includes software developed at +jline (http://jline.sourceforge.net/). +Licensed under the BSD License. + +This product includes software developed at +restlet (http://www.restlet.org/about/legal). +Licensed under the Apache License 2.0. + +This product includes software developed at +Google (http://www.google.com/). +Licensed under the Apache License 2.0. + +This product includes software developed at +snakeyaml (http://www.snakeyaml.org/). +Licensed under the Apache License 2.0. + +This product includes software developed at +zkclient (https://github.com/sgroschupf/zkclient). +Licensed under the Apache License 2.0. + +II. License Summary +- Apache License 2.0 +- BSD License \ No newline at end of file diff --git a/helix-gateway/helix-gateway-1.4.2-SNAPSHOT.ivy b/helix-gateway/helix-gateway-1.4.2-SNAPSHOT.ivy new file mode 100644 index 0000000000..f161540287 --- /dev/null +++ b/helix-gateway/helix-gateway-1.4.2-SNAPSHOT.ivy @@ -0,0 +1,56 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/helix-gateway/pom.xml b/helix-gateway/pom.xml new file mode 100644 index 0000000000..ef6a7f8263 --- /dev/null +++ b/helix-gateway/pom.xml @@ -0,0 +1,257 @@ + + + + + org.apache.helix + helix + 1.4.4-SNAPSHOT + + 4.0.0 + + helix-gateway + bundle + Apache Helix :: Helix Gateway + + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + Apache Software Foundation + http://www.apache.org + + + + 3.24.0 + 3.24.0 + 1.59.1 + + org.slf4j*;version="[1.7,2)", + org.apache.logging.log4j*;version="[2.17,3)", + org.apache.logging.slf4j*;version="[2.17,3)", + * + + org.apache.helix*;proto.org.apache.helix*;version="${project.version};-noimport:=true + + + + + + org.apache.helix + metrics-common + ${project.version} + + + org.apache.helix + zookeeper-api + ${project.version} + + + org.slf4j + slf4j-api + 1.7.32 + + + org.apache.logging.log4j + log4j-slf4j-impl + 2.17.1 + + + org.testng + testng + test + + + org.apache.helix + helix-core + + + org.apache.helix + helix-core + test-jar + test + + + io.grpc + grpc-services + ${grpc.version} + + + io.grpc + grpc-netty-shaded + runtime + ${grpc.version} + + + io.grpc + grpc-protobuf + ${grpc.version} + + + io.grpc + grpc-stub + ${grpc.version} + + + com.google.protobuf + protobuf-java-util + ${protobuf.version} + + + org.apache.tomcat + annotations-api + 6.0.53 + provided + + + io.grpc + grpc-testing + test + ${grpc.version} + + + javax.annotation + javax.annotation-api + 1.3.2 + + + org.mockito + mockito-core + test + + + + + + kr.motd.maven + os-maven-plugin + 1.6.0 + + + + + ${basedir} + + DISCLAIMER + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.12.1 + + + JDK 8 + compile + + compile + + + ${project.build.outputDirectory}_jdk8 + 8 + true + + + + JDK 11 + compile + + compile + + + 11 + true + + + + + + org.apache.maven.plugins + maven-jar-plugin + 3.3.0 + + + default-package-jdk8 + package + + jar + + + ${project.build.outputDirectory}_jdk8 + jdk8 + + + + default-package-jdk11 + package + + jar + + + ${project.build.outputDirectory} + jdk11 + + + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + com.google.protobuf:protoc:${protoc.version}:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} + + + + + compile + compile-custom + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/assemble/assembly.xml + + + + + package + + single + + + + + + + diff --git a/helix-gateway/src/assemble/assembly.xml b/helix-gateway/src/assemble/assembly.xml new file mode 100644 index 0000000000..a3d451fd14 --- /dev/null +++ b/helix-gateway/src/assemble/assembly.xml @@ -0,0 +1,60 @@ + + + + pkg + + tar + + + + target/helix-gateway-pkg/bin + bin + unix + 0755 + 0755 + + + target/helix-gateway-pkg/repo/ + repo + 0755 + 0755 + + **/*.xml + + + + target/helix-gateway-pkg/conf + conf + unix + 0755 + 0755 + + + ${project.basedir} + / + + LICENSE + NOTICE + DISCLAIMER + + 0755 + + + diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/HelixGatewayMain.java b/helix-gateway/src/main/java/org/apache/helix/gateway/HelixGatewayMain.java new file mode 100644 index 0000000000..5ad06d908c --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/HelixGatewayMain.java @@ -0,0 +1,50 @@ +package org.apache.helix.gateway; + +/* + * 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.io.IOException; +import org.apache.helix.gateway.channel.GatewayServiceChannelConfig; +import org.apache.helix.gateway.service.GatewayServiceManager; + +import static java.lang.Integer.*; + + +/** + * Main class for Helix Gateway. + * It starts the Helix Gateway grpc service. + * args0: zk address + * args1: helix gateway groc server port + */ +public final class HelixGatewayMain { + + private HelixGatewayMain() { + } + + public static void main(String[] args) throws IOException { + // Create a new server + GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder builder = + new GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder(); + GatewayServiceManager manager = + new GatewayServiceManager(args[0], builder.setGrpcServerPort(parseInt(args[1])).build()); + + manager.startService(); + } +} + diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/api/constant/GatewayServiceConfigConstant.java b/helix-gateway/src/main/java/org/apache/helix/gateway/api/constant/GatewayServiceConfigConstant.java new file mode 100644 index 0000000000..be5fa1a114 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/api/constant/GatewayServiceConfigConstant.java @@ -0,0 +1,28 @@ +package org.apache.helix.gateway.api.constant; + +/* + * 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. + */ + +public class GatewayServiceConfigConstant { + public static final int DEFAULT_SERVER_HEARTBEAT_INTERVAL = 60; + public static final int DEFAULT_AMX_ALLOWED_CLIENT_HEARTBEAT_INTERVAL = 60; + public static final int DEFAULT_CLIENT_TIMEOUT = 5 * 60; + public static final int DEFAULT_POLL_INTERVAL_SEC = 60; + public static final int DEFAULT_HEALTH_TIMEOUT_SEC = 60; +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/api/constant/GatewayServiceEventType.java b/helix-gateway/src/main/java/org/apache/helix/gateway/api/constant/GatewayServiceEventType.java new file mode 100644 index 0000000000..b5cff99789 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/api/constant/GatewayServiceEventType.java @@ -0,0 +1,26 @@ +package org.apache.helix.gateway.api.constant; + +/* + * 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. + */ + +public enum GatewayServiceEventType { + CONNECT, // init connection to gateway service + UPDATE, // update state transition result + DISCONNECT // shutdown connection to gateway service. +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/api/constant/GatewayServiceManagerConstant.java b/helix-gateway/src/main/java/org/apache/helix/gateway/api/constant/GatewayServiceManagerConstant.java new file mode 100644 index 0000000000..d8fdfec06e --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/api/constant/GatewayServiceManagerConstant.java @@ -0,0 +1,25 @@ +package org.apache.helix.gateway.api.constant; + +/* + * 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. + */ + +public class GatewayServiceManagerConstant { + public static final String TARGET_STATE_ASSIGNMENT_KEY_NAME = "Assignment"; + public static final String TIMESTAMP_KEY = "Timestamp"; +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/api/service/HelixGatewayServiceChannel.java b/helix-gateway/src/main/java/org/apache/helix/gateway/api/service/HelixGatewayServiceChannel.java new file mode 100644 index 0000000000..10055834e5 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/api/service/HelixGatewayServiceChannel.java @@ -0,0 +1,84 @@ +package org.apache.helix.gateway.api.service; + +/* + * 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.io.IOException; +import org.apache.helix.gateway.service.GatewayServiceEvent; +import org.apache.helix.gateway.service.GatewayServiceManager; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass; + + +/** + * Helix Gateway Service channel interface provides API for inbound and outbound communication between + * Gateway service and application instances. + */ +public interface HelixGatewayServiceChannel { + + /** + * Gateway service send a state transition message to a connected participant. + * + * @param instanceName the name of the participant + */ + void sendStateChangeRequests(String instanceName, HelixGatewayServiceOuterClass.ShardChangeRequests shardChangeRequests); + + /** + * Send a GatewayServiceEvent to gateway manager for helix instances changes. + * Event could be a connection closed event (event type DISCONNECT), + * an initial connection establish event that contains a map of current chard states (event type CONNECT), + * or a state transition result message (event type UPDATE). + * + * The default implementation push an event to the Gateway Service Manager. + * + * @param gatewayServiceManager the Gateway Service Manager + * @param event the event to push + */ + default void pushClientEventToGatewayManager(GatewayServiceManager gatewayServiceManager, GatewayServiceEvent event) { + gatewayServiceManager.onGatewayServiceEvent(event); + } + + /** + * Start the gateway service channel. + * + * @throws IOException if the channel cannot be started + */ + public void start() throws IOException; + + /** + * Stop the gateway service channel forcefully. + */ + public void stop(); + + + // TODO: remove the following 2 apis in future changes + /** + * Gateway service close connection with error. This function is called when manager wants to close client + * connection when there is an error. e.g. HelixManager connection is lost. + * @param instanceName instance name + * @param reason reason for closing connection + */ + public void closeConnectionWithError(String clusterName, String instanceName, String reason); + + /** + * Gateway service close client connection with success. This function is called when manager wants to close client + * connection gracefully, e.g., when gateway service is shutting down. + * @param instanceName instance name + */ + public void completeConnection(String clusterName, String instanceName); +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/channel/GatewayServiceChannelConfig.java b/helix-gateway/src/main/java/org/apache/helix/gateway/channel/GatewayServiceChannelConfig.java new file mode 100644 index 0000000000..acdc405cc2 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/channel/GatewayServiceChannelConfig.java @@ -0,0 +1,299 @@ +package org.apache.helix.gateway.channel; + +/* + * 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.Map; +import java.util.Properties; + +import static org.apache.helix.gateway.api.constant.GatewayServiceConfigConstant.*; +import static org.apache.helix.gateway.channel.GatewayServiceChannelConfig.FileBasedConfigType.*; + + +public class GatewayServiceChannelConfig { + // Mode to get helix participant information (inbound information). This included health check and shard state transition response + // We do not support hybrid mode as of now, (i.e. have push mode for participant liveness detection and pull mode for shard state) + public enum ChannelMode { + PUSH_MODE, // The gateway service passively receives participant information + POLL_MODE // The gateway service actively polls participant information + } + + // NOTE: + // For outbound information - stateTransition request, Gateway service will always push the state transition message. + // We do not support participant poll mode for stateTransition request as of now. + + // channel type for the following 3 information - participant liveness detection, shard state transition request and response + // By default, they are all grpc server, user could define them separately. + public enum ChannelType { + GRPC_SERVER, + GRPC_CLIENT, + FILE + } + + // service configs + + // service mode for inbound information. + private final ChannelMode _channelMode; + // channel type for participant liveness detection + private final ChannelType _participantConnectionChannelType; + // channel for sending and receiving shard state transition request and shard state response + private final ChannelType _shardStateChannelType; + + // grpc server configs + private final int _grpcServerPort; + private final int _serverHeartBeatInterval; + private final int _maxAllowedClientHeartBeatInterval; + private final int _clientTimeout; + private final boolean _enableReflectionService; + + // poll mode config + private final int _pollIntervalSec; + private final int _pollStartDelaySec; + private final int _pollHealthCheckTimeoutSec; + private final int _targetFileUpdateIntervalSec; + private final Map> _participantLivenessEndpointMap; + private final Properties _pollModeConfigs; + + public enum FileBasedConfigType { + PARTICIPANT_CURRENT_STATE_PATH, + SHARD_TARGET_STATE_PATH + } + + // getters + public ChannelMode getChannelMode() { + return _channelMode; + } + + public ChannelType getParticipantConnectionChannelType() { + return _participantConnectionChannelType; + } + + public ChannelType getShardStateChannelType() { + return _shardStateChannelType; + } + + public int getGrpcServerPort() { + return _grpcServerPort; + } + + public int getServerHeartBeatInterval() { + return _serverHeartBeatInterval; + } + + public int getMaxAllowedClientHeartBeatInterval() { + return _maxAllowedClientHeartBeatInterval; + } + + public int getClientTimeout() { + return _clientTimeout; + } + + public boolean getEnableReflectionService() { + return _enableReflectionService; + } + + public int getPollIntervalSec() { + return _pollIntervalSec; + } + + public Map> getParticipantLivenessEndpointMap() { + return _participantLivenessEndpointMap; + } + + public int getPollStartDelaySec() { + return _pollStartDelaySec; + } + + public int getPollHealthCheckTimeoutSec() { + return _pollHealthCheckTimeoutSec; + } + + public int getTargetFileUpdateIntervalSec() { + return _targetFileUpdateIntervalSec; + } + + public String getPollModeConfig(FileBasedConfigType type) { + return _pollModeConfigs.getProperty(type.toString()); + } + + private GatewayServiceChannelConfig(int grpcServerPort, ChannelMode channelMode, + ChannelType participantConnectionChannelType, ChannelType shardStateChannelType, int serverHeartBeatInterval, + int maxAllowedClientHeartBeatInterval, int clientTimeout, boolean enableReflectionService, int pollIntervalSec, + int pollStartDelaySec, int pollHealthCheckTimeoutSec, int targetFileUpdateIntervalSec, + Properties pollModeConfigs, Map> participantLivenessEndpointMap) { + _grpcServerPort = grpcServerPort; + _channelMode = channelMode; + _participantConnectionChannelType = participantConnectionChannelType; + _shardStateChannelType = shardStateChannelType; + _serverHeartBeatInterval = serverHeartBeatInterval; + _maxAllowedClientHeartBeatInterval = maxAllowedClientHeartBeatInterval; + _clientTimeout = clientTimeout; + _enableReflectionService = enableReflectionService; + _pollIntervalSec = pollIntervalSec; + _pollStartDelaySec = pollStartDelaySec; + _pollHealthCheckTimeoutSec = pollHealthCheckTimeoutSec; + _targetFileUpdateIntervalSec = targetFileUpdateIntervalSec; + _pollModeConfigs = pollModeConfigs; + _participantLivenessEndpointMap = participantLivenessEndpointMap; + } + + public static class GatewayServiceProcessorConfigBuilder { + + // service configs + private ChannelMode _channelMode = ChannelMode.PUSH_MODE; + private ChannelType _participantConnectionChannelType = ChannelType.GRPC_SERVER; + private ChannelType _shardStatenChannelType = ChannelType.GRPC_SERVER; + + // grpc server configs + private int _grpcServerPort; + private int _serverHeartBeatInterval = DEFAULT_SERVER_HEARTBEAT_INTERVAL; + private int _maxAllowedClientHeartBeatInterval = DEFAULT_AMX_ALLOWED_CLIENT_HEARTBEAT_INTERVAL; + private int _clientTimeout = DEFAULT_CLIENT_TIMEOUT; + private boolean _enableReflectionService = true; + + // poll mode config + private int _pollIntervalSec = DEFAULT_POLL_INTERVAL_SEC; + // poll mode config + private Properties _pollModeConfigs; + private int _pollStartDelaySec = DEFAULT_POLL_INTERVAL_SEC; + private int _pollHealthCheckTimeoutSec = DEFAULT_HEALTH_TIMEOUT_SEC; + private int _targetFileUpdateIntervalSec = DEFAULT_POLL_INTERVAL_SEC; + private Map> _healthCheckEndpointMap; + + public GatewayServiceProcessorConfigBuilder setChannelMode(ChannelMode channelMode) { + _channelMode = channelMode; + return this; + } + + public GatewayServiceProcessorConfigBuilder setParticipantConnectionChannelType(ChannelType channelMode) { + _participantConnectionChannelType = channelMode; + return this; + } + + public GatewayServiceProcessorConfigBuilder setShardStateProcessorType(ChannelType channelMode) { + _shardStatenChannelType = channelMode; + return this; + } + + public GatewayServiceProcessorConfigBuilder setGrpcServerPort(int grpcServerPort) { + _grpcServerPort = grpcServerPort; + return this; + } + + public GatewayServiceProcessorConfigBuilder setServerHeartBeatInterval(int serverHeartBeatInterval) { + _serverHeartBeatInterval = serverHeartBeatInterval; + return this; + } + + public GatewayServiceProcessorConfigBuilder setMaxAllowedClientHeartBeatInterval( + int maxAllowedClientHeartBeatInterval) { + _maxAllowedClientHeartBeatInterval = maxAllowedClientHeartBeatInterval; + return this; + } + + public GatewayServiceProcessorConfigBuilder setClientTimeout(int clientTimeout) { + _clientTimeout = clientTimeout; + return this; + } + + public GatewayServiceProcessorConfigBuilder setEnableReflectionService(boolean enableReflectionService) { + _enableReflectionService = enableReflectionService; + return this; + } + + public GatewayServiceProcessorConfigBuilder setPollIntervalSec(int pollIntervalSec) { + _pollIntervalSec = pollIntervalSec; + return this; + } + + public GatewayServiceProcessorConfigBuilder addPollModeConfig(FileBasedConfigType type, String value) { + if (_pollModeConfigs == null) { + _pollModeConfigs = new Properties(); + } + _pollModeConfigs.put(type.toString(), value); + return this; + } + + public GatewayServiceProcessorConfigBuilder setPollStartDelaySec(int pollStartDelaySec) { + _pollStartDelaySec = pollStartDelaySec; + return this; + } + + public GatewayServiceProcessorConfigBuilder setPollHealthCheckTimeout(int pollHealthCheckTimeout) { + _pollHealthCheckTimeoutSec = pollHealthCheckTimeout; + return this; + } + + public GatewayServiceProcessorConfigBuilder setTargetFileUpdateIntervalSec(int targetFileUpdateIntervalSec) { + _targetFileUpdateIntervalSec = targetFileUpdateIntervalSec; + return this; + } + + public GatewayServiceProcessorConfigBuilder setHealthCheckEndpointMap(Map> healthCheckEndpointMap) { + _healthCheckEndpointMap = healthCheckEndpointMap; + return this; + } + + public void validate() { + switch (_participantConnectionChannelType) { + case GRPC_SERVER: + if (_grpcServerPort == 0) { + throw new IllegalArgumentException("Grpc server port must be set for grpc server channel type"); + } + if (_shardStatenChannelType != ChannelType.GRPC_SERVER) { + throw new IllegalArgumentException( + "In case of GRPC server, Participant connection channel type and shard state channel type must be the same"); + } + break; + case FILE: + if (_healthCheckEndpointMap == null || _healthCheckEndpointMap.isEmpty()) { + throw new IllegalArgumentException("Health check endpoint map must be set for file channel type"); + } + break; + default: + break; + } + + switch (_shardStatenChannelType) { + case GRPC_SERVER: + if (_participantConnectionChannelType != ChannelType.GRPC_SERVER) { + throw new IllegalArgumentException( + "In case of GRPC server, Participant connection channel type and shard state channel type must be the same"); + } + break; + case FILE: + if (_pollModeConfigs == null || _pollModeConfigs.getProperty(SHARD_TARGET_STATE_PATH.name()) == null + || _pollModeConfigs.getProperty(SHARD_TARGET_STATE_PATH.name()).isEmpty()) { + throw new IllegalArgumentException("Current state and target state path must be set for file channel type"); + } + break; + default: + break; + } + } + + public GatewayServiceChannelConfig build() { + validate(); + return new GatewayServiceChannelConfig(_grpcServerPort, _channelMode, _participantConnectionChannelType, + _shardStatenChannelType, _serverHeartBeatInterval, _maxAllowedClientHeartBeatInterval, _clientTimeout, + _enableReflectionService, _pollIntervalSec, _pollStartDelaySec, _pollHealthCheckTimeoutSec, + _targetFileUpdateIntervalSec, _pollModeConfigs, _healthCheckEndpointMap); + } + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/channel/HelixGatewayServiceChannelFactory.java b/helix-gateway/src/main/java/org/apache/helix/gateway/channel/HelixGatewayServiceChannelFactory.java new file mode 100644 index 0000000000..796e844882 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/channel/HelixGatewayServiceChannelFactory.java @@ -0,0 +1,42 @@ +package org.apache.helix.gateway.channel; + +/* + * 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 org.apache.helix.gateway.api.service.HelixGatewayServiceChannel; +import org.apache.helix.gateway.service.GatewayServiceManager; + + +public class HelixGatewayServiceChannelFactory { + + public static HelixGatewayServiceChannel createServiceChannel(GatewayServiceChannelConfig config, + GatewayServiceManager manager) { + + if (config.getChannelMode() == GatewayServiceChannelConfig.ChannelMode.PUSH_MODE) { + if (config.getParticipantConnectionChannelType() == GatewayServiceChannelConfig.ChannelType.GRPC_SERVER) { + return new HelixGatewayServiceGrpcService(manager, config); + } + } else { + return new HelixGatewayServicePollModeChannel(manager, config); + } + throw new IllegalArgumentException( + "Unsupported channel mode and type combination: " + config.getChannelMode() + " , " + + config.getParticipantConnectionChannelType()); + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/channel/HelixGatewayServiceGrpcService.java b/helix-gateway/src/main/java/org/apache/helix/gateway/channel/HelixGatewayServiceGrpcService.java new file mode 100644 index 0000000000..c83b015833 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/channel/HelixGatewayServiceGrpcService.java @@ -0,0 +1,239 @@ +package org.apache.helix.gateway.channel; + +/* + * 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 com.google.common.annotations.VisibleForTesting; +import io.grpc.Server; +import io.grpc.ServerBuilder; +import io.grpc.Status; +import io.grpc.stub.StreamObserver; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.helix.gateway.api.service.HelixGatewayServiceChannel; +import org.apache.helix.gateway.service.GatewayServiceEvent; +import org.apache.helix.gateway.service.GatewayServiceManager; +import org.apache.helix.gateway.util.PerKeyLockRegistry; +import org.apache.helix.gateway.util.StateTransitionMessageTranslateUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import proto.org.apache.helix.gateway.HelixGatewayServiceGrpc; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass.ShardChangeRequests; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass.ShardState; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass.ShardStateMessage; + + +/** + * Helix Gateway Service GRPC UI implementation. + */ +public class HelixGatewayServiceGrpcService extends HelixGatewayServiceGrpc.HelixGatewayServiceImplBase + implements HelixGatewayServiceChannel { + private static final Logger logger = LoggerFactory.getLogger(HelixGatewayServiceGrpcService.class); + + // Map to store the observer for each instance + private final Map> _observerMap = new HashMap<>(); + // A reverse map to store the instance name for each observer. It is used to find the instance when connection is closed. + // map> + private final Map, Pair> _reversedObserverMap = new HashMap<>(); + + private final GatewayServiceManager _manager; + + // A fine grain lock register on instance level + private final PerKeyLockRegistry _lockRegistry = new PerKeyLockRegistry();; + + private final GatewayServiceChannelConfig _config; + + private Server _server; + + public HelixGatewayServiceGrpcService(GatewayServiceManager manager, GatewayServiceChannelConfig config) { + _manager = manager; + _config = config; + } + + /** + * Grpc service end pint. + * Application instances Report the state of the shard or result of transition request to the gateway service. + * + * @param responseObserver the observer to send the response to the client + * @return the observer to receive the state of the shard or result of transition request + */ + @Override + public StreamObserver report( + StreamObserver responseObserver) { + + return new StreamObserver() { + + @Override + public void onNext(ShardStateMessage request) { + logger.info("Receive message from instance: {}", request.toString()); + if (request.hasShardState()) { + ShardState shardState = request.getShardState(); + updateObserver(shardState.getInstanceName(), shardState.getClusterName(), responseObserver); + } + pushClientEventToGatewayManager(_manager, + StateTransitionMessageTranslateUtil.translateShardStateMessageToEventAndUpdateCache(_manager, request)); + } + + @Override + public void onError(Throwable t) { + logger.info("Receive on error, reason: {} message: {}", Status.fromThrowable(t).getCode(), t.getMessage()); + // Notify the gateway manager that the client is closed + Pair instanceInfo = _reversedObserverMap.get(responseObserver); + onClientClose(instanceInfo.getRight(), instanceInfo.getLeft()); + } + + @Override + public void onCompleted() { + logger.info("Receive on complete message"); + // Notify the gateway manager that the client is closed + Pair instanceInfo = _reversedObserverMap.get(responseObserver); + onClientClose(instanceInfo.getRight(), instanceInfo.getLeft()); + } + }; + } + + /** + * Send state transition message to the instance. + * The instance must already have established a connection to the gateway service. + * + * @param instanceName the instance name to send the message to + * @param requests the state transition request to send + */ + @Override + public void sendStateChangeRequests(String instanceName, ShardChangeRequests requests) { + _lockRegistry.withLock(instanceName, () -> { + StreamObserver observer = _observerMap.get(instanceName); + + // If observer is null, this means that the connection is already closed and + // we should not send a ShardChangeRequest + if (observer != null) { + observer.onNext(requests); + } else { + logger.error("Instance {} is not connected to the gateway service", instanceName); + // If the observer is null, we should remove the lock, so we don't keep unnecessary locks + _lockRegistry.removeLock(instanceName); + } + }); + } + + /** + * Close the connection of the instance. If closed because of error, use the error reason to close the connection. + * @param instanceName instance name + * @param errorReason error reason for close + */ + @Override + public void closeConnectionWithError(String clusterName, String instanceName, String errorReason) { + logger.info("Close connection for instance: {} with error reason: {}", instanceName, errorReason); + closeConnectionHelper(instanceName, errorReason, true); + } + + /** + * Complete the connection of the instance. + * @param instanceName instance name + */ + @Override + public void completeConnection(String clusterName, String instanceName) { + logger.info("Complete connection for instance: {}", instanceName); + closeConnectionHelper(instanceName, null, false); + } + + private void closeConnectionHelper(String instanceName, String errorReason, boolean withError) { + _lockRegistry.withLock(instanceName, () -> { + StreamObserver observer = _observerMap.get(instanceName); + + // If observer is null, this means that the connection is already closed and + // we should not try and close it again. + if (observer != null) { + // Depending on whether the connection is closed with error, send different status + if (withError) { + observer.onError(Status.UNAVAILABLE.withDescription(errorReason).asRuntimeException()); + } else { + observer.onCompleted(); + } + + // Clean up the observer and lock + _reversedObserverMap.remove(_observerMap.get(instanceName)); + _observerMap.remove(instanceName); + } + + // We always remove the lock after the connection is closed regardless of if observer is null or not + _lockRegistry.removeLock(instanceName); + }); + } + + private void onClientClose(String clusterName, String instanceName) { + if (instanceName == null || clusterName == null) { + logger.error("Cluster: {} or instance: {} is null while handling onClientClose", clusterName, + instanceName); + return; + } + logger.info("Client close connection for instance: {}", instanceName); + GatewayServiceEvent event = + StateTransitionMessageTranslateUtil.translateClientCloseToEvent(clusterName, instanceName); + pushClientEventToGatewayManager(_manager, event); + } + + private void updateObserver(String instanceName, String clusterName, + StreamObserver streamObserver) { + _lockRegistry.withLock(instanceName, () -> { + _observerMap.put(instanceName, streamObserver); + _reversedObserverMap.put(streamObserver, new ImmutablePair<>(instanceName, clusterName)); + }); + } + + @Override + public void start() throws IOException { + ServerBuilder serverBuilder = ServerBuilder.forPort(_config.getGrpcServerPort()) + .addService(this) + .keepAliveTime(_config.getServerHeartBeatInterval(), + TimeUnit.SECONDS) // HeartBeat time + .keepAliveTimeout(_config.getClientTimeout(), + TimeUnit.SECONDS) // KeepAlive client timeout + .permitKeepAliveTime(_config.getMaxAllowedClientHeartBeatInterval(), + TimeUnit.SECONDS) // Permit min HeartBeat time + .permitKeepAliveWithoutCalls(true); // Allow KeepAlive forever without active RPC + if (_config.getEnableReflectionService()) { + serverBuilder = serverBuilder.addService(io.grpc.protobuf.services.ProtoReflectionService.newInstance()); + } + _server = serverBuilder.build(); + + logger.info("Starting grpc server on port " + _config.getGrpcServerPort() + " now.... Server heart beat interval: " + + _config.getServerHeartBeatInterval() + " seconds, Max allowed client heart beat interval: " + + _config.getMaxAllowedClientHeartBeatInterval() + " seconds, Client timeout: " + _config.getClientTimeout() + + " seconds, Enable reflection service: " + _config.getEnableReflectionService()); + _server.start(); + } + + @Override + public void stop() { + if (_server != null) { + logger.info("Shutting down grpc server now...."); + _server.shutdownNow(); + } + } + + @VisibleForTesting + Server getServer() { + return _server; + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/channel/HelixGatewayServicePollModeChannel.java b/helix-gateway/src/main/java/org/apache/helix/gateway/channel/HelixGatewayServicePollModeChannel.java new file mode 100644 index 0000000000..a62cb6e583 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/channel/HelixGatewayServicePollModeChannel.java @@ -0,0 +1,231 @@ +package org.apache.helix.gateway.channel; + +/* + * 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.io.File; +import java.io.IOException; +import java.nio.file.Files; +import org.apache.commons.lang3.NotImplementedException; +import org.apache.helix.gateway.api.service.HelixGatewayServiceChannel; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.helix.gateway.service.GatewayServiceManager; +import org.apache.helix.gateway.util.StateTransitionMessageTranslateUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.helix.gateway.channel.GatewayServiceChannelConfig.FileBasedConfigType.*; +import static org.apache.helix.gateway.util.PollChannelUtil.*; + + +/** + * Helix Gateway Service Poll mode implementation. + * It periodically polls the current state of the participants and the liveness of the participants. + */ +public class HelixGatewayServicePollModeChannel implements HelixGatewayServiceChannel { + private static final Logger logger = LoggerFactory.getLogger(HelixGatewayServicePollModeChannel.class); + final GatewayServiceManager _manager; + final GatewayServiceChannelConfig _config; + + // cluster -> file for user to report shards' current states + final String _userCurrentStateFilePath; + // cluster -> file path to store the shards' target states + final String _targetStateFilePath; + final GatewayServiceChannelConfig.ChannelType _participantConnectionStatusChannelType; + final GatewayServiceChannelConfig.ChannelType _shardStateChannelType; + + // cluster -> host -> liveness result + final Map> _livenessResults; + // cluster -> host -> endpoint for query liveness + // It is the file pass if _participantConnectionStatusChannelType is FILE, grpc endpoint if it is GRPC_CLIENT + final Map> _livenessCheckEndpointMap; + + ScheduledExecutorService _scheduler; + + public HelixGatewayServicePollModeChannel(GatewayServiceManager manager, GatewayServiceChannelConfig config) { + _manager = manager; + _config = config; + _scheduler = Executors.newSingleThreadScheduledExecutor(); + _participantConnectionStatusChannelType = _config.getParticipantConnectionChannelType(); + _shardStateChannelType = _config.getShardStateChannelType(); + _livenessCheckEndpointMap = _config.getParticipantLivenessEndpointMap(); + _userCurrentStateFilePath = _config.getPollModeConfig(PARTICIPANT_CURRENT_STATE_PATH); + _targetStateFilePath = _config.getPollModeConfig(SHARD_TARGET_STATE_PATH); + _livenessResults = new HashMap<>(); + } + + /** + * Fetch the updates from the participants. + * 1. Get the diff of previous and current shard states, and send the state change event to the gateway manager. + * 2. Compare previous liveness and current liveness, and send the connection event to the gateway manager. + */ + protected void fetchUpdates() { + // 1. get the shard state change + Map>>> currentShardStates = + getChangedParticipantsCurrentState(_userCurrentStateFilePath); + + Map>>> currentStateDiff = new HashMap<>(); + for (String clusterName : currentShardStates.keySet()) { + Map>> clusterDiffMap = + _manager.updateCacheWithNewCurrentStateAndGetDiff(clusterName, currentShardStates.get(clusterName)); + if (clusterDiffMap == null || clusterDiffMap.isEmpty()) { + continue; + } + for (String instanceName : clusterDiffMap.keySet()) { + // if the instance is previously connected, send state change event + if (_livenessResults.get(clusterName) != null && _livenessResults.get(clusterName).get(instanceName)) { + logger.info("Host {} has state change, sending event to gateway manager", instanceName); + pushClientEventToGatewayManager(_manager, + StateTransitionMessageTranslateUtil.translateCurrentStateChangeToEvent(clusterName, instanceName, + clusterDiffMap.get(instanceName))); + } + } + currentStateDiff.put(clusterName, clusterDiffMap); + } + + // 2. fetch host health + for (String clusterName : _livenessCheckEndpointMap.keySet()) { + for (String instanceName : _livenessCheckEndpointMap.get(clusterName).keySet()) { + boolean prevLiveness = + _livenessResults.get(clusterName) != null && _livenessResults.get(clusterName).get(instanceName); + boolean liveness = fetchInstanceLivenessStatus(clusterName, instanceName); + if (prevLiveness && !liveness) { // previously connected, now disconnected + logger.warn("Host {} is not healthy, sending event to gateway manager", instanceName); + pushClientEventToGatewayManager(_manager, + StateTransitionMessageTranslateUtil.translateClientCloseToEvent(clusterName, instanceName)); + } else if (!prevLiveness && liveness) { // new connection. + logger.info("Host {} is newly connected, sending init connection event to gateway manager", instanceName); + pushClientEventToGatewayManager(_manager, + StateTransitionMessageTranslateUtil.translateCurrentStateDiffToInitConnectEvent(clusterName, instanceName, + currentStateDiff.containsKey(clusterName) ? currentStateDiff.get(clusterName).get(instanceName) + : new HashMap<>())); + } + _livenessResults.computeIfAbsent(clusterName, k -> new HashMap<>()).put(instanceName, liveness); + } + } + } + + @Override + public void sendStateChangeRequests(String instanceName, + HelixGatewayServiceOuterClass.ShardChangeRequests shardChangeRequests) { + switch (_shardStateChannelType) { + case FILE: + // we are periodically writing to the file, so no need to write here. + break; + default: + throw new NotImplementedException("Only support file based channel for now"); + } + } + + @Override + public void start() throws IOException { + logger.info("Starting Helix Gateway Service Poll Mode Channel..."); + final Runnable fetchUpdatesTask = new Runnable() { + @Override + public void run() { + fetchUpdates(); + } + }; + _scheduler.scheduleAtFixedRate(fetchUpdatesTask, _config.getPollStartDelaySec(), // init delay + _config.getPollIntervalSec(), // poll interval + TimeUnit.SECONDS); + scheduleTargetStateUpdateTask(); + } + + void scheduleTargetStateUpdateTask() { + if (_shardStateChannelType == GatewayServiceChannelConfig.ChannelType.FILE) { + final Runnable writeTargetStateTask = new Runnable() { + @Override + public void run() { + flushAssignmentToFile(_manager.serializeTargetState(), _targetStateFilePath); + } + }; + _scheduler.scheduleAtFixedRate(writeTargetStateTask, _config.getPollStartDelaySec(), // init delay + _config.getTargetFileUpdateIntervalSec(), // poll interval + TimeUnit.SECONDS); + } + } + + @Override + public void stop() { + logger.info("Stopping Helix Gateway Service Poll Mode Channel..."); + // Shutdown the scheduler gracefully when done (e.g., on app termination) + _scheduler.shutdown(); + try { + if (!_scheduler.awaitTermination(1, TimeUnit.MINUTES)) { + _scheduler.shutdownNow(); + } + } catch (InterruptedException e) { + _scheduler.shutdownNow(); + } + // remove files + if (_shardStateChannelType == GatewayServiceChannelConfig.ChannelType.FILE) { + File file = new File(_targetStateFilePath); + boolean res = file.delete(); + logger.info("Delete target state file: " + file + " res :" + res); + } + } + + @Override + public void closeConnectionWithError(String clusterName, String instanceName, String reason) { + // nothing needed for filed based poll mode + } + + @Override + public void completeConnection(String clusterName, String instanceName) { + // nothing needed for filed based poll mode + } + + /** + * Get current state of the participants. + * Now we only support file based, we will add GRPC based in the future. + */ + protected Map>>> getChangedParticipantsCurrentState( + String userCurrentStateFilePath) { + Map>>> currentShardStates; + switch (_shardStateChannelType) { + case FILE: + currentShardStates = readCurrentStateFromFile(userCurrentStateFilePath); + return currentShardStates; + default: + throw new NotImplementedException("Only support file based channel shard state for now"); + } + } + + /** + * Fetch the liveness status of the instance. + * Now we only support file based, we will add GRPC based in the future. + */ + protected boolean fetchInstanceLivenessStatus(String clusterName, String instanceName) { + String endpoint = _livenessCheckEndpointMap.get(clusterName).get(instanceName); + switch (_participantConnectionStatusChannelType) { + case FILE: + return readInstanceLivenessStatusFromFile(endpoint, _config.getPollHealthCheckTimeoutSec()); + default: + throw new NotImplementedException("Only support grpc based channel for now"); + } + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/participant/HelixGatewayParticipant.java b/helix-gateway/src/main/java/org/apache/helix/gateway/participant/HelixGatewayParticipant.java new file mode 100644 index 0000000000..eeb042c1c1 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/participant/HelixGatewayParticipant.java @@ -0,0 +1,259 @@ +package org.apache.helix.gateway.participant; + +/* + * 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.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.helix.HelixManager; +import org.apache.helix.InstanceType; +import org.apache.helix.gateway.api.service.HelixGatewayServiceChannel; +import org.apache.helix.gateway.channel.HelixGatewayServicePollModeChannel; +import org.apache.helix.gateway.service.GatewayServiceManager; +import org.apache.helix.gateway.statemodel.HelixGatewayMultiTopStateStateModelFactory; +import org.apache.helix.gateway.util.StateTransitionMessageTranslateUtil; +import org.apache.helix.manager.zk.HelixManagerStateListener; +import org.apache.helix.manager.zk.ZKHelixManager; +import org.apache.helix.model.Message; +import org.apache.helix.participant.statemachine.StateTransitionError; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * HelixGatewayParticipant encapsulates the Helix Participant Manager and handles tracking the state + * of a remote participant connected to the Helix Gateway Service. It processes state transitions + * for the participant and updates the state of the participant's shards upon successful state + * transitions signaled by remote participant. + */ +public class HelixGatewayParticipant implements HelixManagerStateListener { + private static final Logger logger = LoggerFactory.getLogger(HelixGatewayParticipant.class); + public static final String UNASSIGNED_STATE = "UNASSIGNED"; + private final HelixGatewayServiceChannel _gatewayServiceChannel; + private final HelixManager _helixManager; + private final Runnable _onDisconnectedCallback; + private final Map> _stateTransitionResultMap; + + private final GatewayServiceManager _gatewayServiceManager; + + private HelixGatewayParticipant(HelixGatewayServiceChannel gatewayServiceChannel, Runnable onDisconnectedCallback, + HelixManager helixManager, Map> initialShardStateMap, + GatewayServiceManager gatewayServiceManager) { + _gatewayServiceChannel = gatewayServiceChannel; + _helixManager = helixManager; + _onDisconnectedCallback = onDisconnectedCallback; + _stateTransitionResultMap = new ConcurrentHashMap<>(); + _gatewayServiceManager = gatewayServiceManager; + } + + public void processStateTransitionMessage(Message message) throws Exception { + String resourceId = message.getResourceName(); + String shardId = message.getPartitionName(); + String toState = message.getToState(); + String concatenatedShardName = resourceId + shardId; + + try { + // update the target state in cache + _gatewayServiceManager.updateTargetState(_helixManager.getClusterName(), _helixManager.getInstanceName(), + resourceId, shardId, toState); + + if (isCurrentStateAlreadyTarget(resourceId, shardId, toState)) { + return; + } + CompletableFuture future = new CompletableFuture<>(); + _stateTransitionResultMap.put(concatenatedShardName, future); + _gatewayServiceChannel.sendStateChangeRequests(_helixManager.getInstanceName(), + StateTransitionMessageTranslateUtil.translateSTMsgToShardChangeRequests(message)); + + if (!toState.equals(future.get())) { + throw new Exception("Failed to transition to state " + toState); + } + } finally { + logger.info("State transition finished for shard: {}{}", resourceId, shardId); + _stateTransitionResultMap.remove(concatenatedShardName); + } + } + + public void handleStateTransitionError(Message message, StateTransitionError error) { + // Remove the stateTransitionResultMap future for the message + String transitionId = message.getMsgId(); + + // Remove the future from the stateTransitionResultMap since we are no longer able + // to process the state transition due to participant manager either timing out + // or failing to process the state transition + _stateTransitionResultMap.remove(transitionId); + + // Notify the HelixGatewayParticipantClient that it is in ERROR state + // TODO: We need a better way than sending the state transition with a toState of ERROR + } + + /** + * Get the instance name of the participant. + * @return participant instance name + */ + public String getInstanceName() { + return _helixManager.getInstanceName(); + } + + /** + * Completes the state transition with the given transitionId. + */ + public void completeStateTransition(String resourceId, String shardId, String currentState) { + logger.info("Completing state transition for shard: {}{} to state: {}", resourceId, shardId, currentState); + String concatenatedShardName = resourceId + shardId; + CompletableFuture future = _stateTransitionResultMap.get(concatenatedShardName); + if (future != null) { + future.complete(currentState); + } + } + + private boolean isCurrentStateAlreadyTarget(String resourceId, String shardId, String targetState) { + return getCurrentState(resourceId, shardId).equals(targetState); + } + + /** + * Get the current state of the shard. + * + * @param resourceId the resource id + * @param shardId the shard id + * @return the current state of the shard or DROPPED if it does not exist + */ + public String getCurrentState(String resourceId, String shardId) { + String currentState = + _gatewayServiceManager.getCurrentState(_helixManager.getClusterName(), _helixManager.getInstanceName(), + resourceId, shardId); + return currentState == null ? UNASSIGNED_STATE : currentState; + } + + /** + * Invoked when the HelixManager connection to zookeeper is established + * + * @param helixManager HelixManager that is successfully connected + */ + public void onConnected(HelixManager helixManager) throws Exception { + // Do nothing + } + + /** + * Invoked when the HelixManager connection to zookeeper is closed unexpectedly. This will not be + * run if the remote participant disconnects from gateway. + * + * @param helixManager HelixManager that fails to be connected + * @param error connection error + */ + @Override + public void onDisconnected(HelixManager helixManager, Throwable error) throws Exception { + _onDisconnectedCallback.run(); + _gatewayServiceChannel.closeConnectionWithError(_helixManager.getClusterName(), _helixManager.getInstanceName(), + error.getMessage()); + } + + public void disconnect() { + logger.info("Disconnecting from HelixManager {}", _helixManager.getInstanceName() ); + if (_helixManager.isConnected()) { + _helixManager.disconnect(); + } + _gatewayServiceChannel.completeConnection(_helixManager.getClusterName(), _helixManager.getInstanceName()); + } + + public static class Builder { + private final HelixGatewayServiceChannel _helixGatewayServiceChannel; + private final String _instanceName; + private final String _clusterName; + private final String _zkAddress; + private final Runnable _onDisconnectedCallback; + private final List _multiTopStateModelDefinitions; + private final Map> _initialShardStateMap; + private final GatewayServiceManager _gatewayServiceManager; + + public Builder(HelixGatewayServiceChannel helixGatewayServiceChannel, String instanceName, String clusterName, + String zkAddress, Runnable onDisconnectedCallback, GatewayServiceManager gatewayServiceManager) { + _helixGatewayServiceChannel = helixGatewayServiceChannel; + _instanceName = instanceName; + _clusterName = clusterName; + _zkAddress = zkAddress; + _gatewayServiceManager = gatewayServiceManager; + _onDisconnectedCallback = onDisconnectedCallback; + _multiTopStateModelDefinitions = new ArrayList<>(); + _initialShardStateMap = new ConcurrentHashMap<>(); + } + + /** + * Add a multi-top state model definition to the participant to be registered in the + * participant's state machine engine. + * + * @param stateModelDefinitionName the state model definition name to add (should be multi-top + * state model) + * @return the builder + */ + public Builder addMultiTopStateStateModelDefinition(String stateModelDefinitionName) { + // TODO: Add validation that the state model definition is a multi-top state model + _multiTopStateModelDefinitions.add(stateModelDefinitionName); + return this; + } + + /** + * Add initial shard state to the participant. This is used to initialize the participant with + * the initial state of the shards in order to reduce unnecessary state transitions from being + * forwarded to the participant. + * + * @param initialShardStateMap the initial shard state map to add + * @return the Builder + */ + public Builder setInitialShardState(Map> initialShardStateMap) { + // TODO: Add handling for shard states that where never assigned to the participant since + // the participant was last online. + // deep copy into the initialShardStateMap into concurrent hash map + initialShardStateMap.forEach((resourceId, shardStateMap) -> { + _initialShardStateMap.put(resourceId, new ConcurrentHashMap<>(shardStateMap)); + }); + + return this; + } + + /** + * Build the HelixGatewayParticipant. This will create a HelixManager for the participant and + * connect to the Helix cluster. The participant will be registered with the multi-top state + * model definitions and initialized with the initial shard state map. + * + * @return the HelixGatewayParticipant + */ + public HelixGatewayParticipant build() { + HelixManager participantManager = + new ZKHelixManager(_clusterName, _instanceName, InstanceType.PARTICIPANT, _zkAddress); + HelixGatewayParticipant participant = + new HelixGatewayParticipant(_helixGatewayServiceChannel, _onDisconnectedCallback, participantManager, + _initialShardStateMap, _gatewayServiceManager); + _multiTopStateModelDefinitions.forEach(stateModelDefinition -> participantManager.getStateMachineEngine() + .registerStateModelFactory(stateModelDefinition, + new HelixGatewayMultiTopStateStateModelFactory(participant))); + try { + participantManager.connect(); + } catch (Exception e) { + // TODO: When API for gracefully triggering disconnect from remote participant + // is available, we should call it here instead of throwing exception. + throw new RuntimeException(e); + } + return participant; + } + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/service/GatewayServiceEvent.java b/helix-gateway/src/main/java/org/apache/helix/gateway/service/GatewayServiceEvent.java new file mode 100644 index 0000000000..1c8c20429f --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/service/GatewayServiceEvent.java @@ -0,0 +1,126 @@ +package org.apache.helix.gateway.service; + +/* + * 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.List; +import java.util.Map; +import org.apache.helix.gateway.api.constant.GatewayServiceEventType; + + +/** + * Event representing message reported by clients to Helix Gateway Service. + */ +public class GatewayServiceEvent { + // event type + private GatewayServiceEventType _eventType; + // event data + private String _clusterName; + private String _instanceName; + // A map where client reports the state of each shard upon connection + private Map> _shardStateMap; + // result for state transition request + private List _stateTransitionResult; + + public static class StateTransitionResult { + + private String resourceName; + private String shardName; + private String shardState; + + public StateTransitionResult(String resourceName,String shardName, String shardState) { + this.shardState = shardState; + this.shardName = shardName; + this.resourceName = resourceName; + } + + public String getShardState() { + return shardState; + } + public String getShardName() { + return shardName; + } + public String getResourceName() { + return resourceName; + } + } + + private GatewayServiceEvent(GatewayServiceEventType eventType, String clusterName, String instanceName, + Map> shardStateMap, List stateTransitionStatusMap) { + _eventType = eventType; + _clusterName = clusterName; + _instanceName = instanceName; + _shardStateMap = shardStateMap; + _stateTransitionResult = stateTransitionStatusMap; + } + + public GatewayServiceEventType getEventType() { + return _eventType; + } + public String getClusterName() { + return _clusterName; + } + public String getInstanceName() { + return _instanceName; + } + public Map> getShardStateMap() { + return _shardStateMap; + } + public List getStateTransitionResult() { + return _stateTransitionResult; + } + + + public static class GateWayServiceEventBuilder { + private GatewayServiceEventType _eventType; + private String _clusterName; + private String _instanceName; + private Map> _shardStateMap; + private List _stateTransitionResult; + + public GateWayServiceEventBuilder(GatewayServiceEventType eventType) { + this._eventType = eventType; + } + + public GateWayServiceEventBuilder setClusterName(String clusterName) { + this._clusterName = clusterName; + return this; + } + + public GateWayServiceEventBuilder setParticipantName(String instanceName) { + this._instanceName = instanceName; + return this; + } + + public GateWayServiceEventBuilder setShardStateMap(Map> shardStateMap) { + this._shardStateMap = shardStateMap; + return this; + } + + public GateWayServiceEventBuilder setStateTransitionStatusMap( + List stateTransitionStatusMap) { + this._stateTransitionResult = stateTransitionStatusMap; + return this; + } + + public GatewayServiceEvent build() { + return new GatewayServiceEvent(_eventType, _clusterName, _instanceName, _shardStateMap, _stateTransitionResult); + } + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/service/GatewayServiceManager.java b/helix-gateway/src/main/java/org/apache/helix/gateway/service/GatewayServiceManager.java new file mode 100644 index 0000000000..d53ee51343 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/service/GatewayServiceManager.java @@ -0,0 +1,269 @@ +package org.apache.helix.gateway.service; + +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.collect.ImmutableSet; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import org.apache.helix.gateway.api.constant.GatewayServiceEventType; +import org.apache.helix.gateway.api.service.HelixGatewayServiceChannel; +import org.apache.helix.gateway.channel.GatewayServiceChannelConfig; +import org.apache.helix.gateway.channel.HelixGatewayServiceChannelFactory; +import org.apache.helix.gateway.participant.HelixGatewayParticipant; +import org.apache.helix.gateway.util.GatewayCurrentStateCache; +import org.apache.helix.gateway.util.PerKeyBlockingExecutor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.helix.gateway.api.constant.GatewayServiceManagerConstant.*; + + +/** + * A top layer class that send/receive messages from Grpc end point, and dispatch them to corrsponding gateway services. + * 1. get event from Grpc service + * 2. Maintain a gateway service registry, one gateway service maps to one Helix cluster + * 3. On init connect, create the participant manager + * 4. For ST reply message, update the tracker + */ +public class GatewayServiceManager { + private static final Logger logger = LoggerFactory.getLogger(GatewayServiceManager.class); + private static final ObjectMapper objectMapper = new ObjectMapper(); + public static final int CONNECTION_EVENT_THREAD_POOL_SIZE = 10; + public static final ImmutableSet SUPPORTED_MULTI_STATE_MODEL_TYPES = + ImmutableSet.of("OnlineOffline"); + private final Map> _helixGatewayParticipantMap; + private final String _zkAddress; + + // a single thread tp for event processing + private final ExecutorService _participantStateTransitionResultUpdator; + + // link to grpc service + private HelixGatewayServiceChannel _gatewayServiceChannel; + + // a per key executor for connection event. All event for the same instance will be executed in sequence. + // It is used to ensure for each instance, the connect/disconnect event won't start until the previous one is done. + private final PerKeyBlockingExecutor _connectionEventProcessor; + + private final Map _currentStateCacheMap; + + public GatewayServiceManager(String zkAddress) { + _helixGatewayParticipantMap = new ConcurrentHashMap<>(); + _zkAddress = zkAddress; + _participantStateTransitionResultUpdator = Executors.newSingleThreadExecutor(); + _connectionEventProcessor = + new PerKeyBlockingExecutor(CONNECTION_EVENT_THREAD_POOL_SIZE); // todo: make it configurable + _currentStateCacheMap = new HashMap<>(); + } + + public GatewayServiceManager(String zkAddress, GatewayServiceChannelConfig gatewayServiceChannelConfig) { + this(zkAddress); + _gatewayServiceChannel = HelixGatewayServiceChannelFactory.createServiceChannel(gatewayServiceChannelConfig, this); + } + + /** + * Set the gateway service channel. This can only be called once. + * The channel is used to send state transition message to the participant. + * + * @param channel the gateway service channel + * @throws IllegalStateException if the channel is already set + */ + public void setGatewayServiceChannel(HelixGatewayServiceChannel channel) { + if (_gatewayServiceChannel != null) { + throw new IllegalStateException( + "Gateway service channel is already set, it can only be set once."); + } + _gatewayServiceChannel = channel; + } + + /** + * Process the event from Grpc service and dispatch to async executor for processing. + * + * @param event + */ + public void onGatewayServiceEvent(GatewayServiceEvent event) { + if (event.getEventType().equals(GatewayServiceEventType.UPDATE)) { + _participantStateTransitionResultUpdator.submit(new ShardStateUpdator(event)); + } else { + _connectionEventProcessor.offerEvent(event.getInstanceName(), new ParticipantConnectionProcessor(event)); + } + } + + public void resetTargetStateCache(String clusterName, String instanceName) { + logger.info("Resetting target state cache for cluster: {}, instance: {}", clusterName, instanceName); + getOrCreateCache(clusterName).resetTargetStateCache(instanceName); + } + + /** + * Overwrite the current state cache with the new current state map, and return the diff of the change. + * @param clusterName + * @param newCurrentStateMap + * @return + */ + public Map>> updateCacheWithNewCurrentStateAndGetDiff(String clusterName, + Map>> newCurrentStateMap) { + return getOrCreateCache(clusterName).updateCacheWithNewCurrentStateAndGetDiff(newCurrentStateMap); + } + + public void updateCurrentState(String clusterName, String instanceName, String resourceId, String shardId, String toState) { + getOrCreateCache(clusterName).updateCurrentStateOfExistingInstance(instanceName, resourceId, shardId, toState); + } + + public synchronized String serializeTargetState() { + ObjectNode targetStateNode = new ObjectMapper().createObjectNode(); + ObjectNode res = new ObjectMapper().createObjectNode(); + for (String clusterName : _currentStateCacheMap.keySet()) { + // add the json node to the target state node + targetStateNode.set(clusterName, getOrCreateCache(clusterName).serializeTargetAssignmentsToJSONNode()); + } + res.set(TARGET_STATE_ASSIGNMENT_KEY_NAME, targetStateNode); + res.set(TIMESTAMP_KEY, objectMapper.valueToTree(System.currentTimeMillis())); + return res.toString(); + } + + public void updateTargetState(String clusterName, String instanceName, String resourceId, String shardId, + String toState) { + getOrCreateCache(clusterName).updateTargetStateOfExistingInstance(instanceName, resourceId, shardId, toState); + } + + public String getCurrentState(String clusterName, String instanceName, String resourceId, String shardId) { + return getOrCreateCache(clusterName).getCurrentState(instanceName, resourceId, shardId); + } + + public String getTargetState(String clusterName, String instanceName, String resourceId, String shardId) { + return getOrCreateCache(clusterName).getTargetState(instanceName, resourceId, shardId); + } + + public Map>> getAllTargetStates(String clusterName) { + return getOrCreateCache(clusterName).getAllTargetStates(); + } + + /** + * Update in memory shard state + */ + class ShardStateUpdator implements Runnable { + + private final GatewayServiceEvent _event; + + private ShardStateUpdator(GatewayServiceEvent event) { + _event = event; + } + + @Override + public void run() { + logger.info("Processing state transition result " + _event.getInstanceName()); + HelixGatewayParticipant participant = + getHelixGatewayParticipant(_event.getClusterName(), _event.getInstanceName()); + if (participant == null) { + // TODO: return error code and throw exception. + return; + } + _event.getStateTransitionResult().forEach(stateTransitionResult -> { + participant.completeStateTransition(stateTransitionResult.getResourceName(), + stateTransitionResult.getShardName(), stateTransitionResult.getShardState()); + }); + } + } + + /** + * Create HelixGatewayService instance and register it to the manager. + * It includes waiting for ZK connection, and also wait for previous LiveInstance to expire. + */ + class ParticipantConnectionProcessor implements Runnable { + GatewayServiceEvent _event; + + public ParticipantConnectionProcessor(GatewayServiceEvent event) { + _event = event; + } + + @Override + public void run() { + if (_event.getEventType().equals(GatewayServiceEventType.CONNECT)) { + createHelixGatewayParticipant(_event.getClusterName(), _event.getInstanceName(), + _event.getShardStateMap()); + } else { + removeHelixGatewayParticipant(_event.getClusterName(), _event.getInstanceName()); + } + } + } + + public void stopManager() { + _connectionEventProcessor.shutdown(); + _participantStateTransitionResultUpdator.shutdown(); + _helixGatewayParticipantMap.forEach((clusterName, participantMap) -> { + participantMap.forEach((instanceName, participant) -> { + participant.disconnect(); + }); + }); + } + + public void startService() throws IOException { + _gatewayServiceChannel.start(); + } + + public void stopService() { + _gatewayServiceChannel.stop(); + stopManager(); + } + + private void createHelixGatewayParticipant(String clusterName, String instanceName, + Map> initialShardStateMap) { + resetTargetStateCache(clusterName, instanceName); + // Create and add the participant to the participant map + HelixGatewayParticipant.Builder participantBuilder = + new HelixGatewayParticipant.Builder(_gatewayServiceChannel, instanceName, clusterName, _zkAddress, + () -> removeHelixGatewayParticipant(clusterName, instanceName), this).setInitialShardState( + initialShardStateMap); + SUPPORTED_MULTI_STATE_MODEL_TYPES.forEach(participantBuilder::addMultiTopStateStateModelDefinition); + _helixGatewayParticipantMap.computeIfAbsent(clusterName, k -> new ConcurrentHashMap<>()) + .put(instanceName, participantBuilder.build()); + } + + private void removeHelixGatewayParticipant(String clusterName, String instanceName) { + logger.info("Removing participant: {} from cluster: {}", instanceName, clusterName); + // Disconnect and remove the participant from the participant map + HelixGatewayParticipant participant = getHelixGatewayParticipant(clusterName, instanceName); + if (participant != null) { + participant.disconnect(); + if (_helixGatewayParticipantMap.containsKey(clusterName)) { + _helixGatewayParticipantMap.get(clusterName).remove(instanceName); + } + } + if (_currentStateCacheMap.containsKey(clusterName)) { + _currentStateCacheMap.get(clusterName).removeInstanceTargetDataFromCache(instanceName); + } + } + + private HelixGatewayParticipant getHelixGatewayParticipant(String clusterName, + String instanceName) { + return _helixGatewayParticipantMap.getOrDefault(clusterName, Collections.emptyMap()) + .get(instanceName); + } + + private synchronized GatewayCurrentStateCache getOrCreateCache(String clusterName) { + return _currentStateCacheMap.computeIfAbsent(clusterName, k -> new GatewayCurrentStateCache(clusterName)); + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/statemodel/HelixGatewayMultiTopStateStateModel.java b/helix-gateway/src/main/java/org/apache/helix/gateway/statemodel/HelixGatewayMultiTopStateStateModel.java new file mode 100644 index 0000000000..37de51b420 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/statemodel/HelixGatewayMultiTopStateStateModel.java @@ -0,0 +1,60 @@ +package org.apache.helix.gateway.statemodel; + +/* + * 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 org.apache.helix.NotificationContext; +import org.apache.helix.gateway.participant.HelixGatewayParticipant; +import org.apache.helix.model.Message; +import org.apache.helix.participant.statemachine.StateModel; +import org.apache.helix.participant.statemachine.StateModelInfo; +import org.apache.helix.participant.statemachine.StateTransitionError; +import org.apache.helix.participant.statemachine.Transition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@StateModelInfo(initialState = "OFFLINE", states = {}) +public class HelixGatewayMultiTopStateStateModel extends StateModel { + private static final Logger _logger = + LoggerFactory.getLogger(HelixGatewayMultiTopStateStateModel.class); + + private final HelixGatewayParticipant _helixGatewayParticipant; + + public HelixGatewayMultiTopStateStateModel( + HelixGatewayParticipant helixGatewayParticipant) { + _helixGatewayParticipant = helixGatewayParticipant; + } + + @Transition(to = "*", from = "*") + public void genericStateTransitionHandler(Message message, NotificationContext context) + throws Exception { + _helixGatewayParticipant.processStateTransitionMessage(message); + } + + @Override + public void reset() { + // no-op we don't want to start from init state again. + } + + @Override + public void rollbackOnError(Message message, NotificationContext context, + StateTransitionError error) { + _helixGatewayParticipant.handleStateTransitionError(message, error); + } +} \ No newline at end of file diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/statemodel/HelixGatewayMultiTopStateStateModelFactory.java b/helix-gateway/src/main/java/org/apache/helix/gateway/statemodel/HelixGatewayMultiTopStateStateModelFactory.java new file mode 100644 index 0000000000..64662998e3 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/statemodel/HelixGatewayMultiTopStateStateModelFactory.java @@ -0,0 +1,38 @@ +package org.apache.helix.gateway.statemodel; + +/* + * 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 org.apache.helix.gateway.participant.HelixGatewayParticipant; +import org.apache.helix.participant.statemachine.StateModelFactory; + +public class HelixGatewayMultiTopStateStateModelFactory extends StateModelFactory { + private final HelixGatewayParticipant _helixGatewayParticipant; + + public HelixGatewayMultiTopStateStateModelFactory( + HelixGatewayParticipant helixGatewayParticipant) { + _helixGatewayParticipant = helixGatewayParticipant; + } + + @Override + public HelixGatewayMultiTopStateStateModel createNewStateModel(String resourceName, + String partitionKey) { + return new HelixGatewayMultiTopStateStateModel(_helixGatewayParticipant); + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/util/GatewayCurrentStateCache.java b/helix-gateway/src/main/java/org/apache/helix/gateway/util/GatewayCurrentStateCache.java new file mode 100644 index 0000000000..8d785aac0a --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/util/GatewayCurrentStateCache.java @@ -0,0 +1,208 @@ +package org.apache.helix.gateway.util; + +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.collect.ImmutableMap; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A cache to store the current target assignment, and the reported current state of the instances in a cluster. + */ +public class GatewayCurrentStateCache { + private static final Logger logger = LoggerFactory.getLogger(GatewayCurrentStateCache.class); + private static final ObjectMapper mapper = new ObjectMapper(); + + // A cache of current state. It should be updated by the HelixGatewayServiceChannel + // instance -> resource state (resource -> shard -> target state) + Map _currentStateMap; + + // A cache of target state. + // instance -> resource state (resource -> shard -> target state) + final Map _targetStateMap; + ObjectNode root = mapper.createObjectNode(); + + public GatewayCurrentStateCache(String clusterName) { + _currentStateMap = new HashMap<>(); + _targetStateMap = new HashMap<>(); + } + + public String getCurrentState(String instance, String resource, String shard) { + ShardStateMap shardStateMap = _currentStateMap.get(instance); + return shardStateMap == null ? null : shardStateMap.getState(resource, shard); + } + + public String getTargetState(String instance, String resource, String shard) { + ShardStateMap shardStateMap = _targetStateMap.get(instance); + return shardStateMap == null ? null : shardStateMap.getState(resource, shard); + } + + public synchronized Map>> getAllTargetStates() { + Map>> result = new HashMap<>(); + for (Map.Entry entry : _targetStateMap.entrySet()) { + result.put(entry.getKey(), new HashMap<>(entry.getValue()._stateMap)); + } + return result; + } + + /** + * Update the cached current state of instances in a cluster, and return the diff of the change. + * @param userCurrentStateMap The new current state map of instances in the cluster + * @return + */ + public Map>> updateCacheWithNewCurrentStateAndGetDiff( + Map>> userCurrentStateMap) { + Map newCurrentStateMap = new HashMap<>(_currentStateMap); + Map>> diff = new HashMap<>(); + for (String instance : userCurrentStateMap.keySet()) { + ShardStateMap oldStateMap = _currentStateMap.get(instance); + Map> instanceDiff = oldStateMap == null ? userCurrentStateMap.get(instance) + : oldStateMap.getDiff(userCurrentStateMap.get(instance)); + if (!instanceDiff.isEmpty()) { + diff.put(instance, instanceDiff); + } + newCurrentStateMap.put(instance, new ShardStateMap(userCurrentStateMap.get(instance))); + } + logger.info("Update current state cache for instances: {}", diff.keySet()); + _currentStateMap = newCurrentStateMap; + return diff; + } + + /** + * Update the current state with the changed current state maps. + */ + public void updateCurrentStateOfExistingInstance(String instance, String resource, String shard, String shardState) { + logger.info("Update current state of instance: {}, resource: {}, shard: {}, state: {}", instance, resource, shard, + shardState); + updateShardStateMapWithDiff(_currentStateMap, instance, resource, shard, shardState); + } + + /** + * Update the target state with the changed target state maps. + * All existing target states remains the same + */ + public void updateTargetStateOfExistingInstance(String instance, String resource, String shard, String shardState) { + logger.info("Update target state of instance: {}, resource: {}, shard: {}, state: {}", instance, resource, shard, + shardState); + updateShardStateMapWithDiff(_targetStateMap, instance, resource, shard, shardState); + } + + private void updateShardStateMapWithDiff(Map stateMap, String instance, String resource, + String shard, String shardState) { + ShardStateMap curStateMap = stateMap.get(instance); + if (curStateMap == null) { + logger.warn("Instance {} is not in the state map, skip updating", instance); + return; + } + curStateMap.updateWithShardState(resource, shard, shardState); + } + + /** + * Serialize the target state assignments to a JSON Node. + * example : {"instance1":{"resource1":{"shard1":"ONLINE","shard2":"OFFLINE"}}}} + */ + public synchronized ObjectNode serializeTargetAssignmentsToJSONNode() { + ObjectNode root = mapper.createObjectNode(); + for (Map.Entry entry : _targetStateMap.entrySet()) { + root.set(entry.getKey(), entry.getValue().toJSONNode()); + } + return root; + } + + /** + * Remove the target state data of an instance from the cache. + */ + public synchronized void removeInstanceTargetDataFromCache(String instance) { + logger.info("Remove instance target data from cache for instance: {}", instance); + _targetStateMap.remove(instance); + root.remove(instance); + } + + /** + * Remove the current state data of an instance from the cache to an empty map. + */ + public synchronized void resetTargetStateCache(String instance) { + logger.info("Reset target state cache for instance: {}", instance); + _targetStateMap.put(instance, new ShardStateMap(new HashMap<>())); + } + + public static class ShardStateMap { + // resource -> shard -> state + Map> _stateMap; + + public ShardStateMap(Map> stateMap) { + _stateMap = new HashMap<>(stateMap); + } + + public String getState(String resource, String shard) { + Map shardStateMap = _stateMap.get(resource); + return shardStateMap == null ? null : shardStateMap.get(shard); + } + + public synchronized void updateWithShardState(String resource, String shard, String shardState) { + logger.info("Update ShardStateMap of resource: {}, shard: {}, state: {}", resource, shard, shardState); + _stateMap.computeIfAbsent(resource, k -> new HashMap<>()).put(shard, shardState); + } + + private Map> getDiff(Map> newCurrentStateMap) { + Map> diff = new HashMap<>(); + for (Map.Entry> entry : newCurrentStateMap.entrySet()) { + String resource = entry.getKey(); + Map newCurrentState = entry.getValue(); + Map oldCurrentState = _stateMap.get(resource); + if (oldCurrentState == null) { + diff.put(resource, newCurrentState); + continue; + } + if (!oldCurrentState.equals(newCurrentState)) { + for (String shard : newCurrentState.keySet()) { + if (oldCurrentState.get(shard) == null || !oldCurrentState.get(shard).equals(newCurrentState.get(shard))) { + diff.computeIfAbsent(resource, k -> new HashMap<>()).put(shard, newCurrentState.get(shard)); + } + } + } + } + return diff; + } + + /** + * Serialize the shard state map to a JSON object. + * @return a JSON object representing the shard state map. Example: {"shard1":"ONLINE","shard2":"OFFLINE"} + */ + public synchronized ObjectNode toJSONNode() { + ObjectNode root = mapper.createObjectNode(); + for (Map.Entry> entry : _stateMap.entrySet()) { + String resource = entry.getKey(); + ObjectNode resourceNode = mapper.createObjectNode(); + for (Map.Entry shardEntry : entry.getValue().entrySet()) { + resourceNode.put(shardEntry.getKey(), shardEntry.getValue()); + } + root.set(resource, resourceNode); + } + return root; + } + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/util/PerKeyBlockingExecutor.java b/helix-gateway/src/main/java/org/apache/helix/gateway/util/PerKeyBlockingExecutor.java new file mode 100644 index 0000000000..c5463d2146 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/util/PerKeyBlockingExecutor.java @@ -0,0 +1,97 @@ +package org.apache.helix.gateway.util; + +/* + * 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.HashMap; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + + +/** + * A per-key blocking executor that ensures that only one event is running for a given key at a time. + */ +public class PerKeyBlockingExecutor { + private final ThreadPoolExecutor _executor; + private final Map> _pendingBlockedEvents; + private final ConcurrentHashMap.KeySetView _runningEvents; + private final Lock _queueLock; + + public PerKeyBlockingExecutor(int maxWorkers) { + this._executor = (ThreadPoolExecutor) Executors.newFixedThreadPool(maxWorkers); + this._pendingBlockedEvents = new HashMap<>(); + this._queueLock = new ReentrantLock(); + this._runningEvents = ConcurrentHashMap.newKeySet(); + } + + /** + * Offer an event to be executed. If an event is already running for the given key, the event will be queued. + * @param key + * @param event + */ + public void offerEvent(String key, Runnable event) { + _queueLock.lock(); + try { + if (!_runningEvents.contains(key)) { + _executor.execute(() -> runEvent(key, event)); + } else { + _pendingBlockedEvents.computeIfAbsent(key, k -> new ConcurrentLinkedQueue<>()); + _pendingBlockedEvents.get(key).offer(event); + } + } finally { + _queueLock.unlock(); + } + } + + private void runEvent(String key, Runnable event) { + try { + _runningEvents.add(key); + event.run(); + } finally { + _queueLock.lock(); + try { + _runningEvents.remove(key); + processQueue(key); + } finally { + _queueLock.unlock(); + } + } + } + + private void processQueue(String key) { + if (!_pendingBlockedEvents.containsKey(key)) { + return; + } + Runnable event = _pendingBlockedEvents.get(key).poll(); + if (event != null) { + _executor.execute(() -> runEvent(key, event)); + } + } + + public void shutdown() { + _executor.shutdown(); + } + +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/util/PerKeyLockRegistry.java b/helix-gateway/src/main/java/org/apache/helix/gateway/util/PerKeyLockRegistry.java new file mode 100644 index 0000000000..63e1400c65 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/util/PerKeyLockRegistry.java @@ -0,0 +1,70 @@ +package org.apache.helix.gateway.util; + +/* + * 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.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantLock; + +/** + * A registry that manages locks per key. + */ +public class PerKeyLockRegistry { + private final ConcurrentHashMap lockMap = new ConcurrentHashMap<>(); + + public void lock(String key) { + ReentrantLock lock = lockMap.computeIfAbsent(key, k -> new ReentrantLock()); + lock.lock(); + } + + public void unlock(String key) { + ReentrantLock lock = lockMap.get(key); + if (lock != null) { + lock.unlock(); + } + } + + /** + * Execute the action with the lock on the key + * @param key + * @param action + */ + public void withLock(String key, Runnable action) { + lock(key); + try { + action.run(); + } finally { + unlock(key); + } + } + + /** + * Remove the lock if it is not being used. + * it must be called after the lock is required + * @param key + */ + public boolean removeLock(String key) { + ReentrantLock lock = lockMap.get(key); + if (lock != null && lock.isHeldByCurrentThread() && !lock.hasQueuedThreads()) { + lockMap.remove(key, lock); + return true; + } + return false; + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/util/PollChannelUtil.java b/helix-gateway/src/main/java/org/apache/helix/gateway/util/PollChannelUtil.java new file mode 100644 index 0000000000..68e8fb2aa6 --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/util/PollChannelUtil.java @@ -0,0 +1,147 @@ +package org.apache.helix.gateway.util; + +/* + * 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 com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.grpc.ManagedChannel; +import io.grpc.ManagedChannelBuilder; +import io.grpc.health.v1.HealthCheckRequest; +import io.grpc.health.v1.HealthCheckResponse; +import io.grpc.health.v1.HealthGrpc; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PollChannelUtil { + private static final ObjectMapper objectMapper = new ObjectMapper(); + private static final Logger logger = LoggerFactory.getLogger(PollChannelUtil.class); + + // return pair + // ManagedChannel need to be shutdown when the connection is no longer needed + public static Pair createGrpcChannel(String endpointPortString) { + String[] endpointPort = endpointPortString.split(":"); + ManagedChannel channel = + ManagedChannelBuilder.forAddress(endpointPort[0], Integer.parseInt(endpointPort[1])).usePlaintext().build(); + + return new ImmutablePair<>(HealthGrpc.newBlockingStub(channel), channel); + } + + /** + * Send Unary RPC to the gRPC service to check the health of the container. Could be liveness or readiness depends on input. + * @param service one of "readiness" or "liveness" + * https://github.com/kubernetes/kubernetes/issues/115651 + * @return + */ + public static boolean fetchLivenessStatusFromGrpcService(String service, HealthGrpc.HealthBlockingStub healthStub) { + HealthCheckRequest request = HealthCheckRequest.newBuilder().setService(service).build(); + HealthCheckResponse response = healthStub.check(request); + return response.getStatus() == HealthCheckResponse.ServingStatus.SERVING; + } + + /** + * Flush the current assignment map to a file. The whole file is re-written every time. + */ + public static void flushAssignmentToFile(String targetAssignment, String filePath) { + try (FileWriter fileWriter = new FileWriter(filePath)) { + fileWriter.write(targetAssignment); + fileWriter.close(); + } catch (IOException e) { + logger.warn("Failed to write to file: " + filePath, e); + } + } + + /** + * read current state from a file, compare with in memory current state, update the in memory current state and return diff. + * Current state file format: {"cluster1" : { "instance_1" : { "resource1" : {"shard1” : “online" }}}} + */ + public static Map>>> readCurrentStateFromFile(String filePath) { + try { + // read from file path + File file = new File(filePath); + return objectMapper.readValue(file, + new TypeReference>>>>() { + }); + } catch (IOException e) { + logger.warn("Failed to read from file: " + filePath, e); + return new HashMap<>(); + } + } + + /** + * Read instance liveness status from a file, return true if the instance is healthy and the last update time is within timeout. + * File format: {"IsAlive": true, "LastUpdateTime": 1629300000} + * @param filePath + * @param timeoutInSec + * @return + */ + public static boolean readInstanceLivenessStatusFromFile(String filePath, int timeoutInSec) { + try { + // read from file path + File file = new File(filePath); + HostLivenessState status = objectMapper.readValue(file, new TypeReference() { + }); + return status.isHealthy() && (System.currentTimeMillis()/1000 - status.getLastUpdatedTime()) < timeoutInSec; + } catch (IOException e) { + logger.warn("Failed to read from file: " + filePath, e); + return false; + } + } + + /** + * Instance health status representation as JSON + */ + public static class HostLivenessState { + @JsonProperty ("IsAlive") + Boolean _isAlive; + @JsonProperty ("LastUpdateTime") + long _lastUpdatedTime; // in epoch second + + public Boolean isHealthy(){ + return _isAlive; + } + public long getLastUpdatedTime(){ + return _lastUpdatedTime; + } + } + + /** + * Target assignments representation as JSON + */ + public static class TargetAssignment { + // cluster -> instance -> resource -> shard -> state + @JsonProperty ("Assignment") + String _targetAssignment; + @JsonProperty ("Timestamp") + long _timestamp; + + public TargetAssignment(String targetAssignment, long timestamp){ + _targetAssignment = targetAssignment; + _timestamp = timestamp; + } + } +} diff --git a/helix-gateway/src/main/java/org/apache/helix/gateway/util/StateTransitionMessageTranslateUtil.java b/helix-gateway/src/main/java/org/apache/helix/gateway/util/StateTransitionMessageTranslateUtil.java new file mode 100644 index 0000000000..5fc319737b --- /dev/null +++ b/helix-gateway/src/main/java/org/apache/helix/gateway/util/StateTransitionMessageTranslateUtil.java @@ -0,0 +1,176 @@ +package org.apache.helix.gateway.util; + +/* + * 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.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.helix.HelixDefinedState; +import org.apache.helix.gateway.api.constant.GatewayServiceEventType; +import org.apache.helix.gateway.participant.HelixGatewayParticipant; +import org.apache.helix.gateway.service.GatewayServiceEvent; +import org.apache.helix.gateway.service.GatewayServiceManager; +import org.apache.helix.model.Message; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass.ShardChangeRequests; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass.ShardState; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass.ShardStateMessage; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass.ShardTransitionStatus; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass.SingleShardChangeRequest; + +public final class StateTransitionMessageTranslateUtil { + /** + * Determine the transition type based on the current state and the target state. + * + * @param currentState current state + * @param toState target state + * @return TransitionType + */ + public static HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType translateStatesToTransitionType( + String currentState, String toState) { + boolean isUnassigned = HelixGatewayParticipant.UNASSIGNED_STATE.equals(currentState); + boolean isToStateDropped = HelixDefinedState.DROPPED.name().equals(toState); + + if (isToStateDropped && !isUnassigned) { + return HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType.DELETE_SHARD; + } + if (!isToStateDropped && isUnassigned) { + return HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType.ADD_SHARD; + } + return HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType.CHANGE_ROLE; + } + + /** + * Translate from Helix ST Message to Helix Gateway Service TransitionMessage. + * + * @param message Message + * @return TransitionMessage + */ + public static ShardChangeRequests translateSTMsgToShardChangeRequests(Message message) { + return ShardChangeRequests.newBuilder().addRequest( + SingleShardChangeRequest.newBuilder() + .setStateChangeRequestType( + translateStatesToTransitionType(message.getFromState(), message.getToState())) + .setResourceName(message.getResourceName()).setShardName(message.getPartitionName()) + .setTargetState(message.getToState()).build()).build(); + } + + /** + * Translate from user sent ShardStateMessage message to Helix Gateway Service event. + * + * @param request ShardStateMessage message + * contains the state of each shard upon connection or result of state transition request. + * @return GatewayServiceEvent + */ + public static GatewayServiceEvent translateShardStateMessageToEventAndUpdateCache( + GatewayServiceManager manager, ShardStateMessage request) { + GatewayServiceEvent.GateWayServiceEventBuilder builder; + if (request.hasShardState()) { // init connection to gateway service + ShardState shardState = request.getShardState(); + Map> shardStateMap = new HashMap<>(); + for (HelixGatewayServiceOuterClass.SingleResourceState resourceState : shardState.getResourceStateList()) { + for (HelixGatewayServiceOuterClass.SingleShardState state : resourceState.getShardStatesList()) { + shardStateMap.computeIfAbsent(resourceState.getResource(), k -> new HashMap<>()) + .put(state.getShardName(), state.getCurrentState()); + } + } + // update current state cache. We always overwrite the current state map for initial connection + Map>> newShardStateMap = new HashMap<>(); + newShardStateMap.put(shardState.getInstanceName(), shardStateMap); + manager.updateCacheWithNewCurrentStateAndGetDiff(shardState.getClusterName(), newShardStateMap); + + builder = new GatewayServiceEvent.GateWayServiceEventBuilder(GatewayServiceEventType.CONNECT).setClusterName( + shardState.getClusterName()).setParticipantName(shardState.getInstanceName()) + .setShardStateMap(shardStateMap); + } else { + ShardTransitionStatus shardTransitionStatus = request.getShardTransitionStatus(); + // this is status update for established connection + List status = + shardTransitionStatus.getShardTransitionStatusList(); + List stResult = new ArrayList<>(); + for (HelixGatewayServiceOuterClass.SingleShardTransitionStatus shardTransition : status) { + GatewayServiceEvent.StateTransitionResult result = + new GatewayServiceEvent.StateTransitionResult(shardTransition.getResourceName(), + shardTransition.getShardName(), shardTransition.getCurrentState()); + // update current state cache + manager.updateCurrentState(shardTransitionStatus.getClusterName(), shardTransitionStatus.getInstanceName(), + shardTransition.getResourceName(), shardTransition.getShardName(), shardTransition.getCurrentState()); + stResult.add(result); + } + builder = new GatewayServiceEvent.GateWayServiceEventBuilder(GatewayServiceEventType.UPDATE).setClusterName( + shardTransitionStatus.getClusterName()) + .setParticipantName(shardTransitionStatus.getInstanceName()) + .setStateTransitionStatusMap(stResult); + } + return builder.build(); + } + + /** + * Translate from client close to Helix Gateway Service event. + * + * @param instanceName the instance name to send the message to + * @param clusterName the cluster name + * @return GatewayServiceEvent + */ + public static GatewayServiceEvent translateClientCloseToEvent(String clusterName, String instanceName) { + GatewayServiceEvent.GateWayServiceEventBuilder builder = + new GatewayServiceEvent.GateWayServiceEventBuilder(GatewayServiceEventType.DISCONNECT).setClusterName( + clusterName).setParticipantName(instanceName); + return builder.build(); + } + + /** + * Translate from current state change to Helix Gateway Service event. + * @param instanceName + * @param clusterName + * @param shardStateMap + * @return + */ + public static GatewayServiceEvent translateCurrentStateChangeToEvent(String clusterName, String instanceName, + Map> shardStateMap) { + List stResult = new ArrayList<>(); + shardStateMap.forEach((resourceName, value) -> value.forEach((key, value1) -> { + GatewayServiceEvent.StateTransitionResult result = + new GatewayServiceEvent.StateTransitionResult(resourceName, key, value1); + stResult.add(result); + })); + GatewayServiceEvent.GateWayServiceEventBuilder builder = + new GatewayServiceEvent.GateWayServiceEventBuilder(GatewayServiceEventType.UPDATE).setClusterName( + clusterName).setParticipantName(instanceName).setStateTransitionStatusMap(stResult); + return builder.build(); + } + + /** + * Create a GatewayServiceEvent to notify the GatewayServiceManager to create a new HelixGatewayParticipant. + * @param instanceName the instance name of the newly connected participant + * @param clusterName the cluster nam + * @param shardStateMap the initial state of shards on the participant. Could be empty map + * @return + */ + public static GatewayServiceEvent translateCurrentStateDiffToInitConnectEvent(String clusterName, String instanceName, + Map> shardStateMap) { + GatewayServiceEvent.GateWayServiceEventBuilder builder = + new GatewayServiceEvent.GateWayServiceEventBuilder(GatewayServiceEventType.CONNECT).setClusterName(clusterName) + .setParticipantName(instanceName) + .setShardStateMap(shardStateMap); + return builder.build(); + } +} diff --git a/helix-gateway/src/main/proto/HelixGatewayService.proto b/helix-gateway/src/main/proto/HelixGatewayService.proto new file mode 100644 index 0000000000..50553e3728 --- /dev/null +++ b/helix-gateway/src/main/proto/HelixGatewayService.proto @@ -0,0 +1,79 @@ +/* + * 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. + */ +syntax = "proto3"; + +package proto.org.apache.helix.gateway; + +message SingleShardChangeRequest { + enum StateChangeRequestType { + ADD_SHARD = 0; + DELETE_SHARD = 1; + CHANGE_ROLE = 2; + } + StateChangeRequestType stateChangeRequestType = 1; // Transition type for shard operations + string resourceName = 2; // Resource ID + string shardName = 3; // Shard to perform operation + string targetState = 4; // Shard target state. +} + +message ShardChangeRequests { + repeated SingleShardChangeRequest request = 1; +} + +message SingleResourceState { + string resource = 1; // name of the resource + repeated SingleShardState shardStates = 2; // State of each shard +} + +message SingleShardState { + string shardName = 1; // Name of the shard + string currentState = 2; // Current state of the shard +} + +message SingleShardTransitionStatus { + string resourceName = 1; // resource name + string shardName = 2; // shard name + string currentState = 3; // If it failed, what is the current state it should reported as. +} + +message ShardTransitionStatus{ + string instanceName = 1; // Name of the application instance + string clusterName = 2; // Name of the cluster to connect to + repeated SingleShardTransitionStatus shardTransitionStatus = 3; // state transition result for a shard +} + +// Application report its state to Helix Gateway upon initial connection +message ShardState{ + string instanceName = 1; // Name of the application instance + string clusterName = 2; // Name of the cluster to connect to + repeated SingleResourceState resourceState = 3; // State of each resource +} + +// Application instance sends message upon initial connection or reply to state transition message +message ShardStateMessage{ + oneof instanceUpdate { + ShardState shardState = 1; + ShardTransitionStatus shardTransitionStatus = 2; + } +} + +service HelixGatewayService { + rpc report(stream ShardStateMessage) returns (stream ShardChangeRequests) {} +} + diff --git a/helix-gateway/src/test/conf/testng.xml b/helix-gateway/src/test/conf/testng.xml new file mode 100644 index 0000000000..f77eab885a --- /dev/null +++ b/helix-gateway/src/test/conf/testng.xml @@ -0,0 +1,27 @@ + + + + + + + + + + diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/TestPerKeyBlockingExecutor.java b/helix-gateway/src/test/java/org/apache/helix/gateway/TestPerKeyBlockingExecutor.java new file mode 100644 index 0000000000..c58a5bfa75 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/TestPerKeyBlockingExecutor.java @@ -0,0 +1,44 @@ +package org.apache.helix.gateway; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.helix.gateway.util.PerKeyBlockingExecutor; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class TestPerKeyBlockingExecutor { + @Test + public void testEventNotAddedIfPending() throws InterruptedException { + CountDownLatch latch1 = new CountDownLatch(1); + CountDownLatch latch2 = new CountDownLatch(1); + CountDownLatch latch3 = new CountDownLatch(1); + + PerKeyBlockingExecutor perKeyBlockingExecutor = new PerKeyBlockingExecutor(3); + + perKeyBlockingExecutor.offerEvent("key1", () -> { + try { + latch1.await(); // Wait for the test to release this latch + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + + perKeyBlockingExecutor.offerEvent("key1", () -> { + latch2.countDown(); + }); + + Thread.sleep(100); // Give time for the second event to be potentially processed + + Assert.assertFalse(latch2.await(100, TimeUnit.MILLISECONDS)); // Event 2 should not run yet + latch1.countDown(); // Release the first latch + Assert.assertTrue(latch2.await(1, TimeUnit.SECONDS)); // Event 2 should run now + + perKeyBlockingExecutor.offerEvent("key1", () -> { + latch3.countDown(); + }); + + Assert.assertTrue(latch3.await(1, TimeUnit.SECONDS)); // Event 3 should run after Event 2 + perKeyBlockingExecutor.shutdown(); + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/TestPerKeyLockRegistry.java b/helix-gateway/src/test/java/org/apache/helix/gateway/TestPerKeyLockRegistry.java new file mode 100644 index 0000000000..3b90dcb101 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/TestPerKeyLockRegistry.java @@ -0,0 +1,55 @@ +package org.apache.helix.gateway; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.helix.gateway.util.PerKeyLockRegistry; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class TestPerKeyLockRegistry { + @Test + public void testConcurrentAccess() { + PerKeyLockRegistry lockRegistry = new PerKeyLockRegistry(); + final AtomicInteger counter = new AtomicInteger(0); + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch doneLatch = new CountDownLatch(2); + + lockRegistry.withLock("key1", () -> { + counter.incrementAndGet(); + // try to acquir the lock for another key + lockRegistry.withLock("key2", () -> { + counter.incrementAndGet(); + }); + }); + + // counter should be 2 + Assert.assertEquals(2, counter.get()); + + // acquire the lock for key + ExecutorService executor = Executors.newFixedThreadPool(2); + lockRegistry.lock("key1"); + executor.submit(() -> { + lockRegistry.withLock("key1", () -> { + //try remove lock + Assert.assertFalse(lockRegistry.removeLock("key1")); + }); + }); + lockRegistry.unlock("key1"); + executor.submit(() -> { + lockRegistry.withLock("key2", () -> { + //try remove lock, should fail because key1 is not locked + Assert.assertFalse(lockRegistry.removeLock("key1")); + }); + }); + executor.submit(() -> { + lockRegistry.withLock("key1", () -> { + //try remove lock, only this tiem it succeded + Assert.assertFalse(lockRegistry.removeLock("key1")); + }); + }); + executor.shutdown(); + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/base/HelixGatewayTestBase.java b/helix-gateway/src/test/java/org/apache/helix/gateway/base/HelixGatewayTestBase.java new file mode 100644 index 0000000000..5ee11e42b0 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/base/HelixGatewayTestBase.java @@ -0,0 +1,140 @@ +package org.apache.helix.gateway.base; + +import java.util.List; +import org.apache.helix.ConfigAccessor; +import org.apache.helix.gateway.base.manager.ClusterControllerManager; +import org.apache.helix.gateway.base.manager.MockParticipantManager; +import org.apache.helix.gateway.base.util.TestHelper; +import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.collections.Lists; + +public class HelixGatewayTestBase extends ZookeeperTestBase { + protected static final int START_PORT = 12918; + protected String _clusterName = "TEST_CLUSTER"; // change the cluster name for each test class + protected int _numParticipants = 3; + protected int _numGatewayInstances = 3; + protected ClusterControllerManager _controller; + protected List _participants; + protected ConfigAccessor _configAccessor; + protected BestPossibleExternalViewVerifier _clusterVerifier; + + @BeforeClass + public void beforeClass() { + _participants = Lists.newArrayList(); + _configAccessor = new ConfigAccessor(ZK_ADDR); + _gSetupTool.getClusterManagementTool().addCluster(_clusterName, true); + controllerManagement(true); + startParticipants(); + startGatewayService(); + } + + @AfterClass + public void afterClass() { + controllerManagement(false); + stopParticipants(true); + stopGatewayService(); + _gSetupTool.getClusterManagementTool().dropCluster(_clusterName); + } + + + /** + * Start or stop the controller + * @param start true to start the controller, false to stop the controller + */ + private void controllerManagement(boolean start) { + String controllerName = CONTROLLER_PREFIX + "_0"; + + if (start) { + _controller = new ClusterControllerManager(ZK_ADDR, _clusterName, controllerName); + _controller.syncStart(); + + _clusterVerifier = + new BestPossibleExternalViewVerifier.Builder(_clusterName).setZkClient(_gZkClient).setWaitTillVerify( + TestHelper.DEFAULT_REBALANCE_PROCESSING_WAIT_TIME).build(); + } else { + _controller.syncStop(); + } + + enablePersistBestPossibleAssignment(_gZkClient, _clusterName, start); + } + + /** + * Create participants with the given number of participants defined by _numParticipants + */ + private void startParticipants() { + for (int i = 0; i < _numParticipants; i++) { + String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); + _gSetupTool.addInstanceToCluster(_clusterName, storageNodeName); + + // start dummy participants + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, _clusterName, storageNodeName); + participant.syncStart(); + _participants.add(participant); + } + } + + /** + * Stop participants and optionally drop the participants + * if dropParticipants is true + * + * @param dropParticipants true to drop the participants, false to stop the participants + */ + private void stopParticipants(boolean dropParticipants) { + for (MockParticipantManager participant : _participants) { + participant.syncStop(); + if (dropParticipants) { + _gSetupTool.getClusterManagementTool().dropInstance(_clusterName, + _configAccessor.getInstanceConfig(_clusterName, participant.getInstanceName())); + } + } + _participants.clear(); + } + + /** + * Create a resource with the given number of partitions and replicas + * WARNING: 1) assume only support OnlineOffline state model + * 2) assume only support FULL_AUTO rebalance mode + * + * Default rebalance strategy is CrushEdRebalanceStrategy + * + * @param resourceName name of the resource + * @param numPartitions number of partitions + * @param numReplicas number of replicas + */ + + protected void createResource(String resourceName, int numPartitions, int numReplicas) { + createResource(resourceName, numPartitions, numReplicas, "org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy");} + + /** + * Create a resource with the given number of partitions, replicas and rebalance strategy + * + * @param resourceName name of the resource + * @param numPartitions number of partitions + * @param numReplicas number of replicas + * @param rebalanceStrategy rebalance strategy + */ + protected void createResource(String resourceName, int numPartitions, int numReplicas, String rebalanceStrategy) { + _gSetupTool.getClusterManagementTool().addResource(_clusterName, resourceName, numPartitions, "OnlineOffline", + "FULL_AUTO", rebalanceStrategy); + _gSetupTool.getClusterManagementTool().rebalance(_clusterName, resourceName, numReplicas); + } + + /** + * Start the gateway service with the given number of gateway instances + * defined by _numGatewayInstances + */ + protected void startGatewayService() { + // Start the gateway service + } + + /** + * Stop the gateway service + */ + protected void stopGatewayService() { + // Stop the gateway service + + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/base/ZookeeperTestBase.java b/helix-gateway/src/test/java/org/apache/helix/gateway/base/ZookeeperTestBase.java new file mode 100644 index 0000000000..ac09eb6ec6 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/base/ZookeeperTestBase.java @@ -0,0 +1,813 @@ +package org.apache.helix.gateway.base; + +import com.google.common.base.Preconditions; +import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.logging.Level; +import javax.management.MBeanServerConnection; +import javax.management.ObjectName; +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.ConfigAccessor; +import org.apache.helix.HelixAdmin; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixManager; +import org.apache.helix.HelixProperty; +import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyPathBuilder; +import org.apache.helix.SystemPropertyKeys; +import org.apache.helix.api.config.HelixConfigProperty; +import org.apache.helix.constants.InstanceConstants; +import org.apache.helix.controller.pipeline.AbstractAsyncBaseStage; +import org.apache.helix.controller.pipeline.Pipeline; +import org.apache.helix.controller.pipeline.Stage; +import org.apache.helix.controller.pipeline.StageContext; +import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer; +import org.apache.helix.controller.rebalancer.strategy.AutoRebalanceStrategy; +import org.apache.helix.controller.rebalancer.waged.WagedRebalancer; +import org.apache.helix.controller.stages.AttributeName; +import org.apache.helix.controller.stages.ClusterEvent; +import org.apache.helix.gateway.base.util.TestHelper; +import org.apache.helix.manager.zk.ZKHelixAdmin; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZNRecordSerializer; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.model.BuiltInStateModelDefinitions; +import org.apache.helix.model.ClusterConfig; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Message; +import org.apache.helix.model.OnlineOfflineSMD; +import org.apache.helix.model.ResourceConfig; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.tools.ClusterSetup; +import org.apache.helix.tools.ClusterStateVerifier; +import org.apache.helix.tools.StateModelConfigGenerator; +import org.apache.helix.zookeeper.api.client.HelixZkClient; +import org.apache.helix.zookeeper.datamodel.ZNRecord; +import org.apache.helix.zookeeper.impl.client.ZkClient; +import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory; +import org.apache.helix.zookeeper.zkclient.ZkServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; +import org.testng.ITestContext; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterSuite; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.BeforeSuite; + +public abstract class ZookeeperTestBase { + private static final Logger LOG = LoggerFactory.getLogger(ZookeeperTestBase.class); + private static final String MULTI_ZK_PROPERTY_KEY = "multiZk"; + private static final String NUM_ZK_PROPERTY_KEY = "numZk"; + + protected static ZkServer _zkServer; + protected static HelixZkClient _gZkClient; + protected static ClusterSetup _gSetupTool; + protected static BaseDataAccessor _baseAccessor; + protected static MBeanServerConnection _server = ManagementFactory.getPlatformMBeanServer(); + + private final Map> _liveInstanceOwners = new HashMap<>(); + + private static final String ZK_PREFIX = "localhost:"; + private static final int ZK_START_PORT = 2283; + public static final String ZK_ADDR = ZK_PREFIX + ZK_START_PORT; + protected static final String CLUSTER_PREFIX = "CLUSTER"; + protected static final String CONTROLLER_CLUSTER_PREFIX = "CONTROLLER_CLUSTER"; + protected final String CONTROLLER_PREFIX = "controller"; + protected final String PARTICIPANT_PREFIX = "localhost"; + private static final long MANUAL_GC_PAUSE = 4000L; + + /* + * Multiple ZK references + */ + // The following maps hold ZK connect string as keys + protected static final Map _zkServerMap = new HashMap<>(); + protected static final Map _helixZkClientMap = new HashMap<>(); + protected static final Map _clusterSetupMap = new HashMap<>(); + protected static final Map _baseDataAccessorMap = new HashMap<>(); + + static public void reportPhysicalMemory() { + com.sun.management.OperatingSystemMXBean os = (com.sun.management.OperatingSystemMXBean) + java.lang.management.ManagementFactory.getOperatingSystemMXBean(); + long physicalMemorySize = os.getTotalPhysicalMemorySize(); + System.out.println("************ SYSTEM Physical Memory:" + physicalMemorySize); + + long MB = 1024 * 1024; + Runtime runtime = Runtime.getRuntime(); + long free = runtime.freeMemory()/MB; + long total = runtime.totalMemory()/MB; + System.out.println("************ total memory:" + total + " free memory:" + free); + } + + @BeforeSuite + public void beforeSuite() throws Exception { + // TODO: use logging.properties file to config java.util.logging.Logger levels + java.util.logging.Logger topJavaLogger = java.util.logging.Logger.getLogger(""); + topJavaLogger.setLevel(Level.WARNING); + + // Due to ZOOKEEPER-2693 fix, we need to specify whitelist for execute zk commends + System.setProperty("zookeeper.4lw.commands.whitelist", "*"); + System.setProperty(SystemPropertyKeys.CONTROLLER_MESSAGE_PURGE_DELAY, "3000"); + + // Start in-memory ZooKeepers + // If multi-ZooKeeper is enabled, start more ZKs. Otherwise, just set up one ZK + int numZkToStart = 1; + String multiZkConfig = System.getProperty(MULTI_ZK_PROPERTY_KEY); + if (multiZkConfig != null && multiZkConfig.equalsIgnoreCase(Boolean.TRUE.toString())) { + String numZkFromConfig = System.getProperty(NUM_ZK_PROPERTY_KEY); + if (numZkFromConfig != null) { + try { + numZkToStart = Math.max(Integer.parseInt(numZkFromConfig), numZkToStart); + } catch (Exception e) { + Assert.fail("Failed to parse the number of ZKs from config!"); + } + } else { + Assert.fail("multiZk config is set but numZk config is missing!"); + } + } + + // Start "numZkFromConfigInt" ZooKeepers + for (int i = 0; i < numZkToStart; i++) { + startZooKeeper(i); + } + + // Set the references for backward-compatibility with a single ZK environment + _zkServer = _zkServerMap.get(ZK_ADDR); + _gZkClient = _helixZkClientMap.get(ZK_ADDR); + _gSetupTool = _clusterSetupMap.get(ZK_ADDR); + _baseAccessor = _baseDataAccessorMap.get(ZK_ADDR); + + // Clean up all JMX objects + for (ObjectName mbean : _server.queryNames(null, null)) { + try { + _server.unregisterMBean(mbean); + } catch (Exception e) { + // OK + } + } + } + + /** + * Starts an additional in-memory ZooKeeper for testing. + * @param i index to be added to the ZK port to avoid conflicts + * @throws Exception + */ + private static synchronized void startZooKeeper(int i) { + String zkAddress = ZK_PREFIX + (ZK_START_PORT + i); + _zkServerMap.computeIfAbsent(zkAddress, ZookeeperTestBase::createZookeeperServer); + _helixZkClientMap.computeIfAbsent(zkAddress, ZookeeperTestBase::createZkClient); + _clusterSetupMap.computeIfAbsent(zkAddress, key -> new ClusterSetup(_helixZkClientMap.get(key))); + _baseDataAccessorMap.computeIfAbsent(zkAddress, key -> new ZkBaseDataAccessor(_helixZkClientMap.get(key))); + } + + private static ZkServer createZookeeperServer(String zkAddress) { + try { + return Preconditions.checkNotNull(TestHelper.startZkServer(zkAddress)); + } catch (Exception e) { + throw new IllegalArgumentException("Failed to start zookeeper server at " + zkAddress, e); + } + } + + private static HelixZkClient createZkClient(String zkAddress) { + HelixZkClient.ZkClientConfig clientConfig = new HelixZkClient.ZkClientConfig(); + clientConfig.setZkSerializer(new ZNRecordSerializer()); + return DedicatedZkClientFactory.getInstance() + .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress), clientConfig); + } + + @AfterSuite + public void afterSuite() throws IOException { + // Clean up all JMX objects + for (ObjectName mbean : _server.queryNames(null, null)) { + try { + _server.unregisterMBean(mbean); + } catch (Exception e) { + // OK + } + } + + synchronized (ZookeeperTestBase.class) { + // Close all ZK resources + _baseDataAccessorMap.values().forEach(BaseDataAccessor::close); + _clusterSetupMap.values().forEach(ClusterSetup::close); + _helixZkClientMap.values().forEach(HelixZkClient::close); + _zkServerMap.values().forEach(TestHelper::stopZkServer); + } + } + + @BeforeClass + public void beforeClass() throws Exception { + cleanupJMXObjects(); + // Giving each test some time to settle (such as gc pause, etc). + // Note that this is the best effort we could make to stabilize tests, not a complete solution + Runtime.getRuntime().gc(); + Thread.sleep(MANUAL_GC_PAUSE); + } + + @BeforeMethod + public void beforeTest(Method testMethod, ITestContext testContext) { + testContext.setAttribute("StartTime", System.currentTimeMillis()); + } + + protected void cleanupJMXObjects() throws IOException { + // Clean up all JMX objects + for (ObjectName mbean : _server.queryNames(null, null)) { + try { + _server.unregisterMBean(mbean); + } catch (Exception e) { + // OK + } + } + } + + protected String getShortClassName() { + return this.getClass().getSimpleName(); + } + + protected String getCurrentLeader(HelixZkClient zkClient, String clusterName) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<>(zkClient)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + if (leader == null) { + return null; + } + return leader.getInstanceName(); + } + + protected void enablePersistBestPossibleAssignment(HelixZkClient zkClient, String clusterName, + Boolean enabled) { + ConfigAccessor configAccessor = new ConfigAccessor(zkClient); + ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName); + clusterConfig.setPersistBestPossibleAssignment(enabled); + configAccessor.setClusterConfig(clusterName, clusterConfig); + } + + protected void enablePersistIntermediateAssignment(HelixZkClient zkClient, String clusterName, + Boolean enabled) { + ConfigAccessor configAccessor = new ConfigAccessor(zkClient); + ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName); + clusterConfig.setPersistIntermediateAssignment(enabled); + configAccessor.setClusterConfig(clusterName, clusterConfig); + } + + protected void enableTopologyAwareRebalance(HelixZkClient zkClient, String clusterName, + Boolean enabled) { + ConfigAccessor configAccessor = new ConfigAccessor(zkClient); + ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName); + clusterConfig.setTopologyAwareEnabled(enabled); + configAccessor.setClusterConfig(clusterName, clusterConfig); + } + + protected void enableDelayRebalanceInCluster(HelixZkClient zkClient, String clusterName, + boolean enabled) { + ConfigAccessor configAccessor = new ConfigAccessor(zkClient); + ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName); + clusterConfig.setDelayRebalaceEnabled(enabled); + configAccessor.setClusterConfig(clusterName, clusterConfig); + } + + protected void enableDelayRebalanceInInstance(HelixZkClient zkClient, String clusterName, + String instanceName, boolean enabled) { + ConfigAccessor configAccessor = new ConfigAccessor(zkClient); + InstanceConfig instanceConfig = configAccessor.getInstanceConfig(clusterName, instanceName); + instanceConfig.setDelayRebalanceEnabled(enabled); + configAccessor.setInstanceConfig(clusterName, instanceName, instanceConfig); + } + + protected void enableDelayRebalanceInCluster(HelixZkClient zkClient, String clusterName, + boolean enabled, long delay) { + ConfigAccessor configAccessor = new ConfigAccessor(zkClient); + ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName); + clusterConfig.setDelayRebalaceEnabled(enabled); + clusterConfig.setRebalanceDelayTime(delay); + configAccessor.setClusterConfig(clusterName, clusterConfig); + } + + protected void enableP2PInCluster(String clusterName, ConfigAccessor configAccessor, + boolean enable) { + // enable p2p message in cluster. + if (enable) { + ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName); + clusterConfig.enableP2PMessage(true); + configAccessor.setClusterConfig(clusterName, clusterConfig); + } else { + ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName); + clusterConfig.getRecord().getSimpleFields() + .remove(HelixConfigProperty.P2P_MESSAGE_ENABLED.name()); + configAccessor.setClusterConfig(clusterName, clusterConfig); + } + } + + protected void enableP2PInResource(String clusterName, ConfigAccessor configAccessor, + String dbName, boolean enable) { + if (enable) { + ResourceConfig resourceConfig = + new ResourceConfig.Builder(dbName).setP2PMessageEnabled(true).build(); + configAccessor.setResourceConfig(clusterName, dbName, resourceConfig); + } else { + // remove P2P Message in resource config + ResourceConfig resourceConfig = configAccessor.getResourceConfig(clusterName, dbName); + if (resourceConfig != null) { + resourceConfig.getRecord().getSimpleFields() + .remove(HelixConfigProperty.P2P_MESSAGE_ENABLED.name()); + configAccessor.setResourceConfig(clusterName, dbName, resourceConfig); + } + } + } + + protected void setDelayTimeInCluster(HelixZkClient zkClient, String clusterName, long delay) { + ConfigAccessor configAccessor = new ConfigAccessor(zkClient); + ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName); + clusterConfig.setRebalanceDelayTime(delay); + configAccessor.setClusterConfig(clusterName, clusterConfig); + } + + protected void setLastOnDemandRebalanceTimeInCluster(HelixZkClient zkClient, + String clusterName, long lastOnDemandTime) { + ConfigAccessor configAccessor = new ConfigAccessor(zkClient); + ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName); + clusterConfig.setLastOnDemandRebalanceTimestamp(lastOnDemandTime); + configAccessor.setClusterConfig(clusterName, clusterConfig); + } + + protected IdealState createResourceWithDelayedRebalance(String clusterName, String db, + String stateModel, int numPartition, int replica, int minActiveReplica, long delay) { + return createResourceWithDelayedRebalance(clusterName, db, stateModel, numPartition, replica, + minActiveReplica, delay, AutoRebalanceStrategy.class.getName()); + } + + protected IdealState createResourceWithDelayedRebalance(String clusterName, String db, + String stateModel, int numPartition, int replica, int minActiveReplica, long delay, + String rebalanceStrategy) { + return createResource(clusterName, db, stateModel, numPartition, replica, minActiveReplica, + delay, DelayedAutoRebalancer.class.getName(), rebalanceStrategy); + } + + protected IdealState createResourceWithWagedRebalance(String clusterName, String db, + String stateModel, int numPartition, int replica, int minActiveReplica) { + return createResource(clusterName, db, stateModel, numPartition, replica, minActiveReplica, + -1, WagedRebalancer.class.getName(), null); + } + + private IdealState createResource(String clusterName, String db, String stateModel, + int numPartition, int replica, int minActiveReplica, long delay, String rebalancerClassName, + String rebalanceStrategy) { + IdealState idealState = + _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db); + if (idealState == null) { + _gSetupTool.addResourceToCluster(clusterName, db, numPartition, stateModel, + IdealState.RebalanceMode.FULL_AUTO + "", rebalanceStrategy); + } + + idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db); + idealState.setMinActiveReplicas(minActiveReplica); + if (!idealState.isDelayRebalanceEnabled()) { + idealState.setDelayRebalanceEnabled(true); + } + if (delay > 0) { + idealState.setRebalanceDelay(delay); + } + idealState.setRebalancerClassName(rebalancerClassName); + _gSetupTool.getClusterManagementTool().setResourceIdealState(clusterName, db, idealState); + _gSetupTool.rebalanceStorageCluster(clusterName, db, replica); + idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db); + + return idealState; + } + + protected IdealState createIdealState(String resourceGroupName, String instanceGroupTag, + List instanceNames, int numPartition, int replica, String rebalanceMode, + String stateModelDef) { + IdealState is = _gSetupTool.createIdealStateForResourceGroup(resourceGroupName, + instanceGroupTag, numPartition, replica, rebalanceMode, stateModelDef); + + // setup initial partition->instance mapping. + int nodeIdx = 0; + int numNode = instanceNames.size(); + assert (numNode >= replica); + for (int i = 0; i < numPartition; i++) { + String partitionName = resourceGroupName + "_" + i; + for (int j = 0; j < replica; j++) { + is.setPartitionState(partitionName, instanceNames.get((nodeIdx + j) % numNode), + OnlineOfflineSMD.States.ONLINE.toString()); + } + nodeIdx++; + } + + return is; + } + + protected void createDBInSemiAuto(ClusterSetup clusterSetup, String clusterName, String dbName, + List preferenceList, String stateModelDef, int numPartition, int replica) { + clusterSetup.addResourceToCluster(clusterName, dbName, numPartition, stateModelDef, + IdealState.RebalanceMode.SEMI_AUTO.toString()); + clusterSetup.rebalanceStorageCluster(clusterName, dbName, replica); + + IdealState is = + _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, dbName); + for (String p : is.getPartitionSet()) { + is.setPreferenceList(p, preferenceList); + } + clusterSetup.getClusterManagementTool().setResourceIdealState(clusterName, dbName, is); + } + + /** + * Validate there should be always minimal active replica and top state replica for each + * partition. + * Also make sure there is always some partitions with only active replica count. + */ + protected void validateMinActiveAndTopStateReplica(IdealState is, ExternalView ev, + int minActiveReplica, int numNodes) { + StateModelDefinition stateModelDef = + BuiltInStateModelDefinitions.valueOf(is.getStateModelDefRef()).getStateModelDefinition(); + String topState = stateModelDef.getStatesPriorityList().get(0); + int replica = Integer.valueOf(is.getReplicas()); + + Map stateCount = stateModelDef.getStateCountMap(numNodes, replica); + Set activeStates = stateCount.keySet(); + + for (String partition : is.getPartitionSet()) { + Map assignmentMap = ev.getRecord().getMapField(partition); + Assert.assertNotNull(assignmentMap, + is.getResourceName() + "'s best possible assignment is null for partition " + partition); + Assert.assertTrue(!assignmentMap.isEmpty(), + is.getResourceName() + "'s partition " + partition + " has no best possible map in IS."); + + boolean hasTopState = false; + int activeReplica = 0; + for (String state : assignmentMap.values()) { + if (topState.equalsIgnoreCase(state)) { + hasTopState = true; + } + if (activeStates.contains(state)) { + activeReplica++; + } + } + + if (activeReplica < minActiveReplica) { + int a = 0; + } + + Assert.assertTrue(hasTopState, String.format("%s missing %s replica", partition, topState)); + Assert.assertTrue(activeReplica >= minActiveReplica, + String.format("%s has less active replica %d then required %d", partition, activeReplica, + minActiveReplica)); + } + } + + protected void runStage(HelixManager manager, ClusterEvent event, Stage stage) throws Exception { + event.addAttribute(AttributeName.helixmanager.name(), manager); + StageContext context = new StageContext(); + stage.init(context); + stage.preProcess(); + + // AbstractAsyncBaseStage will run asynchronously, and it's main logics are implemented in + // execute() function call + if (stage instanceof AbstractAsyncBaseStage) { + ((AbstractAsyncBaseStage) stage).execute(event); + } else { + stage.process(event); + } + stage.postProcess(); + } + + public void verifyInstance(HelixZkClient zkClient, String clusterName, String instance, + boolean wantExists) { + // String instanceConfigsPath = HelixUtil.getConfigPath(clusterName); + String instanceConfigsPath = PropertyPathBuilder.instanceConfig(clusterName); + String instanceConfigPath = instanceConfigsPath + "/" + instance; + String instancePath = PropertyPathBuilder.instance(clusterName, instance); + Assert.assertEquals(wantExists, zkClient.exists(instanceConfigPath)); + Assert.assertEquals(wantExists, zkClient.exists(instancePath)); + } + + public void verifyResource(HelixZkClient zkClient, String clusterName, String resource, + boolean wantExists) { + String resourcePath = PropertyPathBuilder.idealState(clusterName, resource); + Assert.assertEquals(wantExists, zkClient.exists(resourcePath)); + } + + public void verifyEnabled(HelixZkClient zkClient, String clusterName, String instance, + boolean wantEnabled) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + InstanceConfig config = accessor.getProperty(keyBuilder.instanceConfig(instance)); + Assert.assertEquals(wantEnabled, config.getInstanceEnabled()); + } + + public void verifyReplication(HelixZkClient zkClient, String clusterName, String resource, + int repl) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + IdealState idealState = accessor.getProperty(keyBuilder.idealStates(resource)); + for (String partitionName : idealState.getPartitionSet()) { + if (idealState.getRebalanceMode() == IdealState.RebalanceMode.SEMI_AUTO) { + Assert.assertEquals(repl, idealState.getPreferenceList(partitionName).size()); + } else if (idealState.getRebalanceMode() == IdealState.RebalanceMode.CUSTOMIZED) { + Assert.assertEquals(repl, idealState.getInstanceStateMap(partitionName).size()); + } + } + } + + protected void setupStateModel(String clusterName) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<>(_gZkClient)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + StateModelDefinition masterSlave = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + accessor.setProperty(keyBuilder.stateModelDef(masterSlave.getId()), masterSlave); + + StateModelDefinition leaderStandby = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForLeaderStandby()); + accessor.setProperty(keyBuilder.stateModelDef(leaderStandby.getId()), leaderStandby); + + StateModelDefinition onlineOffline = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForOnlineOffline()); + accessor.setProperty(keyBuilder.stateModelDef(onlineOffline.getId()), onlineOffline); + + } + + protected Message createMessage(Message.MessageType type, String msgId, String fromState, + String toState, String resourceName, String tgtName) { + Message msg = new Message(type.toString(), msgId); + msg.setFromState(fromState); + msg.setToState(toState); + msg.getRecord().setSimpleField(Message.Attributes.RESOURCE_NAME.toString(), resourceName); + msg.setTgtName(tgtName); + return msg; + } + + protected List setupIdealState(String clusterName, int[] nodes, String[] resources, + int partitions, int replicas) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<>(_gZkClient)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + List idealStates = new ArrayList<>(); + List instances = new ArrayList<>(); + for (int i : nodes) { + instances.add("localhost_" + i); + } + + for (String resourceName : resources) { + IdealState idealState = new IdealState(resourceName); + for (int p = 0; p < partitions; p++) { + List value = new ArrayList<>(); + for (int r = 0; r < replicas; r++) { + int n = nodes[(p + r) % nodes.length]; + value.add("localhost_" + n); + } + idealState.getRecord().setListField(resourceName + "_" + p, value); + } + + idealState.setReplicas(Integer.toString(replicas)); + idealState.setStateModelDefRef("MasterSlave"); + idealState.setRebalanceMode(IdealState.RebalanceMode.SEMI_AUTO); + idealState.setNumPartitions(partitions); + idealStates.add(idealState); + + // System.out.println(idealState); + accessor.setProperty(keyBuilder.idealStates(resourceName), idealState); + } + return idealStates; + } + + @AfterClass + public void cleanupLiveInstanceOwners() throws InterruptedException { + String testClassName = this.getShortClassName(); + System.out.println("AfterClass: " + testClassName + " called."); + for (String cluster : _liveInstanceOwners.keySet()) { + Map clientMap = _liveInstanceOwners.get(cluster); + for (HelixZkClient client : clientMap.values()) { + client.close(); + } + clientMap.clear(); + } + _liveInstanceOwners.clear(); + } + + protected List setupLiveInstances(String clusterName, int[] liveInstances) { + HelixZkClient.ZkClientConfig clientConfig = new HelixZkClient.ZkClientConfig(); + clientConfig.setZkSerializer(new ZNRecordSerializer()); + + List result = new ArrayList<>(); + + for (int i = 0; i < liveInstances.length; i++) { + String instance = "localhost_" + liveInstances[i]; + _liveInstanceOwners.putIfAbsent(clusterName, new HashMap<>()); + Map clientMap = _liveInstanceOwners.get(clusterName); + clientMap.putIfAbsent(instance, DedicatedZkClientFactory.getInstance() + .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR), clientConfig)); + HelixZkClient client = clientMap.get(instance); + + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<>(client)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + LiveInstance liveInstance = new LiveInstance(instance); + // Keep setting the session id in the deprecated field for ensure the same behavior as a real participant. + // Note the participant is doing so for backward compatibility. + liveInstance.setSessionId(Long.toHexString(client.getSessionId())); + // Please refer to the version requirement here: helix-core/src/main/resources/cluster-manager-version.properties + // Ensuring version compatibility can avoid the warning message during test. + liveInstance.setHelixVersion("0.4"); + accessor.setProperty(keyBuilder.liveInstance(instance), liveInstance); + result.add(accessor.getProperty(keyBuilder.liveInstance(instance))); + } + return result; + } + + protected void deleteLiveInstances(String clusterName) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<>(_gZkClient)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + Map clientMap = _liveInstanceOwners.getOrDefault(clusterName, Collections.emptyMap()); + + for (String liveInstance : accessor.getChildNames(keyBuilder.liveInstances())) { + ZKHelixDataAccessor dataAccessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<>(_gZkClient)); + dataAccessor.removeProperty(keyBuilder.liveInstance(liveInstance)); + + HelixZkClient client = clientMap.remove(liveInstance); + if (client != null) { + client.close(); + } + } + + if (clientMap.isEmpty()) { + _liveInstanceOwners.remove(clusterName); + } + } + + protected void setupInstances(String clusterName, int[] instances) { + HelixAdmin admin = new ZKHelixAdmin(_gZkClient); + for (int i = 0; i < instances.length; i++) { + String instance = "localhost_" + instances[i]; + InstanceConfig instanceConfig = new InstanceConfig(instance); + instanceConfig.setHostName("localhost"); + instanceConfig.setPort("" + instances[i]); + instanceConfig.setInstanceOperation(InstanceConstants.InstanceOperation.ENABLE); + admin.addInstance(clusterName, instanceConfig); + } + } + + protected void runPipeline(ClusterEvent event, Pipeline pipeline, boolean shouldThrowException) + throws Exception { + try { + pipeline.handle(event); + pipeline.finish(); + } catch (Exception e) { + if (shouldThrowException) { + throw e; + } else { + LOG.error("Exception while executing pipeline: {}. Will not continue to next pipeline", + pipeline, e); + } + } + } + + protected void runStage(ClusterEvent event, Stage stage) throws Exception { + StageContext context = new StageContext(); + stage.init(context); + stage.preProcess(); + + // AbstractAsyncBaseStage will run asynchronously, and it's main logics are implemented in + // execute() function call + // TODO (harry): duplicated code in ZkIntegrationTestBase, consider moving runStage() + // to a shared library + if (stage instanceof AbstractAsyncBaseStage) { + ((AbstractAsyncBaseStage) stage).execute(event); + } else { + stage.process(event); + } + stage.postProcess(); + } + + protected void deleteCluster(String clusterName) { + TestHelper.dropCluster(clusterName, _gZkClient, _gSetupTool); + } + + /** + * Poll for the existence (or lack thereof) of a specific Helix property + * @param clazz the HelixProeprty subclass + * @param accessor connected HelixDataAccessor + * @param key the property key to look up + * @param shouldExist true if the property should exist, false otherwise + * @return the property if found, or null if it does not exist + */ + protected T pollForProperty(Class clazz, HelixDataAccessor accessor, + PropertyKey key, boolean shouldExist) throws InterruptedException { + final int POLL_TIMEOUT = 5000; + final int POLL_INTERVAL = 50; + T property = accessor.getProperty(key); + int timeWaited = 0; + while (((shouldExist && property == null) || (!shouldExist && property != null)) + && timeWaited < POLL_TIMEOUT) { + Thread.sleep(POLL_INTERVAL); + timeWaited += POLL_INTERVAL; + property = accessor.getProperty(key); + } + return property; + } + + /** + * Ensures that external view and current state are empty + */ + protected static class EmptyZkVerifier implements ClusterStateVerifier.ZkVerifier { + private final String _clusterName; + private final String _resourceName; + private final HelixZkClient _zkClient; + + /** + * Instantiate the verifier + * @param clusterName the cluster to verify + * @param resourceName the resource to verify + */ + public EmptyZkVerifier(String clusterName, String resourceName) { + _clusterName = clusterName; + _resourceName = resourceName; + + _zkClient = DedicatedZkClientFactory.getInstance() + .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR)); + _zkClient.setZkSerializer(new ZNRecordSerializer()); + } + + @Override + public boolean verify() { + BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_zkClient); + HelixDataAccessor accessor = new ZKHelixDataAccessor(_clusterName, baseAccessor); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + ExternalView externalView = accessor.getProperty(keyBuilder.externalView(_resourceName)); + + // verify external view empty + if (externalView != null) { + for (String partition : externalView.getPartitionSet()) { + Map stateMap = externalView.getStateMap(partition); + if (stateMap != null && !stateMap.isEmpty()) { + LOG.error("External view not empty for " + partition); + return false; + } + } + } + + // verify current state empty + List liveParticipants = accessor.getChildNames(keyBuilder.liveInstances()); + for (String participant : liveParticipants) { + List sessionIds = accessor.getChildNames(keyBuilder.sessions(participant)); + for (String sessionId : sessionIds) { + CurrentState currentState = + accessor.getProperty(keyBuilder.currentState(participant, sessionId, _resourceName)); + Map partitionStateMap = currentState.getPartitionStateMap(); + if (partitionStateMap != null && !partitionStateMap.isEmpty()) { + LOG.error("Current state not empty for " + participant); + return false; + } + } + + List taskSessionIds = + accessor.getChildNames(keyBuilder.taskCurrentStateSessions(participant)); + for (String sessionId : taskSessionIds) { + CurrentState taskCurrentState = accessor + .getProperty(keyBuilder.taskCurrentState(participant, sessionId, _resourceName)); + Map taskPartitionStateMap = taskCurrentState.getPartitionStateMap(); + if (taskPartitionStateMap != null && !taskPartitionStateMap.isEmpty()) { + LOG.error("Task current state not empty for " + participant); + return false; + } + } + } + return true; + } + + @Override + public ZkClient getZkClient() { + return (ZkClient) _zkClient; + } + + @Override + public String getClusterName() { + return _clusterName; + } + } + +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/base/manager/ClusterControllerManager.java b/helix-gateway/src/test/java/org/apache/helix/gateway/base/manager/ClusterControllerManager.java new file mode 100644 index 0000000000..4fecde88f1 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/base/manager/ClusterControllerManager.java @@ -0,0 +1,49 @@ +package org.apache.helix.gateway.base.manager; + +/* + * 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 org.apache.helix.HelixManagerProperty; +import org.apache.helix.InstanceType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The standalone cluster controller class + */ +public class ClusterControllerManager extends ClusterManager { + private static Logger LOG = LoggerFactory.getLogger(ClusterControllerManager.class); + + public ClusterControllerManager(String zkAddr, String clusterName) { + this(zkAddr, clusterName, "controller"); + } + + public ClusterControllerManager(String zkAddr, String clusterName, String controllerName) { + super(zkAddr, clusterName, controllerName, InstanceType.CONTROLLER); + } + + public ClusterControllerManager(String clusterName, HelixManagerProperty helixManagerProperty) { + super(clusterName, "controller", InstanceType.CONTROLLER, null, null, helixManagerProperty); + } + + @Override + public void finalize() { + super.finalize(); + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/base/manager/ClusterManager.java b/helix-gateway/src/test/java/org/apache/helix/gateway/base/manager/ClusterManager.java new file mode 100644 index 0000000000..2d0070ed0a --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/base/manager/ClusterManager.java @@ -0,0 +1,131 @@ +package org.apache.helix.gateway.base.manager; + +/* + * 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.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.helix.HelixManagerProperty; +import org.apache.helix.InstanceType; +import org.apache.helix.manager.zk.HelixManagerStateListener; +import org.apache.helix.manager.zk.ZKHelixManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ClusterManager extends ZKHelixManager implements Runnable { + private static Logger LOG = LoggerFactory.getLogger(ClusterManager.class); + private static final int DISCONNECT_WAIT_TIME_MS = 3000; + + private static AtomicLong UID = new AtomicLong(10000); + private long _uid; + + private final String _clusterName; + private final String _instanceName; + private final InstanceType _type; + + protected CountDownLatch _startCountDown = new CountDownLatch(1); + protected CountDownLatch _stopCountDown = new CountDownLatch(1); + protected CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1); + + protected boolean _started = false; + + protected Thread _watcher; + + protected ClusterManager(String zkAddr, String clusterName, String instanceName, + InstanceType type) { + super(clusterName, instanceName, type, zkAddr); + _clusterName = clusterName; + _instanceName = instanceName; + _type = type; + _uid = UID.getAndIncrement(); + } + protected ClusterManager(String clusterName, String instanceName, InstanceType instanceType, + String zkAddress, HelixManagerStateListener stateListener, + HelixManagerProperty helixManagerProperty) { + super(clusterName, instanceName, instanceType, zkAddress, stateListener, helixManagerProperty); + _clusterName = clusterName; + _instanceName = instanceName; + _type = instanceType; + _uid = UID.getAndIncrement(); + } + + public void syncStop() { + _stopCountDown.countDown(); + try { + _waitStopFinishCountDown.await(); + _started = false; + } catch (InterruptedException e) { + LOG.error("Interrupted waiting for finish", e); + } + } + + // This should not be called more than once because HelixManager.connect() should not be called more than once. + public void syncStart() { + if (_started) { + throw new RuntimeException( + "Helix Controller already started. Do not call syncStart() more than once."); + } else { + _started = true; + } + + _watcher = new Thread(this); + _watcher.setName(String + .format("ClusterManager_Watcher_%s_%s_%s_%d", _clusterName, _instanceName, _type.name(), _uid)); + LOG.debug("ClusterManager_watcher_{}_{}_{}_{} started, stacktrace {}", _clusterName, _instanceName, _type.name(), _uid, Thread.currentThread().getStackTrace()); + _watcher.start(); + + try { + _startCountDown.await(); + } catch (InterruptedException e) { + LOG.error("Interrupted waiting for start", e); + } + } + + @Override + public void run() { + try { + connect(); + _startCountDown.countDown(); + _stopCountDown.await(); + } catch (Exception e) { + LOG.error("exception running controller-manager", e); + } finally { + _startCountDown.countDown(); + disconnect(); + _waitStopFinishCountDown.countDown(); + } + } + + @Override + public void finalize() { + _watcher.interrupt(); + try { + _watcher.join(DISCONNECT_WAIT_TIME_MS); + } catch (InterruptedException e) { + LOG.error("ClusterManager watcher cleanup in the finalize method was interrupted.", e); + } finally { + if (isConnected()) { + LOG.warn( + "The HelixManager ({}-{}-{}) is still connected after {} ms wait. This is a potential resource leakage!", + _clusterName, _instanceName, _type.name(), DISCONNECT_WAIT_TIME_MS); + } + } + } +} + diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/base/manager/MockParticipantManager.java b/helix-gateway/src/test/java/org/apache/helix/gateway/base/manager/MockParticipantManager.java new file mode 100644 index 0000000000..625eb74525 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/base/manager/MockParticipantManager.java @@ -0,0 +1,117 @@ +package org.apache.helix.gateway.base.manager; + +/* + * 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.concurrent.CountDownLatch; +import org.apache.helix.HelixCloudProperty; +import org.apache.helix.HelixManagerProperty; +import org.apache.helix.HelixPropertyFactory; +import org.apache.helix.InstanceType; +import org.apache.helix.gateway.base.statemodel.MockOFModelFactory; +import org.apache.helix.gateway.base.statemodel.MockTransition; +import org.apache.helix.model.BuiltInStateModelDefinitions; +import org.apache.helix.participant.StateMachineEngine; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MockParticipantManager extends ClusterManager { + private static final Logger LOG = LoggerFactory.getLogger(MockParticipantManager.class); + + protected int _transDelay = 10; + + protected MockOFModelFactory _ofModelFactory; + protected HelixCloudProperty _helixCloudProperty; + + public MockParticipantManager(String zkAddr, String clusterName, String instanceName) { + this(zkAddr, clusterName, instanceName, 10); + } + + public MockParticipantManager(String zkAddr, String clusterName, String instanceName, + int transDelay) { + this(zkAddr, clusterName, instanceName, transDelay, null); + } + + public MockParticipantManager(String zkAddr, String clusterName, String instanceName, + int transDelay, HelixCloudProperty helixCloudProperty) { + this(zkAddr, clusterName, instanceName, transDelay, helixCloudProperty, + HelixPropertyFactory.getInstance().getHelixManagerProperty(zkAddr, clusterName)); + } + + public MockParticipantManager(String zkAddr, String clusterName, String instanceName, + int transDelay, HelixCloudProperty helixCloudProperty, + HelixManagerProperty helixManagerProperty) { + super(clusterName, instanceName, InstanceType.PARTICIPANT, zkAddr, null, helixManagerProperty); + _transDelay = transDelay; + _ofModelFactory = new MockOFModelFactory(null); + _helixCloudProperty = helixCloudProperty; + } + + public MockParticipantManager(String clusterName, String instanceName, + HelixManagerProperty helixManagerProperty, int transDelay, + HelixCloudProperty helixCloudProperty) { + super(clusterName, instanceName, InstanceType.PARTICIPANT, null, null, helixManagerProperty); + _transDelay = transDelay; + _ofModelFactory = new MockOFModelFactory(); + _helixCloudProperty = helixCloudProperty; + } + + public void setTransition(MockTransition transition) { + _ofModelFactory.setTrasition(transition); + } + + /** + * This method should be called before syncStart() called after syncStop() + */ + public void reset() { + syncStop(); + _startCountDown = new CountDownLatch(1); + _stopCountDown = new CountDownLatch(1); + _waitStopFinishCountDown = new CountDownLatch(1); + } + + @Override + public void run() { + try { + StateMachineEngine stateMach = getStateMachineEngine(); + stateMach.registerStateModelFactory(BuiltInStateModelDefinitions.OnlineOffline.name(), + _ofModelFactory); + connect(); + _startCountDown.countDown(); + + _stopCountDown.await(); + } catch (InterruptedException e) { + String msg = + "participant: " + getInstanceName() + ", " + Thread.currentThread().getName() + + " is interrupted"; + LOG.info(msg); + } catch (Exception e) { + LOG.error("exception running participant-manager", e); + } finally { + _startCountDown.countDown(); + + disconnect(); + _waitStopFinishCountDown.countDown(); + } + } + @Override + public void finalize() { + super.finalize(); + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/base/statemodel/MockOFModelFactory.java b/helix-gateway/src/test/java/org/apache/helix/gateway/base/statemodel/MockOFModelFactory.java new file mode 100644 index 0000000000..cf656b29d5 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/base/statemodel/MockOFModelFactory.java @@ -0,0 +1,54 @@ +package org.apache.helix.gateway.base.statemodel; + +/* + * 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 org.apache.helix.participant.statemachine.StateModelFactory; + +// mock master slave state model factory +public class MockOFModelFactory extends StateModelFactory { + private MockTransition _transition; + + public MockOFModelFactory() { + this(null); + } + + public MockOFModelFactory(MockTransition transition) { + _transition = transition; + } + + public void setTrasition(MockTransition transition) { + _transition = transition; + + // set existing transition + for (String resource : getResourceSet()) { + for (String partition : getPartitionSet(resource)) { + MockOFStateModel stateModel = getStateModel(resource, partition); + stateModel.setTransition(transition); + } + } + } + + @Override + public MockOFStateModel createNewStateModel(String resourceName, String partitionKey) { + MockOFStateModel model = new MockOFStateModel(_transition); + + return model; + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/base/statemodel/MockOFStateModel.java b/helix-gateway/src/test/java/org/apache/helix/gateway/base/statemodel/MockOFStateModel.java new file mode 100644 index 0000000000..0e0350e1fd --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/base/statemodel/MockOFStateModel.java @@ -0,0 +1,65 @@ +package org.apache.helix.gateway.base.statemodel; + +/* + * 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 org.apache.helix.NotificationContext; +import org.apache.helix.model.Message; +import org.apache.helix.participant.statemachine.StateModel; +import org.apache.helix.participant.statemachine.StateModelInfo; +import org.apache.helix.participant.statemachine.Transition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +// mock master-slave state model +@StateModelInfo(initialState = "OFFLINE", states = { + "ONLINE", "ERROR" +}) +public class MockOFStateModel extends StateModel { + private static Logger LOG = LoggerFactory.getLogger(MockOFStateModel.class); + + protected MockTransition _transition; + + public MockOFStateModel(MockTransition transition) { + _transition = transition; + } + + public void setTransition(MockTransition transition) { + _transition = transition; + } + + @Transition(to = "*", from = "*") + public void generalTransitionHandle(Message message, NotificationContext context) + throws InterruptedException { + LOG.info(String + .format("Resource %s partition %s becomes %s from %s", message.getResourceName(), + message.getPartitionName(), message.getToState(), message.getFromState())); + if (_transition != null) { + _transition.doTransition(message, context); + } + } + + @Override + public void reset() { + LOG.info("Default MockMSStateModel.reset() invoked"); + if (_transition != null) { + _transition.doReset(); + } + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/base/statemodel/MockTransition.java b/helix-gateway/src/test/java/org/apache/helix/gateway/base/statemodel/MockTransition.java new file mode 100644 index 0000000000..d61ba841d9 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/base/statemodel/MockTransition.java @@ -0,0 +1,41 @@ +package org.apache.helix.gateway.base.statemodel; + +/* + * 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 org.apache.helix.NotificationContext; +import org.apache.helix.model.Message; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MockTransition { + private static Logger LOG = LoggerFactory.getLogger(MockTransition.class); + + // called by state model transition functions + public void doTransition(Message message, NotificationContext context) + throws InterruptedException { + LOG.info("default doTransition() invoked"); + } + + // called by state model reset function + public void doReset() { + LOG.info("default doReset() invoked"); + } + +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/base/util/TestHelper.java b/helix-gateway/src/test/java/org/apache/helix/gateway/base/util/TestHelper.java new file mode 100644 index 0000000000..2fd482d7d0 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/base/util/TestHelper.java @@ -0,0 +1,797 @@ +package org.apache.helix.gateway.base.util; + +/* + * 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.io.File; +import java.io.IOException; +import java.lang.reflect.Method; +import java.net.ServerSocket; +import java.util.ArrayList; +import java.util.Arrays; +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 java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.commons.io.FileUtils; +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixManager; +import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy; +import org.apache.helix.controller.rebalancer.strategy.RebalanceStrategy; +import org.apache.helix.manager.zk.ZKHelixAdmin; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZNRecordSerializer; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.Message; +import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.model.StateModelDefinition.StateModelDefinitionProperty; +import org.apache.helix.store.zk.ZNode; +import org.apache.helix.tools.ClusterSetup; +import org.apache.helix.util.ZKClientPool; +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.impl.factory.SharedZkClientFactory; +import org.apache.helix.zookeeper.zkclient.IDefaultNameSpace; +import org.apache.helix.zookeeper.zkclient.ZkClient; +import org.apache.helix.zookeeper.zkclient.ZkServer; +import org.apache.helix.zookeeper.zkclient.exception.ZkNoNodeException; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; + + +public class TestHelper { + private static final Logger LOG = LoggerFactory.getLogger(TestHelper.class); + public static final long WAIT_DURATION = 60 * 1000L; // 60 seconds + public static final int DEFAULT_REBALANCE_PROCESSING_WAIT_TIME = 1500; + + /** + * Returns a unused random port. + */ + public static int getRandomPort() throws IOException { + ServerSocket sock = new ServerSocket(); + sock.bind(null); + int port = sock.getLocalPort(); + sock.close(); + + return port; + } + + static public ZkServer startZkServer(final String zkAddress) throws Exception { + List empty = Collections.emptyList(); + return TestHelper.startZkServer(zkAddress, empty, true); + } + + static public ZkServer startZkServer(final String zkAddress, final String rootNamespace) throws Exception { + List rootNamespaces = new ArrayList(); + rootNamespaces.add(rootNamespace); + return TestHelper.startZkServer(zkAddress, rootNamespaces, true); + } + + static public ZkServer startZkServer(final String zkAddress, final List rootNamespaces) throws Exception { + return startZkServer(zkAddress, rootNamespaces, true); + } + + static public ZkServer startZkServer(final String zkAddress, final List rootNamespaces, boolean overwrite) + throws Exception { + System.out.println("Start zookeeper at " + zkAddress + " in thread " + Thread.currentThread().getName()); + + String zkDir = zkAddress.replace(':', '_'); + final String logDir = "/tmp/" + zkDir + "/logs"; + final String dataDir = "/tmp/" + zkDir + "/dataDir"; + if (overwrite) { + FileUtils.deleteDirectory(new File(dataDir)); + FileUtils.deleteDirectory(new File(logDir)); + } + ZKClientPool.reset(); + + IDefaultNameSpace defaultNameSpace = new IDefaultNameSpace() { + @Override + public void createDefaultNameSpace(ZkClient zkClient) { + if (rootNamespaces == null) { + return; + } + + for (String rootNamespace : rootNamespaces) { + try { + zkClient.deleteRecursive(rootNamespace); + } catch (Exception e) { + LOG.error("fail to deleteRecursive path:" + rootNamespace, e); + } + } + } + }; + + int port = Integer.parseInt(zkAddress.substring(zkAddress.lastIndexOf(':') + 1)); + ZkServer zkServer = new ZkServer(dataDir, logDir, defaultNameSpace, port); + zkServer.start(); + + return zkServer; + } + + static public void stopZkServer(ZkServer zkServer) { + if (zkServer != null) { + zkServer.shutdown(); + System.out.println( + "Shut down zookeeper at port " + zkServer.getPort() + " in thread " + Thread.currentThread().getName()); + } + } + + public static void setupEmptyCluster(HelixZkClient zkClient, String clusterName) { + ZKHelixAdmin admin = new ZKHelixAdmin(zkClient); + admin.addCluster(clusterName, true); + } + + /** + * convert T[] to set + * @param s + * @return + */ + public static Set setOf(T... s) { + Set set = new HashSet(Arrays.asList(s)); + return set; + } + + /** + * generic method for verification with a timeout + * @param verifierName + * @param args + */ + public static void verifyWithTimeout(String verifierName, long timeout, Object... args) { + final long sleepInterval = 1000; // in ms + final int loop = (int) (timeout / sleepInterval) + 1; + try { + boolean result = false; + int i = 0; + for (; i < loop; i++) { + Thread.sleep(sleepInterval); + // verifier should be static method + result = (Boolean) TestHelper.getMethod(verifierName).invoke(null, args); + + if (result == true) { + break; + } + } + + // debug + // LOG.info(verifierName + ": wait " + ((i + 1) * 1000) + "ms to verify (" + // + result + ")"); + System.err.println(verifierName + ": wait " + ((i + 1) * 1000) + "ms to verify " + " (" + result + ")"); + LOG.debug("args:" + Arrays.toString(args)); + // System.err.println("args:" + Arrays.toString(args)); + + if (result == false) { + LOG.error(verifierName + " fails"); + LOG.error("args:" + Arrays.toString(args)); + } + + Assert.assertTrue(result); + } catch (Exception e) { + LOG.error("Exception in verify: " + verifierName, e); + } + } + + private static Method getMethod(String name) { + Method[] methods = TestHelper.class.getMethods(); + for (Method method : methods) { + if (name.equals(method.getName())) { + return method; + } + } + return null; + } + + public static boolean verifyEmptyCurStateAndExtView(String clusterName, String resourceName, + Set instanceNames, String zkAddr) { + HelixZkClient zkClient = + SharedZkClientFactory.getInstance().buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddr)); + zkClient.setZkSerializer(new ZNRecordSerializer()); + + try { + ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); + Builder keyBuilder = accessor.keyBuilder(); + + for (String instanceName : instanceNames) { + List sessionIds = accessor.getChildNames(keyBuilder.sessions(instanceName)); + + for (String sessionId : sessionIds) { + CurrentState curState = accessor.getProperty(keyBuilder.currentState(instanceName, sessionId, resourceName)); + + if (curState != null && curState.getRecord().getMapFields().size() != 0) { + return false; + } + + CurrentState taskCurState = + accessor.getProperty(keyBuilder.taskCurrentState(instanceName, sessionId, resourceName)); + + if (taskCurState != null && taskCurState.getRecord().getMapFields().size() != 0) { + return false; + } + } + + ExternalView extView = accessor.getProperty(keyBuilder.externalView(resourceName)); + + if (extView != null && extView.getRecord().getMapFields().size() != 0) { + return false; + } + } + + return true; + } finally { + zkClient.close(); + } + } + + public static boolean verifyNotConnected(HelixManager manager) { + return !manager.isConnected(); + } + + public static void setupCluster(String clusterName, String zkAddr, int startPort, String participantNamePrefix, + String resourceNamePrefix, int resourceNb, int partitionNb, int nodesNb, int replica, String stateModelDef, + boolean doRebalance) throws Exception { + TestHelper.setupCluster(clusterName, zkAddr, startPort, participantNamePrefix, resourceNamePrefix, resourceNb, + partitionNb, nodesNb, replica, stateModelDef, RebalanceMode.SEMI_AUTO, doRebalance); + } + + public static void setupCluster(String clusterName, String zkAddr, int startPort, String participantNamePrefix, + String resourceNamePrefix, int resourceNb, int partitionNb, int nodesNb, int replica, String stateModelDef, + RebalanceMode mode, boolean doRebalance) { + HelixZkClient zkClient = + SharedZkClientFactory.getInstance().buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddr)); + try { + if (zkClient.exists("/" + clusterName)) { + LOG.warn("Cluster already exists:" + clusterName + ". Deleting it"); + zkClient.deleteRecursively("/" + clusterName); + } + + ClusterSetup setupTool = new ClusterSetup(zkAddr); + setupTool.addCluster(clusterName, true); + + for (int i = 0; i < nodesNb; i++) { + int port = startPort + i; + setupTool.addInstanceToCluster(clusterName, participantNamePrefix + "_" + port); + } + + for (int i = 0; i < resourceNb; i++) { + String resourceName = resourceNamePrefix + i; + setupTool.addResourceToCluster(clusterName, resourceName, partitionNb, stateModelDef, mode.name(), + mode == RebalanceMode.FULL_AUTO ? CrushEdRebalanceStrategy.class.getName() + : RebalanceStrategy.DEFAULT_REBALANCE_STRATEGY); + if (doRebalance) { + setupTool.rebalanceStorageCluster(clusterName, resourceName, replica); + } + } + } finally { + zkClient.close(); + } + } + + public static void dropCluster(String clusterName, RealmAwareZkClient zkClient) { + ClusterSetup setupTool = new ClusterSetup(zkClient); + dropCluster(clusterName, zkClient, setupTool); + } + + public static void dropCluster(String clusterName, RealmAwareZkClient zkClient, ClusterSetup setup) { + String namespace = "/" + clusterName; + if (zkClient.exists(namespace)) { + try { + setup.deleteCluster(clusterName); + } catch (Exception ex) { + // Failed to delete, give some more time for connections to drop + try { + Thread.sleep(3000L); + setup.deleteCluster(clusterName); + } catch (Exception ignored) { + // OK - just ignore + } + } + } + } + + /** + * @param stateMap + * : "ResourceName/partitionKey" -> setOf(instances) + * @param state + * : MASTER|SLAVE|ERROR... + */ + public static void verifyState(String clusterName, String zkAddr, Map> stateMap, String state) { + HelixZkClient zkClient = + SharedZkClientFactory.getInstance().buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddr)); + zkClient.setZkSerializer(new ZNRecordSerializer()); + + try { + ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); + Builder keyBuilder = accessor.keyBuilder(); + + for (String resGroupPartitionKey : stateMap.keySet()) { + Map retMap = getResourceAndPartitionKey(resGroupPartitionKey); + String resGroup = retMap.get("RESOURCE"); + String partitionKey = retMap.get("PARTITION"); + + ExternalView extView = accessor.getProperty(keyBuilder.externalView(resGroup)); + for (String instance : stateMap.get(resGroupPartitionKey)) { + String actualState = extView.getStateMap(partitionKey).get(instance); + Assert.assertNotNull(actualState, + "externalView doesn't contain state for " + resGroup + "/" + partitionKey + " on " + instance + + " (expect " + state + ")"); + + Assert.assertEquals(actualState, state, + "externalView for " + resGroup + "/" + partitionKey + " on " + instance + " is " + actualState + + " (expect " + state + ")"); + } + } + } finally { + zkClient.close(); + } + } + + /** + * @param resourcePartition + * : key is in form of "resource/partitionKey" or "resource_x" + * @return + */ + private static Map getResourceAndPartitionKey(String resourcePartition) { + String resourceName; + String partitionName; + int idx = resourcePartition.indexOf('/'); + if (idx > -1) { + resourceName = resourcePartition.substring(0, idx); + partitionName = resourcePartition.substring(idx + 1); + } else { + idx = resourcePartition.lastIndexOf('_'); + resourceName = resourcePartition.substring(0, idx); + partitionName = resourcePartition; + } + + Map retMap = new HashMap(); + retMap.put("RESOURCE", resourceName); + retMap.put("PARTITION", partitionName); + return retMap; + } + + public static Map startThreadsConcurrently(final int nrThreads, final Callable method, + final long timeout) { + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch finishCounter = new CountDownLatch(nrThreads); + final Map resultsMap = new ConcurrentHashMap(); + final List threadList = new ArrayList(); + + for (int i = 0; i < nrThreads; i++) { + Thread thread = new Thread() { + @Override + public void run() { + try { + boolean isTimeout = !startLatch.await(timeout, TimeUnit.SECONDS); + if (isTimeout) { + LOG.error("Timeout while waiting for start latch"); + } + } catch (InterruptedException ex) { + LOG.error("Interrupted while waiting for start latch"); + } + + try { + T result = method.call(); + if (result != null) { + resultsMap.put("thread_" + this.getId(), result); + } + LOG.debug("result=" + result); + } catch (Exception e) { + LOG.error("Exeption in executing " + method.getClass().getName(), e); + } + + finishCounter.countDown(); + } + }; + threadList.add(thread); + thread.start(); + } + startLatch.countDown(); + + // wait for all thread to complete + try { + boolean isTimeout = !finishCounter.await(timeout, TimeUnit.SECONDS); + if (isTimeout) { + LOG.error("Timeout while waiting for finish latch. Interrupt all threads"); + for (Thread thread : threadList) { + thread.interrupt(); + } + } + } catch (InterruptedException e) { + LOG.error("Interrupted while waiting for finish latch", e); + } + + return resultsMap; + } + + public static Message createMessage(String msgId, String fromState, String toState, String tgtName, + String resourceName, String partitionName) { + Message msg = new Message(MessageType.STATE_TRANSITION, msgId); + msg.setFromState(fromState); + msg.setToState(toState); + msg.setTgtName(tgtName); + msg.setResourceName(resourceName); + msg.setPartitionName(partitionName); + msg.setStateModelDef("MasterSlave"); + + return msg; + } + + public static String getTestMethodName() { + StackTraceElement[] calls = Thread.currentThread().getStackTrace(); + return calls[2].getMethodName(); + } + + public static String getTestClassName() { + StackTraceElement[] calls = Thread.currentThread().getStackTrace(); + String fullClassName = calls[2].getClassName(); + return fullClassName.substring(fullClassName.lastIndexOf('.') + 1); + } + + public static Map startThreadsConcurrently(final List> methods, final long timeout) { + final int nrThreads = methods.size(); + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch finishCounter = new CountDownLatch(nrThreads); + final Map resultsMap = new ConcurrentHashMap(); + final List threadList = new ArrayList(); + + for (int i = 0; i < nrThreads; i++) { + final Callable method = methods.get(i); + + Thread thread = new Thread() { + @Override + public void run() { + try { + boolean isTimeout = !startLatch.await(timeout, TimeUnit.SECONDS); + if (isTimeout) { + LOG.error("Timeout while waiting for start latch"); + } + } catch (InterruptedException ex) { + LOG.error("Interrupted while waiting for start latch"); + } + + try { + T result = method.call(); + if (result != null) { + resultsMap.put("thread_" + this.getId(), result); + } + LOG.debug("result=" + result); + } catch (Exception e) { + LOG.error("Exeption in executing " + method.getClass().getName(), e); + } + + finishCounter.countDown(); + } + }; + threadList.add(thread); + thread.start(); + } + startLatch.countDown(); + + // wait for all thread to complete + try { + boolean isTimeout = !finishCounter.await(timeout, TimeUnit.SECONDS); + if (isTimeout) { + LOG.error("Timeout while waiting for finish latch. Interrupt all threads"); + for (Thread thread : threadList) { + thread.interrupt(); + } + } + } catch (InterruptedException e) { + LOG.error("Interrupted while waiting for finish latch", e); + } + + return resultsMap; + } + + public static void printCache(Map cache) { + System.out.println("START:Print cache"); + TreeMap map = new TreeMap(); + map.putAll(cache); + + for (String key : map.keySet()) { + ZNode node = map.get(key); + TreeSet childSet = new TreeSet(); + childSet.addAll(node.getChildSet()); + System.out.print( + key + "=" + node.getData() + ", " + childSet + ", " + (node.getStat() == null ? "null\n" : node.getStat())); + } + System.out.println("END:Print cache"); + } + + public static void readZkRecursive(String path, Map map, HelixZkClient zkclient) { + try { + Stat stat = new Stat(); + ZNRecord record = zkclient.readData(path, stat); + List childNames = zkclient.getChildren(path); + ZNode node = new ZNode(path, record, stat); + node.addChildren(childNames); + map.put(path, node); + + for (String childName : childNames) { + String childPath = path + "/" + childName; + readZkRecursive(childPath, map, zkclient); + } + } catch (ZkNoNodeException e) { + // OK + } + } + + public static void readZkRecursive(String path, Map map, BaseDataAccessor zkAccessor) { + try { + Stat stat = new Stat(); + ZNRecord record = zkAccessor.get(path, stat, 0); + List childNames = zkAccessor.getChildNames(path, 0); + // System.out.println("childNames: " + childNames); + ZNode node = new ZNode(path, record, stat); + node.addChildren(childNames); + map.put(path, node); + + if (childNames != null && !childNames.isEmpty()) { + for (String childName : childNames) { + String childPath = path + "/" + childName; + readZkRecursive(childPath, map, zkAccessor); + } + } + } catch (ZkNoNodeException e) { + // OK + } + } + + public static boolean verifyZkCache(List paths, BaseDataAccessor zkAccessor, HelixZkClient zkclient, + boolean needVerifyStat) { + // read everything + Map zkMap = new HashMap(); + Map cache = new HashMap(); + for (String path : paths) { + readZkRecursive(path, zkMap, zkclient); + readZkRecursive(path, cache, zkAccessor); + } + // printCache(map); + + return verifyZkCache(paths, null, cache, zkMap, needVerifyStat); + } + + public static boolean verifyZkCache(List paths, Map cache, HelixZkClient zkclient, + boolean needVerifyStat) { + return verifyZkCache(paths, null, cache, zkclient, needVerifyStat); + } + + public static boolean verifyZkCache(List paths, List pathsExcludeForStat, Map cache, + HelixZkClient zkclient, boolean needVerifyStat) { + // read everything on zk under paths + Map zkMap = new HashMap(); + for (String path : paths) { + readZkRecursive(path, zkMap, zkclient); + } + // printCache(map); + + return verifyZkCache(paths, pathsExcludeForStat, cache, zkMap, needVerifyStat); + } + + public static boolean verifyZkCache(List paths, List pathsExcludeForStat, Map cache, + Map zkMap, boolean needVerifyStat) { + // equal size + if (zkMap.size() != cache.size()) { + System.err.println("size mismatch: cacheSize: " + cache.size() + ", zkMapSize: " + zkMap.size()); + System.out.println("cache: (" + cache.size() + ")"); + TestHelper.printCache(cache); + + System.out.println("zkMap: (" + zkMap.size() + ")"); + TestHelper.printCache(zkMap); + + return false; + } + + // everything in cache is also in map + for (String path : cache.keySet()) { + ZNode cacheNode = cache.get(path); + ZNode zkNode = zkMap.get(path); + + if (zkNode == null) { + // in cache but not on zk + System.err.println("path: " + path + " in cache but not on zk: inCacheNode: " + cacheNode); + return false; + } + + if ((zkNode.getData() == null && cacheNode.getData() != null) || (zkNode.getData() != null + && cacheNode.getData() == null) || (zkNode.getData() != null && cacheNode.getData() != null + && !zkNode.getData().equals(cacheNode.getData()))) { + // data not equal + System.err.println( + "data mismatch on path: " + path + ", inCache: " + cacheNode.getData() + ", onZk: " + zkNode.getData()); + return false; + } + + if ((zkNode.getChildSet() == null && cacheNode.getChildSet() != null) || (zkNode.getChildSet() != null + && cacheNode.getChildSet() == null) || (zkNode.getChildSet() != null && cacheNode.getChildSet() != null + && !zkNode.getChildSet().equals(cacheNode.getChildSet()))) { + // childSet not equal + System.err.println("childSet mismatch on path: " + path + ", inCache: " + cacheNode.getChildSet() + ", onZk: " + + zkNode.getChildSet()); + return false; + } + + if (needVerifyStat && pathsExcludeForStat != null && !pathsExcludeForStat.contains(path)) { + if (cacheNode.getStat() == null || !zkNode.getStat().equals(cacheNode.getStat())) { + // stat not equal + System.err.println( + "Stat mismatch on path: " + path + ", inCache: " + cacheNode.getStat() + ", onZk: " + zkNode.getStat()); + return false; + } + } + } + + return true; + } + + public static StateModelDefinition generateStateModelDefForBootstrap() { + ZNRecord record = new ZNRecord("Bootstrap"); + record.setSimpleField(StateModelDefinitionProperty.INITIAL_STATE.toString(), "IDLE"); + List statePriorityList = new ArrayList(); + statePriorityList.add("ONLINE"); + statePriorityList.add("BOOTSTRAP"); + statePriorityList.add("OFFLINE"); + statePriorityList.add("IDLE"); + statePriorityList.add("DROPPED"); + statePriorityList.add("ERROR"); + record.setListField(StateModelDefinitionProperty.STATE_PRIORITY_LIST.toString(), statePriorityList); + for (String state : statePriorityList) { + String key = state + ".meta"; + Map metadata = new HashMap(); + if (state.equals("ONLINE")) { + metadata.put("count", StateModelDefinition.STATE_REPLICA_COUNT_ALL_REPLICAS); + record.setMapField(key, metadata); + } else if (state.equals("BOOTSTRAP")) { + metadata.put("count", "-1"); + record.setMapField(key, metadata); + } else if (state.equals("OFFLINE")) { + metadata.put("count", "-1"); + record.setMapField(key, metadata); + } else if (state.equals("IDLE")) { + metadata.put("count", "-1"); + record.setMapField(key, metadata); + } else if (state.equals("DROPPED")) { + metadata.put("count", "-1"); + record.setMapField(key, metadata); + } else if (state.equals("ERROR")) { + metadata.put("count", "-1"); + record.setMapField(key, metadata); + } + } + + for (String state : statePriorityList) { + String key = state + ".next"; + if (state.equals("ONLINE")) { + Map metadata = new HashMap(); + metadata.put("BOOTSTRAP", "OFFLINE"); + metadata.put("OFFLINE", "OFFLINE"); + metadata.put("DROPPED", "OFFLINE"); + metadata.put("IDLE", "OFFLINE"); + record.setMapField(key, metadata); + } else if (state.equals("BOOTSTRAP")) { + Map metadata = new HashMap(); + metadata.put("ONLINE", "ONLINE"); + metadata.put("OFFLINE", "OFFLINE"); + metadata.put("DROPPED", "OFFLINE"); + metadata.put("IDLE", "OFFLINE"); + record.setMapField(key, metadata); + } else if (state.equals("OFFLINE")) { + Map metadata = new HashMap(); + metadata.put("ONLINE", "BOOTSTRAP"); + metadata.put("BOOTSTRAP", "BOOTSTRAP"); + metadata.put("DROPPED", "IDLE"); + metadata.put("IDLE", "IDLE"); + record.setMapField(key, metadata); + } else if (state.equals("IDLE")) { + Map metadata = new HashMap(); + metadata.put("ONLINE", "OFFLINE"); + metadata.put("BOOTSTRAP", "OFFLINE"); + metadata.put("OFFLINE", "OFFLINE"); + metadata.put("DROPPED", "DROPPED"); + record.setMapField(key, metadata); + } else if (state.equals("ERROR")) { + Map metadata = new HashMap(); + metadata.put("IDLE", "IDLE"); + record.setMapField(key, metadata); + } + } + List stateTransitionPriorityList = new ArrayList(); + stateTransitionPriorityList.add("ONLINE-OFFLINE"); + stateTransitionPriorityList.add("BOOTSTRAP-ONLINE"); + stateTransitionPriorityList.add("OFFLINE-BOOTSTRAP"); + stateTransitionPriorityList.add("BOOTSTRAP-OFFLINE"); + stateTransitionPriorityList.add("OFFLINE-IDLE"); + stateTransitionPriorityList.add("IDLE-OFFLINE"); + stateTransitionPriorityList.add("IDLE-DROPPED"); + stateTransitionPriorityList.add("ERROR-IDLE"); + record.setListField(StateModelDefinitionProperty.STATE_TRANSITION_PRIORITYLIST.toString(), + stateTransitionPriorityList); + return new StateModelDefinition(record); + } + + public static String znrecordToString(ZNRecord record) { + StringBuffer sb = new StringBuffer(); + sb.append(record.getId() + "\n"); + Map simpleFields = record.getSimpleFields(); + if (simpleFields != null) { + sb.append("simpleFields\n"); + for (String key : simpleFields.keySet()) { + sb.append(" " + key + "\t: " + simpleFields.get(key) + "\n"); + } + } + + Map> listFields = record.getListFields(); + sb.append("listFields\n"); + for (String key : listFields.keySet()) { + List list = listFields.get(key); + sb.append(" " + key + "\t: "); + for (String listValue : list) { + sb.append(listValue + ", "); + } + sb.append("\n"); + } + + Map> mapFields = record.getMapFields(); + sb.append("mapFields\n"); + for (String key : mapFields.keySet()) { + Map map = mapFields.get(key); + sb.append(" " + key + "\t: \n"); + for (String mapKey : map.keySet()) { + sb.append(" " + mapKey + "\t: " + map.get(mapKey) + "\n"); + } + } + + return sb.toString(); + } + + public interface Verifier { + boolean verify() throws Exception; + } + + public static boolean verify(Verifier verifier, long timeout) throws Exception { + long start = System.currentTimeMillis(); + do { + boolean result = verifier.verify(); + boolean isTimedout = (System.currentTimeMillis() - start) > timeout; + if (result || isTimedout) { + if (isTimedout && !result) { + LOG.error("verifier time out, consider try longer timeout, stack trace{}", + Arrays.asList(Thread.currentThread().getStackTrace())); + } + return result; + } + Thread.sleep(50); + } while (true); + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/channel/TestGatewayServiceChannelConfig.java b/helix-gateway/src/test/java/org/apache/helix/gateway/channel/TestGatewayServiceChannelConfig.java new file mode 100644 index 0000000000..b0d2f0dc60 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/channel/TestGatewayServiceChannelConfig.java @@ -0,0 +1,88 @@ +package org.apache.helix.gateway.channel; + +/* + * 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 org.testng.Assert; +import org.testng.annotations.Test; + + +public class TestGatewayServiceChannelConfig { + + @Test + public void testGatewayServiceChannelConfigBuilder() { + GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder builder = + new GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder(); + + builder.setChannelMode(GatewayServiceChannelConfig.ChannelMode.PUSH_MODE) + .setParticipantConnectionChannelType(GatewayServiceChannelConfig.ChannelType.GRPC_SERVER) + .setShardStateProcessorType(GatewayServiceChannelConfig.ChannelType.GRPC_SERVER) + .setGrpcServerPort(50051) + .setServerHeartBeatInterval(30) + .setMaxAllowedClientHeartBeatInterval(60) + .setClientTimeout(120) + .setEnableReflectionService(true) + .setPollIntervalSec(10); + + GatewayServiceChannelConfig config = builder.build(); + + Assert.assertEquals(config.getChannelMode(), GatewayServiceChannelConfig.ChannelMode.PUSH_MODE); + Assert.assertEquals(config.getParticipantConnectionChannelType(), + GatewayServiceChannelConfig.ChannelType.GRPC_SERVER); + Assert.assertEquals(config.getShardStateChannelType(), GatewayServiceChannelConfig.ChannelType.GRPC_SERVER); + Assert.assertEquals(config.getGrpcServerPort(), 50051); + Assert.assertEquals(config.getServerHeartBeatInterval(), 30); + Assert.assertEquals(config.getMaxAllowedClientHeartBeatInterval(), 60); + Assert.assertEquals(config.getClientTimeout(), 120); + Assert.assertTrue(config.getEnableReflectionService()); + Assert.assertEquals(config.getPollIntervalSec(), 10); + } + + @Test + public void testInvalidConfigForGrpc() { + GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder builder = + new GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder(); + + builder.setParticipantConnectionChannelType(GatewayServiceChannelConfig.ChannelType.GRPC_SERVER); + + // assert er get an exception + try { + builder.build(); + Assert.fail("Should have thrown an exception"); + } catch (IllegalArgumentException e) { + // expected + } + } + + @Test + public void testInvalidConfigForFile() { + GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder builder = + new GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder(); + + builder.setParticipantConnectionChannelType(GatewayServiceChannelConfig.ChannelType.FILE); + + // assert er get an exception + try { + builder.build(); + Assert.fail("Should have thrown an exception"); + } catch (IllegalArgumentException e) { + // expected + } + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/channel/TestHelixGatewayServicePollModeChannel.java b/helix-gateway/src/test/java/org/apache/helix/gateway/channel/TestHelixGatewayServicePollModeChannel.java new file mode 100644 index 0000000000..98941e94dc --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/channel/TestHelixGatewayServicePollModeChannel.java @@ -0,0 +1,90 @@ +package org.apache.helix.gateway.channel; + +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import org.apache.helix.gateway.api.constant.GatewayServiceEventType; +import org.apache.helix.gateway.service.GatewayServiceEvent; +import org.apache.helix.gateway.service.GatewayServiceManager; +import org.junit.Assert; +import org.testng.annotations.Test; + +import static org.apache.helix.gateway.channel.GatewayServiceChannelConfig.ChannelMode.*; +import static org.mockito.Mockito.*; + + +public class TestHelixGatewayServicePollModeChannel { + private HelixGatewayServicePollModeChannel pollModeChannel; + private GatewayServiceManager manager; + private ScheduledExecutorService scheduler; + + int connectEventCount = 0; + int disconnectEventCount = 0; + int updateEventCount = 0; + + @Test + public void testFetchUpdates() { + scheduler = mock(ScheduledExecutorService.class); + GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder builder = + new GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder().setChannelMode(POLL_MODE) + .setHealthCheckEndpointMap(Map.of("cluster1", Map.of("instance1", "endpoint1"))) + .setParticipantConnectionChannelType(GatewayServiceChannelConfig.ChannelType.FILE) + .setShardStateProcessorType(GatewayServiceChannelConfig.ChannelType.FILE) + .addPollModeConfig(GatewayServiceChannelConfig.FileBasedConfigType.PARTICIPANT_CURRENT_STATE_PATH, + "CurrentStatePath") + .addPollModeConfig(GatewayServiceChannelConfig.FileBasedConfigType.SHARD_TARGET_STATE_PATH, + "shardTargetStatePath") + .setPollIntervalSec(60 * 1000) // set a larger number to avoid recurrent polling + .setPollStartDelaySec(60 * 1000) + .setTargetFileUpdateIntervalSec(60 * 1000); + + manager = new DummyGatewayServiceManager(builder.build()); + pollModeChannel = spy(new HelixGatewayServicePollModeChannel(manager, builder.build())); + pollModeChannel._scheduler = scheduler; // Inject the mocked scheduler + + // Mock the necessary methods and data + doReturn(true).when(pollModeChannel).fetchInstanceLivenessStatus("cluster1", "instance1"); + Map>>> currentStateMap = + Map.of("cluster1", Map.of("instance1", Map.of("resource1", Map.of("shard", "ONLINE")))); + doReturn(currentStateMap).when(pollModeChannel).getChangedParticipantsCurrentState(any()); + + // 1. Call fetch update for first time, verify we got a init connect event + pollModeChannel.fetchUpdates(); + + Assert.assertEquals(1, connectEventCount); + + // 2. Change currentStateMap, Call fetch update for second time, verify we got an update event + Map>>> currentStateMap2 = + Map.of("cluster1", Map.of("instance1", Map.of("resource1", Map.of("shard", "OFFLINE")))); + doReturn(currentStateMap2).when(pollModeChannel).getChangedParticipantsCurrentState(any()); + pollModeChannel.fetchUpdates(); + Assert.assertEquals(1, updateEventCount); + + // call pne more time with same shard state, verify no new event + pollModeChannel.fetchUpdates(); + Assert.assertEquals(1, updateEventCount); + + // 3. Change health check result, Call fetch update for third time, verify we got a disconnect event + doReturn(false).when(pollModeChannel).fetchInstanceLivenessStatus("cluster1", "instance1"); + pollModeChannel.fetchUpdates(); + Assert.assertEquals(1, disconnectEventCount); + } + + class DummyGatewayServiceManager extends GatewayServiceManager { + + public DummyGatewayServiceManager(GatewayServiceChannelConfig gatewayServiceChannelConfig) { + super("dummyZkAddress", gatewayServiceChannelConfig); + } + + @Override + public void onGatewayServiceEvent(GatewayServiceEvent event) { + if (event.getEventType().equals(GatewayServiceEventType.CONNECT)) { + connectEventCount++; + } else if (event.getEventType().equals(GatewayServiceEventType.DISCONNECT)) { + disconnectEventCount++; + } else if (event.getEventType().equals(GatewayServiceEventType.UPDATE)) { + updateEventCount++; + } + System.out.println("Received event: " + event.getEventType()); + } + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/integration/TestFilePullChannelE2E.java b/helix-gateway/src/test/java/org/apache/helix/gateway/integration/TestFilePullChannelE2E.java new file mode 100644 index 0000000000..fa4c948fbf --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/integration/TestFilePullChannelE2E.java @@ -0,0 +1,258 @@ +package org.apache.helix.gateway.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.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import org.apache.helix.ConfigAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.gateway.base.HelixGatewayTestBase; +import org.apache.helix.gateway.channel.GatewayServiceChannelConfig; +import org.apache.helix.gateway.service.GatewayServiceManager; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZKHelixManager; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.model.ClusterConfig; +import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier; +import org.testng.Assert; +import org.testng.annotations.AfterTest; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.apache.helix.gateway.channel.GatewayServiceChannelConfig.ChannelMode.*; + + +public class TestFilePullChannelE2E extends HelixGatewayTestBase { + + private static final String CLUSTER_NAME = "CLUSTER_" + TestFilePullChannelE2E.class.getSimpleName(); + private static final int START_NUM_NODE = 3; + private static final String TEST_DB = "TestDB"; + private static final String TEST_STATE_MODEL = "OnlineOffline"; + private static final String CONTROLLER_PREFIX = "controller"; + private static final String currentStatePath = "tmpcurrentState"; + private static final String targetStatePath = "tmptargetState"; + GatewayServiceManager manager1, manager2, manager0; + ArrayList csPaths = new ArrayList(); + ArrayList targetPaths = new ArrayList(); + ArrayList healthPaths = new ArrayList(); + private ClusterControllerManager _controller; + + @BeforeClass + public void beforeClass() { + super.beforeClass(); + + // Set up the Helix cluster + ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient); + _gSetupTool.addCluster(CLUSTER_NAME, true); + + ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME); + clusterConfig.getRecord().setSimpleField(ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, "true"); + configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig); + + // Start the controller + String controllerName = CONTROLLER_PREFIX + '_' + CLUSTER_NAME; + _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName); + _controller.syncStart(); + + // Enable best possible assignment persistence + enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true); + } + + @Test + public void testE2E() throws Exception { + // create files for health state + try { + for (int i = 0; i < START_NUM_NODE; i++) { + csPaths.add(createTempFile(currentStatePath + i, ".txt", "")); + targetPaths.add(createTempFile(targetStatePath + i, ".txt", "")); + String currentTime = String.valueOf(System.currentTimeMillis()/1000); + String content = "{\"IsAlive\":" + true + ",\"LastUpdateTime\":" + currentTime + "}"; + healthPaths.add(createTempFile("tmphealthCheck" + i, ".txt", content)); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + + String fileName0 = healthPaths.get(0).toAbsolutePath().toString(); + String fileName1 = healthPaths.get(1).toAbsolutePath().toString(); + String fileName2 = healthPaths.get(2).toAbsolutePath().toString(); + + GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder builder = + new GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder().setChannelMode(POLL_MODE) + .setParticipantConnectionChannelType(GatewayServiceChannelConfig.ChannelType.FILE) + .setShardStateProcessorType(GatewayServiceChannelConfig.ChannelType.FILE) + .setPollIntervalSec(1) // set a larger number to avoid recurrent polling + .setPollStartDelaySec(1) + .setTargetFileUpdateIntervalSec(1); + + // create empty file for shard state + + // create 3 manager instances + manager0 = new GatewayServiceManager(ZK_ADDR, + builder.addPollModeConfig(GatewayServiceChannelConfig.FileBasedConfigType.PARTICIPANT_CURRENT_STATE_PATH, + csPaths.get(0).toAbsolutePath().toString()) + .addPollModeConfig(GatewayServiceChannelConfig.FileBasedConfigType.SHARD_TARGET_STATE_PATH, + targetPaths.get(0).toAbsolutePath().toString()) + .setHealthCheckEndpointMap(Map.of(CLUSTER_NAME, Map.of("instance0", fileName0))) + .build()); + manager1 = new GatewayServiceManager(ZK_ADDR, + builder.addPollModeConfig(GatewayServiceChannelConfig.FileBasedConfigType.PARTICIPANT_CURRENT_STATE_PATH, + csPaths.get(1).toAbsolutePath().toString()) + .addPollModeConfig(GatewayServiceChannelConfig.FileBasedConfigType.SHARD_TARGET_STATE_PATH, + targetPaths.get(1).toAbsolutePath().toString()) + .setHealthCheckEndpointMap(Map.of(CLUSTER_NAME, Map.of("instance1", fileName1))) + .build()); + manager2 = new GatewayServiceManager(ZK_ADDR, + builder.addPollModeConfig(GatewayServiceChannelConfig.FileBasedConfigType.PARTICIPANT_CURRENT_STATE_PATH, + csPaths.get(2).toAbsolutePath().toString()) + .addPollModeConfig(GatewayServiceChannelConfig.FileBasedConfigType.SHARD_TARGET_STATE_PATH, + targetPaths.get(2).toAbsolutePath().toString()) + .setHealthCheckEndpointMap(Map.of(CLUSTER_NAME, Map.of("instance2", fileName2))) + .build()); + + System.out.println("Starting all managers"); + manager0.startService(); + manager1.startService(); + manager2.startService(); + + // verify we see live instances + verifyInstances(CLUSTER_NAME, List.of("instance0", "instance1", "instance2")); + + // create an DB on cluster + createDB(); + + // read the target state file and verify the target state is updated + verifyTargetState(); + + // write current state to file + for (int i = 0; i < 3; i++) { + String content = + "{\"" + CLUSTER_NAME + "\" : { \"instance" + i + "\" : { \"TestDB\" : {\"TestDB_0\" : \"ONLINE\" }}}} "; + Files.write(csPaths.get(i), content.getBytes()); + } + + // check no pending messages for partitions + verifyNoPendingMessages(List.of("instance0", "instance1", "instance2")); + + // change health state to false on two instances + String currentTime = String.valueOf(System.currentTimeMillis()/1000 - 100); + String content = "{\"IsAlive\":" + true + ",\"LastUpdateTime\":" + currentTime + "}"; + Files.write(healthPaths.get(0), content.getBytes()); + + String content2 = "{\"IsAlive\":" + false + ",\"LastUpdateTime\":" + currentTime + "}"; + Files.write(healthPaths.get(1), content2.getBytes()); + + // check live instance for that instance is gone + Assert.assertTrue(TestHelper.verify(() -> { + List liveInstance = getLiveInstances(); + return !liveInstance.contains("instance0") && !liveInstance.contains("instance1") && liveInstance.contains( + "instance2"); + }, TestHelper.WAIT_DURATION)); + + // stop all manager + manager0.stopService(); + manager1.stopService(); + manager2.stopService(); + + + // check target state files are gone + for (int i = 0; i < 3; i++) { + Assert.assertFalse(Files.exists(targetPaths.get(i))); + } + + // check all live instances are gone + Assert.assertTrue(TestHelper.verify(() -> { + List liveInstance = getLiveInstances(); + return !liveInstance.contains("instance0") && !liveInstance.contains("instance1") && !liveInstance.contains( + "instance2"); + }, TestHelper.WAIT_DURATION)); + + for (int i = 0; i < 3; i++) { + try { + Files.deleteIfExists(csPaths.get(i)); + Files.deleteIfExists(targetPaths.get(i)); + Files.deleteIfExists(healthPaths.get(i)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + + private void verifyInstances(String clusterName, List instance0) throws Exception { + for (String instance : instance0) { + Assert.assertTrue(TestHelper.verify( + () -> _gSetupTool.getClusterManagementTool().getInstancesInCluster(clusterName).contains(instance), + TestHelper.WAIT_DURATION)); + } + } + + private List getLiveInstances() { + ZKHelixDataAccessor dataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<>(_gZkClient)); + PropertyKey liveInstances = dataAccessor.keyBuilder().liveInstances(); + return dataAccessor.getChildNames(liveInstances); + } + + private void verifyNoPendingMessages(List participants) throws Exception { + ZKHelixDataAccessor dataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<>(_gZkClient)); + for (String participant : participants) { + PropertyKey messagesNode = dataAccessor.keyBuilder().messages(participant); + Assert.assertTrue( + TestHelper.verify(() -> dataAccessor.getChildNames(messagesNode).isEmpty(), TestHelper.WAIT_DURATION)); + } + } + + private void verifyTargetState() throws Exception { + for (int i = 0; i < 3; i++) { + int finalI = i; + Assert.assertTrue(TestHelper.verify(() -> { + String content = Files.readString(targetPaths.get(finalI)); + return content.contains("{\"TestDB\":{\"TestDB_0\":\"ONLINE\"}}"); + }, TestHelper.WAIT_DURATION)); + } + } + + public static Path createTempFile(String prefix, String suffix, String content) throws IOException { + // Create a temporary file + Path tempFile = Files.createTempFile(prefix, suffix); + + // Write content to the temporary file + Files.write(tempFile, content.getBytes()); + + return tempFile; + } + + private void createDB() { + createDBInSemiAuto(_gSetupTool, CLUSTER_NAME, TEST_DB, List.of("instance0", "instance1", "instance2"), + TEST_STATE_MODEL, 1, 3); + + _clusterVerifier = new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR) + .setResources(new HashSet<>(_gSetupTool.getClusterManagementTool().getResourcesInCluster(CLUSTER_NAME))) + .setWaitTillVerify(TestHelper.DEFAULT_REBALANCE_PROCESSING_WAIT_TIME) + .build(); + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/participant/TestHelixGatewayParticipant.java b/helix-gateway/src/test/java/org/apache/helix/gateway/participant/TestHelixGatewayParticipant.java new file mode 100644 index 0000000000..128a228576 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/participant/TestHelixGatewayParticipant.java @@ -0,0 +1,444 @@ +package org.apache.helix.gateway.participant; + +/* + * 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.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import org.apache.helix.ConfigAccessor; +import org.apache.helix.TestHelper; +import org.apache.helix.common.ZkTestBase; +import org.apache.helix.gateway.api.service.HelixGatewayServiceChannel; +import org.apache.helix.gateway.channel.GatewayServiceChannelConfig; +import org.apache.helix.gateway.service.GatewayServiceManager; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.manager.zk.ZKHelixManager; +import org.apache.helix.model.ClusterConfig; +import org.apache.helix.model.IdealState; +import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier; +import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; +import org.testng.collections.Lists; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass.ShardChangeRequests; + + +public class TestHelixGatewayParticipant extends ZkTestBase { + private static final String CLUSTER_NAME = TestHelixGatewayParticipant.class.getSimpleName(); + private static final int START_NUM_NODE = 3; + private static final String TEST_DB = "TestDB"; + private static final String TEST_STATE_MODEL = "OnlineOffline"; + private static final String CONTROLLER_PREFIX = "controller"; + private static final String PARTICIPANT_PREFIX = "participant"; + + private ZkHelixClusterVerifier _clusterVerifier; + private ClusterControllerManager _controller; + private int _nextStartPort = 12000; + private final List _participants = Lists.newArrayList(); + private final Map _pendingMessageMap = new ConcurrentHashMap<>(); + private final AtomicInteger _onDisconnectCallbackCount = new AtomicInteger(); + + private GatewayServiceManager _gatewayServiceManager; + + @BeforeClass + public void beforeClass() { + GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder builder = + new GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder(); + + builder.setParticipantConnectionChannelType(GatewayServiceChannelConfig.ChannelType.GRPC_SERVER).setGrpcServerPort(5001); + GatewayServiceChannelConfig config = builder.build(); + _gatewayServiceManager = new GatewayServiceManager(ZK_ADDR, config); + + // Set up the Helix cluster + ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient); + _gSetupTool.addCluster(CLUSTER_NAME, true); + + ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME); + clusterConfig.getRecord().setSimpleField(ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, "true"); + configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig); + + // Start initial participants + for (int i = 0; i < START_NUM_NODE; i++) { + addParticipant(); + } + + // Start the controller + String controllerName = CONTROLLER_PREFIX + '_' + CLUSTER_NAME; + _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName); + _controller.syncStart(); + + // Enable best possible assignment persistence + enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true); + } + + @AfterClass + public void afterClass() { + // Clean up by disconnecting the controller and participants + _controller.disconnect(); + for (HelixGatewayParticipant participant : _participants) { + participant.disconnect(); + } + } + + /** + * Add a participant with a specific initial state map. + */ + private HelixGatewayParticipant addParticipant(String participantName, + Map> initialShardMap) { + _gatewayServiceManager.resetTargetStateCache(CLUSTER_NAME, participantName); + HelixGatewayParticipant participant = + new HelixGatewayParticipant.Builder(new MockHelixGatewayServiceChannel(_pendingMessageMap), participantName, + CLUSTER_NAME, ZK_ADDR, _onDisconnectCallbackCount::incrementAndGet, _gatewayServiceManager).addMultiTopStateStateModelDefinition( + TEST_STATE_MODEL).setInitialShardState(initialShardMap).build(); + _participants.add(participant); + return participant; + } + + /** + * Add a participant with an empty initial state map. + */ + private HelixGatewayParticipant addParticipant() { + String participantName = PARTICIPANT_PREFIX + "_" + _nextStartPort++; + return addParticipant(participantName, Collections.emptyMap()); + } + + /** + * Remove a participant from the cluster. + */ + private void deleteParticipant(HelixGatewayParticipant participant) { + participant.disconnect(); + _participants.remove(participant); + } + + /** + * Add a participant to the IdealState's preference list. + */ + private void addToPreferenceList(HelixGatewayParticipant participant) { + IdealState idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB); + idealState.getPreferenceLists() + .values() + .forEach(preferenceList -> preferenceList.add(participant.getInstanceName())); + idealState.setReplicas(String.valueOf(Integer.parseInt(idealState.getReplicas()) + 1)); + _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, TEST_DB, idealState); + } + + /** + * Remove a participant from the IdealState's preference list. + */ + private void removeFromPreferenceList(HelixGatewayParticipant participant) { + IdealState idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB); + idealState.getPreferenceLists() + .values() + .forEach(preferenceList -> preferenceList.remove(participant.getInstanceName())); + idealState.setReplicas(String.valueOf(Integer.parseInt(idealState.getReplicas()) - 1)); + _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, TEST_DB, idealState); + } + + /** + * Create a test database in the cluster with a semi-automatic state model. + */ + private void createDB() { + createDBInSemiAuto(_gSetupTool, CLUSTER_NAME, TEST_DB, + _participants.stream().map(HelixGatewayParticipant::getInstanceName).collect(Collectors.toList()), + TEST_STATE_MODEL, 1, _participants.size()); + + _clusterVerifier = new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR) + .setResources(new HashSet<>(_gSetupTool.getClusterManagementTool().getResourcesInCluster(CLUSTER_NAME))) + .setWaitTillVerify(TestHelper.DEFAULT_REBALANCE_PROCESSING_WAIT_TIME) + .build(); + } + + /** + * Retrieve a pending message for a specific participant. + */ + private ShardChangeRequests getPendingMessage(String instanceName) { + return _pendingMessageMap.get(instanceName); + } + + /** + * Process the pending message for a participant. + */ + private void processPendingMessage(HelixGatewayParticipant participant, boolean isSuccess, String toState) { + ShardChangeRequests requests = _pendingMessageMap.remove(participant.getInstanceName()); + + Map>> newSInstanceStateMap = new HashMap<>(); + newSInstanceStateMap.put(participant.getInstanceName(), + createSingleShardStateMap(requests.getRequest(0).getResourceName(), requests.getRequest(0).getShardName(), + isSuccess ? toState : "ERROR")); + _gatewayServiceManager.updateCacheWithNewCurrentStateAndGetDiff(CLUSTER_NAME, newSInstanceStateMap); + + participant.completeStateTransition(requests.getRequest(0).getResourceName(), requests.getRequest(0).getShardName(), + isSuccess ? toState : "ERROR"); + } + + /** + * Create a single shard state map. + */ + Map> createSingleShardStateMap( String resource, String shard, String state) { + + Map> resourceStateMap = new HashMap<>(); + Map shardStateMap = new HashMap<>(); + shardStateMap.put(shard, state); + resourceStateMap.put(resource, shardStateMap); + return resourceStateMap; + } + + /** + * Get the current state of a Helix shard. + */ + private String getHelixCurrentState(String instanceName, String resourceName, String shardId) { + return _gSetupTool.getClusterManagementTool() + .getResourceExternalView(CLUSTER_NAME, resourceName) + .getStateMap(shardId) + .getOrDefault(instanceName, HelixGatewayParticipant.UNASSIGNED_STATE); + } + + /** + * Verify that all specified participants have pending messages. + */ + private void verifyPendingMessages(List participants) throws Exception { + Assert.assertTrue(TestHelper.verify( + () -> participants.stream().allMatch(participant -> getPendingMessage(participant.getInstanceName()) != null), + TestHelper.WAIT_DURATION)); + } + + /** + * Verify that the gateway state matches the Helix state for all participants. + */ + private void verifyGatewayStateMatchesHelixState() throws Exception { + Assert.assertTrue(TestHelper.verify(() -> _participants.stream().allMatch(participant -> { + String instanceName = participant.getInstanceName(); + for (String resourceName : _gSetupTool.getClusterManagementTool().getResourcesInCluster(CLUSTER_NAME)) { + for (String shardId : _gSetupTool.getClusterManagementTool() + .getResourceIdealState(CLUSTER_NAME, resourceName) + .getPartitionSet()) { + String helixCurrentState = getHelixCurrentState(instanceName, resourceName, shardId); + if (!participant.getCurrentState(resourceName, shardId).equals(helixCurrentState) && !( + participant.getCurrentState(resourceName, shardId).equals("DROPPED") && helixCurrentState.equals( + "UNASSIGNED"))) { + return false; + } + } + } + return true; + }), TestHelper.WAIT_DURATION)); + } + + private void verifyGatewayTargetStateMatchHelixTargetState() throws Exception { + Assert.assertTrue(TestHelper.verify(() -> _participants.stream().allMatch(participant -> { + String instanceName = participant.getInstanceName(); + for (String resourceName : _gSetupTool.getClusterManagementTool().getResourcesInCluster(CLUSTER_NAME)) { + for (String shardId : _gSetupTool.getClusterManagementTool() + .getResourceIdealState(CLUSTER_NAME, resourceName) + .getPartitionSet()) { + String helixTargetState = getHelixCurrentState(instanceName, resourceName, shardId); + if (_gatewayServiceManager.getTargetState(CLUSTER_NAME, instanceName, resourceName, shardId) == null) { + System.out.println("Gateway target state is null for instance: " + instanceName + ", resource: " + resourceName + ", shard: " + shardId); + } + if (!participant.getCurrentState(resourceName, shardId).equals(helixTargetState) && !( + participant.getCurrentState(resourceName, shardId).equals("DROPPED") && helixTargetState.equals( + "UNASSIGNED"))) { + return false; + } + } + } + return true; + }), 6000L)); + } + + /** + * Verify that all shards for a given instance are in a specific state. + */ + private void verifyHelixPartitionStates(String instanceName, String state) throws Exception { + Assert.assertTrue(TestHelper.verify(() -> { + for (String resourceName : _gSetupTool.getClusterManagementTool().getResourcesInCluster(CLUSTER_NAME)) { + for (String shardId : _gSetupTool.getClusterManagementTool() + .getResourceIdealState(CLUSTER_NAME, resourceName) + .getPartitionSet()) { + if (!getHelixCurrentState(instanceName, resourceName, shardId).equals(state)) { + return false; + } + } + } + return true; + }, TestHelper.WAIT_DURATION)); + } + + @Test + public void testProcessStateTransitionMessageSuccess() throws Exception { + createDB(); + verifyPendingMessages(_participants); + + // Verify that all pending messages have the toState "ONLINE" + for (HelixGatewayParticipant participant : _participants) { + HelixGatewayServiceOuterClass.SingleShardChangeRequest request = getPendingMessage(participant.getInstanceName()).getRequest(0); + Assert.assertNotNull(request); + Assert.assertEquals(request.getTargetState(), "ONLINE"); + } + + // Process all pending messages successfully + for (HelixGatewayParticipant participant : _participants) { + processPendingMessage(participant, true, "ONLINE"); + } + + // Verify that the cluster converges and all states are "ONLINE" + Assert.assertTrue(_clusterVerifier.verify()); + verifyGatewayStateMatchesHelixState(); + verifyGatewayTargetStateMatchHelixTargetState(); + } + + @Test(dependsOnMethods = "testProcessStateTransitionMessageSuccess") + public void testProcessStateTransitionMessageFailure() throws Exception { + // Add a new participant and include it in the preference list + HelixGatewayParticipant participant = addParticipant(); + addToPreferenceList(participant); + verifyPendingMessages(List.of(participant)); + + // Verify the pending message has the toState "ONLINE" + HelixGatewayServiceOuterClass.SingleShardChangeRequest request = getPendingMessage(participant.getInstanceName()).getRequest(0); + Assert.assertNotNull(request); + Assert.assertEquals(request.getTargetState(), "ONLINE"); + + // Process the message with failure + processPendingMessage(participant, false, "ONLINE"); + + // Verify that the cluster converges and states reflect the failure (e.g., "OFFLINE") + Assert.assertTrue(_clusterVerifier.verify()); + verifyGatewayStateMatchesHelixState(); + + // Remove the participant from the preference list and delete it + removeFromPreferenceList(participant); + deleteParticipant(participant); + Assert.assertTrue(_clusterVerifier.verify()); + } + + @Test(dependsOnMethods = "testProcessStateTransitionMessageFailure") + public void testProcessStateTransitionAfterReconnect() throws Exception { + // Remove the first participant + HelixGatewayParticipant participant = _participants.get(0); + deleteParticipant(participant); + + // Verify the Helix state transitions to "UNASSIGNED_STATE" for the participant + verifyHelixPartitionStates(participant.getInstanceName(), HelixGatewayParticipant.UNASSIGNED_STATE); + + // Re-add the participant with its initial state + addParticipant(participant.getInstanceName(), createSingleShardStateMap(TEST_DB, "TestDB_0", + _gatewayServiceManager.getCurrentState(CLUSTER_NAME, participant.getInstanceName(), TEST_DB, + "TestDB_0"))); + Assert.assertTrue(_clusterVerifier.verify()); + + // Verify the Helix state is "ONLINE" + verifyHelixPartitionStates(participant.getInstanceName(), "ONLINE"); + } + + @Test(dependsOnMethods = "testProcessStateTransitionAfterReconnect") + public void testProcessStateTransitionAfterReconnectAfterDroppingPartition() throws Exception { + // Remove the first participant and verify state + HelixGatewayParticipant participant = _participants.get(0); + deleteParticipant(participant); + verifyHelixPartitionStates(participant.getInstanceName(), HelixGatewayParticipant.UNASSIGNED_STATE); + + // Remove shard preference and re-add the participant + removeFromPreferenceList(participant); + HelixGatewayParticipant participantReplacement = addParticipant(participant.getInstanceName(), + createSingleShardStateMap(TEST_DB, "TestDB_0", + _gatewayServiceManager.getCurrentState(CLUSTER_NAME, participant.getInstanceName(), TEST_DB, + "TestDB_0"))); + verifyPendingMessages(List.of(participantReplacement)); + + // Process the pending message successfully + processPendingMessage(participantReplacement, true, "DROPPED"); + + // Verify that the cluster converges and states are correctly updated to "ONLINE" + Assert.assertTrue(_clusterVerifier.verify()); + verifyGatewayStateMatchesHelixState(); + verifyGatewayTargetStateMatchHelixTargetState(); + } + + @Test(dependsOnMethods = "testProcessStateTransitionAfterReconnectAfterDroppingPartition") + public void testGatewayParticipantDisconnectGracefully() { + int gracefulDisconnectCount = MockHelixGatewayServiceChannel._gracefulDisconnectCount.get(); + // Remove the first participant + HelixGatewayParticipant participant = _participants.get(0); + deleteParticipant(participant); + + Assert.assertEquals(MockHelixGatewayServiceChannel._gracefulDisconnectCount.get(), gracefulDisconnectCount + 1); + } + + @Test(dependsOnMethods = "testGatewayParticipantDisconnectGracefully") + public void testGatewayParticipantDisconnectWithError() throws Exception { + int errorDisconnectCount = MockHelixGatewayServiceChannel._errorDisconnectCount.get(); + int onDisconnectCallbackCount = _onDisconnectCallbackCount.get(); + + // Call on disconnect with error for all participants + for (HelixGatewayParticipant participant : _participants) { + participant.onDisconnected(null, new Exception("Test error")); + } + + Assert.assertEquals(MockHelixGatewayServiceChannel._errorDisconnectCount.get(), + errorDisconnectCount + _participants.size()); + Assert.assertEquals(_onDisconnectCallbackCount.get(), onDisconnectCallbackCount + _participants.size()); + } + + public static class MockHelixGatewayServiceChannel implements HelixGatewayServiceChannel { + private final Map _pendingMessageMap; + private static final AtomicInteger _gracefulDisconnectCount = new AtomicInteger(); + private static final AtomicInteger _errorDisconnectCount = new AtomicInteger(); + + public MockHelixGatewayServiceChannel(Map pendingMessageMap) { + _pendingMessageMap = pendingMessageMap; + } + + @Override + public void sendStateChangeRequests(String instanceName, + HelixGatewayServiceOuterClass.ShardChangeRequests shardChangeRequests) { + _pendingMessageMap.put(instanceName, shardChangeRequests); + } + + @Override + public void start() throws IOException { + + } + + @Override + public void stop() { + + } + + @Override + public void closeConnectionWithError(String clusterName, String instanceName, String reason) { + _errorDisconnectCount.incrementAndGet(); + } + @Override + public void completeConnection(String clusterName, String instanceName) { + _gracefulDisconnectCount.incrementAndGet(); + } + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/service/TestGatewayServiceConnection.java b/helix-gateway/src/test/java/org/apache/helix/gateway/service/TestGatewayServiceConnection.java new file mode 100644 index 0000000000..8253444a89 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/service/TestGatewayServiceConnection.java @@ -0,0 +1,158 @@ +package org.apache.helix.gateway.service; + +/* + * 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 io.grpc.ManagedChannel; +import io.grpc.ManagedChannelBuilder; +import io.grpc.stub.StreamObserver; +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import org.apache.helix.gateway.api.service.HelixGatewayServiceChannel; +import org.apache.helix.gateway.base.HelixGatewayTestBase; +import org.apache.helix.gateway.channel.GatewayServiceChannelConfig; +import org.apache.helix.gateway.api.constant.GatewayServiceEventType; +import org.apache.helix.gateway.channel.HelixGatewayServiceChannelFactory; +import org.testng.Assert; +import org.testng.annotations.Test; +import proto.org.apache.helix.gateway.HelixGatewayServiceGrpc; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass; + + + +public class TestGatewayServiceConnection extends HelixGatewayTestBase { + CountDownLatch connectLatch = new CountDownLatch(1); + CountDownLatch disconnectLatch = new CountDownLatch(1); + + @Test + public void testLivenessDetection() throws IOException, InterruptedException { + // start the gateway service + GatewayServiceChannelConfig config = + new GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder().setGrpcServerPort(50051).build(); + GatewayServiceManager mng = new DummyGatewayServiceManager(config); + mng.startService(); + + // start the client + HelixGatewayClient client = new HelixGatewayClient("localhost", 50051); + client.connect(); + // assert we get connect event + Assert.assertTrue(connectLatch.await(5, TimeUnit.SECONDS)); + + // assert we have disconnect event + client.shutdownGracefully(); + Assert.assertTrue(disconnectLatch.await(5, TimeUnit.SECONDS)); + + connectLatch = new CountDownLatch(1); + disconnectLatch = new CountDownLatch(1); + + // start the client + HelixGatewayClient client2 = new HelixGatewayClient("localhost", 50051); + client2.connect(); + // assert we get connect event + Assert.assertTrue(connectLatch.await(5, TimeUnit.SECONDS)); + + // assert we have disconnect event when shutting down ungracefully + client2.shutdownByClosingConnection(); + Assert.assertTrue(disconnectLatch.await(5, TimeUnit.SECONDS)); + + mng.stopService(); + } + + public class HelixGatewayClient { + + private final ManagedChannel channel; + private final HelixGatewayServiceGrpc.HelixGatewayServiceStub asyncStub; + + private StreamObserver _requestObserver; + + public HelixGatewayClient(String host, int port) { + this.channel = ManagedChannelBuilder.forAddress(host, port) + .usePlaintext() + .keepAliveTime(30, TimeUnit.SECONDS) // KeepAlive time + .keepAliveTimeout(3, TimeUnit.MINUTES) // KeepAlive timeout + .keepAliveWithoutCalls(true) // Allow KeepAlive without active RPCs + .build(); + this.asyncStub = HelixGatewayServiceGrpc.newStub(channel); + } + + public void connect() { + _requestObserver = asyncStub.report(new StreamObserver() { + @Override + public void onNext(HelixGatewayServiceOuterClass.ShardChangeRequests value) { + // Handle response from server + } + + @Override + public void onError(Throwable t) { + // Handle error + } + + @Override + public void onCompleted() { + // Handle stream completion + } + }); + + // Send initial ShardStateMessage + HelixGatewayServiceOuterClass.ShardStateMessage initialMessage = + HelixGatewayServiceOuterClass.ShardStateMessage.newBuilder() + .setShardState(HelixGatewayServiceOuterClass.ShardState.newBuilder() + .setInstanceName("instance1") + .setClusterName("TEST_CLUSTER") + .build()) + .build(); + _requestObserver.onNext(initialMessage); + + // Add more logic to send additional messages if needed + } + + public void shutdownGracefully() { + // graceful shutdown + _requestObserver.onCompleted(); + channel.shutdown().shutdownNow(); + } + + public void shutdownByClosingConnection() { + //ungraceful shutdown + channel.shutdown().shutdownNow(); + } + } + + class DummyGatewayServiceManager extends GatewayServiceManager { + + public DummyGatewayServiceManager(GatewayServiceChannelConfig gatewayServiceChannelConfig) { + super("dummyZkAddress"); + this.setGatewayServiceChannel( + HelixGatewayServiceChannelFactory.createServiceChannel(gatewayServiceChannelConfig, + this)); + } + + @Override + public void onGatewayServiceEvent(GatewayServiceEvent event) { + if (event.getEventType().equals(GatewayServiceEventType.CONNECT)) { + connectLatch.countDown(); + } else if (event.getEventType().equals(GatewayServiceEventType.DISCONNECT)) { + disconnectLatch.countDown(); + } + System.out.println("Received event: " + event.getEventType()); + } + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/service/TestGatewayServiceManager.java b/helix-gateway/src/test/java/org/apache/helix/gateway/service/TestGatewayServiceManager.java new file mode 100644 index 0000000000..c8f441f096 --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/service/TestGatewayServiceManager.java @@ -0,0 +1,79 @@ +package org.apache.helix.gateway.service; + +import java.util.HashMap; +import java.util.Map; +import org.apache.helix.gateway.channel.GatewayServiceChannelConfig; +import org.apache.helix.gateway.channel.HelixGatewayServiceGrpcService; +import org.testng.annotations.Test; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass; + +import static org.mockito.ArgumentMatchers.*; +import static org.mockito.Mockito.*; +import static org.testng.Assert.*; + + +public class TestGatewayServiceManager { + + private GatewayServiceManager manager; + + @Test + public void testConnectionAndDisconnectionEvents() { + + manager = mock(GatewayServiceManager.class); + GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder builder = new GatewayServiceChannelConfig.GatewayServiceProcessorConfigBuilder().setGrpcServerPort(50051); + HelixGatewayServiceGrpcService grpcService = new HelixGatewayServiceGrpcService(manager,builder.build()); + // Mock a connection event + HelixGatewayServiceOuterClass.ShardStateMessage connectionEvent = + HelixGatewayServiceOuterClass.ShardStateMessage.newBuilder() + .setShardState(HelixGatewayServiceOuterClass.ShardState.newBuilder() + .setInstanceName("instance1") + .setClusterName("cluster1") + .build()) + .build(); + + // Mock a disconnection event + HelixGatewayServiceOuterClass.ShardStateMessage disconnectionEvent = + HelixGatewayServiceOuterClass.ShardStateMessage.newBuilder() + .setShardState(HelixGatewayServiceOuterClass.ShardState.newBuilder() + .setInstanceName("instance1") + .setClusterName("cluster1") + .build()) + .build(); + + // Process connection event + grpcService.report(null).onNext(connectionEvent); + + // Process disconnection event + grpcService.report(null).onNext(disconnectionEvent); + // Verify the events were processed in sequence + verify(manager, times(2)).onGatewayServiceEvent(any()); + + grpcService.stop(); + } + @Test + public void testGetAllTargetStates() { + GatewayServiceManager gatewayServiceManager = new GatewayServiceManager("localhost:2181"); + String clusterName = "TestCluster"; + String instanceName = "instance1"; + String resourceId = "resource1"; + String shardId = "shard1"; + String state = "ONLINE"; + + // Add target state + gatewayServiceManager.updateTargetState(clusterName, instanceName, resourceId, shardId, state); + + // Expected target states + Map>> expectedTargetStates = new HashMap<>(); + Map> instanceMap = new HashMap<>(); + Map shardMap = new HashMap<>(); + shardMap.put(shardId, state); + instanceMap.put(resourceId, shardMap); + expectedTargetStates.put(instanceName, instanceMap); + + // Get all target states + Map>> actualTargetStates = gatewayServiceManager.getAllTargetStates(clusterName); + + // Verify the target states + assertEquals(actualTargetStates, expectedTargetStates); + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/util/TestGatewayCurrentStateCache.java b/helix-gateway/src/test/java/org/apache/helix/gateway/util/TestGatewayCurrentStateCache.java new file mode 100644 index 0000000000..17aa25637b --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/util/TestGatewayCurrentStateCache.java @@ -0,0 +1,78 @@ +package org.apache.helix.gateway.util; + +/* + * 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.HashMap; +import java.util.Map; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class TestGatewayCurrentStateCache { + private GatewayCurrentStateCache cache; + + @BeforeMethod + public void setUp() { + cache = new GatewayCurrentStateCache("TestCluster"); + } + + @Test + public void testUpdateCacheWithNewCurrentStateAndGetDiff() { + Map>> newState = new HashMap<>(); + Map> instanceState = new HashMap<>(); + Map shardState = new HashMap<>(); + shardState.put("shard1", "ONLINE"); + instanceState.put("resource1", shardState); + newState.put("instance1", instanceState); + + Map>> diff = cache.updateCacheWithNewCurrentStateAndGetDiff(newState); + + Assert.assertNotNull(diff); + Assert.assertEquals(diff.size(), 1); + Assert.assertEquals(diff.get("instance1").get("resource1").get("shard1"), "ONLINE"); + } + + @Test + public void testUpdateCacheWithExistingStateAndGetDiff() { + // Initial state + Map>> initialState = new HashMap<>(); + Map> instanceState = new HashMap<>(); + Map shardState = new HashMap<>(); + shardState.put("shard1", "ONLINE"); + instanceState.put("resource1", shardState); + initialState.put("instance1", instanceState); + cache.updateCacheWithNewCurrentStateAndGetDiff(initialState); + + // New state with a change + Map>> newState = new HashMap<>(); + Map> newInstanceState = new HashMap<>(); + Map newShardState = new HashMap<>(); + newShardState.put("shard1", "OFFLINE"); + newInstanceState.put("resource1", newShardState); + newState.put("instance1", newInstanceState); + + Map>> diff = cache.updateCacheWithNewCurrentStateAndGetDiff(newState); + + Assert.assertNotNull(diff); + Assert.assertEquals(diff.size(), 1); + Assert.assertEquals(diff.get("instance1").get("resource1").get("shard1"), "OFFLINE"); + } +} diff --git a/helix-gateway/src/test/java/org/apache/helix/gateway/util/TestStateTransitionMessageTranslateUtil.java b/helix-gateway/src/test/java/org/apache/helix/gateway/util/TestStateTransitionMessageTranslateUtil.java new file mode 100644 index 0000000000..d65031a5fa --- /dev/null +++ b/helix-gateway/src/test/java/org/apache/helix/gateway/util/TestStateTransitionMessageTranslateUtil.java @@ -0,0 +1,69 @@ +package org.apache.helix.gateway.util; + +/* + * 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 org.apache.helix.HelixDefinedState; + +import org.apache.helix.gateway.participant.HelixGatewayParticipant; +import org.testng.Assert; +import org.testng.annotations.Test; +import proto.org.apache.helix.gateway.HelixGatewayServiceOuterClass; + +public class TestStateTransitionMessageTranslateUtil { + + @Test + public void testTranslateStatesToTransitionType_DeleteShard() { + String currentState = "ONLINE"; + String toState = HelixDefinedState.DROPPED.name(); + + HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType result = + StateTransitionMessageTranslateUtil.translateStatesToTransitionType(currentState, toState); + + Assert.assertEquals(result, + HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType.DELETE_SHARD, + "Expected DELETE_SHARD when transitioning to DROPPED state from a non-DROPPED state."); + } + + @Test + public void testTranslateStatesToTransitionType_AddShard() { + String currentState = HelixGatewayParticipant.UNASSIGNED_STATE; + String toState = "ONLINE"; + + HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType result = + StateTransitionMessageTranslateUtil.translateStatesToTransitionType(currentState, toState); + + Assert.assertEquals(result, + HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType.ADD_SHARD, + "Expected ADD_SHARD when transitioning from DROPPED state to a non-DROPPED state."); + } + + @Test + public void testTranslateStatesToTransitionType_ChangeRole() { + String currentState = "ONLINE"; + String toState = "OFFLINE"; + + HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType result = + StateTransitionMessageTranslateUtil.translateStatesToTransitionType(currentState, toState); + + Assert.assertEquals(result, + HelixGatewayServiceOuterClass.SingleShardChangeRequest.StateChangeRequestType.CHANGE_ROLE, + "Expected CHANGE_ROLE when transitioning between non-DROPPED states."); + } +} diff --git a/helix-gateway/src/test/resources/log4j2.properties b/helix-gateway/src/test/resources/log4j2.properties new file mode 100644 index 0000000000..5aa8df88ad --- /dev/null +++ b/helix-gateway/src/test/resources/log4j2.properties @@ -0,0 +1,63 @@ +# +# 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. +# + +# See https://logging.apache.org/log4j/2.0/manual/configuration.html#Properties +rootLogger.level = error +rootLogger.appenderRef.stdout.ref = STDOUT + +appender.console.name = STDOUT +appender.console.type = Console +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n + +appender.rolling.name = R +appender.rolling.type = RollingFile +appender.rolling.layout.type = PatternLayout +appender.rolling.layout.pattern = %5p [%C:%M] (%F:%L) - %m%n +appender.rolling.fileName = target/ClusterManagerLogs/log.txt +appender.rolling.filePattern = target/ClusterManagerLogs/log.%d{dd-MMM-hh}.txt.gz +appender.rolling.policies.type = Policies +appender.rolling.policies.startup.type = OnStartupTriggeringPolicy + +# Via https://logging.apache.org/log4j/2.x/manual/migration.html +# 'The Log4j 1.x SimpleLayout can be emulated with PatternLayout "%level - %m%n"' +appender.statusdump.name = STATUSDUMP +appender.statusdump.type = RollingFile +appender.statusdump.layout.type = PatternLayout +appender.statusdump.layout.pattern = "%level - %m%n" +appender.statusdump.fileName = target/ClusterManagerLogs/statusUpdates.log +appender.statusdump.filePattern = target/ClusterManagerLogs/statusUpdates.%d{dd-MMM-hh}.log.gz +appender.statusdump.policies.type = Policies +appender.statusdump.policies.startup.type = OnStartupTriggeringPolicy + +logger.i0itec.name = org.I0Itec +logger.i0itec.level = error + +logger.apache.name = org.apache +logger.apache.level = error + +logger.noelios.name = com.noelios +logger.noelios.level = error + +logger.restlet.name = org.restlet +logger.restlet.level = error + +logger.helixzkdatadump.name = org.apache.helix.monitoring.ZKPathDataDumpTask +logger.helixzkdatadump.level = error +logger.helixzkdatadump.appenderRef.statusdump.ref = STATUSDUMP diff --git a/pom.xml b/pom.xml index dd09e90057..c31af010c9 100644 --- a/pom.xml +++ b/pom.xml @@ -318,6 +318,7 @@ recipes helix-view-aggregator meta-client + helix-gateway