diff --git a/.gitignore b/.gitignore index 9097189e44..2eb84c13ef 100644 --- a/.gitignore +++ b/.gitignore @@ -1,3 +1,21 @@ +# +# 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. +# cscope.* .classpath .project diff --git a/deploySite.sh b/deploySite.sh old mode 100644 new mode 100755 index 8abe7906ac..47d685f93c --- a/deploySite.sh +++ b/deploySite.sh @@ -21,4 +21,4 @@ read -s -p "Enter Apache Username: " myusername echo "" read -s -p "Enter Apache Password: " mypassword -mvn clean site-deploy scm-publish:publish-scm -Dusername=$myusername -Dpassword=$mypassword -DskipTests $@ +mvn clean site-deploy scm-publish:publish-scm -Dusername="$myusername" -Dpassword="$mypassword" -DskipTests $@ diff --git a/helix-admin-webapp/pom.xml b/helix-admin-webapp/pom.xml index 544a51337b..b4f38b5fde 100644 --- a/helix-admin-webapp/pom.xml +++ b/helix-admin-webapp/pom.xml @@ -21,7 +21,7 @@ under the License. org.apache.helix helix - 0.6.2-incubating-SNAPSHOT + 0.7.1-incubating-SNAPSHOT 4.0.0 @@ -47,13 +47,8 @@ under the License. helix-core - com.noelios.restlet - com.noelios.restlet - - - com.thoughtworks.xstream - xstream - 1.3.1 + org.restlet.jse + org.restlet org.codehaus.jackson diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/RestAdminApplication.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/RestAdminApplication.java index ce951cd93f..0940c39c24 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/RestAdminApplication.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/RestAdminApplication.java @@ -19,6 +19,8 @@ * under the License. */ +import java.util.logging.Level; + import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.GnuParser; @@ -52,13 +54,13 @@ import org.apache.helix.webapp.resources.ZkPathResource; import org.restlet.Application; import org.restlet.Context; +import org.restlet.Request; +import org.restlet.Response; import org.restlet.Restlet; -import org.restlet.Router; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.StringRepresentation; -import org.restlet.util.Template; +import org.restlet.representation.StringRepresentation; +import org.restlet.routing.Router; +import org.restlet.routing.Template; public class RestAdminApplication extends Application { public static final String HELP = "help"; @@ -67,6 +69,10 @@ public class RestAdminApplication extends Application { public static final String ZKCLIENT = "zkClient"; public static final int DEFAULT_PORT = 8100; + static { + org.restlet.engine.Engine.setLogLevel(Level.SEVERE); + } + public RestAdminApplication() { super(); } @@ -76,7 +82,7 @@ public RestAdminApplication(Context context) { } @Override - public Restlet createRoot() { + public Restlet createInboundRoot() { Router router = new Router(getContext()); router.setDefaultMatchingMode(Template.MODE_EQUALS); router.attach("/clusters", ClustersResource.class); diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterRepresentationUtil.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterRepresentationUtil.java index ccac49a92e..f227801a87 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterRepresentationUtil.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterRepresentationUtil.java @@ -31,9 +31,9 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixProperty; import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.PropertyType; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkBaseDataAccessor; diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterResource.java index 91b991d7ae..86459eb2ae 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterResource.java @@ -20,57 +20,39 @@ */ import java.io.IOException; +import java.util.HashSet; import java.util.List; +import java.util.Set; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixException; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZkClient; +import org.apache.helix.model.Leader; import org.apache.helix.model.LiveInstance; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.webapp.RestAdminApplication; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; +import org.restlet.data.Method; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; - -public class ClusterResource extends Resource { - public ClusterResource(Context context, Request request, Response response) { - super(context, request, response); +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; + +public class ClusterResource extends ServerResource { + + public ClusterResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return true; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); @@ -80,7 +62,6 @@ public Representation represent(Variant variant) { catch (Exception e) { String error = ClusterRepresentationUtil.getErrorAsJsonStringFromException(e); presentation = new StringRepresentation(error, MediaType.APPLICATION_JSON); - e.printStackTrace(); } return presentation; @@ -107,7 +88,7 @@ StringRepresentation getClusterRepresentation(String clusterName) throws JsonGen ClusterRepresentationUtil.getClusterDataAccessor(zkClient, clusterName); Builder keyBuilder = accessor.keyBuilder(); - LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); if (leader != null) { clusterSummayRecord.setSimpleField("LEADER", leader.getInstanceName()); } else { @@ -121,7 +102,7 @@ StringRepresentation getClusterRepresentation(String clusterName) throws JsonGen } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); ZkClient zkClient = @@ -160,10 +141,11 @@ public void acceptRepresentation(Representation entity) { MediaType.APPLICATION_JSON); getResponse().setStatus(Status.SUCCESS_OK); } + return getResponseEntity(); } @Override - public void removeRepresentations() { + public Representation delete() { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); ZkClient zkClient = @@ -176,5 +158,6 @@ public void removeRepresentations() { MediaType.APPLICATION_JSON); getResponse().setStatus(Status.SUCCESS_OK); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClustersResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClustersResource.java index 84678abf9a..4b1205486a 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClustersResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClustersResource.java @@ -30,43 +30,25 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ClustersResource extends Resource { +public class ClustersResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ClustersResource.class); - public ClustersResource(Context context, Request request, Response response) { - super(context, request, response); + public ClustersResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); // handle(request,response); } @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return true; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { presentation = getClustersRepresentation(); @@ -96,7 +78,7 @@ StringRepresentation getClustersRepresentation() throws JsonGenerationException, } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { JsonParameters jsonParameters = new JsonParameters(entity); String command = jsonParameters.getCommand(); @@ -124,10 +106,11 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("Error in posting " + entity, e); } + return null; } @Override - public void removeRepresentations() { - + public Representation delete() { + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConfigResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConfigResource.java index c19ae9dca5..3c384d46ec 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConfigResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConfigResource.java @@ -20,7 +20,6 @@ */ import java.util.Arrays; -import java.util.HashSet; import java.util.List; import java.util.Map; @@ -28,32 +27,29 @@ import org.apache.helix.HelixException; import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZkClient; -import org.apache.helix.model.ConfigScope; import org.apache.helix.model.HelixConfigScope; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; -import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.helix.model.builder.HelixConfigScopeBuilder; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.webapp.RestAdminApplication; import org.apache.log4j.Logger; import org.restlet.Context; +import org.restlet.Request; +import org.restlet.Response; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ConfigResource extends Resource { +public class ConfigResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ConfigResource.class); - public ConfigResource(Context context, Request request, Response response) { - super(context, request, response); + public ConfigResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); - setModifiable(true); + setNegotiated(false); } String getValue(String key) { @@ -122,7 +118,7 @@ StringRepresentation getConfigs(// ConfigScope scope, } @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation representation = null; String clusterName = getValue("clusterName"); @@ -221,12 +217,13 @@ void setConfigs(Representation entity, ConfigScopeProperty type, String scopeArg } - getResponse().setEntity(represent()); + getResponse().setEntity(get()); getResponse().setStatus(Status.SUCCESS_OK); } - @Override - public void acceptRepresentation(Representation entity) { + +@Override + public Representation post(Representation entity) { String clusterName = getValue("clusterName"); String scopeStr = getValue("scope").toUpperCase(); @@ -268,5 +265,6 @@ public void acceptRepresentation(Representation entity) { MediaType.APPLICATION_JSON); getResponse().setStatus(Status.SUCCESS_OK); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConstraintResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConstraintResource.java index 4cb4a4815f..675d0ec915 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConstraintResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConstraintResource.java @@ -30,24 +30,23 @@ import org.apache.helix.webapp.RestAdminApplication; import org.apache.log4j.Logger; import org.restlet.Context; +import org.restlet.Request; +import org.restlet.Response; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ConstraintResource extends Resource { +public class ConstraintResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ConstraintResource.class); - public ConstraintResource(Context context, Request request, Response response) { - super(context, request, response); + public ConstraintResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); - setModifiable(true); + setNegotiated(false); } // TODO move to a util function @@ -56,7 +55,7 @@ String getValue(String key) { } @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation representation = null; String clusterName = getValue("clusterName"); String constraintTypeStr = getValue("constraintType").toUpperCase(); @@ -104,7 +103,7 @@ public Representation represent(Variant variant) { } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { String clusterName = getValue("clusterName"); String constraintTypeStr = getValue("constraintType").toUpperCase(); String constraintId = getValue("constraintId"); @@ -124,10 +123,11 @@ public void acceptRepresentation(Representation entity) { MediaType.APPLICATION_JSON); getResponse().setStatus(Status.SUCCESS_OK); } + return null; } @Override - public void removeRepresentations() { + public Representation delete() { String clusterName = getValue("clusterName"); String constraintTypeStr = getValue("constraintType").toUpperCase(); String constraintId = getValue("constraintId"); @@ -145,6 +145,6 @@ public void removeRepresentations() { MediaType.APPLICATION_JSON); getResponse().setStatus(Status.SUCCESS_OK); } - + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerResource.java index a6c7c29a6e..8e81a08ad6 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerResource.java @@ -34,6 +34,7 @@ import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.manager.zk.ZkClient; +import org.apache.helix.model.Leader; import org.apache.helix.model.LiveInstance; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.util.StatusUpdateUtil.Level; @@ -41,43 +42,22 @@ import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; import org.restlet.Context; +import org.restlet.Request; +import org.restlet.Response; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ControllerResource extends Resource { - - public ControllerResource(Context context, Request request, Response response) +public class ControllerResource extends ServerResource { + public ControllerResource() { - super(context, request, response); getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); - } - - @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; + setNegotiated(false); } StringRepresentation getControllerRepresentation(String clusterName) @@ -89,7 +69,7 @@ StringRepresentation getControllerRepresentation(String clusterName) new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); ZNRecord record = null; - LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); if (leader != null) { record = leader.getRecord(); } else { @@ -112,7 +92,7 @@ record = new ZNRecord(""); } @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); @@ -126,7 +106,7 @@ public Representation represent(Variant variant) { } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); ZkClient zkClient = @@ -156,5 +136,7 @@ public void acceptRepresentation(Representation entity) { MediaType.APPLICATION_JSON); getResponse().setStatus(Status.SUCCESS_OK); } + + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerStatusUpdateResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerStatusUpdateResource.java index d95e581540..be96fad69b 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerStatusUpdateResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerStatusUpdateResource.java @@ -28,46 +28,23 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ControllerStatusUpdateResource extends Resource { +public class ControllerStatusUpdateResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ControllerStatusUpdateResource.class); - public ControllerStatusUpdateResource(Context context, Request request, Response response) { - super(context, request, response); + public ControllerStatusUpdateResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return false; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String zkServer = diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/CurrentStateResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/CurrentStateResource.java index 797b6631a9..6b63cb5955 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/CurrentStateResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/CurrentStateResource.java @@ -28,46 +28,23 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class CurrentStateResource extends Resource { +public class CurrentStateResource extends ServerResource { private final static Logger LOG = Logger.getLogger(CurrentStateResource.class); - public CurrentStateResource(Context context, Request request, Response response) { - super(context, request, response); + public CurrentStateResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return false; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/CurrentStatesResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/CurrentStatesResource.java index 3b50e0c85a..84c3aaa749 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/CurrentStatesResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/CurrentStatesResource.java @@ -27,41 +27,22 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class CurrentStatesResource extends Resource { +public class CurrentStatesResource extends ServerResource { private final static Logger LOG = Logger.getLogger(CurrentStatesResource.class); - public CurrentStatesResource(Context context, Request request, Response response) { - super(context, request, response); + public CurrentStatesResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } - public boolean allowGet() { - return true; - } - - public boolean allowPost() { - return false; - } - - public boolean allowPut() { - return false; - } - - public boolean allowDelete() { - return false; - } - - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ErrorResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ErrorResource.java index f680e08cf2..9ff7f7582c 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ErrorResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ErrorResource.java @@ -28,46 +28,23 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ErrorResource extends Resource { +public class ErrorResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ErrorResource.class); - public ErrorResource(Context context, Request request, Response response) { - super(context, request, response); + public ErrorResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return false; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ErrorsResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ErrorsResource.java index 06cf64f94d..46b5aa1b91 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ErrorsResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ErrorsResource.java @@ -27,41 +27,22 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ErrorsResource extends Resource { +public class ErrorsResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ErrorsResource.class); - public ErrorsResource(Context context, Request request, Response response) { - super(context, request, response); + public ErrorsResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } - public boolean allowGet() { - return true; - } - - public boolean allowPost() { - return false; - } - - public boolean allowPut() { - return false; - } - - public boolean allowDelete() { - return false; - } - - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); @@ -73,6 +54,7 @@ public Representation represent(Variant variant) { LOG.error("", e); } + return presentation; } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ExternalViewResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ExternalViewResource.java index 68587d9f78..6ec28dc8bf 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ExternalViewResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ExternalViewResource.java @@ -28,46 +28,23 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ExternalViewResource extends Resource { +public class ExternalViewResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ExternalViewResource.class); - public ExternalViewResource(Context context, Request request, Response response) { - super(context, request, response); + public ExternalViewResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return false; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/IdealStateResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/IdealStateResource.java index 87e6cd7478..49df073d92 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/IdealStateResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/IdealStateResource.java @@ -25,8 +25,8 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixException; import org.apache.helix.PropertyKey; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.model.IdealState; import org.apache.helix.tools.ClusterSetup; @@ -34,47 +34,24 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class IdealStateResource extends Resource { +public class IdealStateResource extends ServerResource { private final static Logger LOG = Logger.getLogger(IdealStateResource.class); - public IdealStateResource(Context context, Request request, Response response) { - super(context, request, response); + public IdealStateResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); @@ -107,7 +84,7 @@ StringRepresentation getIdealStateRepresentation(String clusterName, String reso } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); String resourceName = (String) getRequest().getAttributes().get("resourceName"); @@ -155,5 +132,6 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("Error in posting " + entity, e); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/InstanceResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/InstanceResource.java index 8439cfb8b3..e62fe5e2ef 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/InstanceResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/InstanceResource.java @@ -31,47 +31,24 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class InstanceResource extends Resource { +public class InstanceResource extends ServerResource { private final static Logger LOG = Logger.getLogger(InstanceResource.class); - public InstanceResource(Context context, Request request, Response response) { - super(context, request, response); + public InstanceResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return true; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { presentation = getInstanceRepresentation(); @@ -102,7 +79,7 @@ StringRepresentation getInstanceRepresentation() throws JsonGenerationException, } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); String instanceName = (String) getRequest().getAttributes().get("instanceName"); @@ -179,10 +156,11 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("", e); } + return null; } @Override - public void removeRepresentations() { + public Representation delete() { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); String instanceName = (String) getRequest().getAttributes().get("instanceName"); @@ -198,5 +176,6 @@ public void removeRepresentations() { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("Error in remove", e); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/InstancesResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/InstancesResource.java index 784a3e9d41..0e98f4462a 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/InstancesResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/InstancesResource.java @@ -36,46 +36,27 @@ import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; import org.restlet.Context; +import org.restlet.Request; +import org.restlet.Response; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class InstancesResource extends Resource { +public class InstancesResource extends ServerResource { private final static Logger LOG = Logger.getLogger(InstancesResource.class); - public InstancesResource(Context context, Request request, Response response) { - super(context, request, response); + public InstancesResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } - @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); @@ -128,7 +109,7 @@ StringRepresentation getInstancesRepresentation(String clusterName) } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); JsonParameters jsonParameters = new JsonParameters(entity); @@ -173,5 +154,6 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("", e); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JsonParameters.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JsonParameters.java index c8e8fc6702..5f405d8459 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JsonParameters.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JsonParameters.java @@ -33,7 +33,7 @@ import org.apache.helix.tools.ClusterSetup; import org.codehaus.jackson.map.ObjectMapper; import org.restlet.data.Form; -import org.restlet.resource.Representation; +import org.restlet.representation.Representation; public class JsonParameters { // json parameter key diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ResourceGroupResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ResourceGroupResource.java index 365b21282b..055f64a3d3 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ResourceGroupResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ResourceGroupResource.java @@ -31,47 +31,24 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ResourceGroupResource extends Resource { +public class ResourceGroupResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ResourceGroupResource.class); - public ResourceGroupResource(Context context, Request request, Response response) { - super(context, request, response); + public ResourceGroupResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return true; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); @@ -104,7 +81,7 @@ StringRepresentation getIdealStateRepresentation(String clusterName, String reso } @Override - public void removeRepresentations() { + public Representation delete() { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); String resourceGroupName = (String) getRequest().getAttributes().get("resourceName"); @@ -120,10 +97,11 @@ public void removeRepresentations() { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("", e); } + return null; } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); String resourceName = (String) getRequest().getAttributes().get("resourceName"); @@ -146,6 +124,7 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("", e); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ResourceGroupsResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ResourceGroupsResource.java index f137c07235..01ea37a5d3 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ResourceGroupsResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ResourceGroupsResource.java @@ -31,47 +31,24 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ResourceGroupsResource extends Resource { +public class ResourceGroupsResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ResourceGroupsResource.class); - public ResourceGroupsResource(Context context, Request request, Response response) { - super(context, request, response); + public ResourceGroupsResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); @@ -106,7 +83,7 @@ StringRepresentation getHostedEntitiesRepresentation(String clusterName) } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); @@ -168,5 +145,6 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("Error in posting " + entity, e); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/SchedulerTasksResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/SchedulerTasksResource.java index 59d9174ad8..b48d0cb241 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/SchedulerTasksResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/SchedulerTasksResource.java @@ -31,8 +31,11 @@ import org.apache.helix.InstanceType; import org.apache.helix.PropertyPathConfig; import org.apache.helix.PropertyType; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory; import org.apache.helix.manager.zk.ZkClient; +import org.apache.helix.model.Leader; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageType; @@ -41,55 +44,32 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.Form; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; /** * This resource can be used to send scheduler tasks to the controller. */ -public class SchedulerTasksResource extends Resource { +public class SchedulerTasksResource extends ServerResource { private final static Logger LOG = Logger.getLogger(SchedulerTasksResource.class); public static String CRITERIA = "Criteria"; public static String MESSAGETEMPLATE = "MessageTemplate"; public static String TASKQUEUENAME = "TaskQueueName"; - public SchedulerTasksResource(Context context, Request request, Response response) { - super(context, request, response); + public SchedulerTasksResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { presentation = getSchedulerTasksRepresentation(); @@ -117,7 +97,6 @@ StringRepresentation getSchedulerTasksRepresentation() throws JsonGenerationExce ClusterRepresentationUtil.getClusterDataAccessor(zkClient, clusterName); LiveInstance liveInstance = accessor.getProperty(accessor.keyBuilder().liveInstance(instanceName)); - String sessionId = liveInstance.getSessionId(); StringRepresentation representation = new StringRepresentation("");// (ClusterRepresentationUtil.ObjectToJson(instanceConfigs), // MediaType.APPLICATION_JSON); @@ -126,7 +105,7 @@ StringRepresentation getSchedulerTasksRepresentation() throws JsonGenerationExce } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); Form form = new Form(entity); @@ -147,18 +126,18 @@ public void acceptRepresentation(Representation entity) { } HelixDataAccessor accessor = ClusterRepresentationUtil.getClusterDataAccessor(zkClient, clusterName); - LiveInstance leader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); + Leader leader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); if (leader == null) { throw new HelixException("There is no leader for the cluster " + clusterName); } Message schedulerMessage = - new Message(MessageType.SCHEDULER_MSG, UUID.randomUUID().toString()); + new Message(MessageType.SCHEDULER_MSG, MessageId.from(UUID.randomUUID().toString())); schedulerMessage.getRecord().getSimpleFields().put(CRITERIA, criteriaString); schedulerMessage.getRecord().getMapFields().put(MESSAGETEMPLATE, messageTemplate); - schedulerMessage.setTgtSessionId(leader.getSessionId()); + schedulerMessage.setTgtSessionId(SessionId.from(leader.getTypedSessionId().stringify())); schedulerMessage.setTgtName("CONTROLLER"); schedulerMessage.setSrcInstanceType(InstanceType.CONTROLLER); String taskQueueName = @@ -167,22 +146,23 @@ public void acceptRepresentation(Representation entity) { schedulerMessage.getRecord().setSimpleField( DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, taskQueueName); } - accessor.setProperty(accessor.keyBuilder().controllerMessage(schedulerMessage.getMsgId()), + accessor.setProperty( + accessor.keyBuilder().controllerMessage(schedulerMessage.getMessageId().stringify()), schedulerMessage); Map resultMap = new HashMap(); resultMap.put("StatusUpdatePath", PropertyPathConfig.getPath( PropertyType.STATUSUPDATES_CONTROLLER, clusterName, MessageType.SCHEDULER_MSG.toString(), - schedulerMessage.getMsgId())); + schedulerMessage.getMessageId().stringify())); resultMap.put("MessageType", Message.MessageType.SCHEDULER_MSG.toString()); - resultMap.put("MsgId", schedulerMessage.getMsgId()); + resultMap.put("MsgId", schedulerMessage.getMessageId().stringify()); // Assemble the rest URL for task status update String ipAddress = InetAddress.getLocalHost().getCanonicalHostName(); String url = "http://" + ipAddress + ":" + getContext().getAttributes().get(RestAdminApplication.PORT) + "/clusters/" + clusterName + "/Controller/statusUpdates/SCHEDULER_MSG/" - + schedulerMessage.getMsgId(); + + schedulerMessage.getMessageId(); resultMap.put("statusUpdateUrl", url); getResponse().setEntity(ClusterRepresentationUtil.ObjectToJson(resultMap), @@ -194,5 +174,6 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("", e); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StateModelResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StateModelResource.java index 9abded8eb6..d8d2ef02ad 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StateModelResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StateModelResource.java @@ -24,8 +24,8 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixException; import org.apache.helix.PropertyKey; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.model.StateModelDefinition; import org.apache.helix.tools.ClusterSetup; @@ -33,47 +33,24 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class StateModelResource extends Resource { +public class StateModelResource extends ServerResource { private final static Logger LOG = Logger.getLogger(StateModelResource.class); - public StateModelResource(Context context, Request request, Response response) { - super(context, request, response); + public StateModelResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); @@ -106,7 +83,7 @@ StringRepresentation getStateModelRepresentation(String clusterName, String mode } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); String modelName = (String) getRequest().getAttributes().get("modelName"); @@ -136,5 +113,6 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("Error in posting " + entity, e); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StateModelsResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StateModelsResource.java index 28e5685bf2..2e744b954c 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StateModelsResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StateModelsResource.java @@ -32,47 +32,24 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class StateModelsResource extends Resource { +public class StateModelsResource extends ServerResource { private final static Logger LOG = Logger.getLogger(StateModelsResource.class); - public StateModelsResource(Context context, Request request, Response response) { - super(context, request, response); + public StateModelsResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { presentation = getStateModelsRepresentation(); @@ -84,6 +61,7 @@ public Representation represent(Variant variant) { LOG.error("", e); } + return presentation; } @@ -106,7 +84,7 @@ StringRepresentation getStateModelsRepresentation() throws JsonGenerationExcepti } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); ZkClient zkClient = @@ -137,5 +115,6 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("Error in posting " + entity, e); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StatusUpdateResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StatusUpdateResource.java index 852fcfa6ca..8e9f689f45 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StatusUpdateResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StatusUpdateResource.java @@ -28,46 +28,23 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class StatusUpdateResource extends Resource { +public class StatusUpdateResource extends ServerResource { private final static Logger LOG = Logger.getLogger(StatusUpdateResource.class); - public StatusUpdateResource(Context context, Request request, Response response) { - super(context, request, response); + public StatusUpdateResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return false; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StatusUpdatesResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StatusUpdatesResource.java index 22f14d75e2..fa311881c3 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StatusUpdatesResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/StatusUpdatesResource.java @@ -27,46 +27,23 @@ import org.apache.log4j.Logger; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class StatusUpdatesResource extends Resource { +public class StatusUpdatesResource extends ServerResource { private final static Logger LOG = Logger.getLogger(StatusUpdatesResource.class); - public StatusUpdatesResource(Context context, Request request, Response response) { - super(context, request, response); + public StatusUpdatesResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return false; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; try { String clusterName = (String) getRequest().getAttributes().get("clusterName"); diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ZkChildResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ZkChildResource.java index 7c99a24039..6d50ade2fd 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ZkChildResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ZkChildResource.java @@ -26,43 +26,20 @@ import org.apache.helix.webapp.RestAdminApplication; import org.apache.log4j.Logger; import org.apache.zookeeper.data.Stat; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ZkChildResource extends Resource { +public class ZkChildResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ZkChildResource.class); - public ZkChildResource(Context context, Request request, Response response) { - super(context, request, response); + public ZkChildResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); - } - - @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return false; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return true; + setNegotiated(false); } private String getZKPath() { @@ -80,7 +57,7 @@ private String getZKPath() { } @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; String zkPath = getZKPath(); @@ -126,7 +103,7 @@ private ZNRecord readZkChild(String zkPath, ZkClient zkClient) { } @Override - public void removeRepresentations() { + public Representation delete() { String zkPath = getZKPath(); try { ZkClient zkClient = @@ -147,5 +124,6 @@ public void removeRepresentations() { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("Error in delete zkChild: " + zkPath, e); } + return null; } } diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ZkPathResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ZkPathResource.java index 8f03e12ce4..0ab29bbaf6 100644 --- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ZkPathResource.java +++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ZkPathResource.java @@ -24,49 +24,24 @@ import org.apache.helix.HelixException; import org.apache.helix.ZNRecord; -import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.manager.zk.ZkClient; -import org.apache.helix.tools.ClusterSetup; import org.apache.helix.webapp.RestAdminApplication; import org.apache.log4j.Logger; import org.apache.zookeeper.data.Stat; -import org.restlet.Context; import org.restlet.data.MediaType; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; -import org.restlet.resource.StringRepresentation; -import org.restlet.resource.Variant; +import org.restlet.representation.Representation; +import org.restlet.representation.StringRepresentation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; -public class ZkPathResource extends Resource { +public class ZkPathResource extends ServerResource { private final static Logger LOG = Logger.getLogger(ZkPathResource.class); - public ZkPathResource(Context context, Request request, Response response) { - super(context, request, response); + public ZkPathResource() { getVariants().add(new Variant(MediaType.TEXT_PLAIN)); getVariants().add(new Variant(MediaType.APPLICATION_JSON)); - } - - @Override - public boolean allowGet() { - return true; - } - - @Override - public boolean allowPost() { - return true; - } - - @Override - public boolean allowPut() { - return false; - } - - @Override - public boolean allowDelete() { - return true; + setNegotiated(false); } private String getZKPath() { @@ -84,7 +59,7 @@ private String getZKPath() { } @Override - public void acceptRepresentation(Representation entity) { + public Representation post(Representation entity) { String zkPath = getZKPath(); try { @@ -114,10 +89,11 @@ public void acceptRepresentation(Representation entity) { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("Error in post zkPath: " + zkPath, e); } + return null; } @Override - public Representation represent(Variant variant) { + public Representation get() { StringRepresentation presentation = null; String zkPath = getZKPath(); @@ -166,7 +142,7 @@ private ZNRecord readZkDataStatAndChild(String zkPath, ZkClient zkClient) { } @Override - public void removeRepresentations() { + public Representation delete() { String zkPath = getZKPath(); try { ZkClient zkClient = @@ -180,6 +156,7 @@ public void removeRepresentations() { getResponse().setStatus(Status.SUCCESS_OK); LOG.error("Error in delete zkPath: " + zkPath, e); } + return null; } } diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestHelixAdminScenariosRest.java b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestHelixAdminScenariosRest.java index 4730e76c1d..e4307d36de 100644 --- a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestHelixAdminScenariosRest.java +++ b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestHelixAdminScenariosRest.java @@ -19,30 +19,24 @@ * under the License. */ -/* - * Simulate all the admin tasks needed by using command line tool - * - * */ import java.io.FileWriter; import java.io.IOException; import java.io.PrintWriter; import java.io.StringReader; import java.io.StringWriter; import java.util.HashMap; -import java.util.List; import java.util.Map; import org.apache.helix.HelixDataAccessor; -import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.ClusterDistributedController; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKUtil; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState.IdealStateProperty; +import org.apache.helix.model.Leader; import org.apache.helix.model.LiveInstance; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier; import org.apache.helix.webapp.RestAdminApplication; @@ -55,19 +49,21 @@ import org.codehaus.jackson.map.SerializationConfig; import org.restlet.Client; import org.restlet.Component; +import org.restlet.Request; +import org.restlet.Response; import org.restlet.data.MediaType; import org.restlet.data.Method; import org.restlet.data.Protocol; import org.restlet.data.Reference; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; -import org.restlet.resource.Representation; +import org.restlet.representation.Representation; import org.testng.Assert; import org.testng.annotations.Test; +/** + * Simulate all the admin tasks needed by using command line tool + */ public class TestHelixAdminScenariosRest extends AdminTestBase { - Map _startCMResultMap = new HashMap(); RestAdminApplication _adminApp; Component _component; String _tag1 = "tag1123"; @@ -92,55 +88,6 @@ public static T JsonToObject(Class clazz, String jsonStrin return mapper.readValue(sr, clazz); } - @Test - public void testAddDeleteClusterAndInstanceAndResource() throws Exception { - // Helix bug helix-102 - // ZKPropertyTransferServer.PERIOD = 500; - // ZkPropertyTransferClient.SEND_PERIOD = 500; - // ZKPropertyTransferServer.getInstance().init(19999, ZK_ADDR); - - /** ======================= Add clusters ============================== */ - - testAddCluster(); - - /** ================= Add / drop some resources =========================== */ - - testAddResource(); - - /** ====================== Add / delete instances =========================== */ - - testAddInstance(); - - /** ===================== Rebalance resource =========================== */ - - testRebalanceResource(); - - /** ==================== start the clusters ============================= */ - - testStartCluster(); - - /** ==================== drop add resource in live clusters =================== */ - testDropAddResource(); - - /** ======================Operations with live node ============================ */ - - testInstanceOperations(); - - /** ======================Operations with partitions ============================ */ - - testEnablePartitions(); - - /** ============================ expand cluster =========================== */ - - testExpandCluster(); - - /** ============================ deactivate cluster =========================== */ - testDeactivateCluster(); - - // wait all zk callbacks done - Thread.sleep(1000); - } - static String assertSuccessPostOperation(String url, Map jsonParameters, boolean hasException) throws IOException { Reference resourceRef = new Reference(url); @@ -228,41 +175,46 @@ void assertClusterSetupException(String command) { Assert.assertTrue(exceptionThrown); } + private Map addClusterCmd(String clusterName) { + Map parameters = new HashMap(); + parameters.put(JsonParameters.CLUSTER_NAME, clusterName); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addCluster); + + return parameters; + } + + private void addCluster(String clusterName) throws IOException { + String url = "http://localhost:" + ADMIN_PORT + "/clusters"; + String response = assertSuccessPostOperation(url, addClusterCmd(clusterName), false); + Assert.assertTrue(response.contains(clusterName)); + } + + @Test public void testAddCluster() throws Exception { String url = "http://localhost:" + ADMIN_PORT + "/clusters"; - Map paraMap = new HashMap(); // Normal add - paraMap.put(JsonParameters.CLUSTER_NAME, "clusterTest"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addCluster); - - String response = assertSuccessPostOperation(url, paraMap, false); + String response = assertSuccessPostOperation(url, addClusterCmd("clusterTest"), false); Assert.assertTrue(response.contains("clusterTest")); // malformed cluster name - paraMap.put(JsonParameters.CLUSTER_NAME, "/ClusterTest"); - response = assertSuccessPostOperation(url, paraMap, true); + response = assertSuccessPostOperation(url, addClusterCmd("/ClusterTest"), true); // Add the grand cluster - paraMap.put(JsonParameters.CLUSTER_NAME, "Klazt3rz"); - response = assertSuccessPostOperation(url, paraMap, false); + response = assertSuccessPostOperation(url, addClusterCmd("Klazt3rz"), false); Assert.assertTrue(response.contains("Klazt3rz")); - paraMap.put(JsonParameters.CLUSTER_NAME, "\\ClusterTest"); - response = assertSuccessPostOperation(url, paraMap, false); + response = assertSuccessPostOperation(url, addClusterCmd("\\ClusterTest"), false); Assert.assertTrue(response.contains("\\ClusterTest")); // Add already exist cluster - paraMap.put(JsonParameters.CLUSTER_NAME, "clusterTest"); - response = assertSuccessPostOperation(url, paraMap, true); + response = assertSuccessPostOperation(url, addClusterCmd("clusterTest"), true); // delete cluster without resource and instance Assert.assertTrue(ZKUtil.isClusterSetup("Klazt3rz", _gZkClient)); Assert.assertTrue(ZKUtil.isClusterSetup("clusterTest", _gZkClient)); Assert.assertTrue(ZKUtil.isClusterSetup("\\ClusterTest", _gZkClient)); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.dropCluster); - String clusterUrl = getClusterUrl("\\ClusterTest"); deleteUrl(clusterUrl, false); @@ -286,94 +238,180 @@ public void testAddCluster() throws Exception { Assert.assertFalse(_gZkClient.exists("/clusterTest1")); Assert.assertFalse(_gZkClient.exists("/clusterTestOK")); - paraMap.put(JsonParameters.CLUSTER_NAME, "clusterTest1"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addCluster); - response = assertSuccessPostOperation(url, paraMap, false); + response = assertSuccessPostOperation(url, addClusterCmd("clusterTest1"), false); response = getUrl(clustersUrl); Assert.assertTrue(response.contains("clusterTest1")); } - public void testAddResource() throws Exception { - String reourcesUrl = "http://localhost:" + ADMIN_PORT + "/clusters/clusterTest1/resourceGroups"; + private Map addResourceCmd(String resourceName, String stateModelDef, + int partition) { + Map parameters = new HashMap(); - Map paraMap = new HashMap(); - paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_22"); - paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave"); - paraMap.put(JsonParameters.PARTITIONS, "144"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addResource); + parameters.put(JsonParameters.RESOURCE_GROUP_NAME, resourceName); + parameters.put(JsonParameters.STATE_MODEL_DEF_REF, stateModelDef); + parameters.put(JsonParameters.PARTITIONS, "" + partition); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addResource); - String response = assertSuccessPostOperation(reourcesUrl, paraMap, false); - Assert.assertTrue(response.contains("db_22")); + return parameters; + } + + private void addResource(String clusterName, String resourceName, int partitions) + throws IOException { + final String reourcesUrl = + "http://localhost:" + ADMIN_PORT + "/clusters/" + clusterName + "/resourceGroups"; + String response = + assertSuccessPostOperation(reourcesUrl, + addResourceCmd(resourceName, "MasterSlave", partitions), false); + Assert.assertTrue(response.contains(resourceName)); + } - paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_11"); - paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave"); - paraMap.put(JsonParameters.PARTITIONS, "44"); + @Test + public void testAddResource() throws Exception { + final String clusterName = "clusterTestAddResource"; + addCluster(clusterName); + + String reourcesUrl = + "http://localhost:" + ADMIN_PORT + "/clusters/" + clusterName + "/resourceGroups"; + String response = + assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_22", "MasterSlave", 144), false); + Assert.assertTrue(response.contains("db_22")); - response = assertSuccessPostOperation(reourcesUrl, paraMap, false); + response = + assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_11", "MasterSlave", 44), false); Assert.assertTrue(response.contains("db_11")); // Add duplicate resource - paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_22"); - paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "OnlineOffline"); - paraMap.put(JsonParameters.PARTITIONS, "55"); - - response = assertSuccessPostOperation(reourcesUrl, paraMap, true); + response = + assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_22", "OnlineOffline", 55), true); // drop resource now - String resourceUrl = getResourceUrl("clusterTest1", "db_11"); + String resourceUrl = getResourceUrl(clusterName, "db_11"); deleteUrl(resourceUrl, false); - Assert.assertFalse(_gZkClient.exists("/clusterTest1/IDEALSTATES/db_11")); + Assert.assertFalse(_gZkClient.exists("/" + clusterName + "/IDEALSTATES/db_11")); - paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_11"); - paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave"); - paraMap.put(JsonParameters.PARTITIONS, "44"); - response = assertSuccessPostOperation(reourcesUrl, paraMap, false); + response = + assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_11", "MasterSlave", 44), false); Assert.assertTrue(response.contains("db_11")); - Assert.assertTrue(_gZkClient.exists("/clusterTest1/IDEALSTATES/db_11")); + Assert.assertTrue(_gZkClient.exists("/" + clusterName + "/IDEALSTATES/db_11")); - paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_33"); - response = assertSuccessPostOperation(reourcesUrl, paraMap, false); + response = + assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_33", "MasterSlave", 44), false); Assert.assertTrue(response.contains("db_33")); - paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_44"); - response = assertSuccessPostOperation(reourcesUrl, paraMap, false); + response = + assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_44", "MasterSlave", 44), false); Assert.assertTrue(response.contains("db_44")); } - private void testDeactivateCluster() throws Exception, InterruptedException { - HelixDataAccessor accessor; - String path; - // deactivate cluster - String clusterUrl = getClusterUrl("clusterTest1"); - Map paraMap = new HashMap(); - paraMap.put(JsonParameters.ENABLED, "false"); - paraMap.put(JsonParameters.GRAND_CLUSTER, "Klazt3rz"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.activateCluster); + private Map activateClusterCmd(String grandClusterName, boolean enabled) { + Map parameters = new HashMap(); + parameters.put(JsonParameters.GRAND_CLUSTER, grandClusterName); + parameters.put(JsonParameters.ENABLED, "" + enabled); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.activateCluster); - String response = assertSuccessPostOperation(clusterUrl, paraMap, false); + return parameters; + } + + @Test + public void testDeactivateCluster() throws Exception { + final String clusterName = "clusterTestDeactivateCluster"; + final String controllerClusterName = "controllerClusterTestDeactivateCluster"; + + Map participants = + new HashMap(); + Map distControllers = + new HashMap(); + + // setup cluster + addCluster(clusterName); + addInstancesToCluster(clusterName, "localhost:123", 6, null); + addResource(clusterName, "db_11", 16); + rebalanceResource(clusterName, "db_11"); + + addCluster(controllerClusterName); + addInstancesToCluster(controllerClusterName, "controller_900", 2, null); + + // start mock nodes + for (int i = 0; i < 6; i++) { + String instanceName = "localhost_123" + i; + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participant.syncStart(); + participants.put(instanceName, participant); + } + + // start controller nodes + for (int i = 0; i < 2; i++) { + String controllerName = "controller_900" + i; + ClusterDistributedController distController = + new ClusterDistributedController(ZK_ADDR, controllerClusterName, controllerName); + distController.syncStart(); + distControllers.put(controllerName, distController); + } + + String clusterUrl = getClusterUrl(clusterName); + + // activate cluster + assertSuccessPostOperation(clusterUrl, activateClusterCmd(controllerClusterName, true), false); + boolean verifyResult = + ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, + controllerClusterName)); + Assert.assertTrue(verifyResult); + + verifyResult = + ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(verifyResult); + + // deactivate cluster + assertSuccessPostOperation(clusterUrl, activateClusterCmd(controllerClusterName, false), false); Thread.sleep(6000); - Assert.assertFalse(_gZkClient.exists("/Klazt3rz/IDEALSTATES/clusterTest1")); + Assert.assertFalse(_gZkClient.exists("/" + controllerClusterName + "/IDEALSTATES/" + + clusterName)); - accessor = _startCMResultMap.get("localhost_1231")._manager.getHelixDataAccessor(); - path = accessor.keyBuilder().controllerLeader().getPath(); + HelixDataAccessor accessor = participants.get("localhost_1231").getHelixDataAccessor(); + String path = accessor.keyBuilder().controllerLeader().getPath(); Assert.assertFalse(_gZkClient.exists(path)); deleteUrl(clusterUrl, true); + Assert.assertTrue(_gZkClient.exists("/" + clusterName)); - Assert.assertTrue(_gZkClient.exists("/clusterTest1")); // leader node should be gone - for (StartCMResult result : _startCMResultMap.values()) { - result._manager.disconnect(); - result._thread.interrupt(); + for (MockParticipantManager participant : participants.values()) { + participant.syncStop(); } deleteUrl(clusterUrl, false); - Assert.assertFalse(_gZkClient.exists("/clusterTest1")); + Assert.assertFalse(_gZkClient.exists("/" + clusterName)); + + // clean up + for (ClusterDistributedController controller : distControllers.values()) { + controller.syncStop(); + } + + for (MockParticipantManager participant : participants.values()) { + participant.syncStop(); + } + } + + private Map addIdealStateCmd() { + Map parameters = new HashMap(); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addIdealState); + + return parameters; } - private void testDropAddResource() throws Exception { - ZNRecord record = _gSetupTool._admin.getResourceIdealState("clusterTest1", "db_11").getRecord(); + @Test + public void testDropAddResource() throws Exception { + final String clusterName = "clusterTestDropAddResource"; + + // setup cluster + addCluster(clusterName); + addResource(clusterName, "db_11", 22); + addInstancesToCluster(clusterName, "localhost_123", 6, null); + rebalanceResource(clusterName, "db_11"); + ZNRecord record = _gSetupTool._admin.getResourceIdealState(clusterName, "db_11").getRecord(); String x = ObjectToJson(record); FileWriter fos = new FileWriter("/tmp/temp.log"); @@ -381,218 +419,371 @@ private void testDropAddResource() throws Exception { pw.write(x); pw.close(); - String resourceUrl = getResourceUrl("clusterTest1", "db_11"); - deleteUrl(resourceUrl, false); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_9900"); + controller.syncStart(); + // start mock nodes + Map participants = + new HashMap(); + for (int i = 0; i < 6; i++) { + String instanceName = "localhost_123" + i; + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participant.syncStart(); + participants.put(instanceName, participant); + } boolean verifyResult = ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - "clusterTest1")); + clusterName)); Assert.assertTrue(verifyResult); - Map paraMap = new HashMap(); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addResource); - paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_11"); - paraMap.put(JsonParameters.PARTITIONS, "22"); - paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave"); - String response = - assertSuccessPostOperation(getClusterUrl("clusterTest1") + "/resourceGroups", paraMap, - false); - String idealStateUrl = getResourceUrl("clusterTest1", "db_11") + "/idealState"; - Assert.assertTrue(response.contains("db_11")); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addIdealState); + String resourceUrl = getResourceUrl(clusterName, "db_11"); + deleteUrl(resourceUrl, false); + + verifyResult = + ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(verifyResult); + addResource(clusterName, "db_11", 22); + + String idealStateUrl = getResourceUrl(clusterName, "db_11") + "/idealState"; Map extraform = new HashMap(); extraform.put(JsonParameters.NEW_IDEAL_STATE, x); - response = assertSuccessPostOperation(idealStateUrl, paraMap, extraform, false); + assertSuccessPostOperation(idealStateUrl, addIdealStateCmd(), extraform, false); verifyResult = ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - "clusterTest1")); + clusterName)); Assert.assertTrue(verifyResult); - ZNRecord record2 = - _gSetupTool._admin.getResourceIdealState("clusterTest1", "db_11").getRecord(); + ZNRecord record2 = _gSetupTool._admin.getResourceIdealState(clusterName, "db_11").getRecord(); Assert.assertTrue(record2.equals(record)); + + // clean up + controller.syncStop(); + for (MockParticipantManager participant : participants.values()) { + participant.syncStop(); + } } - private void testExpandCluster() throws Exception { - boolean verifyResult; + private Map addInstanceCmd(String instances) { + Map parameters = new HashMap(); + parameters.put(JsonParameters.INSTANCE_NAMES, instances); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance); - String clusterUrl = getClusterUrl("clusterTest1"); - String instancesUrl = clusterUrl + "/instances"; + return parameters; + } + + private Map expandClusterCmd() { + Map parameters = new HashMap(); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.expandCluster); + + return parameters; + } + + @Test + public void testExpandCluster() throws Exception { - Map paraMap = new HashMap(); - paraMap.put(JsonParameters.INSTANCE_NAMES, - "localhost:12331;localhost:12341;localhost:12351;localhost:12361"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance); + final String clusterName = "clusterTestExpandCluster"; - String response = assertSuccessPostOperation(instancesUrl, paraMap, false); - String[] hosts = "localhost:12331;localhost:12341;localhost:12351;localhost:12361".split(";"); + // setup cluster + addCluster(clusterName); + addInstancesToCluster(clusterName, "localhost:123", 6, null); + addResource(clusterName, "db_11", 22); + rebalanceResource(clusterName, "db_11"); + + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_9900"); + controller.syncStart(); + + // start mock nodes + Map participants = + new HashMap(); + for (int i = 0; i < 6; i++) { + String instanceName = "localhost_123" + i; + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participant.syncStart(); + participants.put(instanceName, participant); + } + + boolean verifyResult = + ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(verifyResult); + + String clusterUrl = getClusterUrl(clusterName); + String instancesUrl = clusterUrl + "/instances"; + + String instances = "localhost:12331;localhost:12341;localhost:12351;localhost:12361"; + String response = assertSuccessPostOperation(instancesUrl, addInstanceCmd(instances), false); + String[] hosts = instances.split(";"); for (String host : hosts) { Assert.assertTrue(response.contains(host.replace(':', '_'))); } - paraMap.clear(); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.expandCluster); - response = assertSuccessPostOperation(clusterUrl, paraMap, false); + + response = assertSuccessPostOperation(clusterUrl, expandClusterCmd(), false); for (int i = 3; i <= 6; i++) { - StartCMResult result = - TestHelper.startDummyProcess(ZK_ADDR, "clusterTest1", "localhost_123" + i + "1"); - _startCMResultMap.put("localhost_123" + i + "1", result); + String instanceName = "localhost_123" + i + "1"; + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participant.syncStart(); + participants.put(instanceName, participant); } verifyResult = - ClusterStateVerifier.verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, - "clusterTest1")); + ClusterStateVerifier + .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, clusterName)); Assert.assertTrue(verifyResult); verifyResult = ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - "clusterTest1")); + clusterName)); Assert.assertTrue(verifyResult); + + // clean up + controller.syncStop(); + for (MockParticipantManager participant : participants.values()) { + participant.syncStop(); + } } - private void testEnablePartitions() throws IOException, InterruptedException { - HelixDataAccessor accessor; - accessor = _startCMResultMap.get("localhost_1231")._manager.getHelixDataAccessor(); + private Map enablePartitionCmd(String resourceName, String partitions, + boolean enabled) { + Map parameters = new HashMap(); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enablePartition); + parameters.put(JsonParameters.ENABLED, "" + enabled); + parameters.put(JsonParameters.PARTITION, partitions); + parameters.put(JsonParameters.RESOURCE, resourceName); + + return parameters; + } + + @Test + public void testEnablePartitions() throws IOException, InterruptedException { + final String clusterName = "clusterTestEnablePartitions"; + + // setup cluster + addCluster(clusterName); + addInstancesToCluster(clusterName, "localhost:123", 6, null); + addResource(clusterName, "db_11", 22); + rebalanceResource(clusterName, "db_11"); + + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_9900"); + controller.syncStart(); + + // start mock nodes + Map participants = + new HashMap(); + for (int i = 0; i < 6; i++) { + String instanceName = "localhost_123" + i; + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participant.syncStart(); + participants.put(instanceName, participant); + } + + HelixDataAccessor accessor = participants.get("localhost_1231").getHelixDataAccessor(); // drop node should fail as not disabled String hostName = "localhost_1231"; - String instanceUrl = getInstanceUrl("clusterTest1", hostName); + String instanceUrl = getInstanceUrl(clusterName, hostName); ExternalView ev = accessor.getProperty(accessor.keyBuilder().externalView("db_11")); - Map paraMap = new HashMap(); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enablePartition); - paraMap.put(JsonParameters.ENABLED, "false"); - paraMap.put(JsonParameters.PARTITION, "db_11_0;db_11_15"); - paraMap.put(JsonParameters.RESOURCE, "db_11"); - - String response = assertSuccessPostOperation(instanceUrl, paraMap, false); + String response = + assertSuccessPostOperation(instanceUrl, + enablePartitionCmd("db_11", "db_11_0;db_11_11", false), false); Assert.assertTrue(response.contains("DISABLED_PARTITION")); Assert.assertTrue(response.contains("db_11_0")); - Assert.assertTrue(response.contains("db_11_15")); + Assert.assertTrue(response.contains("db_11_11")); boolean verifyResult = ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - "clusterTest1")); + clusterName)); Assert.assertTrue(verifyResult); ev = accessor.getProperty(accessor.keyBuilder().externalView("db_11")); Assert.assertEquals(ev.getStateMap("db_11_0").get(hostName), "OFFLINE"); - Assert.assertEquals(ev.getStateMap("db_11_15").get(hostName), "OFFLINE"); + Assert.assertEquals(ev.getStateMap("db_11_11").get(hostName), "OFFLINE"); - paraMap.put(JsonParameters.ENABLED, "true"); - response = assertSuccessPostOperation(instanceUrl, paraMap, false); + response = + assertSuccessPostOperation(instanceUrl, + enablePartitionCmd("db_11", "db_11_0;db_11_11", true), false); Assert.assertFalse(response.contains("db_11_0")); - Assert.assertFalse(response.contains("db_11_15")); + Assert.assertFalse(response.contains("db_11_11")); verifyResult = ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - "clusterTest1")); + clusterName)); Assert.assertTrue(verifyResult); ev = accessor.getProperty(accessor.keyBuilder().externalView("db_11")); Assert.assertEquals(ev.getStateMap("db_11_0").get(hostName), "MASTER"); - Assert.assertEquals(ev.getStateMap("db_11_15").get(hostName), "SLAVE"); + Assert.assertEquals(ev.getStateMap("db_11_11").get(hostName), "SLAVE"); + + // clean up + controller.syncStop(); + for (MockParticipantManager participant : participants.values()) { + participant.syncStop(); + } + } + + private Map enableInstanceCmd(boolean enabled) { + Map parameters = new HashMap(); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enableInstance); + parameters.put(JsonParameters.ENABLED, "" + enabled); + return parameters; } - private void testInstanceOperations() throws Exception { + private Map swapInstanceCmd(String oldInstance, String newInstance) { + Map parameters = new HashMap(); + + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.swapInstance); + parameters.put(JsonParameters.OLD_INSTANCE, oldInstance); + parameters.put(JsonParameters.NEW_INSTANCE, newInstance); + + return parameters; + } + + @Test + public void testInstanceOperations() throws Exception { + final String clusterName = "clusterTestInstanceOperations"; + + // setup cluster + addCluster(clusterName); + addInstancesToCluster(clusterName, "localhost:123", 6, null); + addResource(clusterName, "db_11", 8); + rebalanceResource(clusterName, "db_11"); + + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_9900"); + controller.syncStart(); + + // start mock nodes + Map participants = + new HashMap(); + for (int i = 0; i < 6; i++) { + String instanceName = "localhost_123" + i; + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participant.syncStart(); + participants.put(instanceName, participant); + } + HelixDataAccessor accessor; // drop node should fail as not disabled - String instanceUrl = getInstanceUrl("clusterTest1", "localhost_1232"); + String instanceUrl = getInstanceUrl(clusterName, "localhost_1232"); deleteUrl(instanceUrl, true); // disabled node - Map paraMap = new HashMap(); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enableInstance); - paraMap.put(JsonParameters.ENABLED, "false"); - String response = assertSuccessPostOperation(instanceUrl, paraMap, false); + String response = assertSuccessPostOperation(instanceUrl, enableInstanceCmd(false), false); Assert.assertTrue(response.contains("false")); // Cannot drop / swap deleteUrl(instanceUrl, true); - String instancesUrl = getClusterUrl("clusterTest1") + "/instances"; - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.swapInstance); - paraMap.put(JsonParameters.OLD_INSTANCE, "localhost_1232"); - paraMap.put(JsonParameters.NEW_INSTANCE, "localhost_12320"); - response = assertSuccessPostOperation(instancesUrl, paraMap, true); + String instancesUrl = getClusterUrl(clusterName) + "/instances"; + response = + assertSuccessPostOperation(instancesUrl, + swapInstanceCmd("localhost_1232", "localhost_12320"), true); // disconnect the node - _startCMResultMap.get("localhost_1232")._manager.disconnect(); - _startCMResultMap.get("localhost_1232")._thread.interrupt(); + participants.get("localhost_1232").syncStop(); // add new node then swap instance - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance); - paraMap.put(JsonParameters.INSTANCE_NAME, "localhost_12320"); - response = assertSuccessPostOperation(instancesUrl, paraMap, false); + response = assertSuccessPostOperation(instancesUrl, addInstanceCmd("localhost_12320"), false); Assert.assertTrue(response.contains("localhost_12320")); // swap instance. The instance get swapped out should not exist anymore - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.swapInstance); - paraMap.put(JsonParameters.OLD_INSTANCE, "localhost_1232"); - paraMap.put(JsonParameters.NEW_INSTANCE, "localhost_12320"); - response = assertSuccessPostOperation(instancesUrl, paraMap, false); + response = + assertSuccessPostOperation(instancesUrl, + swapInstanceCmd("localhost_1232", "localhost_12320"), false); Assert.assertTrue(response.contains("localhost_12320")); Assert.assertFalse(response.contains("localhost_1232\"")); - accessor = _startCMResultMap.get("localhost_1231")._manager.getHelixDataAccessor(); + accessor = participants.get("localhost_1231").getHelixDataAccessor(); String path = accessor.keyBuilder().instanceConfig("localhost_1232").getPath(); Assert.assertFalse(_gZkClient.exists(path)); - _startCMResultMap.put("localhost_12320", - TestHelper.startDummyProcess(ZK_ADDR, "clusterTest1", "localhost_12320")); + MockParticipantManager newParticipant = + new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12320"); + newParticipant.syncStart(); + participants.put("localhost_12320", newParticipant); + + boolean verifyResult = + ClusterStateVerifier + .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, clusterName)); + Assert.assertTrue(verifyResult); + + // clean up + controller.syncStop(); + for (MockParticipantManager participant : participants.values()) { + participant.syncStop(); + } } - private void testStartCluster() throws Exception, InterruptedException { + @Test + public void testStartCluster() throws Exception { + final String clusterName = "clusterTestStartCluster"; + final String controllerClusterName = "controllerClusterTestStartCluster"; + + Map participants = + new HashMap(); + Map distControllers = + new HashMap(); + + // setup cluster + addCluster(clusterName); + addInstancesToCluster(clusterName, "localhost:123", 6, null); + addResource(clusterName, "db_11", 8); + rebalanceResource(clusterName, "db_11"); + + addCluster(controllerClusterName); + addInstancesToCluster(controllerClusterName, "controller_900", 2, null); + // start mock nodes for (int i = 0; i < 6; i++) { - StartCMResult result = - TestHelper.startDummyProcess(ZK_ADDR, "clusterTest1", "localhost_123" + i); - _startCMResultMap.put("localhost_123" + i, result); + String instanceName = "localhost_123" + i; + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participant.syncStart(); + participants.put(instanceName, participant); } // start controller nodes for (int i = 0; i < 2; i++) { - StartCMResult result = - TestHelper.startController("Klazt3rz", "controller_900" + i, ZK_ADDR, - HelixControllerMain.DISTRIBUTED); - - _startCMResultMap.put("controller_900" + i, result); + String controllerName = "controller_900" + i; + ClusterDistributedController distController = + new ClusterDistributedController(ZK_ADDR, controllerClusterName, controllerName); + distController.syncStart(); + distControllers.put(controllerName, distController); } Thread.sleep(100); // activate clusters // wrong grand clustername - - String clusterUrl = getClusterUrl("clusterTest1"); - Map paraMap = new HashMap(); - paraMap.put(JsonParameters.ENABLED, "true"); - paraMap.put(JsonParameters.GRAND_CLUSTER, "Klazters"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.activateCluster); - - String response = assertSuccessPostOperation(clusterUrl, paraMap, true); + String clusterUrl = getClusterUrl(clusterName); + assertSuccessPostOperation(clusterUrl, activateClusterCmd("nonExistCluster", true), true); // wrong cluster name - clusterUrl = getClusterUrl("clusterTest2"); - paraMap.put(JsonParameters.GRAND_CLUSTER, "Klazt3rz"); - response = assertSuccessPostOperation(clusterUrl, paraMap, true); + clusterUrl = getClusterUrl("nonExistCluster"); + assertSuccessPostOperation(clusterUrl, activateClusterCmd(controllerClusterName, true), true); - paraMap.put(JsonParameters.ENABLED, "true"); - paraMap.put(JsonParameters.GRAND_CLUSTER, "Klazt3rz"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.activateCluster); - clusterUrl = getClusterUrl("clusterTest1"); - response = assertSuccessPostOperation(clusterUrl, paraMap, false); + clusterUrl = getClusterUrl(clusterName); + assertSuccessPostOperation(clusterUrl, activateClusterCmd(controllerClusterName, true), false); Thread.sleep(500); deleteUrl(clusterUrl, true); // verify leader node - HelixDataAccessor accessor = - _startCMResultMap.get("controller_9001")._manager.getHelixDataAccessor(); - LiveInstance controllerLeader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); + HelixDataAccessor accessor = distControllers.get("controller_9001").getHelixDataAccessor(); + Leader controllerLeader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); Assert.assertTrue(controllerLeader.getInstanceName().startsWith("controller_900")); - accessor = _startCMResultMap.get("localhost_1232")._manager.getHelixDataAccessor(); - LiveInstance leader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); + accessor = participants.get("localhost_1232").getHelixDataAccessor(); + Leader leader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); for (int i = 0; i < 5; i++) { if (leader != null) { break; @@ -603,81 +794,98 @@ private void testStartCluster() throws Exception, InterruptedException { Assert.assertTrue(leader.getInstanceName().startsWith("controller_900")); boolean verifyResult = - ClusterStateVerifier.verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, - "clusterTest1")); + ClusterStateVerifier + .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, clusterName)); Assert.assertTrue(verifyResult); verifyResult = ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - "clusterTest1")); + clusterName)); Assert.assertTrue(verifyResult); + Thread.sleep(1000); + + // clean up + for (ClusterDistributedController controller : distControllers.values()) { + controller.syncStop(); + } + for (MockParticipantManager participant : participants.values()) { + participant.syncStop(); + } + } + + private Map rebalanceCmd(int replicas, String prefix, String tag) { + Map parameters = new HashMap(); + parameters.put(JsonParameters.REPLICAS, "" + replicas); + if (prefix != null) { + parameters.put(JsonParameters.RESOURCE_KEY_PREFIX, prefix); + } + if (tag != null) { + parameters.put(ClusterSetup.instanceGroupTag, tag); + } + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance); + + return parameters; } - private void testRebalanceResource() throws Exception { - String resourceUrl = getResourceUrl("clusterTest1", "db_11"); - Map paraMap = new HashMap(); - paraMap.put(JsonParameters.REPLICAS, "3"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance); + private void rebalanceResource(String clusterName, String resourceName) throws IOException { + String resourceUrl = getResourceUrl(clusterName, resourceName); + String idealStateUrl = resourceUrl + "/idealState"; + + assertSuccessPostOperation(idealStateUrl, rebalanceCmd(3, null, null), false); + } + + @Test + public void testRebalanceResource() throws Exception { + // add a normal cluster + final String clusterName = "clusterTestRebalanceResource"; + addCluster(clusterName); - String ISUrl = resourceUrl + "/idealState"; - String response = assertSuccessPostOperation(ISUrl, paraMap, false); + addInstancesToCluster(clusterName, "localhost:123", 3, _tag1); + addResource(clusterName, "db_11", 44); + + String resourceUrl = getResourceUrl(clusterName, "db_11"); + + String idealStateUrl = resourceUrl + "/idealState"; + String response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(3, null, null), false); ZNRecord record = JsonToObject(ZNRecord.class, response); Assert.assertTrue(record.getId().equalsIgnoreCase("db_11")); - Assert - .assertTrue((((List) (record.getListFields().values().toArray()[0]))).size() == 3); - Assert.assertTrue((((Map) (record.getMapFields().values().toArray()[0]))) - .size() == 3); + Assert.assertEquals(record.getListField("db_11_0").size(), 3); + Assert.assertEquals(record.getMapField("db_11_0").size(), 3); deleteUrl(resourceUrl, false); // re-add and rebalance - String reourcesUrl = "http://localhost:" + ADMIN_PORT + "/clusters/clusterTest1/resourceGroups"; + final String reourcesUrl = + "http://localhost:" + ADMIN_PORT + "/clusters/" + clusterName + "/resourceGroups"; + response = getUrl(reourcesUrl); Assert.assertFalse(response.contains("db_11")); - paraMap = new HashMap(); - paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_11"); - paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave"); - paraMap.put(JsonParameters.PARTITIONS, "48"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addResource); - - response = assertSuccessPostOperation(reourcesUrl, paraMap, false); - Assert.assertTrue(response.contains("db_11")); - - ISUrl = resourceUrl + "/idealState"; - paraMap.put(JsonParameters.REPLICAS, "3"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance); - response = assertSuccessPostOperation(ISUrl, paraMap, false); + addResource(clusterName, "db_11", 48); + idealStateUrl = resourceUrl + "/idealState"; + response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(3, null, null), false); record = JsonToObject(ZNRecord.class, response); Assert.assertTrue(record.getId().equalsIgnoreCase("db_11")); - Assert - .assertTrue((((List) (record.getListFields().values().toArray()[0]))).size() == 3); - Assert.assertTrue((((Map) (record.getMapFields().values().toArray()[0]))) - .size() == 3); + Assert.assertEquals(record.getListField("db_11_0").size(), 3); + Assert.assertEquals(record.getMapField("db_11_0").size(), 3); // rebalance with key prefix - resourceUrl = getResourceUrl("clusterTest1", "db_22"); - ISUrl = resourceUrl + "/idealState"; - paraMap.put(JsonParameters.REPLICAS, "2"); - paraMap.put(JsonParameters.RESOURCE_KEY_PREFIX, "alias"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance); - response = assertSuccessPostOperation(ISUrl, paraMap, false); + addResource(clusterName, "db_22", 55); + resourceUrl = getResourceUrl(clusterName, "db_22"); + idealStateUrl = resourceUrl + "/idealState"; + response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(2, "alias", null), false); record = JsonToObject(ZNRecord.class, response); Assert.assertTrue(record.getId().equalsIgnoreCase("db_22")); - Assert - .assertTrue((((List) (record.getListFields().values().toArray()[0]))).size() == 2); - Assert.assertTrue((((Map) (record.getMapFields().values().toArray()[0]))) - .size() == 2); + Assert.assertEquals(record.getListField("alias_0").size(), 2); + Assert.assertEquals(record.getMapField("alias_0").size(), 2); Assert.assertTrue((((String) (record.getMapFields().keySet().toArray()[0]))) .startsWith("alias_")); Assert.assertFalse(response.contains(IdealStateProperty.INSTANCE_GROUP_TAG.toString())); - resourceUrl = getResourceUrl("clusterTest1", "db_33"); - ISUrl = resourceUrl + "/idealState"; - paraMap.put(JsonParameters.REPLICAS, "2"); - paraMap.remove(JsonParameters.RESOURCE_KEY_PREFIX); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance); - paraMap.put(ClusterSetup.instanceGroupTag, _tag1); - response = assertSuccessPostOperation(ISUrl, paraMap, false); + + addResource(clusterName, "db_33", 44); + resourceUrl = getResourceUrl(clusterName, "db_33"); + idealStateUrl = resourceUrl + "/idealState"; + response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(2, null, _tag1), false); Assert.assertTrue(response.contains(IdealStateProperty.INSTANCE_GROUP_TAG.toString())); Assert.assertTrue(response.contains(_tag1)); @@ -690,14 +898,10 @@ record = JsonToObject(ZNRecord.class, response); } } - resourceUrl = getResourceUrl("clusterTest1", "db_44"); - ISUrl = resourceUrl + "/idealState"; - paraMap.put(JsonParameters.REPLICAS, "2"); - paraMap.remove(JsonParameters.RESOURCE_KEY_PREFIX); - paraMap.put(JsonParameters.RESOURCE_KEY_PREFIX, "alias"); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance); - paraMap.put(ClusterSetup.instanceGroupTag, _tag1); - response = assertSuccessPostOperation(ISUrl, paraMap, false); + addResource(clusterName, "db_44", 44); + resourceUrl = getResourceUrl(clusterName, "db_44"); + idealStateUrl = resourceUrl + "/idealState"; + response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(2, "alias", _tag1), false); Assert.assertTrue(response.contains(IdealStateProperty.INSTANCE_GROUP_TAG.toString())); Assert.assertTrue(response.contains(_tag1)); @@ -715,24 +919,66 @@ record = JsonToObject(ZNRecord.class, response); } } - private void testAddInstance() throws Exception { - String clusterUrl = getClusterUrl("clusterTest1"); - Map paraMap = new HashMap(); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance); - String response = null; - // Add instances to cluster + private void addInstancesToCluster(String clusterName, String instanceNamePrefix, int n, + String tag) throws IOException { + Map parameters = new HashMap(); + final String clusterUrl = getClusterUrl(clusterName); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance); + + // add instances to cluster String instancesUrl = clusterUrl + "/instances"; - for (int i = 0; i < 3; i++) { + for (int i = 0; i < n; i++) { - paraMap.put(JsonParameters.INSTANCE_NAME, "localhost:123" + i); - response = assertSuccessPostOperation(instancesUrl, paraMap, false); - Assert.assertTrue(response.contains(("localhost:123" + i).replace(':', '_'))); + parameters.put(JsonParameters.INSTANCE_NAME, instanceNamePrefix + i); + String response = assertSuccessPostOperation(instancesUrl, parameters, false); + Assert.assertTrue(response.contains((instanceNamePrefix + i).replace(':', '_'))); } - paraMap.remove(JsonParameters.INSTANCE_NAME); - paraMap.put(JsonParameters.INSTANCE_NAMES, - "localhost:1233;localhost:1234;localhost:1235;localhost:1236"); - response = assertSuccessPostOperation(instancesUrl, paraMap, false); + // add tag to instance + if (tag != null && !tag.isEmpty()) { + parameters.clear(); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstanceTag); + parameters.put(ClusterSetup.instanceGroupTag, tag); + for (int i = 0; i < n; i++) { + String instanceUrl = instancesUrl + "/" + (instanceNamePrefix + i).replace(':', '_'); + String response = assertSuccessPostOperation(instanceUrl, parameters, false); + Assert.assertTrue(response.contains(_tag1)); + } + } + + } + + private Map addInstanceTagCmd(String tag) { + Map parameters = new HashMap(); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstanceTag); + parameters.put(ClusterSetup.instanceGroupTag, tag); + + return parameters; + } + + private Map removeInstanceTagCmd(String tag) { + Map parameters = new HashMap(); + parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.removeInstanceTag); + parameters.put(ClusterSetup.instanceGroupTag, tag); + + return parameters; + } + + @Test + public void testAddInstance() throws Exception { + final String clusterName = "clusterTestAddInstance"; + + // add normal cluster + addCluster(clusterName); + + String clusterUrl = getClusterUrl(clusterName); + + // Add instances to cluster + String instancesUrl = clusterUrl + "/instances"; + addInstancesToCluster(clusterName, "localhost:123", 3, null); + + String instances = "localhost:1233;localhost:1234;localhost:1235;localhost:1236"; + String response = assertSuccessPostOperation(instancesUrl, addInstanceCmd(instances), false); for (int i = 3; i <= 6; i++) { Assert.assertTrue(response.contains("localhost_123" + i)); } @@ -751,42 +997,34 @@ private void testAddInstance() throws Exception { // disable node instanceUrl = instancesUrl + "/localhost_1236"; - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enableInstance); - paraMap.put(JsonParameters.ENABLED, "false"); - response = assertSuccessPostOperation(instanceUrl, paraMap, false); + response = assertSuccessPostOperation(instanceUrl, enableInstanceCmd(false), false); Assert.assertTrue(response.contains("false")); deleteUrl(instanceUrl, false); + // add controller cluster + final String controllerClusterName = "controllerClusterTestAddInstance"; + addCluster(controllerClusterName); + // add node to controller cluster - paraMap.remove(JsonParameters.INSTANCE_NAME); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance); - paraMap.put(JsonParameters.INSTANCE_NAMES, "controller:9000;controller:9001"); - String controllerUrl = getClusterUrl("Klazt3rz") + "/instances"; - response = assertSuccessPostOperation(controllerUrl, paraMap, false); + String controllers = "controller:9000;controller:9001"; + String controllerUrl = getClusterUrl(controllerClusterName) + "/instances"; + response = assertSuccessPostOperation(controllerUrl, addInstanceCmd(controllers), false); Assert.assertTrue(response.contains("controller_9000")); Assert.assertTrue(response.contains("controller_9001")); - // add a dup host - paraMap.remove(JsonParameters.INSTANCE_NAMES); - paraMap.put(JsonParameters.INSTANCE_NAME, "localhost:1234"); - response = assertSuccessPostOperation(instancesUrl, paraMap, true); - - // add tags + // add a duplicated host + response = assertSuccessPostOperation(instancesUrl, addInstanceCmd("localhost:1234"), true); - paraMap.clear(); - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstanceTag); - paraMap.put(ClusterSetup.instanceGroupTag, _tag1); + // add/remove tags for (int i = 0; i < 4; i++) { instanceUrl = instancesUrl + "/localhost_123" + i; - response = assertSuccessPostOperation(instanceUrl, paraMap, false); + response = assertSuccessPostOperation(instanceUrl, addInstanceTagCmd(_tag1), false); Assert.assertTrue(response.contains(_tag1)); - } - paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.removeInstanceTag); + instanceUrl = instancesUrl + "/localhost_1233"; - response = assertSuccessPostOperation(instanceUrl, paraMap, false); + response = assertSuccessPostOperation(instanceUrl, removeInstanceTagCmd(_tag1), false); Assert.assertFalse(response.contains(_tag1)); - } } diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetInstance.java b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetInstance.java index 9534cf5768..fd12080592 100644 --- a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetInstance.java +++ b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetInstance.java @@ -25,8 +25,8 @@ import java.util.Set; import org.apache.helix.TestHelper; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.ErrTransition; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; @@ -53,12 +53,8 @@ public void testResetInstance() throws Exception { 3, // replicas "MasterSlave", true); // do rebalance - // // start admin thread - // AdminThread adminThread = new AdminThread(ZK_ADDR, _port); - // adminThread.start(); - // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); Map> errPartitions = new HashMap>() { @@ -69,16 +65,16 @@ public void testResetInstance() throws Exception { }; // start mock participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); if (i == 0) { participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, - new ErrTransition(errPartitions)); + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errPartitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -111,9 +107,6 @@ public void testResetInstance() throws Exception { Assert.assertTrue(result, "Cluster verification fails"); // clean up - // wait for all zk callbacks done - Thread.sleep(1000); - // adminThread.stop(); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java index c8099a4e76..82a26073c3 100644 --- a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java +++ b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java @@ -29,10 +29,11 @@ import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.mock.participant.ErrTransition; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; @@ -67,9 +68,9 @@ public ErrTransitionWithResetCnt(Map> errPartitions) { @Override public void doTransition(Message message, NotificationContext context) { super.doTransition(message, context); - String fromState = message.getFromState(); - String toState = message.getToState(); - if (fromState.equals("ERROR") && toState.equals("OFFLINE")) { + State fromState = message.getTypedFromState(); + State toState = message.getTypedToState(); + if (fromState.toString().equals("ERROR") && toState.toString().equals("OFFLINE")) { // System.err.println("doReset() invoked"); _errToOfflineInvoked.incrementAndGet(); } @@ -94,12 +95,8 @@ public void testResetPartitionState() throws Exception { 3, // replicas "MasterSlave", true); // do rebalance - // start admin thread - // AdminThread adminThread = new AdminThread(ZK_ADDR, _port); - // adminThread.start(); - // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); Map> errPartitions = new HashMap>(); @@ -107,16 +104,16 @@ public void testResetPartitionState() throws Exception { errPartitions.put("OFFLINE-SLAVE", TestHelper.setOf("TestDB0_8")); // start mock participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); if (i == 0) { participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, - new ErrTransition(errPartitions)); + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errPartitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -170,9 +167,6 @@ public void testResetPartitionState() throws Exception { Assert.assertEquals(_errToOfflineInvoked.get(), 2, "reset() should be invoked 2 times"); // clean up - // wait for all zk callbacks done - Thread.sleep(1000); - // adminThread.stop(); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); @@ -184,15 +178,14 @@ public void testResetPartitionState() throws Exception { private void clearStatusUpdate(String clusterName, String instance, String resource, String partition) { // clear status update for error partition so verify() will not fail on - // old - // errors + // old errors ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance(instance)); - accessor.removeProperty(keyBuilder.stateTransitionStatus(instance, liveInstance.getSessionId(), - resource, partition)); + accessor.removeProperty(keyBuilder.stateTransitionStatus(instance, liveInstance.getTypedSessionId() + .stringify(), resource, partition)); } diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetResource.java b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetResource.java index 96f4f6c56e..db9e9bba60 100644 --- a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetResource.java +++ b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetResource.java @@ -25,8 +25,8 @@ import java.util.Set; import org.apache.helix.TestHelper; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.ErrTransition; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; @@ -53,12 +53,8 @@ public void testResetNode() throws Exception { 3, // replicas "MasterSlave", true); // do rebalance - // start admin thread - // AdminThread adminThread = new AdminThread(ZK_ADDR, _port); - // adminThread.start(); - // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); Map> errPartitions = new HashMap>() { @@ -69,16 +65,16 @@ public void testResetNode() throws Exception { }; // start mock participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); if (i == 0) { participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, - new ErrTransition(errPartitions)); + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errPartitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -112,15 +108,11 @@ public void testResetNode() throws Exception { Assert.assertTrue(result, "Cluster verification fails"); // clean up - // wait for all zk callbacks done - Thread.sleep(1000); - // adminThread.stop(); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); - } } diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestClusterManagementWebapp.java b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestClusterManagementWebapp.java index 236f327204..c1251de261 100644 --- a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestClusterManagementWebapp.java +++ b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestClusterManagementWebapp.java @@ -30,8 +30,8 @@ import org.apache.helix.PropertyPathConfig; import org.apache.helix.PropertyType; import org.apache.helix.ZNRecord; -import org.apache.helix.model.StateModelDefinition; import org.apache.helix.model.InstanceConfig.InstanceConfigProperty; +import org.apache.helix.model.StateModelDefinition; import org.apache.helix.tools.AdminTestBase; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.webapp.resources.ClusterRepresentationUtil; @@ -41,13 +41,13 @@ import org.codehaus.jackson.map.ObjectMapper; import org.codehaus.jackson.type.TypeReference; import org.restlet.Client; +import org.restlet.Request; +import org.restlet.Response; import org.restlet.data.MediaType; import org.restlet.data.Method; import org.restlet.data.Protocol; import org.restlet.data.Reference; -import org.restlet.data.Request; -import org.restlet.data.Response; -import org.restlet.resource.Representation; +import org.restlet.representation.Representation; import org.testng.Assert; import org.testng.AssertJUnit; import org.testng.annotations.Test; diff --git a/helix-agent/DISCLAIMER b/helix-agent/DISCLAIMER new file mode 100644 index 0000000000..2001d3154d --- /dev/null +++ b/helix-agent/DISCLAIMER @@ -0,0 +1,15 @@ +Apache Helix is an effort undergoing incubation at the Apache Software +Foundation (ASF), sponsored by the Apache Incubator PMC. + +Incubation is required of all newly accepted projects until a further review +indicates that the infrastructure, communications, and decision making process +have stabilized in a manner consistent with other successful ASF projects. + +While incubation status is not necessarily a reflection of the completeness +or stability of the code, it does indicate that the project has yet to be +fully endorsed by the ASF. + +For more information about the incubation status of the Apache Helix project you +can go to the following page: + +http://incubator.apache.org/projects/helix.html diff --git a/helix-agent/LICENSE b/helix-agent/LICENSE new file mode 100644 index 0000000000..413913f132 --- /dev/null +++ b/helix-agent/LICENSE @@ -0,0 +1,273 @@ + + 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. + + + diff --git a/helix-agent/NOTICE b/helix-agent/NOTICE new file mode 100644 index 0000000000..e070e15573 --- /dev/null +++ b/helix-agent/NOTICE @@ -0,0 +1,30 @@ +Apache Helix +Copyright 2012 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 +josql (http://sourceforge.net/projects/josql/). +Licensed under the Apache License 2.0. + +This product includes software developed at +restlet (http://www.restlet.org/about/legal). +Licensed under the Apache License 2.0. + + +II. License Summary +- Apache License 2.0 +- BSD License diff --git a/helix-agent/pom.xml b/helix-agent/pom.xml index 6d690e53a6..7d2a0ce14d 100644 --- a/helix-agent/pom.xml +++ b/helix-agent/pom.xml @@ -22,7 +22,7 @@ under the License. org.apache.helix helix - 0.6.2-incubating-SNAPSHOT + 0.7.1-incubating-SNAPSHOT helix-agent bundle @@ -44,8 +44,8 @@ under the License. helix-core - com.noelios.restlet - com.noelios.restlet + org.restlet.jse + org.restlet org.testng @@ -79,6 +79,23 @@ under the License. + + org.apache.maven.plugins + maven-assembly-plugin + + + src/assemble/assembly.xml + + + + + package + + single + + + + diff --git a/helix-agent/src/assemble/assembly.xml b/helix-agent/src/assemble/assembly.xml new file mode 100644 index 0000000000..c2d08a1cda --- /dev/null +++ b/helix-agent/src/assemble/assembly.xml @@ -0,0 +1,60 @@ + + + + pkg + + tar + + + + ${project.build.directory}/${project.artifactId}-pkg/bin + bin + unix + 0755 + 0755 + + + ${project.build.directory}/${project.artifactId}-pkg/repo/ + repo + 0755 + 0755 + + **/*.xml + + + + ${project.build.directory}/${project.artifactId}-pkg/conf + conf + unix + 0755 + 0755 + + + ${project.basedir} + / + + LICENSE + NOTICE + DISCLAIMER + + 0755 + + + \ No newline at end of file diff --git a/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModel.java b/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModel.java index 313f43069f..d227ac3dc2 100644 --- a/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModel.java +++ b/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModel.java @@ -29,9 +29,10 @@ import org.apache.helix.ExternalCommand; import org.apache.helix.HelixManager; import org.apache.helix.NotificationContext; +import org.apache.helix.api.State; import org.apache.helix.model.HelixConfigScope; -import org.apache.helix.model.Message; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.model.Message; import org.apache.helix.model.builder.HelixConfigScopeBuilder; import org.apache.helix.participant.statemachine.StateModel; import org.apache.helix.participant.statemachine.StateModelInfo; @@ -71,21 +72,24 @@ public void genericStateTransitionHandler(Message message, NotificationContext c HelixManager manager = context.getManager(); String clusterName = manager.getClusterName(); - String fromState = message.getFromState(); - String toState = message.getToState(); + State fromState = message.getTypedFromState(); + State toState = message.getTypedToState(); // construct keys for command-config - String cmdKey = buildKey(fromState, toState, CommandAttribute.COMMAND); - String workingDirKey = buildKey(fromState, toState, CommandAttribute.WORKING_DIR); - String timeoutKey = buildKey(fromState, toState, CommandAttribute.TIMEOUT); - String pidFileKey = buildKey(fromState, toState, CommandAttribute.PID_FILE); + String cmdKey = buildKey(fromState.toString(), toState.toString(), CommandAttribute.COMMAND); + String workingDirKey = + buildKey(fromState.toString(), toState.toString(), CommandAttribute.WORKING_DIR); + String timeoutKey = + buildKey(fromState.toString(), toState.toString(), CommandAttribute.TIMEOUT); + String pidFileKey = + buildKey(fromState.toString(), toState.toString(), CommandAttribute.PID_FILE); List cmdConfigKeys = Arrays.asList(cmdKey, workingDirKey, timeoutKey, pidFileKey); // read command from resource-scope configures if (cmd == null) { HelixConfigScope resourceScope = new HelixConfigScopeBuilder(ConfigScopeProperty.RESOURCE).forCluster(clusterName) - .forResource(message.getResourceName()).build(); + .forResource(message.getResourceId().stringify()).build(); Map cmdKeyValueMap = manager.getConfigAccessor().get(resourceScope, cmdConfigKeys); if (cmdKeyValueMap != null) { @@ -112,8 +116,8 @@ public void genericStateTransitionHandler(Message message, NotificationContext c } if (cmd == null) { - throw new Exception("Unable to find command for transition from:" + message.getFromState() - + " to:" + message.getToState()); + throw new Exception("Unable to find command for transition from:" + message.getTypedFromState() + + " to:" + message.getTypedToState()); } _logger.info("Executing command: " + cmd + ", using workingDir: " + workingDir + ", timeout: " + timeout + ", on " + manager.getInstanceName()); diff --git a/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModelFactory.java b/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModelFactory.java index a0e00a3762..69d45ae6d7 100644 --- a/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModelFactory.java +++ b/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModelFactory.java @@ -19,12 +19,13 @@ * under the License. */ -import org.apache.helix.participant.statemachine.StateModelFactory; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.participant.statemachine.HelixStateModelFactory; -public class AgentStateModelFactory extends StateModelFactory { +public class AgentStateModelFactory extends HelixStateModelFactory { @Override - public AgentStateModel createNewStateModel(String partitionKey) { + public AgentStateModel createNewStateModel(PartitionId partitionKey) { AgentStateModel model = new AgentStateModel(); return model; } diff --git a/helix-agent/src/main/java/org/apache/helix/agent/HelixAgentMain.java b/helix-agent/src/main/java/org/apache/helix/agent/HelixAgentMain.java index c90054608a..ded2d9b479 100644 --- a/helix-agent/src/main/java/org/apache/helix/agent/HelixAgentMain.java +++ b/helix-agent/src/main/java/org/apache/helix/agent/HelixAgentMain.java @@ -31,6 +31,7 @@ import org.apache.commons.cli.ParseException; import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.participant.StateMachineEngine; import org.apache.log4j.Logger; @@ -128,7 +129,8 @@ public static void main(String[] args) throws Exception { new ZKHelixManager(clusterName, instance, InstanceType.PARTICIPANT, zkAddress); StateMachineEngine stateMach = manager.getStateMachineEngine(); - stateMach.registerStateModelFactory(stateModelName, new AgentStateModelFactory()); + stateMach.registerStateModelFactory(StateModelDefId.from(stateModelName), + new AgentStateModelFactory()); Runtime.getRuntime().addShutdownHook(new HelixAgentShutdownHook(manager)); diff --git a/helix-agent/src/main/java/org/apache/helix/agent/ProcessMonitorThread.java b/helix-agent/src/main/java/org/apache/helix/agent/ProcessMonitorThread.java index e1a3d72a6a..09a89df719 100644 --- a/helix-agent/src/main/java/org/apache/helix/agent/ProcessMonitorThread.java +++ b/helix-agent/src/main/java/org/apache/helix/agent/ProcessMonitorThread.java @@ -1,6 +1,5 @@ package org.apache.helix.agent; -import java.io.File; import java.util.Random; import java.util.concurrent.TimeUnit; diff --git a/helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java b/helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java index e7232969d7..27b4d36b66 100644 --- a/helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java +++ b/helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java @@ -20,30 +20,31 @@ */ import java.io.File; -import java.io.IOException; import java.util.Date; -import java.util.concurrent.TimeUnit; +import java.util.HashMap; +import java.util.Map; import org.apache.helix.ConfigAccessor; import org.apache.helix.ExternalCommand; import org.apache.helix.ScriptTestHelper; import org.apache.helix.TestHelper; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.manager.zk.ZNRecordSerializer; -import org.apache.helix.manager.zk.ZkClient; -import org.apache.helix.mock.controller.ClusterController; +import org.apache.helix.integration.manager.ClusterControllerManager; import org.apache.helix.model.HelixConfigScope; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; import org.apache.helix.model.builder.HelixConfigScopeBuilder; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; +import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class TestHelixAgent extends ZkUnitTestBase { + private final static Logger LOG = Logger.getLogger(TestHelixAgent.class); + final String workingDir = ScriptTestHelper.getPrefix() + ScriptTestHelper.INTEGRATION_SCRIPT_DIR; ExternalCommand serverCmd = null; @@ -96,13 +97,9 @@ public void test() throws Exception { "MasterSlave", true); // do rebalance // set cluster config - ZkClient client = - new ZkClient(zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT, ZkClient.DEFAULT_CONNECTION_TIMEOUT, - new ZNRecordSerializer()); - HelixConfigScope scope = new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(clusterName).build(); - ConfigAccessor configAccessor = new ConfigAccessor(client); + ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient); // String pidFile = ScriptTestHelper.getPrefix() + ScriptTestHelper.INTEGRATION_LOG_DIR + // "/default/foo_{PARTITION_NAME}_pid.txt"; @@ -150,10 +147,11 @@ public void test() throws Exception { configAccessor.set(scope, cmdConfig.toKeyValueMap()); // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", zkAddr); + ClusterControllerManager controller = new ClusterControllerManager(zkAddr, clusterName, "controller_0"); controller.syncStart(); // start helix-agent + Map agents = new HashMap(); for (int i = 0; i < n; i++) { final String instanceName = "localhost_" + (12918 + i); Thread agentThread = new Thread() { @@ -165,11 +163,11 @@ public void run() { "--stateModel", "MasterSlave" }); } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Exception start helix-agent", e); } } }; + agents.put(instanceName, agentThread); agentThread.start(); // wait participant thread to start @@ -199,6 +197,11 @@ public void run() { clusterName)); Assert.assertTrue(result); + // clean up + controller.syncStop(); + for (Thread agentThread : agents.values()) { + agentThread.interrupt(); + } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } diff --git a/helix-core/pom.xml b/helix-core/pom.xml index e57c3ffef9..6f2aeb9827 100644 --- a/helix-core/pom.xml +++ b/helix-core/pom.xml @@ -21,7 +21,7 @@ under the License. org.apache.helix helix - 0.6.2-incubating-SNAPSHOT + 0.7.1-incubating-SNAPSHOT 4.0.0 @@ -46,7 +46,7 @@ under the License. org.apache.zookeeper*;version="[3.3,4)", org.codehaus.jackson*;version="[1.8,2)", org.josql*;version="[1.5,2)", - org.restlet;version="[1.1,2)", + org.restlet;version="[2.1.4,3]", * @@ -118,7 +118,7 @@ under the License. org.apache.camel camel-josql - 2.5.0 + 2.12.1 org.testng @@ -137,19 +137,20 @@ under the License. 2.1 - com.noelios.restlet - com.noelios.restlet - - - org.restlet + org.restlet.jse org.restlet - 1.1.10 + 2.1.4 com.google.guava guava - r09 + 15.0 + + org.yaml + snakeyaml + 1.12 + @@ -201,10 +202,6 @@ under the License. org.apache.helix.tools.ZkLogCSVFormatter zk-log-csv-formatter - - org.apache.helix.examples.ExampleProcess - start-helix-participant - org.apache.helix.tools.LocalZKServer start-standalone-zookeeper @@ -213,14 +210,18 @@ under the License. org.apache.helix.tools.ZkLogAnalyzer zk-log-analyzer - - org.apache.helix.examples.Quickstart - quickstart - org.apache.helix.tools.JmxDumper JmxDumper + + org.apache.helix.tools.YAMLClusterSetup + yaml-cluster-setup + + + org.apache.helix.tools.IntegrationTestUtil + test-util + diff --git a/helix-core/src/main/java/org/apache/helix/ConfigAccessor.java b/helix-core/src/main/java/org/apache/helix/ConfigAccessor.java index 34aef49b8d..f46e537674 100644 --- a/helix-core/src/main/java/org/apache/helix/ConfigAccessor.java +++ b/helix-core/src/main/java/org/apache/helix/ConfigAccessor.java @@ -27,11 +27,11 @@ import java.util.Map; import java.util.TreeMap; +import org.apache.helix.manager.zk.ZKUtil; +import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.model.ConfigScope; import org.apache.helix.model.HelixConfigScope; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; -import org.apache.helix.manager.zk.ZKUtil; -import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.util.StringTemplate; import org.apache.log4j.Logger; diff --git a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java index d5c62fab94..ee1bb46917 100644 --- a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java +++ b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java @@ -22,8 +22,15 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import org.apache.helix.model.*; + +import org.apache.helix.model.ClusterConstraints; import org.apache.helix.model.ClusterConstraints.ConstraintType; +import org.apache.helix.model.ConstraintItem; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.HelixConfigScope; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.StateModelDefinition; /* * Helix cluster management @@ -228,34 +235,6 @@ void resetPartition(String clusterName, String instanceName, String resourceName */ void dropResource(String clusterName, String resourceName); - /** - * Add a statistics to a cluster - * @param clusterName - * @param statName - */ - void addStat(String clusterName, String statName); - - /** - * Add an alert to a cluster - * @param clusterName - * @param alertName - */ - void addAlert(String clusterName, String alertName); - - /** - * Drop statistics from a cluster - * @param clusterName - * @param statName - */ - void dropStat(String clusterName, String statName); - - /** - * Drop an alert from a cluster - * @param clusterName - * @param alertName - */ - void dropAlert(String clusterName, String alertName); - /** * Get a list of state model definitions in a cluster * @param clusterName @@ -417,4 +396,13 @@ void rebalance(String clusterName, String resourceName, int replica, String keyP * Release resources */ void close(); + + /** + * Swap instances + * @param clusterName + * @param oldInstanceName + * @param newInstanceName + */ + void swapInstance(String clusterName, String oldInstanceName, String newInstanceName); + } diff --git a/helix-core/src/main/java/org/apache/helix/HelixAutoController.java b/helix-core/src/main/java/org/apache/helix/HelixAutoController.java new file mode 100644 index 0000000000..04cbf2dd0e --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/HelixAutoController.java @@ -0,0 +1,61 @@ +package org.apache.helix; + +/* + * 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.api.id.ControllerId; +import org.apache.helix.participant.StateMachineEngine; + +/** + * Autonomous controller + */ +public interface HelixAutoController extends HelixRole, HelixService, HelixConnectionStateListener { + /** + * get controller id + * @return controller id + */ + ControllerId getControllerId(); + + /** + * get state machine engine + * @return state machine engine + */ + StateMachineEngine getStateMachineEngine(); + + /** + * add pre-connect callback + * @param callback + */ + void addPreConnectCallback(PreConnectCallback callback); + + /** + * Add a LiveInstanceInfoProvider that is invoked before creating liveInstance.
+ * This allows applications to provide additional information that will be published to zookeeper + * and become available for discovery
+ * @see LiveInstanceInfoProvider#getAdditionalLiveInstanceInfo() + * @param liveInstanceInfoProvider + */ + void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider); + + /** + * tell if this controller is leader of cluster + * @return + */ + boolean isLeader(); +} diff --git a/helix-core/src/main/java/org/apache/helix/HelixConnection.java b/helix-core/src/main/java/org/apache/helix/HelixConnection.java new file mode 100644 index 0000000000..c56b01adcb --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/HelixConnection.java @@ -0,0 +1,270 @@ +package org.apache.helix; + +/* + * 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.api.accessor.ClusterAccessor; +import org.apache.helix.api.accessor.ParticipantAccessor; +import org.apache.helix.api.accessor.ResourceAccessor; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.store.HelixPropertyStore; + +/** + * Helix connection (aka helix manager) + */ +public interface HelixConnection { + + /** + * start connection + */ + void connect(); + + /** + * close connection + */ + void disconnect(); + + /** + * test if connection is started + * @return true if connection is started, false otherwise + */ + boolean isConnected(); + + /** + * get session id + * @return session id of current connection + */ + SessionId getSessionId(); + + /** + * get session timeout + * @return session timeout in millisecond + */ + int getSessionTimeout(); + + /** + * create a helix-participant + * @param clusterId + * @param participantId + * @return helix-participant + */ + HelixParticipant createParticipant(ClusterId clusterId, ParticipantId participantId); + + /** + * create a helix-controller + * @param clusterId + * @param controllerId + * @return helix-controller + */ + HelixController createController(ClusterId clusterId, ControllerId controllerId); + + /** + * create an autonomous helix-controller + * @param clusterId + * @param controllerId + * @return + */ + HelixAutoController createAutoController(ClusterId clusterId, ControllerId controllerId); + + /** + * create a cluster-accessor + * @param clusterId + * @return cluster-accessor + */ + ClusterAccessor createClusterAccessor(ClusterId clusterId); + + /** + * create a resource accessor + * @param clusterId + * @return resource accessor + */ + ResourceAccessor createResourceAccessor(ClusterId clusterId); + + /** + * create a participant accessor + * @param clusterId + * @return participant-accessor + */ + ParticipantAccessor createParticipantAccessor(ClusterId clusterId); + + /** + * Provides admin interface to setup and modify cluster + * @return instantiated HelixAdmin + */ + HelixAdmin createClusterManagementTool(); + + /** + * create a default property-store for a cluster + * @param clusterId + * @return property-store + */ + HelixPropertyStore createPropertyStore(ClusterId clusterId); + + /** + * create a data-accessor + * @param clusterId + * @return data-accessor + */ + HelixDataAccessor createDataAccessor(ClusterId clusterId); + + /** + * get config accessor + * TODO replace with new ConfigAccessor + * @return config accessor + */ + @Deprecated + ConfigAccessor getConfigAccessor(); + + /** + * add ideal state change listener + * @param role + * @param listener + * @param clusterId + */ + void addIdealStateChangeListener(HelixRole role, IdealStateChangeListener listener, + ClusterId clusterId); + + /** + * add controller message listener + * @param role + * @param listener + * @param clusterId + */ + void addControllerMessageListener(HelixRole role, MessageListener listener, ClusterId clusterId); + + /** + * add controller listener + * @param role + * @param listener + * @param clusterId + */ + void addControllerListener(HelixRole role, ControllerChangeListener listener, ClusterId clusterId); + + /** + * add live-instance listener using this connection + * @param role + * @param listener + * @param clusterId + */ + void addLiveInstanceChangeListener(HelixRole role, LiveInstanceChangeListener listener, + ClusterId clusterId); + + /** + * add message listener + * @param role + * @param listener + * @param clusterId + * @param participantId + */ + void addMessageListener(HelixRole role, MessageListener listener, ClusterId clusterId, + ParticipantId participantId); + + /** + * add config change listener + * @param role + * @param listener + * @param clusterId + */ + @Deprecated + void addConfigChangeListener(HelixRole role, ConfigChangeListener listener, ClusterId clusterId); + + /** + * add instance config change listener + * @see InstanceConfigChangeListener#onInstanceConfigChange(List, NotificationContext) + * @param role + * @param listener + * @param clusterId + */ + void addInstanceConfigChangeListener(HelixRole role, InstanceConfigChangeListener listener, + ClusterId clusterId); + + /** + * add config change listener for a scope + * @see ScopedConfigChangeListener#onConfigChange(List, NotificationContext) + * @param role + * @param listener + * @param clusterId + * @param scope + */ + void addConfigChangeListener(HelixRole role, ScopedConfigChangeListener listener, + ClusterId clusterId, ConfigScopeProperty scope); + + /** + * add current state change listener + * @param role + * @param listener + * @param clusterId + * @param participantId + * @param sessionId + */ + void addCurrentStateChangeListener(HelixRole role, CurrentStateChangeListener listener, + ClusterId clusterId, ParticipantId participantId, SessionId sessionId); + + /** + * add external view change listener + * @see ExternalViewChangeListener#onExternalViewChange(List, NotificationContext) + * @param listener + */ + void addExternalViewChangeListener(HelixRole role, ExternalViewChangeListener listener, + ClusterId clusterId); + + /** + * remove a listener + * @param role + * @param listener + * @param key + * @return + */ + boolean removeListener(HelixRole role, Object listener, PropertyKey key); + + /** + * add connection state listener + * @param listener + */ + void addConnectionStateListener(HelixConnectionStateListener listener); + + /** + * remove connection state listener + * @param listener + */ + void removeConnectionStateListener(HelixConnectionStateListener listener); + + /** + * create messasing service using this connection + * @param role + * @return messaging-service + */ + ClusterMessagingService createMessagingService(HelixRole role); + + /** + * get helix version + * @return helix version + */ + String getHelixVersion(); + + /** + * get helix properties + * @return helix-properties + */ + HelixManagerProperties getHelixProperties(); +} diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItemMBean.java b/helix-core/src/main/java/org/apache/helix/HelixConnectionStateListener.java similarity index 78% rename from helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItemMBean.java rename to helix-core/src/main/java/org/apache/helix/HelixConnectionStateListener.java index 573391d299..044a3a7c0c 100644 --- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItemMBean.java +++ b/helix-core/src/main/java/org/apache/helix/HelixConnectionStateListener.java @@ -1,4 +1,4 @@ -package org.apache.helix.monitoring.mbeans; +package org.apache.helix; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,12 +19,14 @@ * under the License. */ -public interface ClusterAlertItemMBean { - String getSensorName(); +public interface HelixConnectionStateListener { + /** + * called after connection is established + */ + void onConnected(); - double getAlertValue(); - - int getAlertFired(); - - String getAdditionalInfo(); + /** + * called before disconnect + */ + void onDisconnecting(); } diff --git a/helix-core/src/main/java/org/apache/helix/HelixConstants.java b/helix-core/src/main/java/org/apache/helix/HelixConstants.java index 6b6287c79c..b58dcbfc35 100644 --- a/helix-core/src/main/java/org/apache/helix/HelixConstants.java +++ b/helix-core/src/main/java/org/apache/helix/HelixConstants.java @@ -35,7 +35,6 @@ enum ChangeType { EXTERNAL_VIEW, CONTROLLER, MESSAGES_CONTROLLER, - HEALTH // @formatter:on } diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationType.java b/helix-core/src/main/java/org/apache/helix/HelixController.java similarity index 69% rename from helix-core/src/main/java/org/apache/helix/healthcheck/AggregationType.java rename to helix-core/src/main/java/org/apache/helix/HelixController.java index 29f59210a8..9565cfba30 100644 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationType.java +++ b/helix-core/src/main/java/org/apache/helix/HelixController.java @@ -1,4 +1,4 @@ -package org.apache.helix.healthcheck; +package org.apache.helix; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,13 +19,19 @@ * under the License. */ -public interface AggregationType { +import org.apache.helix.api.id.ControllerId; - // public abstract T merge(T iv, T ev); +public interface HelixController extends HelixRole, HelixService, HelixConnectionStateListener { - public final static String DELIM = "#"; + /** + * get controller id + * @return controller id + */ + ControllerId getControllerId(); - public abstract String merge(String incomingVal, String existingVal, long prevTimestamp); - - public abstract String getName(); + /** + * tell if this controller is leader of cluster + * @return + */ + boolean isLeader(); } diff --git a/helix-core/src/main/java/org/apache/helix/HelixDataAccessor.java b/helix-core/src/main/java/org/apache/helix/HelixDataAccessor.java index ad2c4bc99e..96e8ace1b5 100644 --- a/helix-core/src/main/java/org/apache/helix/HelixDataAccessor.java +++ b/helix-core/src/main/java/org/apache/helix/HelixDataAccessor.java @@ -47,7 +47,7 @@ public interface HelixDataAccessor { * creating the node. * @param key * @param value - * @true if the operation was successful + * @return true if the operation was successful */ boolean setProperty(PropertyKey key, T value); diff --git a/helix-core/src/main/java/org/apache/helix/HelixManager.java b/helix-core/src/main/java/org/apache/helix/HelixManager.java index 808705b3b1..eb18ddcede 100644 --- a/helix-core/src/main/java/org/apache/helix/HelixManager.java +++ b/helix-core/src/main/java/org/apache/helix/HelixManager.java @@ -22,7 +22,7 @@ import java.util.List; import org.apache.helix.controller.GenericHelixController; -import org.apache.helix.healthcheck.ParticipantHealthReportCollector; +import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; import org.apache.helix.participant.HelixStateMachineEngine; import org.apache.helix.participant.StateMachineEngine; @@ -33,13 +33,13 @@ * Class that represents the Helix Agent. * First class Object any process will interact with
* General flow
+ * *
  * manager = HelixManagerFactory.getZKHelixManager(
  *    clusterName, instanceName, ROLE, zkAddr);
  *    // ROLE can be participant, spectator or a controller
* manager.connect(); * manager.addSOMEListener(); - * manager.start() * After start is invoked the subsequent interactions will be via listener onChange callbacks * There will be 3 scenarios for onChange callback, which can be determined using NotificationContext.type * INIT -> will be invoked the first time the listener is added @@ -47,6 +47,7 @@ * FINALIZE -> will be invoked when listener is removed or session expires * manager.disconnect() *
+ * *
Default implementations available * @see HelixStateMachineEngine HelixStateMachineEngine for participant * @see RoutingTableProvider RoutingTableProvider for spectator @@ -54,7 +55,8 @@ */ public interface HelixManager { - public static final String ALLOW_PARTICIPANT_AUTO_JOIN = "ALLOW_PARTICIPANT_AUTO_JOIN"; + public static final String ALLOW_PARTICIPANT_AUTO_JOIN = + ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN; /** * Start participating in the cluster operations. All listeners will be @@ -98,6 +100,7 @@ public interface HelixManager { * @param listener * @deprecated replaced by addInstanceConfigChangeListener() */ + @Deprecated void addConfigChangeListener(ConfigChangeListener listener) throws Exception; /** @@ -130,14 +133,6 @@ void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopePro void addCurrentStateChangeListener(CurrentStateChangeListener listener, String instanceName, String sessionId) throws Exception; - /** - * @see HealthStateChangeListener#onHealthChange(String, List, NotificationContext) - * @param listener - * @param instanceName - */ - void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName) - throws Exception; - /** * @see ExternalViewChangeListener#onExternalViewChange(List, NotificationContext) * @param listener @@ -150,6 +145,12 @@ void addHealthStateChangeListener(HealthStateChangeListener listener, String ins */ void addControllerListener(ControllerChangeListener listener); + /** + * Add message listener for controller + * @param listener + */ + void addControllerMessageListener(MessageListener listener); + /** * Removes the listener. If the same listener was used for multiple changes, * all change notifications will be removed.
@@ -224,13 +225,6 @@ void addHealthStateChangeListener(HealthStateChangeListener listener, String ins */ ClusterMessagingService getMessagingService(); - /** - * Participant only component that periodically update participant health - * report to cluster manager server. - * @return ParticipantHealthReportCollector - */ - ParticipantHealthReportCollector getHealthReportCollector(); - /** * Get cluster manager instance type * @return instance type (e.g. PARTICIPANT, CONTROLLER, SPECTATOR) diff --git a/helix-core/src/main/java/org/apache/helix/HelixManagerProperties.java b/helix-core/src/main/java/org/apache/helix/HelixManagerProperties.java index 2318d205f2..aca2f30bea 100644 --- a/helix-core/src/main/java/org/apache/helix/HelixManagerProperties.java +++ b/helix-core/src/main/java/org/apache/helix/HelixManagerProperties.java @@ -96,7 +96,7 @@ public String getProperty(String key) { */ static boolean versionNoLessThan(String version1, String version2) { if (version1 == null || version2 == null) { - LOG.warn("fail to compare versions. version1: " + version1 + ", version2: " + version2); + LOG.warn("Skip null version check. version1: " + version1 + ", version2: " + version2); return true; } @@ -105,7 +105,8 @@ static boolean versionNoLessThan(String version1, String version2) { if (version1Splits == null || version1Splits.length == 0 || version2Splits == null || version2Splits.length == 0) { - LOG.warn("fail to compare versions. version1: " + version1 + ", version2: " + version2); + LOG.warn("Skip empty version check. version1: " + version1 + ", version2: " + version2); + return true; } for (int i = 0; i < version1Splits.length && i < version2Splits.length; i++) { @@ -115,9 +116,13 @@ static boolean versionNoLessThan(String version1, String version2) { if (versionNum1 < versionNum2) { return false; + } else if (versionNum1 > versionNum2) { + return true; } } catch (Exception e) { // ignore non-numerical strings and strings after non-numerical strings + LOG.warn("Skip non-numerical version check. version1: " + version1 + ", version2: " + + version2); break; } } diff --git a/helix-core/src/main/java/org/apache/helix/HelixParticipant.java b/helix-core/src/main/java/org/apache/helix/HelixParticipant.java new file mode 100644 index 0000000000..678da4ac88 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/HelixParticipant.java @@ -0,0 +1,56 @@ +package org.apache.helix; + +/* + * 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.api.id.ParticipantId; +import org.apache.helix.participant.StateMachineEngine; + +/** + * Helix participant + */ +public interface HelixParticipant extends HelixRole, HelixService, HelixConnectionStateListener { + /** + * get participant id + * @return participant id + */ + ParticipantId getParticipantId(); + + /** + * get state machine engine + * @return state machine engine + */ + StateMachineEngine getStateMachineEngine(); + + /** + * add pre-connect callback + * @param callback + */ + void addPreConnectCallback(PreConnectCallback callback); + + /** + * Add a LiveInstanceInfoProvider that is invoked before creating liveInstance.
+ * This allows applications to provide additional information that will be published to zookeeper + * and become available for discovery
+ * @see LiveInstanceInfoProvider#getAdditionalLiveInstanceInfo() + * @param liveInstanceInfoProvider + */ + void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider); + +} diff --git a/helix-core/src/main/java/org/apache/helix/HelixProperty.java b/helix-core/src/main/java/org/apache/helix/HelixProperty.java index 2e1923199a..00189b69ed 100644 --- a/helix-core/src/main/java/org/apache/helix/HelixProperty.java +++ b/helix-core/src/main/java/org/apache/helix/HelixProperty.java @@ -27,10 +27,15 @@ import java.util.List; import java.util.Map; +import org.apache.helix.api.config.NamespacedConfig; +import org.apache.log4j.Logger; + /** * A wrapper class for ZNRecord. Used as a base class for IdealState, CurrentState, etc. */ public class HelixProperty { + private static Logger LOG = Logger.getLogger(HelixProperty.class); + public enum HelixPropertyAttribute { BUCKET_SIZE, BATCH_MESSAGE_MODE @@ -54,6 +59,14 @@ public HelixProperty(ZNRecord record) { _record = new ZNRecord(record); } + /** + * Initialize the property by copying from another property + * @param property + */ + public HelixProperty(HelixProperty property) { + _record = new ZNRecord(property.getRecord()); + } + /** * Get the property identifier * @return the property id @@ -128,8 +141,7 @@ public static T convertToTypedInstance(Class clazz, }); return constructor.newInstance(record); } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Exception convert znrecord: " + record + " to class: " + clazz, e); } return null; @@ -217,6 +229,14 @@ public boolean getBatchMessageMode() { } } + /** + * Add namespaced configuration properties to this property + * @param namespacedConfig namespaced properties + */ + public void addNamespacedConfig(NamespacedConfig namespacedConfig) { + NamespacedConfig.addConfigToProperty(this, namespacedConfig); + } + /** * Get property validity * @return true if valid, false if invalid diff --git a/helix-core/src/main/java/org/apache/helix/HelixRole.java b/helix-core/src/main/java/org/apache/helix/HelixRole.java new file mode 100644 index 0000000000..ffcb70074a --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/HelixRole.java @@ -0,0 +1,59 @@ +package org.apache.helix; + +/* + * 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.api.id.ClusterId; +import org.apache.helix.api.id.Id; + +/** + * helix-role i.e. participant, controller, auto-controller + */ +public interface HelixRole { + /** + * get the underlying connection + * @return helix-connection + */ + HelixConnection getConnection(); + + /** + * get cluster id to which this role belongs + * @return cluster id + */ + ClusterId getClusterId(); + + /** + * get id of this helix-role + * @return id + */ + Id getId(); + + /** + * helix-role type + * @return + */ + InstanceType getType(); + + /** + * get the messaging-service + * @return messaging-service + */ + ClusterMessagingService getMessagingService(); + +} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/package-info.java b/helix-core/src/main/java/org/apache/helix/HelixService.java similarity index 78% rename from helix-core/src/main/java/org/apache/helix/healthcheck/package-info.java rename to helix-core/src/main/java/org/apache/helix/HelixService.java index f584b5b09a..a1ce0ecebf 100644 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/package-info.java +++ b/helix-core/src/main/java/org/apache/helix/HelixService.java @@ -1,3 +1,5 @@ +package org.apache.helix; + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -16,8 +18,18 @@ * specific language governing permissions and limitations * under the License. */ + /** - * Helix health check classes - * + * Operational methods of a helix role */ -package org.apache.helix.healthcheck; \ No newline at end of file +public interface HelixService { + /** + * start helix service async + */ + void startAsync(); + + /** + * stop helix service async + */ + void stopAsync(); +} diff --git a/helix-core/src/main/java/org/apache/helix/PropertyKey.java b/helix-core/src/main/java/org/apache/helix/PropertyKey.java index 0874958fad..7f9eb1cafe 100644 --- a/helix-core/src/main/java/org/apache/helix/PropertyKey.java +++ b/helix-core/src/main/java/org/apache/helix/PropertyKey.java @@ -19,16 +19,13 @@ * under the License. */ -import static org.apache.helix.PropertyType.ALERTS; -import static org.apache.helix.PropertyType.ALERT_HISTORY; -import static org.apache.helix.PropertyType.ALERT_STATUS; +import static org.apache.helix.PropertyType.CLUSTER; import static org.apache.helix.PropertyType.CONFIGS; import static org.apache.helix.PropertyType.CONTROLLER; import static org.apache.helix.PropertyType.CURRENTSTATES; import static org.apache.helix.PropertyType.ERRORS; import static org.apache.helix.PropertyType.ERRORS_CONTROLLER; import static org.apache.helix.PropertyType.EXTERNALVIEW; -import static org.apache.helix.PropertyType.HEALTHREPORT; import static org.apache.helix.PropertyType.HISTORY; import static org.apache.helix.PropertyType.IDEALSTATES; import static org.apache.helix.PropertyType.LEADER; @@ -36,29 +33,32 @@ import static org.apache.helix.PropertyType.MESSAGES; import static org.apache.helix.PropertyType.MESSAGES_CONTROLLER; import static org.apache.helix.PropertyType.PAUSE; -import static org.apache.helix.PropertyType.PERSISTENTSTATS; +import static org.apache.helix.PropertyType.PROPERTYSTORE; +import static org.apache.helix.PropertyType.RESOURCEASSIGNMENTS; import static org.apache.helix.PropertyType.STATEMODELDEFS; import static org.apache.helix.PropertyType.STATUSUPDATES; import static org.apache.helix.PropertyType.STATUSUPDATES_CONTROLLER; import java.util.Arrays; -import org.apache.helix.model.AlertHistory; -import org.apache.helix.model.AlertStatus; -import org.apache.helix.model.Alerts; +import org.apache.helix.model.AlertConfig; +import org.apache.helix.model.ClusterConfiguration; import org.apache.helix.model.ClusterConstraints; import org.apache.helix.model.CurrentState; import org.apache.helix.model.Error; import org.apache.helix.model.ExternalView; -import org.apache.helix.model.HealthStat; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; import org.apache.helix.model.IdealState; import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.Leader; import org.apache.helix.model.LeaderHistory; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; +import org.apache.helix.model.MonitoringConfig; +import org.apache.helix.model.PartitionConfiguration; import org.apache.helix.model.PauseSignal; -import org.apache.helix.model.PersistentStats; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.ResourceConfiguration; import org.apache.helix.model.StateModelDefinition; import org.apache.helix.model.StatusUpdate; import org.apache.log4j.Logger; @@ -143,6 +143,14 @@ public Builder(String clusterName) { _clusterName = clusterName; } + /** + * Get the key for the root node + * @return cluster node + */ + public PropertyKey cluster() { + return new PropertyKey(CLUSTER, null, _clusterName); + } + /** * Get a property key associated with {@link IdealState} * @return {@link PropertyKey} @@ -160,6 +168,25 @@ public PropertyKey idealStates(String resourceName) { return new PropertyKey(IDEALSTATES, IdealState.class, _clusterName, resourceName); } + /** + * Get a property key associated with all {@link ResourceAssignment}s on the cluster + * @return {@link PropertyKey} + */ + public PropertyKey resourceAssignments() { + return new PropertyKey(RESOURCEASSIGNMENTS, ResourceAssignment.class, _clusterName); + } + + /** + * Get a property key associated with {@link ResourceAssignment} representing the most recent + * assignment + * @param resourceName name of the resource + * @return {@link PropertyKey} + */ + public PropertyKey resourceAssignment(String resourceName) { + return new PropertyKey(RESOURCEASSIGNMENTS, ResourceAssignment.class, _clusterName, + resourceName); + } + /** * Get a property key associated with {@link StateModelDefinition} * @return {@link PropertyKey} @@ -184,7 +211,7 @@ public PropertyKey stateModelDef(String stateModelName) { */ public PropertyKey clusterConfigs() { - return new PropertyKey(CONFIGS, ConfigScopeProperty.CLUSTER, HelixProperty.class, + return new PropertyKey(CONFIGS, ConfigScopeProperty.CLUSTER, ClusterConfiguration.class, _clusterName, ConfigScopeProperty.CLUSTER.toString()); } @@ -193,7 +220,7 @@ public PropertyKey clusterConfigs() { * @return {@link PropertyKey} */ public PropertyKey clusterConfig() { - return new PropertyKey(CONFIGS, ConfigScopeProperty.CLUSTER, HelixProperty.class, + return new PropertyKey(CONFIGS, ConfigScopeProperty.CLUSTER, ClusterConfiguration.class, _clusterName, ConfigScopeProperty.CLUSTER.toString(), _clusterName); } @@ -221,7 +248,7 @@ public PropertyKey instanceConfig(String instanceName) { * @return {@link PropertyKey} */ public PropertyKey resourceConfigs() { - return new PropertyKey(CONFIGS, ConfigScopeProperty.RESOURCE, HelixProperty.class, + return new PropertyKey(CONFIGS, ConfigScopeProperty.RESOURCE, ResourceConfiguration.class, _clusterName, ConfigScopeProperty.RESOURCE.toString()); } @@ -231,19 +258,29 @@ public PropertyKey resourceConfigs() { * @return {@link PropertyKey} */ public PropertyKey resourceConfig(String resourceName) { - return new PropertyKey(CONFIGS, ConfigScopeProperty.RESOURCE, HelixProperty.class, + return new PropertyKey(CONFIGS, ConfigScopeProperty.RESOURCE, ResourceConfiguration.class, _clusterName, ConfigScopeProperty.RESOURCE.toString(), resourceName); } /** - * Get a property key associated with a partition + * Get a property key associated with all partition configurations + * @param resourceName + * @return {@link PropertyKey} + */ + public PropertyKey partitionConfigs(String resourceName) { + return new PropertyKey(CONFIGS, ConfigScopeProperty.RESOURCE, PartitionConfiguration.class, + _clusterName, ConfigScopeProperty.RESOURCE.toString(), resourceName); + } + + /** + * Get a property key associated with a partition configuration * @param resourceName * @param partitionName * @return {@link PropertyKey} */ public PropertyKey partitionConfig(String resourceName, String partitionName) { - return new PropertyKey(CONFIGS, ConfigScopeProperty.RESOURCE, HelixProperty.class, - _clusterName, ConfigScopeProperty.RESOURCE.toString(), resourceName); + return new PropertyKey(CONFIGS, ConfigScopeProperty.RESOURCE, PartitionConfiguration.class, + _clusterName, ConfigScopeProperty.RESOURCE.toString(), resourceName, partitionName); } /** @@ -255,8 +292,8 @@ public PropertyKey partitionConfig(String resourceName, String partitionName) { */ public PropertyKey partitionConfig(String instanceName, String resourceName, String partitionName) { - return new PropertyKey(CONFIGS, ConfigScopeProperty.RESOURCE, HelixProperty.class, - _clusterName, ConfigScopeProperty.RESOURCE.toString(), resourceName); + return new PropertyKey(CONFIGS, ConfigScopeProperty.RESOURCE, PartitionConfiguration.class, + _clusterName, ConfigScopeProperty.RESOURCE.toString(), resourceName, partitionName); } /** @@ -273,12 +310,49 @@ public PropertyKey constraints() { * @param constraintType * @return {@link PropertyKey} */ - public PropertyKey constraint(String constraintType) { return new PropertyKey(CONFIGS, ClusterConstraints.class, _clusterName, ConfigScopeProperty.CONSTRAINT.toString(), constraintType); } + /** + * Get a property key associated with all {@link MonitoringConfig} + * @return {@link PropertyKey} + */ + public PropertyKey monitoringConfigs() { + return new PropertyKey(CONFIGS, ConfigScopeProperty.MONITORING, MonitoringConfig.class, + _clusterName, ConfigScopeProperty.MONITORING.toString()); + } + + /** + * Get a property key associated with a single {@link MonitoringConfig} + * @param monitoringConfigName name of the configuration + * @return {@link PropertyKey} + */ + public PropertyKey monitoringConfig(String monitoringConfigName) { + return new PropertyKey(CONFIGS, ConfigScopeProperty.MONITORING, MonitoringConfig.class, + _clusterName, ConfigScopeProperty.MONITORING.toString(), monitoringConfigName); + } + + /** + * Get a property key associated with all {@link AlertConfig} + * @return {@link PropertyKey} + */ + public PropertyKey alertConfigs() { + return new PropertyKey(CONFIGS, ConfigScopeProperty.ALERT, AlertConfig.class, + _clusterName, ConfigScopeProperty.ALERT.name()); + } + + /** + * Get a property key associated with a single {@link AlertConfig} + * @param alertConfigName name of the configuration + * @return {@link PropertyKey} + */ + public PropertyKey alertConfig(String alertConfigName) { + return new PropertyKey(CONFIGS, ConfigScopeProperty.ALERT, AlertConfig.class, + _clusterName, ConfigScopeProperty.ALERT.name(), alertConfigName); + } + /** * Get a property key associated with {@link LiveInstance} * @return {@link PropertyKey} @@ -304,6 +378,14 @@ public PropertyKey instances() { return new PropertyKey(PropertyType.INSTANCES, null, _clusterName); } + /** + * Get a property key associated with all instances + * @return {@link PropertyKey} + */ + public PropertyKey instance(String instanceName) { + return new PropertyKey(PropertyType.INSTANCES, null, _clusterName, instanceName); + } + /** * Get a property key associated with {@link Message} for an instance * @param instanceName @@ -332,6 +414,16 @@ public PropertyKey sessions(String instanceName) { return new PropertyKey(CURRENTSTATES, CurrentState.class, _clusterName, instanceName); } + /** + * Get a property key associated with {@link CurrentState} of an instance and session + * @param instanceName + * @param sessionId + * @return {@link PropertyKey} + */ + public PropertyKey currentStates(String instanceName) { + return new PropertyKey(CURRENTSTATES, CurrentState.class, _clusterName, instanceName); + } + /** * Get a property key associated with {@link CurrentState} of an instance and session * @param instanceName @@ -377,6 +469,15 @@ public PropertyKey currentState(String instanceName, String sessionId, String re } } + /** + * Get a property key representing the root of all status updates + * @param instanceName the participant the status updates belong to + * @return {@link PropertyKey} + */ + public PropertyKey statusUpdates(String instanceName) { + return new PropertyKey(STATUSUPDATES, null, _clusterName, instanceName); + } + /** * Get a property key associated with {@link StatusUpdate} of an instance, session, resource, * and partition @@ -431,6 +532,15 @@ public PropertyKey taskStatus(String instanceName, String sessionId, String msgT sessionId, msgType, msgId); } + /** + * Get a property key representing the root of all persisted participant errors + * @param instanceName the participant of interest + * @return {@link PropertyKey} + */ + public PropertyKey participantErrors(String instanceName) { + return new PropertyKey(ERRORS, null, _clusterName, instanceName); + } + /** * Get a property key associated with {@link Error} of an instance, session, resource, * and partition @@ -514,6 +624,14 @@ public PropertyKey controllerTaskError(String errorId) { return new PropertyKey(ERRORS_CONTROLLER, Error.class, _clusterName, errorId); } + /** + * Get the root of all controller status updates + * @return {@link PropertyKey} + */ + public PropertyKey controllerTaskStatuses() { + return new PropertyKey(STATUSUPDATES_CONTROLLER, StatusUpdate.class, _clusterName); + } + /** * Get a property key associated with {@link StatusUpdate} of controller status updates * @param subPath @@ -560,11 +678,11 @@ public PropertyKey controllerLeaderHistory() { } /** - * Get a property key associated with a {@link LiveInstance} leader + * Get a property key associated with a {@link Leader} leader * @return {@link PropertyKey} */ public PropertyKey controllerLeader() { - return new PropertyKey(LEADER, LiveInstance.class, _clusterName); + return new PropertyKey(LEADER, Leader.class, _clusterName); } /** @@ -576,56 +694,12 @@ public PropertyKey pause() { } /** - * Get a property key associated with {@link PersistentStats} - * @return {@link PropertyKey} - */ - public PropertyKey persistantStat() { - return new PropertyKey(PERSISTENTSTATS, PersistentStats.class, _clusterName); - } - - /** - * Get a property key associated with {@link Alerts} - * @return {@link PropertyKey} - */ - public PropertyKey alerts() { - return new PropertyKey(ALERTS, Alerts.class, _clusterName); - } - - /** - * Get a property key associated with {@link AlertStatus} - * @return {@link PropertyKey} + * Get a propertykey associated with the root of the Helix property store + * @return {@link PropertyStore} */ - public PropertyKey alertStatus() { - return new PropertyKey(ALERT_STATUS, AlertStatus.class, _clusterName); + public PropertyKey propertyStore() { + return new PropertyKey(PROPERTYSTORE, null, _clusterName); } - - /** - * Get a property key associated with {@link AlertHistory} - * @return {@link PropertyKey} - */ - public PropertyKey alertHistory() { - return new PropertyKey(ALERT_HISTORY, AlertHistory.class, _clusterName); - } - - /** - * Get a property key associated with a {@link HealthStat} for an instance - * @param instanceName - * @param id identifies the statistics - * @return {@link PropertyKey} - */ - public PropertyKey healthReport(String instanceName, String id) { - return new PropertyKey(HEALTHREPORT, HealthStat.class, _clusterName, instanceName, id); - } - - /** - * Get a property key associated with {@link HealthStat}s for an instance - * @param instanceName - * @return {@link PropertyKey} - */ - public PropertyKey healthReports(String instanceName) { - return new PropertyKey(HEALTHREPORT, HealthStat.class, _clusterName, instanceName); - } - } /** diff --git a/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java b/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java index 96e186d86e..97cbbb6c10 100644 --- a/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java +++ b/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java @@ -19,17 +19,15 @@ * under the License. */ -import static org.apache.helix.PropertyType.ALERTS; -import static org.apache.helix.PropertyType.ALERT_STATUS; import static org.apache.helix.PropertyType.CONFIGS; import static org.apache.helix.PropertyType.CURRENTSTATES; import static org.apache.helix.PropertyType.EXTERNALVIEW; -import static org.apache.helix.PropertyType.HEALTHREPORT; import static org.apache.helix.PropertyType.HISTORY; import static org.apache.helix.PropertyType.IDEALSTATES; import static org.apache.helix.PropertyType.LIVEINSTANCES; import static org.apache.helix.PropertyType.MESSAGES; import static org.apache.helix.PropertyType.PAUSE; +import static org.apache.helix.PropertyType.RESOURCEASSIGNMENTS; import static org.apache.helix.PropertyType.STATEMODELDEFS; import static org.apache.helix.PropertyType.STATUSUPDATES; @@ -39,17 +37,15 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.helix.model.AlertStatus; -import org.apache.helix.model.Alerts; import org.apache.helix.model.CurrentState; import org.apache.helix.model.ExternalView; -import org.apache.helix.model.HealthStat; import org.apache.helix.model.IdealState; import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.LeaderHistory; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; import org.apache.helix.model.PauseSignal; +import org.apache.helix.model.ResourceAssignment; import org.apache.helix.model.StateModelDefinition; import org.apache.helix.model.StatusUpdate; import org.apache.log4j.Logger; @@ -74,15 +70,15 @@ public class PropertyPathConfig { typeToClassMapping.put(CURRENTSTATES, CurrentState.class); typeToClassMapping.put(STATUSUPDATES, StatusUpdate.class); typeToClassMapping.put(HISTORY, LeaderHistory.class); - typeToClassMapping.put(HEALTHREPORT, HealthStat.class); - typeToClassMapping.put(ALERTS, Alerts.class); - typeToClassMapping.put(ALERT_STATUS, AlertStatus.class); typeToClassMapping.put(PAUSE, PauseSignal.class); + typeToClassMapping.put(RESOURCEASSIGNMENTS, ResourceAssignment.class); // @formatter:off + addEntry(PropertyType.CLUSTER, 1, "/{clusterName}"); addEntry(PropertyType.CONFIGS, 1, "/{clusterName}/CONFIGS"); addEntry(PropertyType.CONFIGS, 2, "/{clusterName}/CONFIGS/{scope}"); addEntry(PropertyType.CONFIGS, 3, "/{clusterName}/CONFIGS/{scope}/{scopeKey}"); + addEntry(PropertyType.CONFIGS, 4, "/{clusterName}/CONFIGS/{scope}/{scopeKey}/{subScopeKey}"); // addEntry(PropertyType.CONFIGS,2,"/{clusterName}/CONFIGS/{instanceName}"); addEntry(PropertyType.LIVEINSTANCES, 1, "/{clusterName}/LIVEINSTANCES"); addEntry(PropertyType.LIVEINSTANCES, 2, "/{clusterName}/LIVEINSTANCES/{instanceName}"); @@ -90,6 +86,9 @@ public class PropertyPathConfig { addEntry(PropertyType.INSTANCES, 2, "/{clusterName}/INSTANCES/{instanceName}"); addEntry(PropertyType.IDEALSTATES, 1, "/{clusterName}/IDEALSTATES"); addEntry(PropertyType.IDEALSTATES, 2, "/{clusterName}/IDEALSTATES/{resourceName}"); + addEntry(PropertyType.RESOURCEASSIGNMENTS, 1, "/{clusterName}/RESOURCEASSIGNMENTS"); + addEntry(PropertyType.RESOURCEASSIGNMENTS, 2, + "/{clusterName}/RESOURCEASSIGNMENTS/{resourceName}"); addEntry(PropertyType.EXTERNALVIEW, 1, "/{clusterName}/EXTERNALVIEW"); addEntry(PropertyType.EXTERNALVIEW, 2, "/{clusterName}/EXTERNALVIEW/{resourceName}"); addEntry(PropertyType.STATEMODELDEFS, 1, "/{clusterName}/STATEMODELDEFS"); diff --git a/helix-core/src/main/java/org/apache/helix/PropertyType.java b/helix-core/src/main/java/org/apache/helix/PropertyType.java index f6145ed1b0..680dc06c2f 100644 --- a/helix-core/src/main/java/org/apache/helix/PropertyType.java +++ b/helix-core/src/main/java/org/apache/helix/PropertyType.java @@ -36,10 +36,12 @@ public enum PropertyType { // @formatter:off // CLUSTER PROPERTIES + CLUSTER(Type.CLUSTER, true, false, false, true, true), CONFIGS(Type.CLUSTER, true, false, false, false, true), LIVEINSTANCES(Type.CLUSTER, false, false, false, true, true), INSTANCES(Type.CLUSTER, true, false), IDEALSTATES(Type.CLUSTER, true, false, false, false, true), + RESOURCEASSIGNMENTS(Type.CLUSTER, true, false), EXTERNALVIEW(Type.CLUSTER, true, false), STATEMODELDEFS(Type.CLUSTER, true, false, false, false, true), CONTROLLER(Type.CLUSTER, true, false), diff --git a/helix-core/src/main/java/org/apache/helix/ZNRecord.java b/helix-core/src/main/java/org/apache/helix/ZNRecord.java index 56a6cf2fd4..37cd5eb41b 100644 --- a/helix-core/src/main/java/org/apache/helix/ZNRecord.java +++ b/helix-core/src/main/java/org/apache/helix/ZNRecord.java @@ -120,6 +120,15 @@ public void setPayloadSerializer(PayloadSerializer serializer) { _serializer = serializer; } + /** + * Get the {@link PayloadSerializer} that will serialize/deserialize the payload + * @return serializer + */ + @JsonIgnore(true) + public PayloadSerializer getPayloadSerializer() { + return _serializer; + } + /** * Set the list of updates to this ZNRecord * @param deltaList diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AccumulateAggregator.java b/helix-core/src/main/java/org/apache/helix/alerts/AccumulateAggregator.java deleted file mode 100644 index 2967f0510f..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/AccumulateAggregator.java +++ /dev/null @@ -1,67 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.HelixException; - -public class AccumulateAggregator extends Aggregator { - - public AccumulateAggregator() { - _numArgs = 0; - } - - @Override - public void merge(Tuple currValTup, Tuple newValTup, Tuple currTimeTup, - Tuple newTimeTup, String... args) { - - double currVal = 0; - double currTime = -1; - double newVal; - double newTime; - double mergedVal; - double mergedTime; - - if (currValTup == null || newValTup == null || currTimeTup == null || newTimeTup == null) { - throw new HelixException("Tuples cannot be null"); - } - - // old tuples may be empty, indicating no value/time exist - if (currValTup.size() > 0 && currTimeTup.size() > 0) { - currVal = Double.parseDouble(currValTup.iterator().next()); - currTime = Double.parseDouble(currTimeTup.iterator().next()); - } - newVal = Double.parseDouble(newValTup.iterator().next()); - newTime = Double.parseDouble(newTimeTup.iterator().next()); - - if (newTime > currTime) { // if old doesn't exist, we end up here - mergedVal = currVal + newVal; // if old doesn't exist, it has value "0" - mergedTime = newTime; - } else { - mergedVal = currVal; - mergedTime = currTime; - } - - currValTup.clear(); - currValTup.add(Double.toString(mergedVal)); - currTimeTup.clear(); - currTimeTup.add(Double.toString(mergedTime)); - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertParser.java b/helix-core/src/main/java/org/apache/helix/alerts/AlertParser.java deleted file mode 100644 index ceb4d2e455..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/AlertParser.java +++ /dev/null @@ -1,138 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.apache.helix.HelixException; -import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory.ActionOnError; -import org.apache.log4j.Logger; - -public class AlertParser { - private static Logger logger = Logger.getLogger(AlertParser.class); - - public static final String EXPRESSION_NAME = "EXP"; - public static final String COMPARATOR_NAME = "CMP"; - public static final String CONSTANT_NAME = "CON"; - public static final String ACTION_NAME = "ACTION"; - - static Map comparatorMap = new HashMap(); - - static { - - addComparatorEntry("GREATER", new GreaterAlertComparator()); - } - - private static void addComparatorEntry(String label, AlertComparator comp) { - if (!comparatorMap.containsKey(label)) { - comparatorMap.put(label, comp); - } - logger.info("Adding comparator: " + comp); - } - - public static AlertComparator getComparator(String compName) { - compName = compName.replaceAll("\\s+", ""); // remove white space - if (!comparatorMap.containsKey(compName)) { - throw new HelixException("Comparator type <" + compName + "> unknown"); - } - return comparatorMap.get(compName); - } - - public static String getComponent(String component, String alert) throws HelixException { - // find EXP and keep going until paren are closed - int expStartPos = alert.indexOf(component); - if (expStartPos < 0) { - throw new HelixException(alert + " does not contain component " + component); - } - expStartPos += (component.length() + 1); // advance length of string and one for open paren - int expEndPos = expStartPos; - int openParenCount = 1; - while (openParenCount > 0) { - if (alert.charAt(expEndPos) == '(') { - openParenCount++; - } else if (alert.charAt(expEndPos) == ')') { - openParenCount--; - } - expEndPos++; - } - if (openParenCount != 0) { - throw new HelixException(alert + " does not contain valid " + component + " component, " - + "parentheses do not close"); - } - // return what is in between paren - return alert.substring(expStartPos, expEndPos - 1); - } - - public static boolean validateAlert(String alert) throws HelixException { - // TODO: decide if toUpperCase is going to cause problems with stuff like db name - alert = alert.replaceAll("\\s+", ""); // remove white space - String exp = getComponent(EXPRESSION_NAME, alert); - String cmp = getComponent(COMPARATOR_NAME, alert); - String val = getComponent(CONSTANT_NAME, alert); - logger.debug("exp: " + exp); - logger.debug("cmp: " + cmp); - logger.debug("val: " + val); - - // separately validate each portion - ExpressionParser.validateExpression(exp); - - // validate comparator - if (!comparatorMap.containsKey(cmp.toUpperCase())) { - throw new HelixException("Unknown comparator type " + cmp); - } - String actionValue = null; - try { - actionValue = AlertParser.getComponent(AlertParser.ACTION_NAME, alert); - } catch (Exception e) { - logger.info("No action specified in " + alert); - } - - if (actionValue != null) { - validateActionValue(actionValue); - } - // ValParser. Probably don't need this. Just make sure it's a valid tuple. But would also be - // good - // to validate that the tuple is same length as exp's output...maybe leave that as future todo - // not sure we can really do much here though...anything can be in a tuple. - - // TODO: try to compare tuple width of CON against tuple width of agg type! Not a good idea, - // what if - // is not at full width yet, like with window - - // if all of this passes, then we can safely record the alert in zk. still need to implement zk - // location - - return false; - } - - public static void validateActionValue(String actionValue) { - try { - ActionOnError actionVal = ActionOnError.valueOf(actionValue); - } catch (Exception e) { - String validActions = ""; - for (ActionOnError action : ActionOnError.values()) { - validActions = validActions + action + " "; - } - throw new HelixException("Unknown cmd type " + actionValue + ", valid types : " - + validActions); - } - } -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertProcessor.java b/helix-core/src/main/java/org/apache/helix/alerts/AlertProcessor.java deleted file mode 100644 index 2bb1976425..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/AlertProcessor.java +++ /dev/null @@ -1,311 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.ListIterator; -import java.util.Map; -import java.util.Set; - -import org.apache.helix.HelixException; -import org.apache.helix.healthcheck.StatHealthReportProvider; -import org.apache.log4j.Logger; - -public class AlertProcessor { - private static Logger logger = Logger.getLogger(AlertProcessor.class); - - private static final String bindingDelim = ","; - public static final String noWildcardAlertKey = "*"; - - StatsHolder _statsHolder; - - // AlertsHolder _alertsHolder; - - /* - * public AlertProcessor(StatHealthReportProvider statProvider) { - * } - */ - - public AlertProcessor(StatsHolder sh) { - _statsHolder = sh; - } - - public static Map>> initAlertStatTuples(Alert alert) { - // get the stats out of the alert - String[] alertStats = ExpressionParser.getBaseStats(alert.getExpression()); - // init a tuple list for each alert stat - Map>> alertStatTuples = new HashMap>>(); - for (String currAlertStat : alertStats) { - List> currList = new ArrayList>(); - alertStatTuples.put(currAlertStat, currList); - } - return alertStatTuples; - } - - /* - * //this function is all messed up!!! public static void - * populateAlertStatTuples(Map>> tupleLists, - * List persistentStats) { Set alertStatNames = - * tupleLists.keySet(); for (Stat persistentStat : persistentStats) { //ignore - * stats with wildcards, they don't have values...they are just there to catch - * new actual stats if - * (ExpressionParser.statContainsWildcards(persistentStat.getName())) { - * continue; } Iterator alertStatIter = alertStatNames.iterator(); - * while (alertStatIter.hasNext()) { String currAlertStat = - * alertStatIter.next(); if - * (ExpressionParser.isAlertStatExactMatch(currAlertStat, - * persistentStat.getName()) || - * ExpressionParser.isAlertStatWildcardMatch(currAlertStat, - * persistentStat.getName())) { - * tupleLists.get(currAlertStat).add(persistentStat.getValue()); } } } } - */ - - public static String formAlertKey(ArrayList bindings) { - if (bindings.size() == 0) { - return null; - } - StringBuilder alertKey = new StringBuilder(); - boolean emptyKey = true; - for (String binding : bindings) { - if (!emptyKey) { - alertKey.append(bindingDelim); - } - alertKey.append(binding); - emptyKey = false; - } - return alertKey.toString(); - } - - // XXX: major change here. return ArrayList of Stats instead of ArrayList of - // Tuple's - public static Map>> populateAlertStatTuples(String[] alertStats, - List persistentStats) { - Map>> tupleSets = - new HashMap>>(); - - // check each persistentStat, alertStat pair - for (Stat persistentStat : persistentStats) { - // ignore stats with wildcards, they don't have values...they are just - // there to catch new actual stats - if (ExpressionParser.statContainsWildcards(persistentStat.getName())) { - continue; - } - for (int i = 0; i < alertStats.length; i++) { - String alertStat = alertStats[i]; - ArrayList wildcardBindings = new ArrayList(); - // if match, then proceed. If the match is wildcard, additionally fill - // in the wildcard bindings - if (ExpressionParser.isAlertStatExactMatch(alertStat, persistentStat.getName()) - || ExpressionParser.isAlertStatWildcardMatch(alertStat, persistentStat.getName(), - wildcardBindings)) { - String alertKey; - if (wildcardBindings.size() == 0) { - alertKey = noWildcardAlertKey; - } else { - alertKey = formAlertKey(wildcardBindings); - } - if (!tupleSets.containsKey(alertKey)) { // don't have an entry for alertKey yet, create - // one - ArrayList> tuples = new ArrayList>(alertStats.length); - for (int j = 0; j < alertStats.length; j++) { // init all entries to null - tuples.add(j, null); - } - tupleSets.put(alertKey, tuples); // add to map - } - tupleSets.get(alertKey).set(i, persistentStat.getValue()); - } - } - } - - // post-processing step to discard any rows with null vals... - // TODO: decide if this is best thing to do with incomplete rows - List selectedKeysToRemove = new ArrayList(); - for (String setKey : tupleSets.keySet()) { - ArrayList> tupleSet = tupleSets.get(setKey); - for (Tuple tup : tupleSet) { - if (tup == null) { - selectedKeysToRemove.add(setKey); - break; // move on to next setKey - } - } - } - for (String keyToRemove : selectedKeysToRemove) { - tupleSets.remove(keyToRemove); - } - - // convert above to a series of iterators - - return tupleSets; - } - - public static List>> convertTupleRowsToTupleColumns( - Map>> tupleMap) { - // input is a map of key -> list of tuples. each tuple list is same length - // output should be a list of iterators. each column in input becomes - // iterator in output - - ArrayList>> columns = new ArrayList>>(); - ArrayList>> columnIters = new ArrayList>>(); - for (String currStat : tupleMap.keySet()) { - List> currSet = tupleMap.get(currStat); - for (int i = 0; i < currSet.size(); i++) { - if (columns.size() < (i + 1)) { - ArrayList> col = new ArrayList>(); - columns.add(col); - } - columns.get(i).add(currSet.get(i)); - } - } - for (ArrayList> al : columns) { - columnIters.add(al.iterator()); - } - return columnIters; - - } - - public static Iterator> executeOperatorPipeline( - List>> tupleIters, String[] operators) { - List>> nextIters = tupleIters; - if (operators != null) { - for (String opName : operators) { - Operator op = ExpressionParser.getOperator(opName); - nextIters = op.execute(nextIters); - } - } - - if (nextIters.size() != 1) { - throw new HelixException("operator pipeline produced " + nextIters.size() - + " tuple sets instead of exactly 1"); - } - - return nextIters.get(0); - } - - /* - * TODO: consider returning actual values, rather than bools. Could just - * return the triggered alerts - */ - public static ArrayList executeComparator(Iterator> tuples, - String comparatorName, Tuple constant) { - ArrayList results = new ArrayList(); - AlertComparator cmp = AlertParser.getComparator(comparatorName); - - while (tuples.hasNext()) { - Tuple currTup = tuples.next(); - boolean fired = cmp.evaluate(currTup, constant); - results.add(new AlertValueAndStatus(currTup, fired)); - // results.add(cmp.evaluate(currTup, constant)); - } - return results; - - } - - /* - * public static void executeAlert(Alert alert, List stats) { //init - * tuple lists and populate them Map>> - * alertStatTupleSets = initAlertStatTuples(alert); - * populateAlertStatTuples(alertStatTupleSets, stats); //TODO: not sure I am - * being careful enough with sticking stats that match each other in this - * list! //convert to operator friendly format List>> - * tupleIters = convertTupleSetsToTupleIterators(alertStatTupleSets); //get - * the operators String[] operators = - * ExpressionParser.getOperators(alert.getExpression()); //do operator - * pipeline Iterator> opResultTuples = - * executeOperatorPipeline(tupleIters, operators); //execute comparator for - * tuple list ArrayList evalResults = - * executeComparator(opResultTuples, alert.getComparator(), - * alert.getConstant()); - * //TODO: convey this back to execute all - * } - */ - - public static HashMap generateResultMap( - Set alertStatBindings, ArrayList evalResults) { - HashMap resultMap = new HashMap(); - Iterator bindingIter = alertStatBindings.iterator(); - Iterator resultIter = evalResults.iterator(); - if (alertStatBindings.size() != evalResults.size()) { - // can't match up alerts bindings to results - while (resultIter.hasNext()) { - resultMap.put(noWildcardAlertKey, resultIter.next()); - } - } else { - // they do match up - while (resultIter.hasNext()) { - resultMap.put(bindingIter.next(), resultIter.next()); - } - } - return resultMap; - } - - public static HashMap executeAlert(Alert alert, - List persistedStats) { - // init tuple lists and populate them - // Map>> alertStatTupleSets = - // initAlertStatTuples(alert); - - String[] alertStats = ExpressionParser.getBaseStats(alert.getExpression()); - - Map>> alertsToTupleRows = - populateAlertStatTuples(alertStats, persistedStats); - - if (alertsToTupleRows.size() == 0) { - return null; - } - // convert to operator friendly format - List>> tupleIters = convertTupleRowsToTupleColumns(alertsToTupleRows); - // get the operators - String[] operators = ExpressionParser.getOperators(alert.getExpression()); - // do operator pipeline - Iterator> opResultTuples = executeOperatorPipeline(tupleIters, operators); - // execute comparator for tuple list - ArrayList evalResults = - executeComparator(opResultTuples, alert.getComparator(), alert.getConstant()); - - // stitch alert bindings back together with final result - // XXX: there is a non-critical bug here. if we have an aggregating - // operator, but that operator only takes one input, - // we bind to original wildcard binding, instead of to "*" - - HashMap alertBindingsToResult = - generateResultMap(alertsToTupleRows.keySet(), evalResults); - - return alertBindingsToResult; - - } - - public static Map> executeAllAlerts(List alerts, - List stats) { - Map> alertsResults = - new HashMap>(); - - for (Alert alert : alerts) { - HashMap result = executeAlert(alert, stats); - // TODO: decide if sticking null results in here is ok - alertsResults.put(alert.getName(), result); - } - - return alertsResults; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertsHolder.java b/helix-core/src/main/java/org/apache/helix/alerts/AlertsHolder.java deleted file mode 100644 index 8bfaae6677..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/AlertsHolder.java +++ /dev/null @@ -1,264 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixException; -import org.apache.helix.HelixManager; -import org.apache.helix.HelixProperty; -import org.apache.helix.PropertyKey; -import org.apache.helix.PropertyType; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.stages.HealthDataCache; -import org.apache.helix.model.AlertStatus; -import org.apache.helix.model.Alerts; -import org.apache.log4j.Logger; - -public class AlertsHolder { - - private static final Logger logger = Logger.getLogger(AlertsHolder.class.getName()); - - HelixDataAccessor _accessor; - HealthDataCache _cache; - Map> _alertsMap; // not sure if map or set yet - Map> _alertStatusMap; - // Alerts _alerts; - HashSet alerts; - StatsHolder _statsHolder; - - private final HelixManager _manager; - - private Builder _keyBuilder; - - public AlertsHolder(HelixManager manager, HealthDataCache cache) { - this(manager, cache, new StatsHolder(manager, cache)); - } - - public AlertsHolder(HelixManager manager, HealthDataCache cache, StatsHolder statHolder) { - _manager = manager; - _accessor = manager.getHelixDataAccessor(); - _cache = cache; - _statsHolder = statHolder; - _keyBuilder = new PropertyKey.Builder(_manager.getClusterName()); - updateCache(_cache); - } - - public void refreshAlerts() { - _cache.refresh(_accessor); - updateCache(_cache); - - /* - * _alertsMap = _cache.getAlerts(); - * //TODO: confirm this a good place to init the _statMap when null - * if (_alertsMap == null) { - * _alertsMap = new HashMap>(); - * }\ - */ - } - - public void refreshAlertStatus() { - AlertStatus alertStatusRecord = _cache.getAlertStatus(); - if (alertStatusRecord != null) { - _alertStatusMap = alertStatusRecord.getMapFields(); - } else { - _alertStatusMap = new HashMap>(); - } - } - - public void persistAlerts() { - // XXX: Am I using _accessor too directly here? - - Alerts alerts = _accessor.getProperty(_keyBuilder.alerts()); - if (alerts == null) { - alerts = new Alerts(Alerts.nodeName); // TODO: fix naming of this record, if it matters - } - alerts.getRecord().setMapFields(_alertsMap); - boolean retVal = _accessor.setProperty(_keyBuilder.alerts(), alerts); - logger.debug("persistAlerts retVal: " + retVal); - } - - public void persistAlertStatus() { - // XXX: Am I using _accessor too directly here? - AlertStatus alertStatus = _accessor.getProperty(_keyBuilder.alertStatus()); - if (alertStatus == null) { - alertStatus = new AlertStatus(AlertStatus.nodeName); // TODO: fix naming of this record, if it - // matters - } - alertStatus.getRecord().setMapFields(_alertStatusMap); - boolean retVal = _accessor.setProperty(_keyBuilder.alertStatus(), alertStatus); - logger.debug("persistAlerts retVal: " + retVal); - } - - // read alerts from cm state - private void readExistingAlerts() { - - } - - public void addAlert(String alert) throws HelixException { - alert = alert.replaceAll("\\s+", ""); // remove white space - AlertParser.validateAlert(alert); - refreshAlerts(); - // stick the 3 alert fields in map - Map alertFields = new HashMap(); - alertFields.put(AlertParser.EXPRESSION_NAME, - AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert)); - alertFields.put(AlertParser.COMPARATOR_NAME, - AlertParser.getComponent(AlertParser.COMPARATOR_NAME, alert)); - alertFields.put(AlertParser.CONSTANT_NAME, - AlertParser.getComponent(AlertParser.CONSTANT_NAME, alert)); - try { - alertFields.put(AlertParser.ACTION_NAME, - AlertParser.getComponent(AlertParser.ACTION_NAME, alert)); - } catch (Exception e) { - logger.info("No action specified in " + alert); - } - // store the expression as stat - _statsHolder.addStat(alertFields.get(AlertParser.EXPRESSION_NAME)); - _statsHolder.persistStats(); - - // naming the alert with the full name - _alertsMap.put(alert, alertFields); - persistAlerts(); - } - - /* - * Add a set of alert statuses to ZK - */ - public void addAlertStatusSet(Map> statusSet) - throws HelixException { - if (_alertStatusMap == null) { - _alertStatusMap = new HashMap>(); - } - _alertStatusMap.clear(); // clear map. all alerts overwrite old alerts - for (String alert : statusSet.keySet()) { - Map currStatus = statusSet.get(alert); - if (currStatus != null) { - addAlertStatus(alert, currStatus); - } - } - - AlertStatus alertStatus = _accessor.getProperty(_keyBuilder.alertStatus()); - int alertStatusSize = 0; - if (alertStatus != null) { - alertStatusSize = alertStatus.getMapFields().size(); - } - // no need to persist alerts if there are none to persist and none are currently persisted - if (_alertStatusMap.size() > 0 || alertStatusSize > 0) { - persistAlertStatus(); // save statuses in zk - } - } - - private void addAlertStatus(String parentAlertKey, Map alertStatus) - throws HelixException { - // _alertStatusMap = new HashMap>(); - for (String alertName : alertStatus.keySet()) { - String mapAlertKey; - mapAlertKey = parentAlertKey; - if (!alertName.equals(ExpressionParser.wildcardChar)) { - mapAlertKey = mapAlertKey + " : (" + alertName + ")"; - } - AlertValueAndStatus vs = alertStatus.get(alertName); - Map alertFields = new HashMap(); - alertFields.put(AlertValueAndStatus.VALUE_NAME, vs.getValue().toString()); - alertFields.put(AlertValueAndStatus.FIRED_NAME, String.valueOf(vs.isFired())); - _alertStatusMap.put(mapAlertKey, alertFields); - } - } - - public AlertValueAndStatus getAlertValueAndStatus(String alertName) { - Map alertFields = _alertStatusMap.get(alertName); - String val = alertFields.get(AlertValueAndStatus.VALUE_NAME); - Tuple valTup = new Tuple(); - valTup.add(val); - boolean fired = Boolean.valueOf(alertFields.get(AlertValueAndStatus.FIRED_NAME)); - AlertValueAndStatus vs = new AlertValueAndStatus(valTup, fired); - return vs; - } - - public static void parseAlert(String alert, StringBuilder statsName, - Map alertFields) throws HelixException { - alert = alert.replaceAll("\\s+", ""); // remove white space - AlertParser.validateAlert(alert); - // alertFields = new HashMap(); - alertFields.put(AlertParser.EXPRESSION_NAME, - AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert)); - alertFields.put(AlertParser.COMPARATOR_NAME, - AlertParser.getComponent(AlertParser.COMPARATOR_NAME, alert)); - alertFields.put(AlertParser.CONSTANT_NAME, - AlertParser.getComponent(AlertParser.CONSTANT_NAME, alert)); - try { - alertFields.put(AlertParser.ACTION_NAME, - AlertParser.getComponent(AlertParser.ACTION_NAME, alert)); - } catch (Exception e) { - logger.info("No action specified in " + alert); - } - statsName.append(alertFields.get(AlertParser.EXPRESSION_NAME)); - } - - /* - * public void evaluateAllAlerts() - * { - * for (String alert : _alertsMap.keySet()) { - * Map alertFields = _alertsMap.get(alert); - * String exp = alertFields.get(AlertParser.EXPRESSION_NAME); - * String comp = alertFields.get(AlertParser.COMPARATOR_NAME); - * String con = alertFields.get(AlertParser.CONSTANT_NAME); - * //TODO: test the fields for null and fail if needed - * AlertProcessor.execute(exp, comp, con, sh); - * } - * } - */ - - public List getAlertList() { - List alerts = new LinkedList(); - for (String alert : _alertsMap.keySet()) { - Map alertFields = _alertsMap.get(alert); - String exp = alertFields.get(AlertParser.EXPRESSION_NAME); - String comp = alertFields.get(AlertParser.COMPARATOR_NAME); - Tuple con = Tuple.fromString(alertFields.get(AlertParser.CONSTANT_NAME)); - // TODO: test the fields for null and fail if needed - - Alert a = new Alert(alert, exp, comp, con); - alerts.add(a); - } - return alerts; - } - - public void updateCache(HealthDataCache cache) { - _cache = cache; - Alerts alertsRecord = _cache.getAlerts(); - if (alertsRecord != null) { - _alertsMap = alertsRecord.getMapFields(); - } else { - _alertsMap = new HashMap>(); - } - } - - public Map> getAlertsMap() { - return _alertsMap; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/DecayAggregator.java b/helix-core/src/main/java/org/apache/helix/alerts/DecayAggregator.java deleted file mode 100644 index be17ca9b03..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/DecayAggregator.java +++ /dev/null @@ -1,78 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; - -import org.apache.helix.HelixException; - -public class DecayAggregator extends Aggregator { - - double _decayWeight; - - public DecayAggregator(double weight) { - _decayWeight = weight; - } - - public DecayAggregator() { - _numArgs = 1; - } - - @Override - public void merge(Tuple currValTup, Tuple newValTup, Tuple currTimeTup, - Tuple newTimeTup, String... args) { - - _decayWeight = Double.parseDouble(args[0]); - - double currVal = 0; - double currTime = -1; - double newVal; - double newTime; - double mergedVal; - double mergedTime; - - if (currValTup == null || newValTup == null || currTimeTup == null || newTimeTup == null) { - throw new HelixException("Tuples cannot be null"); - } - - // old tuples may be empty, indicating no value/time exist - if (currValTup.size() > 0 && currTimeTup.size() > 0) { - currVal = Double.parseDouble(currValTup.iterator().next()); - currTime = Double.parseDouble(currTimeTup.iterator().next()); - } - newVal = Double.parseDouble(newValTup.iterator().next()); - newTime = Double.parseDouble(newTimeTup.iterator().next()); - - if (newTime > currTime) { // if old doesn't exist, we end up here - mergedVal = (1 - _decayWeight) * currVal + _decayWeight * newVal; // if old doesn't exist, it - // has value "0" - mergedTime = newTime; - } else { - mergedVal = currVal; - mergedTime = currTime; - } - - currValTup.clear(); - currValTup.add(Double.toString(mergedVal)); - currTimeTup.clear(); - currTimeTup.add(Double.toString(mergedTime)); - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/ExpressionParser.java b/helix-core/src/main/java/org/apache/helix/alerts/ExpressionParser.java deleted file mode 100644 index ca64be53e8..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/ExpressionParser.java +++ /dev/null @@ -1,496 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Map; -import java.util.Set; -import java.util.StringTokenizer; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import org.apache.helix.HelixException; -import org.apache.log4j.Logger; - -public class ExpressionParser { - private static Logger logger = Logger.getLogger(ExpressionParser.class); - - final static String opDelim = "|"; - final static String opDelimForSplit = "\\|"; - final static String argDelim = ","; - final public static String statFieldDelim = "."; - final static String wildcardChar = "*"; - - // static Map operatorMap = new - // HashMap(); - - static Map operatorMap = new HashMap(); - static Map aggregatorMap = new HashMap(); - - static { - - addOperatorEntry("EXPAND", new ExpandOperator()); - addOperatorEntry("DIVIDE", new DivideOperator()); - addOperatorEntry("SUM", new SumOperator()); - addOperatorEntry("SUMEACH", new SumEachOperator()); - - addAggregatorEntry("ACCUMULATE", new AccumulateAggregator()); - addAggregatorEntry("DECAY", new DecayAggregator()); - addAggregatorEntry("WINDOW", new WindowAggregator()); - /* - * addEntry("EACH", ExpressionOperatorType.EACH); addEntry("SUM", - * ExpressionOperatorType.SUM); addEntry("DIVIDE", - * ExpressionOperatorType.DIVIDE); addEntry("ACCUMULATE", - * ExpressionOperatorType.ACCUMULATE); - */ - } - - // static Pattern pattern = Pattern.compile("(\\{.+?\\})"); - - private static void addOperatorEntry(String label, Operator op) { - if (!operatorMap.containsKey(label)) { - operatorMap.put(label, op); - } - logger.info("Adding operator: " + op); - } - - private static void addAggregatorEntry(String label, Aggregator agg) { - if (!aggregatorMap.containsKey(label.toUpperCase())) { - aggregatorMap.put(label.toUpperCase(), agg); - } - logger.info("Adding aggregator: " + agg); - } - - /* - * private static void addEntry(String label, ExpressionOperatorType type) { - * if (!operatorMap.containsKey(label)) { operatorMap.put(label, type); } - * logger.info("Adding operator type: "+type); } - */ - - public static boolean isExpressionNested(String expression) { - return expression.contains("("); - } - - /* - * public static Operator getOperatorType(String expression) throws Exception - * { String op = expression.substring(0,expression.indexOf("(")); if - * (!operatorMap.containsKey(op)) { throw new - * Exception(op+" is not a valid op type"); } return operatorMap.get(op); } - */ - - public static String getInnerExpression(String expression) { - return expression.substring(expression.indexOf("(") + 1, expression.lastIndexOf(")")); - } - - /* - * public static String[] getBaseStats(ExpressionOperatorType type, String - * expression) throws Exception { String[] items = null; if - * (isExpressionNested(expression)) { ExpressionOperatorType nextType = - * getOperatorType(expression); String innerExp = - * getInnerExpression(expression); items = getBaseStats(nextType, innerExp); } - * else { //base class, no nesting items = expression.split(","); } - * if (type != null && type.isBaseOp()) { //surround items with type. for (int - * i=0; i= lastMatchEnd + 1) { // lastMatchEnd is pos 1 past the pattern. check - // if there are paren there - if (expression.substring(lastMatchEnd).contains("(") - || expression.substring(lastMatchEnd).contains(")")) { - throw new HelixException(expression + " has extra parenthesis"); - } - } - - // check wildcard locations. each part can have at most 1 wildcard, and must - // be at end - // String expStatNamePart = expression.substring(expression.) - StringTokenizer fieldTok = new StringTokenizer(statComponent, statFieldDelim); - while (fieldTok.hasMoreTokens()) { - String currTok = fieldTok.nextToken(); - if (currTok.contains(wildcardChar)) { - if (currTok.indexOf(wildcardChar) != currTok.length() - 1 - || currTok.lastIndexOf(wildcardChar) != currTok.length() - 1) { - throw new HelixException(currTok - + " is illegal stat name. Single wildcard must appear at end."); - } - } - } - } - - public static boolean statContainsWildcards(String stat) { - return stat.contains(wildcardChar); - } - - /* - * Return true if stat name matches exactly...incomingStat has no agg type - * currentStat can have any - * Function can match for 2 cases extractStatFromAgg=false. Match - * accumulate()(dbFoo.partition10.latency) with - * accumulate()(dbFoo.partition10.latency)...trival extractStatFromAgg=true. - * Match accumulate()(dbFoo.partition10.latency) with - * dbFoo.partition10.latency - */ - public static boolean isExactMatch(String currentStat, String incomingStat, - boolean extractStatFromAgg) { - String currentStatName = currentStat; - if (extractStatFromAgg) { - currentStatName = getSingleAggregatorStat(currentStat); - } - return (incomingStat.equals(currentStatName)); - } - - /* - * Return true if incomingStat matches wildcardStat except currentStat has 1+ - * fields with "*" a*.c* matches a5.c7 a*.c* does not match a5.b6.c7 - * Function can match for 2 cases extractStatFromAgg=false. Match - * accumulate()(dbFoo.partition*.latency) with - * accumulate()(dbFoo.partition10.latency) extractStatFromAgg=true. Match - * accumulate()(dbFoo.partition*.latency) with dbFoo.partition10.latency - */ - public static boolean isWildcardMatch(String currentStat, String incomingStat, - boolean statCompareOnly, ArrayList bindings) { - if (!statCompareOnly) { // need to check for match on agg type and stat - String currentStatAggType = (currentStat.split("\\)"))[0]; - String incomingStatAggType = (incomingStat.split("\\)"))[0]; - if (!currentStatAggType.equals(incomingStatAggType)) { - return false; - } - } - // now just get the stats - String currentStatName = getSingleAggregatorStat(currentStat); - String incomingStatName = getSingleAggregatorStat(incomingStat); - - if (!currentStatName.contains(wildcardChar)) { // no wildcards in stat name - return false; - } - - String currentStatNamePattern = currentStatName.replace(".", "\\."); - currentStatNamePattern = currentStatNamePattern.replace("*", ".*"); - boolean result = Pattern.matches(currentStatNamePattern, incomingStatName); - if (result && bindings != null) { - bindings.add(incomingStatName); - } - return result; - /* - * StringTokenizer currentStatTok = new StringTokenizer(currentStatName, - * statFieldDelim); - * StringTokenizer incomingStatTok = new StringTokenizer(incomingStatName, - * statFieldDelim); - * if (currentStatTok.countTokens() != incomingStatTok.countTokens()) - * { // stat names different numbers of fields - * return false; - * } - * // for each token, if not wildcarded, must be an exact match - * while (currentStatTok.hasMoreTokens()) - * { - * String currTok = currentStatTok.nextToken(); - * String incomingTok = incomingStatTok.nextToken(); - * logger.debug("curTok: " + currTok); - * logger.debug("incomingTok: " + incomingTok); - * if (!currTok.contains(wildcardChar)) - * { // no wildcard, but have exact match - * if (!currTok.equals(incomingTok)) - * { // not exact match - * return false; - * } - * } - * else - * { // currTok has a wildcard - * if (currTok.indexOf(wildcardChar) != currTok.length() - 1 - * || currTok.lastIndexOf(wildcardChar) != currTok.length() - 1) - * { - * throw new HelixException(currTok - * + " is illegal stat name. Single wildcard must appear at end."); - * } - * // for wildcard matching, need to escape parentheses on currTok, so - * // regex works - * // currTok = currTok.replace("(", "\\("); - * // currTok = currTok.replace(")", "\\)"); - * // incomingTok = incomingTok.replace("(", "\\("); - * // incomingTok = incomingTok.replace(")", "\\)"); - * String currTokPreWildcard = currTok.substring(0, currTok.length() - 1); - * // TODO: if current token has a "(" in it, pattern compiling throws - * // error - * // Pattern pattern = Pattern.compile(currTokPreWildcard+".+"); //form - * // pattern...wildcard part can be anything - * // Matcher matcher = pattern.matcher(incomingTok); //see if incomingTok - * // matches - * if (incomingTok.indexOf(currTokPreWildcard) != 0) - * { - * // if (!matcher.find()) { //no match on one tok, return false - * return false; - * } - * // get the binding - * if (bindings != null) - * { - * // TODO: debug me! - * String wildcardBinding = incomingTok.substring(incomingTok - * .indexOf(currTokPreWildcard) + currTokPreWildcard.length()); - * bindings.add(wildcardBinding); - * } - * } - * } - * // all fields match or wildcard match...return true! - * return true; - */ - } - - /* - * For checking if an incoming stat (no agg type defined) matches a persisted - * stat (with agg type defined) - */ - public static boolean isIncomingStatExactMatch(String currentStat, String incomingStat) { - return isExactMatch(currentStat, incomingStat, true); - } - - /* - * For checking if an incoming stat (no agg type defined) wildcard matches a - * persisted stat (with agg type defined) The persisted stat may have - * wildcards - */ - public static boolean isIncomingStatWildcardMatch(String currentStat, String incomingStat) { - return isWildcardMatch(currentStat, incomingStat, true, null); - } - - /* - * For checking if a persisted stat matches a stat defined in an alert - */ - public static boolean isAlertStatExactMatch(String alertStat, String currentStat) { - return isExactMatch(alertStat, currentStat, false); - } - - /* - * For checking if a maintained stat wildcard matches a stat defined in an - * alert. The alert may have wildcards - */ - public static boolean isAlertStatWildcardMatch(String alertStat, String currentStat, - ArrayList wildcardBindings) { - return isWildcardMatch(alertStat, currentStat, false, wildcardBindings); - } - - public static Aggregator getAggregator(String aggStr) throws HelixException { - aggStr = aggStr.toUpperCase(); - Aggregator agg = aggregatorMap.get(aggStr); - if (agg == null) { - throw new HelixException("Unknown aggregator type " + aggStr); - } - return agg; - } - - public static String getAggregatorStr(String expression) throws HelixException { - if (!expression.contains("(")) { - throw new HelixException(expression - + " does not contain a valid aggregator. No parentheses found"); - } - String aggName = expression.substring(0, expression.indexOf("(")); - if (!aggregatorMap.containsKey(aggName.toUpperCase())) { - throw new HelixException("aggregator <" + aggName + "> is unknown type"); - } - return aggName; - } - - public static String[] getAggregatorArgs(String expression) throws HelixException { - String aggregator = getAggregatorStr(expression); - String argsStr = getAggregatorArgsStr(expression); - String[] args = argsStr.split(argDelim); - logger.debug("args size: " + args.length); - int numArgs = (argsStr.length() == 0) ? 0 : args.length; - // String[] argList = (expression.substring(expression.indexOf("(")+1, - // expression.indexOf(")"))).split(argDelim); - // verify correct number of args - int requiredNumArgs = aggregatorMap.get(aggregator.toUpperCase()).getRequiredNumArgs(); - if (numArgs != requiredNumArgs) { - throw new HelixException(expression + " contains " + args.length - + " arguments, but requires " + requiredNumArgs); - } - return args; - } - - /* - * public static String[] getAggregatorArgsList(String expression) { String - * argsStr = getAggregatorArgsStr(expression); String[] args = - * argsStr.split(argDelim); return args; } - */ - - public static String getAggregatorArgsStr(String expression) { - return expression.substring(expression.indexOf("(") + 1, expression.indexOf(")")); - } - - public static String[] getAggregatorStats(String expression) throws HelixException { - String justStats = expression; - if (expression.contains("(") && expression.contains(")")) { - justStats = - (expression.substring(expression.lastIndexOf("(") + 1, expression.lastIndexOf(")"))); - } - String[] statList = justStats.split(argDelim); - if (statList.length < 1) { - throw new HelixException(expression + " does not contain any aggregator stats"); - } - return statList; - } - - public static String getSingleAggregatorStat(String expression) throws HelixException { - String[] stats = getAggregatorStats(expression); - if (stats.length > 1) { - throw new HelixException(expression + " contains more than 1 stat"); - } - return stats[0]; - } - - public static String getWildcardStatSubstitution(String wildcardStat, String fixedStat) { - int lastOpenParenLoc = wildcardStat.lastIndexOf("("); - int lastCloseParenLoc = wildcardStat.lastIndexOf(")"); - StringBuilder builder = new StringBuilder(); - builder.append(wildcardStat.substring(0, lastOpenParenLoc + 1)); - builder.append(fixedStat); - builder.append(")"); - logger.debug("wildcardStat: " + wildcardStat); - logger.debug("fixedStat: " + fixedStat); - logger.debug("subbedStat: " + builder.toString()); - return builder.toString(); - } - - // XXX: each op type should have number of inputs, number of outputs. do - // validation. - // (dbFoo.partition*.latency, dbFoo.partition*.count)|EACH|ACCUMULATE|DIVIDE - public static String[] getBaseStats(String expression) throws HelixException { - expression = expression.replaceAll("\\s+", ""); - validateAggregatorFormat(expression); - - String aggName = getAggregatorStr(expression); - String[] aggArgs = getAggregatorArgs(expression); - String[] aggStats = getAggregatorStats(expression); - - // form aggArgs - String aggArgList = getAggregatorArgsStr(expression); - - String[] baseStats = new String[aggStats.length]; - for (int i = 0; i < aggStats.length; i++) { - StringBuilder stat = new StringBuilder(); - stat.append(aggName); - stat.append("("); - stat.append(aggArgList); - stat.append(")"); - stat.append("("); - stat.append(aggStats[i]); - stat.append(")"); - baseStats[i] = stat.toString(); - } - return baseStats; - } - - public static String[] getOperators(String expression) throws HelixException { - String[] ops = null; - int numAggStats = (getAggregatorStats(expression)).length; - int opDelimLoc = expression.indexOf(opDelim); - if (opDelimLoc < 0) { - return null; - } - logger.debug("ops str: " + expression.substring(opDelimLoc + 1)); - ops = expression.substring(opDelimLoc + 1).split(opDelimForSplit); - - // validate this string of ops - // verify each op exists - // take num input tuples sets and verify ops will output exactly 1 tuple - // sets - int currNumTuples = numAggStats; - for (String op : ops) { - logger.debug("op: " + op); - if (!operatorMap.containsKey(op.toUpperCase())) { - throw new HelixException("<" + op + "> is not a valid operator type"); - } - Operator currOpType = operatorMap.get(op.toUpperCase()); - if (currNumTuples < currOpType.minInputTupleLists - || currNumTuples > currOpType.maxInputTupleLists) { - throw new HelixException("<" + op + "> cannot process " + currNumTuples + " input tuples"); - } - // reset num tuples to this op's output size - if (!currOpType.inputOutputTupleListsCountsEqual) { // if equal, this number does not change - currNumTuples = currOpType.numOutputTupleLists; - } - } - if (currNumTuples != 1) { - throw new HelixException(expression + " does not terminate in a single tuple set"); - } - return ops; - } - - public static void validateOperators(String expression) throws HelixException { - getOperators(expression); - } - - public static Operator getOperator(String opName) throws HelixException { - if (!operatorMap.containsKey(opName)) { - throw new HelixException(opName + " is unknown op type"); - } - return operatorMap.get(opName); - } - - public static void validateExpression(String expression) throws HelixException { - // 1. extract stats part and validate - validateAggregatorFormat(expression); - // 2. extract ops part and validate the ops exist and the inputs/outputs are - // correct - validateOperators(expression); - } -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/MultiplyOperator.java b/helix-core/src/main/java/org/apache/helix/alerts/MultiplyOperator.java deleted file mode 100644 index 74a468854b..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/MultiplyOperator.java +++ /dev/null @@ -1,60 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; -import java.util.List; - -public class MultiplyOperator extends Operator { - - public MultiplyOperator() { - minInputTupleLists = 1; - maxInputTupleLists = Integer.MAX_VALUE; - inputOutputTupleListsCountsEqual = false; - numOutputTupleLists = 1; - } - - public List>> singleSetToIter(ArrayList> input) { - List out = new ArrayList(); - out.add(input.iterator()); - return out; - } - - @Override - public List>> execute(List>> input) { - ArrayList> output = new ArrayList>(); - if (input == null || input.size() == 0) { - return singleSetToIter(output); - } - while (true) { // loop through set of iters, return when 1 runs out (not completing the row in - // progress) - Tuple rowProduct = null; - for (Iterator> it : input) { - if (!it.hasNext()) { // when any iterator runs out, we are done - return singleSetToIter(output); - } - rowProduct = multiplyTuples(rowProduct, it.next()); - } - output.add(rowProduct); - } - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Operator.java b/helix-core/src/main/java/org/apache/helix/alerts/Operator.java deleted file mode 100644 index 0612cf3535..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/Operator.java +++ /dev/null @@ -1,111 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; -import java.util.List; - -public abstract class Operator { - - public int minInputTupleLists; - public int maxInputTupleLists; - public int numOutputTupleLists = -1; - public boolean inputOutputTupleListsCountsEqual = false; - - public Operator() { - - } - - public Tuple multiplyTuples(Tuple tup1, Tuple tup2) { - if (tup1 == null) { - return tup2; - } - if (tup2 == null) { - return tup1; - } - Tuple outputTup = new Tuple(); - - // sum staggers if the tuples are same length - // e.g. 1,2,3 + 4,5 = 1,6,8 - // so this is a bit tricky - Tuple largerTup; - Tuple smallerTup; - if (tup1.size() >= tup2.size()) { - largerTup = tup1; - smallerTup = tup2; - } else { - largerTup = tup2; - smallerTup = tup1; - } - int gap = largerTup.size() - smallerTup.size(); - - for (int i = 0; i < largerTup.size(); i++) { - if (i < gap) { - outputTup.add(largerTup.getElement(i)); - } else { - double elementProduct = 0; - elementProduct = - Double.parseDouble(largerTup.getElement(i)) - * Double.parseDouble(smallerTup.getElement(i - gap)); - outputTup.add(String.valueOf(elementProduct)); - } - } - return outputTup; - } - - public Tuple sumTuples(Tuple tup1, Tuple tup2) { - if (tup1 == null) { - return tup2; - } - if (tup2 == null) { - return tup1; - } - Tuple outputTup = new Tuple(); - - // sum staggers if the tuples are same length - // e.g. 1,2,3 + 4,5 = 1,6,8 - // so this is a bit tricky - Tuple largerTup; - Tuple smallerTup; - if (tup1.size() >= tup2.size()) { - largerTup = tup1; - smallerTup = tup2; - } else { - largerTup = tup2; - smallerTup = tup1; - } - int gap = largerTup.size() - smallerTup.size(); - - for (int i = 0; i < largerTup.size(); i++) { - if (i < gap) { - outputTup.add(largerTup.getElement(i)); - } else { - double elementSum = 0; - elementSum = - Double.parseDouble(largerTup.getElement(i)) - + Double.parseDouble(smallerTup.getElement(i - gap)); - outputTup.add(String.valueOf(elementSum)); - } - } - return outputTup; - } - - public abstract List>> execute(List>> input); -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Stat.java b/helix-core/src/main/java/org/apache/helix/alerts/Stat.java deleted file mode 100644 index 6895128aa4..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/Stat.java +++ /dev/null @@ -1,44 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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 Stat { - String _name; - Tuple _value; - Tuple _timestamp; - - public Stat(String name, Tuple value, Tuple timestamp) { - _name = name; - _value = value; - _timestamp = timestamp; - } - - public String getName() { - return _name; - } - - public Tuple getValue() { - return _value; - } - - public Tuple getTimestamp() { - return _timestamp; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/StatsHolder.java b/helix-core/src/main/java/org/apache/helix/alerts/StatsHolder.java deleted file mode 100644 index 97ae463ffd..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/StatsHolder.java +++ /dev/null @@ -1,310 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.regex.Pattern; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixException; -import org.apache.helix.HelixManager; -import org.apache.helix.PropertyKey; -import org.apache.helix.PropertyType; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.stages.HealthDataCache; -import org.apache.helix.model.PersistentStats; -import org.apache.log4j.Logger; - -public class StatsHolder { - enum MatchResult { - WILDCARDMATCH, - EXACTMATCH, - NOMATCH - }; - - private static final Logger logger = Logger.getLogger(StatsHolder.class.getName()); - - public static final String VALUE_NAME = "value"; - public static final String TIMESTAMP_NAME = "TimeStamp"; - - HelixDataAccessor _accessor; - HealthDataCache _cache; - - Map> _statMap; - Map> _statAlertMatchResult; - - private Builder _keyBuilder; - - // PersistentStats _persistentStats; - - public StatsHolder(HelixManager manager, HealthDataCache cache) { - _accessor = manager.getHelixDataAccessor(); - _cache = cache; - _keyBuilder = new PropertyKey.Builder(manager.getClusterName()); - updateCache(_cache); - _statAlertMatchResult = new HashMap>(); - - } - - public void refreshStats() { - logger.info("Refreshing cached stats"); - _cache.refresh(_accessor); - updateCache(_cache); - } - - public void persistStats() { - // XXX: Am I using _accessor too directly here? - // took around 35 ms from desktop to ESV4 machine - PersistentStats stats = _accessor.getProperty(_keyBuilder.persistantStat()); - if (stats == null) { - stats = new PersistentStats(PersistentStats.nodeName); // TODO: fix naming of - // this record, if it - // matters - } - stats.getRecord().setMapFields(_statMap); - boolean retVal = _accessor.setProperty(_keyBuilder.persistantStat(), stats); - } - - public void getStatsFromCache(boolean refresh) { - long refreshStartTime = System.currentTimeMillis(); - if (refresh) { - _cache.refresh(_accessor); - } - PersistentStats persistentStatRecord = _cache.getPersistentStats(); - if (persistentStatRecord != null) { - _statMap = persistentStatRecord.getMapFields(); - } else { - _statMap = new HashMap>(); - } - /* - * if (_cache.getPersistentStats() != null) { - * _statMap = _cache.getPersistentStats(); - * } - */ - // TODO: confirm this a good place to init the _statMap when null - /* - * if (_statMap == null) { - * _statMap = new HashMap>(); - * } - */ - System.out.println("Refresh stats done: " + (System.currentTimeMillis() - refreshStartTime)); - } - - public Iterator getAllStats() { - return null; - } - - /* - * TODO: figure out pre-conditions here. I think not allowing anything to be - * null on input - */ - public Map mergeStats(String statName, Map existingStat, - Map incomingStat) throws HelixException { - if (existingStat == null) { - throw new HelixException("existing stat for merge is null"); - } - if (incomingStat == null) { - throw new HelixException("incoming stat for merge is null"); - } - // get agg type and arguments, then get agg object - String aggTypeStr = ExpressionParser.getAggregatorStr(statName); - String[] aggArgs = ExpressionParser.getAggregatorArgs(statName); - Aggregator agg = ExpressionParser.getAggregator(aggTypeStr); - // XXX: some of below lines might fail with null exceptions - - // get timestamps, values out of zk maps - String existingTime = existingStat.get(TIMESTAMP_NAME); - String existingVal = existingStat.get(VALUE_NAME); - String incomingTime = incomingStat.get(TIMESTAMP_NAME); - String incomingVal = incomingStat.get(VALUE_NAME); - // parse values into tuples, if the values exist. else, tuples are null - Tuple existingTimeTuple = - (existingTime != null) ? Tuple.fromString(existingTime) : null; - Tuple existingValueTuple = (existingVal != null) ? Tuple.fromString(existingVal) : null; - Tuple incomingTimeTuple = - (incomingTime != null) ? Tuple.fromString(incomingTime) : null; - Tuple incomingValueTuple = (incomingVal != null) ? Tuple.fromString(incomingVal) : null; - - // dp merge - agg.merge(existingValueTuple, incomingValueTuple, existingTimeTuple, incomingTimeTuple, aggArgs); - // put merged tuples back in map - Map mergedMap = new HashMap(); - if (existingTimeTuple.size() == 0) { - throw new HelixException("merged time tuple has size zero"); - } - if (existingValueTuple.size() == 0) { - throw new HelixException("merged value tuple has size zero"); - } - - mergedMap.put(TIMESTAMP_NAME, existingTimeTuple.toString()); - mergedMap.put(VALUE_NAME, existingValueTuple.toString()); - return mergedMap; - } - - /* - * Find all persisted stats this stat matches. Update those stats. An incoming - * stat can match multiple stats exactly (if that stat has multiple agg types) - * An incoming stat can match multiple wildcard stats - */ - - // need to do a time check here! - - public void applyStat(String incomingStatName, Map statFields) { - // TODO: consider locking stats here - // refreshStats(); //will have refreshed by now during stage - - Map> pendingAdds = new HashMap>(); - - if (!_statAlertMatchResult.containsKey(incomingStatName)) { - _statAlertMatchResult.put(incomingStatName, new HashMap()); - } - Map resultMap = _statAlertMatchResult.get(incomingStatName); - // traverse through all persistent stats - for (String key : _statMap.keySet()) { - if (resultMap.containsKey(key)) { - MatchResult cachedMatchResult = resultMap.get(key); - if (cachedMatchResult == MatchResult.EXACTMATCH) { - processExactMatch(key, statFields); - } else if (cachedMatchResult == MatchResult.WILDCARDMATCH) { - processWildcardMatch(incomingStatName, key, statFields, pendingAdds); - } - // don't care about NOMATCH - continue; - } - // exact match on stat and stat portion of persisted stat, just update - if (ExpressionParser.isIncomingStatExactMatch(key, incomingStatName)) { - processExactMatch(key, statFields); - resultMap.put(key, MatchResult.EXACTMATCH); - } - // wildcard match - else if (ExpressionParser.isIncomingStatWildcardMatch(key, incomingStatName)) { - processWildcardMatch(incomingStatName, key, statFields, pendingAdds); - resultMap.put(key, MatchResult.WILDCARDMATCH); - } else { - resultMap.put(key, MatchResult.NOMATCH); - } - } - _statMap.putAll(pendingAdds); - } - - void processExactMatch(String key, Map statFields) { - Map mergedStat = mergeStats(key, _statMap.get(key), statFields); - // update in place, no problem with hash map - _statMap.put(key, mergedStat); - } - - void processWildcardMatch(String incomingStatName, String key, Map statFields, - Map> pendingAdds) { - - // make sure incoming stat doesn't already exist, either in previous - // round or this round - // form new key (incomingStatName with agg type from the wildcarded - // stat) - String statToAdd = ExpressionParser.getWildcardStatSubstitution(key, incomingStatName); - // if the stat already existed in _statMap, we have/will apply it as an - // exact match - // if the stat was added this round to pendingAdds, no need to recreate - // (it would have same value) - if (!_statMap.containsKey(statToAdd) && !pendingAdds.containsKey(statToAdd)) { - // add this stat to persisted stats - Map mergedStat = mergeStats(statToAdd, getEmptyStat(), statFields); - // add to pendingAdds so we don't mess up ongoing traversal of - // _statMap - pendingAdds.put(statToAdd, mergedStat); - } - } - - // add parsing of stat (or is that in expression holder?) at least add - // validate - public void addStat(String exp) throws HelixException { - refreshStats(); // get current stats - - String[] parsedStats = ExpressionParser.getBaseStats(exp); - - for (String stat : parsedStats) { - if (_statMap.containsKey(stat)) { - logger.debug("Stat " + stat + " already exists; not adding"); - continue; - } - _statMap.put(stat, getEmptyStat()); // add new stat to map - } - } - - public static Map> parseStat(String exp) throws HelixException { - String[] parsedStats = ExpressionParser.getBaseStats(exp); - Map> statMap = new HashMap>(); - - for (String stat : parsedStats) { - if (statMap.containsKey(stat)) { - logger.debug("Stat " + stat + " already exists; not adding"); - continue; - } - statMap.put(stat, getEmptyStat()); // add new stat to map - } - return statMap; - } - - public static Map getEmptyStat() { - Map statFields = new HashMap(); - statFields.put(TIMESTAMP_NAME, ""); - statFields.put(VALUE_NAME, ""); - return statFields; - } - - public List getStatsList() { - List stats = new LinkedList(); - for (String stat : _statMap.keySet()) { - Map statFields = _statMap.get(stat); - Tuple valTup = Tuple.fromString(statFields.get(VALUE_NAME)); - Tuple timeTup = Tuple.fromString(statFields.get(TIMESTAMP_NAME)); - Stat s = new Stat(stat, valTup, timeTup); - stats.add(s); - } - return stats; - } - - public Map> getStatsMap() { - // refreshStats(); //don't refresh, stage will have refreshed by this time - HashMap> stats = new HashMap>(); - for (String stat : _statMap.keySet()) { - Map statFields = _statMap.get(stat); - Tuple valTup = Tuple.fromString(statFields.get(VALUE_NAME)); - Tuple timeTup = Tuple.fromString(statFields.get(TIMESTAMP_NAME)); - stats.put(stat, valTup); - } - return stats; - } - - public void updateCache(HealthDataCache cache) { - _cache = cache; - PersistentStats persistentStatRecord = _cache.getPersistentStats(); - if (persistentStatRecord != null) { - _statMap = persistentStatRecord.getMapFields(); - } else { - _statMap = new HashMap>(); - } - } -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/SumEachOperator.java b/helix-core/src/main/java/org/apache/helix/alerts/SumEachOperator.java deleted file mode 100644 index 2cc733fa1b..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/SumEachOperator.java +++ /dev/null @@ -1,51 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; -import java.util.List; - -public class SumEachOperator extends Operator { - - public SumEachOperator() { - minInputTupleLists = 1; - maxInputTupleLists = Integer.MAX_VALUE; - inputOutputTupleListsCountsEqual = true; - numOutputTupleLists = -1; - } - - // for each column, generate sum - @Override - public List>> execute(List>> input) { - List>> out = new ArrayList>>(); - for (Iterator> currIt : input) { - Tuple currSum = null; - while (currIt.hasNext()) { - currSum = sumTuples(currSum, currIt.next()); - } - ArrayList> currOutList = new ArrayList>(); - currOutList.add(currSum); - out.add(currOutList.iterator()); - } - return out; - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/SumOperator.java b/helix-core/src/main/java/org/apache/helix/alerts/SumOperator.java deleted file mode 100644 index 90c9ab0d58..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/SumOperator.java +++ /dev/null @@ -1,59 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; -import java.util.List; - -public class SumOperator extends Operator { - - public SumOperator() { - minInputTupleLists = 1; - maxInputTupleLists = Integer.MAX_VALUE; - inputOutputTupleListsCountsEqual = false; - numOutputTupleLists = 1; - } - - public List>> singleSetToIter(ArrayList> input) { - List out = new ArrayList(); - out.add(input.iterator()); - return out; - } - - @Override - public List>> execute(List>> input) { - ArrayList> output = new ArrayList>(); - if (input == null || input.size() == 0) { - return singleSetToIter(output); - } - while (true) { // loop through set of iters, return when 1 runs out (not completing the row in - // progress) - Tuple rowSum = null; - for (Iterator> it : input) { - if (!it.hasNext()) { // when any iterator runs out, we are done - return singleSetToIter(output); - } - rowSum = sumTuples(rowSum, it.next()); - } - output.add(rowSum); - } - } -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Tuple.java b/helix-core/src/main/java/org/apache/helix/alerts/Tuple.java deleted file mode 100644 index 9a876a45cb..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/Tuple.java +++ /dev/null @@ -1,88 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.StringTokenizer; -import java.util.Vector; - -public class Tuple { - List elements; - - public Tuple() { - elements = new ArrayList(); - } - - public int size() { - return elements.size(); - } - - public void add(T entry) { - elements.add(entry); - } - - public void addAll(Tuple incoming) { - elements.addAll(incoming.getElements()); - } - - public Iterator iterator() { - return elements.listIterator(); - } - - public T getElement(int ind) { - return elements.get(ind); - } - - public List getElements() { - return elements; - } - - public void clear() { - elements.clear(); - } - - public static Tuple fromString(String in) { - Tuple tup = new Tuple(); - if (in.length() > 0) { - String[] elements = in.split(","); - for (String element : elements) { - tup.add(element); - } - } - return tup; - } - - public String toString() { - StringBuilder out = new StringBuilder(); - Iterator it = iterator(); - boolean outEmpty = true; - while (it.hasNext()) { - if (!outEmpty) { - out.append(","); - } - out.append(it.next()); - outEmpty = false; - } - return out.toString(); - } -} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/WindowAggregator.java b/helix-core/src/main/java/org/apache/helix/alerts/WindowAggregator.java deleted file mode 100644 index ff4340cf5f..0000000000 --- a/helix-core/src/main/java/org/apache/helix/alerts/WindowAggregator.java +++ /dev/null @@ -1,93 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; - -import org.apache.helix.HelixException; - -public class WindowAggregator extends Aggregator { - - int _windowSize; - - public WindowAggregator(String windowSize) { - _windowSize = Integer.parseInt(windowSize); - _numArgs = 1; - } - - public WindowAggregator() { - this("1"); - } - - @Override - public void merge(Tuple currValTup, Tuple newValTup, Tuple currTimeTup, - Tuple newTimeTup, String... args) { - - _windowSize = Integer.parseInt(args[0]); - - // figure out how many curr tuple values we displace - Tuple mergedTimeTuple = new Tuple(); - Tuple mergedValTuple = new Tuple(); - - Iterator currTimeIter = currTimeTup.iterator(); - Iterator currValIter = currValTup.iterator(); - Iterator newTimeIter = newTimeTup.iterator(); - Iterator newValIter = newValTup.iterator(); - int currCtr = 0; - // traverse current vals - double currTime = -1; - double currVal; - while (currTimeIter.hasNext()) { - currTime = Double.parseDouble(currTimeIter.next()); - currVal = Double.parseDouble(currValIter.next()); - currCtr++; - // number of evicted currVals equal to total size of both minus _windowSize - if (currCtr > (newTimeTup.size() + currTimeTup.size() - _windowSize)) { // non-evicted - // element, just bump - // down - mergedTimeTuple.add(String.valueOf(currTime)); - mergedValTuple.add(String.valueOf(currVal)); - } - } - - double newVal; - double newTime; - while (newTimeIter.hasNext()) { - newVal = Double.parseDouble(newValIter.next()); - newTime = Double.parseDouble(newTimeIter.next()); - if (newTime <= currTime) { // oldest new time older than newest curr time. we will not apply - // new tuple! - return; // curr tuples remain the same - } - currCtr++; - if (currCtr > (newTimeTup.size() + currTimeTup.size() - _windowSize)) { // non-evicted element - mergedTimeTuple.add(String.valueOf(newTime)); - mergedValTuple.add(String.valueOf(newVal)); - } - } - // set curr tuples to merged tuples - currTimeTup.clear(); - currTimeTup.addAll(mergedTimeTuple); - currValTup.clear(); - currValTup.addAll(mergedValTuple); - // TODO: see if we can do merger in place on curr - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/api/Cluster.java b/helix-core/src/main/java/org/apache/helix/api/Cluster.java new file mode 100644 index 0000000000..856c09b257 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/Cluster.java @@ -0,0 +1,288 @@ +package org.apache.helix.api; + +/* + * 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.Collections; +import java.util.Map; + +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SpectatorId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.model.ClusterConstraints; +import org.apache.helix.model.ClusterConstraints.ConstraintType; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.model.Transition; + +import com.google.common.base.Function; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + +/** + * Represent a logical helix cluster + */ +public class Cluster { + + /** + * map of resource-id to resource + */ + private final Map _resourceMap; + + /** + * map of participant-id to participant + */ + private final Map _participantMap; + + /** + * map of participant-id to live participant + */ + private final Map _liveParticipantMap; + + /** + * map of controller-id to controller + */ + private final Map _controllerMap; + + /** + * map of spectator-id to spectator + */ + private final Map _spectatorMap; + + private final ControllerId _leaderId; + + private final ClusterConfig _config; + + /** + * construct a cluster + * @param id + * @param resourceMap + * @param participantMap + * @param controllerMap + * @param leaderId + * @param constraintMap + * @param stateModelMap + * @param stats + * @param alerts + * @param userConfig + * @param isPaused + * @param autoJoinAllowed + */ + public Cluster(ClusterId id, Map resourceMap, + Map participantMap, Map controllerMap, + ControllerId leaderId, Map constraintMap, + Map stateModelMap, UserConfig userConfig, + boolean isPaused, boolean autoJoinAllowed) { + + // build the config + // Guava's transform and "copy" functions really return views so the maps all reflect each other + Map resourceConfigMap = + Maps.transformValues(resourceMap, new Function() { + @Override + public ResourceConfig apply(Resource resource) { + return resource.getConfig(); + } + }); + Map participantConfigMap = + Maps.transformValues(participantMap, new Function() { + @Override + public ParticipantConfig apply(Participant participant) { + return participant.getConfig(); + } + }); + _config = + new ClusterConfig.Builder(id).addResources(resourceConfigMap.values()) + .addParticipants(participantConfigMap.values()).addConstraints(constraintMap.values()) + .addStateModelDefinitions(stateModelMap.values()).pausedStatus(isPaused) + .userConfig(userConfig).autoJoin(autoJoinAllowed).build(); + + _resourceMap = ImmutableMap.copyOf(resourceMap); + + _participantMap = ImmutableMap.copyOf(participantMap); + + // Build the subset of participants that is live + ImmutableMap.Builder liveParticipantBuilder = + new ImmutableMap.Builder(); + for (Participant participant : participantMap.values()) { + if (participant.isAlive()) { + liveParticipantBuilder.put(participant.getId(), participant); + } + } + _liveParticipantMap = liveParticipantBuilder.build(); + + _leaderId = leaderId; + + // TODO impl this when we persist controllers and spectators on zookeeper + _controllerMap = ImmutableMap.copyOf(controllerMap); + _spectatorMap = Collections.emptyMap(); + } + + /** + * Get cluster id + * @return cluster id + */ + public ClusterId getId() { + return _config.getId(); + } + + /** + * Get resources in the cluster + * @return a map of resource id to resource, or empty map if none + */ + public Map getResourceMap() { + return _resourceMap; + } + + /** + * Get resource given resource id + * @param resourceId + * @return resource or null if not exist + */ + public Resource getResource(ResourceId resourceId) { + return _resourceMap.get(resourceId); + } + + /** + * Get participants of the cluster + * @return a map of participant id to participant, or empty map if none + */ + public Map getParticipantMap() { + return _participantMap; + } + + /** + * Get live participants of the cluster + * @return a map of participant id to participant, or empty map if none is live + */ + public Map getLiveParticipantMap() { + return _liveParticipantMap; + } + + /** + * Get controllers of the cluster + * @return a map of controller id to controller, or empty map if none + */ + public Map getControllerMap() { + return _controllerMap; + } + + /** + * Get the leader of the cluster + * @return the leader or null if not exist + */ + public Controller getLeader() { + return _controllerMap.get(_leaderId); + } + + /** + * Get spectators of the cluster + * @return a map of spectator id to spectator, or empty map if none + */ + public Map getSpectatorMap() { + return _spectatorMap; + } + + /** + * Get all the constraints on the cluster + * @return map of constraint type to constraints + */ + public Map getConstraintMap() { + return _config.getConstraintMap(); + } + + /** + * Get all the state model definitions on the cluster + * @return map of state model definition id to state model definition + */ + public Map getStateModelMap() { + return _config.getStateModelMap(); + } + + /** + * Get user-specified configuration properties of this cluster + * @return UserConfig properties + */ + public UserConfig getUserConfig() { + return _config.getUserConfig(); + } + + /** + * Get a cluster constraint + * @param type the type of constrant to query + * @return cluster constraints, or null if none + */ + public ClusterConstraints getConstraint(ConstraintType type) { + return _config.getConstraintMap().get(type); + } + + /** + * Get the maximum number of participants that can be in a state + * @param scope the scope for the bound + * @param stateModelDefId the state model of the state + * @param state the constrained state + * @return The upper bound, which can be "-1" if unspecified, a numerical upper bound, "R" for + * number of replicas, or "N" for number of participants + */ + public String getStateUpperBoundConstraint(Scope scope, StateModelDefId stateModelDefId, + State state) { + return _config.getStateUpperBoundConstraint(scope, stateModelDefId, state); + } + + /** + * Get the limit of simultaneous execution of a transition + * @param scope the scope under which the transition is constrained + * @param stateModelDefId the state model of which the transition is a part + * @param transition the constrained transition + * @return the limit, or Integer.MAX_VALUE if there is no limit + */ + public int getTransitionConstraint(Scope scope, StateModelDefId stateModelDefId, + Transition transition) { + return _config.getTransitionConstraint(scope, stateModelDefId, transition); + } + + /** + * Check the paused status of the cluster + * @return true if paused, false otherwise + */ + public boolean isPaused() { + return _config.isPaused(); + } + + /** + * Check if the cluster supports participants automatically joining + * @return true if allowed, false if disallowed + */ + public boolean autoJoinAllowed() { + return _config.autoJoinAllowed(); + } + + /** + * Get the ClusterConfig specifying this cluster + * @return ClusterConfig + */ + public ClusterConfig getConfig() { + return _config; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/Controller.java b/helix-core/src/main/java/org/apache/helix/api/Controller.java new file mode 100644 index 0000000000..1218287eac --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/Controller.java @@ -0,0 +1,74 @@ +package org.apache.helix.api; + +/* + * 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.api.id.ControllerId; +import org.apache.helix.model.LiveInstance; + +/** + * A helix controller + */ +public class Controller { + private final ControllerId _id; + private final RunningInstance _runningInstance; + private final boolean _isLeader; + + /** + * Construct a controller + * @param id + */ + public Controller(ControllerId id, LiveInstance liveInstance, boolean isLeader) { + _id = id; + + if (liveInstance != null) { + _runningInstance = + new RunningInstance(liveInstance.getTypedSessionId(), liveInstance.getTypedHelixVersion(), + liveInstance.getProcessId()); + } else { + _runningInstance = null; + } + + _isLeader = isLeader; + } + + /** + * Get controller id + * @return controller id + */ + public ControllerId getId() { + return _id; + } + + /** + * Check if the controller is leader + * @return true if leader or false otherwise + */ + public boolean isLeader() { + return _isLeader; + } + + /** + * Get the running instance + * @return running instance or null if not running + */ + public RunningInstance getRunningInstance() { + return _runningInstance; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/HelixVersion.java b/helix-core/src/main/java/org/apache/helix/api/HelixVersion.java new file mode 100644 index 0000000000..b50031d1fc --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/HelixVersion.java @@ -0,0 +1,68 @@ +package org.apache.helix.api; + +/* + * 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. + */ + +/** + * Helix version (e.g. 0.6.1.5) + */ +public class HelixVersion { + final String _version; + + /** + * Construct with a version string (e.g. 0.6.1.5) + * @param version + */ + public HelixVersion(String version) { + _version = version; + } + + /** + * Get major version (e.g. 6 in 0.6.1.5) + * @return major version number + */ + public String getMajor() { + return null; + } + + /** + * Get minor version (e.g. 1 in 0.6.1.5) + * @return minor version number + */ + public String getMinor() { + return null; + } + + @Override + public String toString() { + return _version; + } + + /** + * Create a version from a version string + * @param version string in the form of a.b.c.d + * @return HelixVersion + */ + public static HelixVersion from(String version) { + if (version == null) { + return null; + } + return new HelixVersion(version); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/Participant.java b/helix-core/src/main/java/org/apache/helix/api/Participant.java new file mode 100644 index 0000000000..53f40386ed --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/Participant.java @@ -0,0 +1,174 @@ +package org.apache.helix.api; + +/* + * 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.Set; + +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.Message; + +import com.google.common.collect.ImmutableMap; + +/** + * A cluster participant + */ +public class Participant { + private final ParticipantConfig _config; + + private final RunningInstance _runningInstance; + + /** + * map of resource-id to current-state + */ + private final Map _currentStateMap; + + /** + * map of message-id to message + */ + private final Map _messageMap; + + /** + * Construct a participant + * @param config + */ + public Participant(ParticipantId id, String hostName, int port, boolean isEnabled, + Set disabledPartitionIdSet, Set tags, RunningInstance runningInstance, + Map currentStateMap, Map messageMap, + UserConfig userConfig) { + _config = + new ParticipantConfig(id, hostName, port, isEnabled, disabledPartitionIdSet, tags, + userConfig); + _runningInstance = runningInstance; + _currentStateMap = ImmutableMap.copyOf(currentStateMap); + _messageMap = ImmutableMap.copyOf(messageMap); + } + + /** + * Get the host name of the participant + * @return host name, or null if not applicable + */ + public String getHostName() { + return _config.getHostName(); + } + + /** + * Get the port of the participant + * @return port number, or -1 if not applicable + */ + public int getPort() { + return _config.getPort(); + } + + /** + * Get if the participant is enabled + * @return true if enabled or false otherwise + */ + public boolean isEnabled() { + return _config.isEnabled(); + } + + /** + * Get if the participant is alive + * @return true if running or false otherwise + */ + public boolean isAlive() { + return _runningInstance != null; + } + + /** + * Get the running instance + * @return running instance or null if not running + */ + public RunningInstance getRunningInstance() { + return _runningInstance; + } + + /** + * Get disabled partition id's + * @return set of disabled partition id's, or empty set if none + */ + public Set getDisabledPartitionIds() { + return _config.getDisabledPartitions(); + } + + /** + * Get tags + * @return set of tags + */ + public Set getTags() { + return _config.getTags(); + } + + /** + * Check if participant has a tag + * @param tag tag to check + * @return true if tagged, false otherwise + */ + public boolean hasTag(String tag) { + return _config.hasTag(tag); + } + + /** + * Get message map + * @return message map + */ + public Map getMessageMap() { + return _messageMap; + } + + /** + * Get the current states of the resource + * @return map of resource-id to current state, or empty map if none + */ + public Map getCurrentStateMap() { + return _currentStateMap; + } + + /** + * Get user-specified configuration properties of this participant + * @return UserConfig properties + */ + public UserConfig getUserConfig() { + return _config.getUserConfig(); + } + + /** + * Get the participant id + * @return ParticipantId + */ + public ParticipantId getId() { + return _config.getId(); + } + + /** + * Get the participant configuration + * @return ParticipantConfig that backs this participant + */ + public ParticipantConfig getConfig() { + return _config; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/AlertHistory.java b/helix-core/src/main/java/org/apache/helix/api/Partition.java similarity index 59% rename from helix-core/src/main/java/org/apache/helix/model/AlertHistory.java rename to helix-core/src/main/java/org/apache/helix/api/Partition.java index 58db71dc92..b295d72cd8 100644 --- a/helix-core/src/main/java/org/apache/helix/model/AlertHistory.java +++ b/helix-core/src/main/java/org/apache/helix/api/Partition.java @@ -1,4 +1,8 @@ -package org.apache.helix.model; +package org.apache.helix.api; + +import org.apache.helix.api.id.PartitionId; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -18,27 +22,33 @@ * specific language governing permissions and limitations * under the License. */ -import org.apache.helix.HelixProperty; -import org.apache.helix.ZNRecord; /** - * Maintains a history of alerts that have been fired, as well as actions taken, if any. + * A partition of a resource */ -public class AlertHistory extends HelixProperty { +public class Partition { + @JsonProperty("id") + private final PartitionId _id; /** - * Instantiate the alert history with a pre-populated record - * @param record ZNRecord representing the alert history + * Construct a partition + * @param id */ - public AlertHistory(ZNRecord record) { - super(record); - // TODO Auto-generated constructor stub + @JsonCreator + public Partition(@JsonProperty("id") PartitionId id) { + _id = id; } - @Override - public boolean isValid() { - // TODO Auto-generated method stub - return true; + /** + * Get partition id + * @return partition id + */ + public PartitionId getId() { + return _id; } + @Override + public String toString() { + return _id.toString(); + } } diff --git a/helix-core/src/main/java/org/apache/helix/api/Resource.java b/helix-core/src/main/java/org/apache/helix/api/Resource.java new file mode 100644 index 0000000000..79a1e09223 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/Resource.java @@ -0,0 +1,212 @@ +package org.apache.helix.api; + +/* + * 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.Set; + +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.ResourceConfig.ResourceType; +import org.apache.helix.api.config.SchedulerTaskConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.Message; +import org.apache.helix.model.ResourceAssignment; + +/** + * Represent a resource entity in helix cluster + */ +public class Resource { + private final ResourceConfig _config; + private final ExternalView _externalView; + private final ResourceAssignment _resourceAssignment; + + /** + * Construct a resource + * @param id resource id + * @param type ResourceType type + * @param idealState ideal state of the resource + * @param externalView external view of the resource + * @param resourceAssignment current resource assignment of the cluster + * @param rebalancerContext contextual parameters that the rebalancer should be aware of + * @param userConfig any resource user-defined configuration + * @param bucketSize the bucket size to use for physically saved state + * @param batchMessageMode true if batch messaging allowed, false otherwise + */ + public Resource(ResourceId id, ResourceType type, IdealState idealState, + ResourceAssignment resourceAssignment, ExternalView externalView, + RebalancerContext rebalancerContext, UserConfig userConfig, int bucketSize, + boolean batchMessageMode) { + SchedulerTaskConfig schedulerTaskConfig = schedulerTaskConfig(idealState); + RebalancerConfig rebalancerConfig = new RebalancerConfig(rebalancerContext); + + _config = + new ResourceConfig(id, type, schedulerTaskConfig, rebalancerConfig, userConfig, bucketSize, + batchMessageMode); + _externalView = externalView; + _resourceAssignment = resourceAssignment; + } + + /** + * Extract scheduler-task config from ideal-state if state-model-def is SchedulerTaskQueue + * @param idealState + * @return scheduler-task config or null if state-model-def is not SchedulerTaskQueue + */ + SchedulerTaskConfig schedulerTaskConfig(IdealState idealState) { + if (idealState == null) { + return null; + } + // TODO refactor get timeout + Map transitionTimeoutMap = new HashMap(); + for (String simpleKey : idealState.getRecord().getSimpleFields().keySet()) { + if (simpleKey.indexOf(Message.Attributes.TIMEOUT.name()) != -1) { + try { + String timeoutStr = idealState.getRecord().getSimpleField(simpleKey); + int timeout = Integer.parseInt(timeoutStr); + transitionTimeoutMap.put(simpleKey, timeout); + } catch (Exception e) { + // ignore + } + } + } + + Map innerMsgMap = new HashMap(); + if (idealState.getStateModelDefId().equalsIgnoreCase(StateModelDefId.SchedulerTaskQueue)) { + for (PartitionId partitionId : idealState.getPartitionIdSet()) { + // TODO refactor: scheduler-task-queue state model uses map-field to store inner-messages + // this is different from all other state-models + Map innerMsgStrMap = + idealState.getRecord().getMapField(partitionId.stringify()); + if (innerMsgStrMap != null) { + Message innerMsg = Message.toMessage(innerMsgStrMap); + innerMsgMap.put(partitionId, innerMsg); + } + } + } + + // System.out.println("transitionTimeoutMap: " + transitionTimeoutMap); + // System.out.println("innerMsgMap: " + innerMsgMap); + return new SchedulerTaskConfig(transitionTimeoutMap, innerMsgMap); + } + + /** + * Get the subunits of the resource + * @return map of subunit id to partition or empty map if none + */ + public Map getSubUnitMap() { + return _config.getSubUnitMap(); + } + + /** + * Get a subunit that the resource contains + * @param subUnitId the subunit id to look up + * @return Partition or null if none is present with the given id + */ + public Partition getSubUnit(PartitionId subUnitId) { + return _config.getSubUnit(subUnitId); + } + + /** + * Get the set of subunit ids that the resource contains + * @return subunit id set, or empty if none + */ + public Set getSubUnitSet() { + return _config.getSubUnitSet(); + } + + /** + * Get the external view of the resource + * @return the external view of the resource + */ + public ExternalView getExternalView() { + return _externalView; + } + + /** + * Get the current resource assignment + * @return ResourceAssignment, or null if no current assignment + */ + public ResourceAssignment getResourceAssignment() { + return _resourceAssignment; + } + + /** + * Get the resource properties configuring rebalancing + * @return RebalancerConfig properties + */ + public RebalancerConfig getRebalancerConfig() { + return _config.getRebalancerConfig(); + } + + /** + * Get user-specified configuration properties of this resource + * @return UserConfig properties + */ + public UserConfig getUserConfig() { + return _config.getUserConfig(); + } + + /** + * Get the resource id + * @return ResourceId + */ + public ResourceId getId() { + return _config.getId(); + } + + /** + * Get the properties configuring scheduler tasks + * @return SchedulerTaskConfig properties + */ + public SchedulerTaskConfig getSchedulerTaskConfig() { + return _config.getSchedulerTaskConfig(); + } + + /** + * Get bucket size + * @return bucket size + */ + public int getBucketSize() { + return _config.getBucketSize(); + } + + /** + * Get batch message mode + * @return true if in batch message mode, false otherwise + */ + public boolean getBatchMessageMode() { + return _config.getBatchMessageMode(); + } + + /** + * Get the configuration of this resource + * @return ResourceConfig that backs this Resource + */ + public ResourceConfig getConfig() { + return _config; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/RunningInstance.java b/helix-core/src/main/java/org/apache/helix/api/RunningInstance.java new file mode 100644 index 0000000000..4e5aedba41 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/RunningInstance.java @@ -0,0 +1,69 @@ +package org.apache.helix.api; + +import org.apache.helix.api.id.ProcId; +import org.apache.helix.api.id.SessionId; + +/* + * 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. + */ + +/** + * A running attributes of a helix instance + */ +public class RunningInstance { + private final SessionId _sessionId; + private final HelixVersion _version; + private final ProcId _pid; + + /** + * Construct running instance + * @param sessionId zookeeper session-id + * @param version helix-version + * @param pid running jvm name + */ + public RunningInstance(SessionId sessionId, HelixVersion version, ProcId pid) { + _sessionId = sessionId; + _version = version; + _pid = pid; + } + + /** + * Get session id of the running instance + * session id is the zookeeper session id + * @return session id + */ + public SessionId getSessionId() { + return _sessionId; + } + + /** + * Get helix version of the running instance + * @return helix version + */ + public HelixVersion getVersion() { + return _version; + } + + /** + * Get the name of the running jvm of the running instance + * @return running jvm name (e.g. 1111@host) + */ + public ProcId getPid() { + return _pid; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/Scope.java b/helix-core/src/main/java/org/apache/helix/api/Scope.java new file mode 100644 index 0000000000..26e09a9a87 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/Scope.java @@ -0,0 +1,125 @@ +package org.apache.helix.api; + +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.Id; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; + +/* + * 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. + */ + +/** + * Represents the scope of an object. For instance, a configuration can belong to a specific scope + * of cluster, participant, partition, or resource, but not more than one of these at any time. + */ +public class Scope { + public enum ScopeType { + CLUSTER, + PARTICIPANT, + PARTITION, + RESOURCE + } + + private final T _id; + + /** + * Private: instantiate a scope with an id + * @param id any object that extends Id + */ + private Scope(T id) { + _id = id; + } + + /** + * Get the scope that is tracked + * @return The id of the scoped object + */ + public T getScopedId() { + return _id; + } + + @Override + public String toString() { + return getType() + "{" + getScopedId() + "}"; + } + + @Override + public boolean equals(Object that) { + if (that instanceof Scope) { + return this.toString().equals(that.toString()); + } + return false; + } + + /** + * Get the Helix entity type that this scope covers + * @return scope type + */ + public ScopeType getType() { + Class idClass = _id.getClass(); + if (ClusterId.class.equals(idClass)) { + return ScopeType.CLUSTER; + } else if (ParticipantId.class.equals(idClass)) { + return ScopeType.PARTICIPANT; + } else if (PartitionId.class.equals(idClass)) { + return ScopeType.PARTITION; + } else if (ResourceId.class.equals(idClass)) { + return ScopeType.RESOURCE; + } else { + return null; + } + } + + /** + * Get a cluster scope + * @param clusterId the id of the cluster that is scoped + * @return cluster scope + */ + public static Scope cluster(ClusterId clusterId) { + return new Scope(clusterId); + } + + /** + * Get a participant scope + * @param participantId the id of the participant that is scoped + * @return participant scope + */ + public static Scope participant(ParticipantId participantId) { + return new Scope(participantId); + } + + /** + * Get a partition scope + * @param partitionId the id of the partition that is scoped + * @return partition scope + */ + public static Scope partition(PartitionId partitionId) { + return new Scope(partitionId); + } + + /** + * Get a resource scope + * @param resourceId the id of the resource that is scoped + * @return resource scope + */ + public static Scope resource(ResourceId resourceId) { + return new Scope(resourceId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/ExpressionOperatorType.java b/helix-core/src/main/java/org/apache/helix/api/Spectator.java similarity index 66% rename from helix-core/src/main/java/org/apache/helix/alerts/ExpressionOperatorType.java rename to helix-core/src/main/java/org/apache/helix/api/Spectator.java index 6c4f913f93..993de2f767 100644 --- a/helix-core/src/main/java/org/apache/helix/alerts/ExpressionOperatorType.java +++ b/helix-core/src/main/java/org/apache/helix/api/Spectator.java @@ -1,4 +1,6 @@ -package org.apache.helix.alerts; +package org.apache.helix.api; + +import org.apache.helix.api.id.SpectatorId; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,26 +21,25 @@ * under the License. */ -public enum ExpressionOperatorType { - // each - EACH(true), - // standard math - SUM(false), - MULTIPLY(false), - SUBTRACT(false), - DIVIDE(false), - // aggregation types - ACCUMULATE(true), - DECAY(false), - WINDOW(false); - - boolean isBase; +/** + * A cluster spectator that listen on cluster changes + */ +public class Spectator { + private final SpectatorId _id; - private ExpressionOperatorType(boolean isBase) { - this.isBase = isBase; + /** + * Construct a spectator with id + * @param id + */ + public Spectator(SpectatorId id) { + _id = id; } - boolean isBaseOp() { - return isBase; + /** + * Spectator id + * @return spectator id + */ + public SpectatorId getId() { + return _id; } } diff --git a/helix-core/src/main/java/org/apache/helix/api/State.java b/helix-core/src/main/java/org/apache/helix/api/State.java new file mode 100644 index 0000000000..3315987797 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/State.java @@ -0,0 +1,85 @@ +package org.apache.helix.api; + +import org.apache.helix.HelixDefinedState; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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. + */ + +/** + * A state in a state model managed by Helix + */ +public class State { + @JsonProperty("state") + private final String _state; + + /** + * Create a state + * @param state string representing a state + */ + @JsonCreator + public State(@JsonProperty("state") String state) { + _state = state.toUpperCase(); + } + + @Override + public String toString() { + return _state; + } + + @Override + public boolean equals(Object that) { + if (that instanceof State) { + return this.toString().equals(((State) that).toString()); + } else if (that instanceof String) { + return _state.equals(that); + } + return false; + } + + @Override + public int hashCode() { + return _state.hashCode(); + } + + /** + * Get a State from a state name + * @param state state name + * @return State + */ + public static State from(String state) { + if (state == null) { + return null; + } + return new State(state); + } + + /** + * Get a State from a HelixDefinedState + * @param state HelixDefinedState + * @return State + */ + public static State from(HelixDefinedState state) { + if (state == null) { + return null; + } + return new State(state.toString()); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicClusterAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicClusterAccessor.java new file mode 100644 index 0000000000..37cc47eda4 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicClusterAccessor.java @@ -0,0 +1,260 @@ +package org.apache.helix.api.accessor; + +/* + * 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 java.util.Set; + +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Resource; +import org.apache.helix.api.Scope; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.lock.HelixLock; +import org.apache.helix.lock.HelixLockable; +import org.apache.log4j.Logger; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +/** + * An atomic version of the ClusterAccessor. If atomic operations are required, use instances of + * this class. Atomicity is not guaranteed when using instances of ClusterAccessor alongside + * instances of this class. Furthermore, depending on the semantics of the lock, lock acquisition + * may fail, in which case users should handle the return value of each function if necessary.
+ *
+ * Using this class is quite expensive; it should thus be used sparingly and only in systems where + * contention on these operations is expected. For most systems running Helix, this is typically not + * the case. + */ +public class AtomicClusterAccessor extends ClusterAccessor { + private static final Logger LOG = Logger.getLogger(AtomicClusterAccessor.class); + + private final HelixLockable _lockProvider; + private final HelixDataAccessor _accessor; + private final PropertyKey.Builder _keyBuilder; + private final ClusterId _clusterId; + + /** + * Non-atomic instance to protect against reentrant locking via polymorphism + */ + private final ClusterAccessor _clusterAccessor; + + /** + * Instantiate the accessor + * @param clusterId the cluster to access + * @param accessor a HelixDataAccessor for the physical properties + * @param lockProvider a lock provider + */ + public AtomicClusterAccessor(ClusterId clusterId, HelixDataAccessor accessor, + HelixLockable lockProvider) { + super(clusterId, accessor); + _lockProvider = lockProvider; + _accessor = accessor; + _keyBuilder = accessor.keyBuilder(); + _clusterId = clusterId; + _clusterAccessor = new ClusterAccessor(clusterId, accessor); + } + + @Override + public boolean createCluster(ClusterConfig cluster) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.cluster(_clusterId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _clusterAccessor.createCluster(cluster); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public boolean dropCluster() { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.cluster(_clusterId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _clusterAccessor.dropCluster(); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public Cluster readCluster() { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.cluster(_clusterId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _clusterAccessor.readCluster(); + } finally { + lock.unlock(); + } + } + return null; + } + + @Override + public boolean addParticipantToCluster(ParticipantConfig participant) { + if (participant == null) { + LOG.error("Participant config cannot be null"); + return false; + } + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participant.getId())); + boolean locked = lock.lock(); + if (locked) { + try { + return _clusterAccessor.addParticipantToCluster(participant); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public boolean dropParticipantFromCluster(ParticipantId participantId) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participantId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _clusterAccessor.dropParticipantFromCluster(participantId); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public boolean addResourceToCluster(ResourceConfig resource) { + if (resource == null) { + LOG.error("Resource config cannot be null"); + return false; + } + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.resource(resource.getId())); + boolean locked = lock.lock(); + if (locked) { + try { + return _clusterAccessor.addResourceToCluster(resource); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public boolean dropResourceFromCluster(ResourceId resourceId) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.resource(resourceId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _clusterAccessor.dropResourceFromCluster(resourceId); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public ClusterConfig updateCluster(ClusterConfig.Delta clusterDelta) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.cluster(_clusterId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _clusterAccessor.updateCluster(clusterDelta); + } finally { + lock.unlock(); + } + } + return null; + } + + /** + * Read resources atomically. This is resource-atomic, not cluster-atomic + */ + @Override + public Map readResources() { + // read resources individually instead of together to maintain the equality link between ideal + // state and resource config + Map resources = Maps.newHashMap(); + Set idealStateNames = + Sets.newHashSet(_accessor.getChildNames(_keyBuilder.idealStates())); + Set resourceConfigNames = + Sets.newHashSet(_accessor.getChildNames(_keyBuilder.resourceConfigs())); + resourceConfigNames.addAll(idealStateNames); + ResourceAccessor accessor = new AtomicResourceAccessor(_clusterId, _accessor, _lockProvider); + for (String resourceName : resourceConfigNames) { + ResourceId resourceId = ResourceId.from(resourceName); + Resource resource = accessor.readResource(resourceId); + if (resource != null) { + resources.put(resourceId, resource); + } + } + return resources; + } + + /** + * Read participants atomically. This is participant-atomic, not cluster-atomic + */ + @Override + public Map readParticipants() { + // read participants individually to keep configs consistent with current state and messages + Map participants = Maps.newHashMap(); + ParticipantAccessor accessor = + new AtomicParticipantAccessor(_clusterId, _accessor, _lockProvider); + List participantNames = _accessor.getChildNames(_keyBuilder.instanceConfigs()); + for (String participantName : participantNames) { + ParticipantId participantId = ParticipantId.from(participantName); + Participant participant = accessor.readParticipant(participantId); + if (participant != null) { + participants.put(participantId, participant); + } + } + return participants; + } + + @Override + public void initClusterStructure() { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.cluster(_clusterId)); + boolean locked = lock.lock(); + if (locked) { + try { + _clusterAccessor.initClusterStructure(); + } finally { + lock.unlock(); + } + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicParticipantAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicParticipantAccessor.java new file mode 100644 index 0000000000..1c734e3477 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicParticipantAccessor.java @@ -0,0 +1,211 @@ +package org.apache.helix.api.accessor; + +/* + * 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.Set; + +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Scope; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.lock.HelixLock; +import org.apache.helix.lock.HelixLockable; +import org.apache.helix.model.Message; +import org.apache.log4j.Logger; + +/** + * An atomic version of the ParticipantAccessor. If atomic operations are required, use instances of + * this class. Atomicity is not guaranteed when using instances of ParticipantAccessor alongside + * instances of this class. Furthermore, depending on the semantics of the lock, lock acquisition + * may fail, in which case users should handle the return value of each function if necessary.
+ *
+ * Using this class is quite expensive; it should thus be used sparingly and only in systems where + * contention on these operations is expected. For most systems running Helix, this is typically not + * the case. + */ +public class AtomicParticipantAccessor extends ParticipantAccessor { + private static final Logger LOG = Logger.getLogger(AtomicParticipantAccessor.class); + + private final ClusterId _clusterId; + private final HelixDataAccessor _accessor; + private final HelixLockable _lockProvider; + + /** + * Non-atomic instance to protect against reentrant locking via polymorphism + */ + private final ParticipantAccessor _participantAccessor; + + /** + * Instantiate the accessor + * @param clusterId the cluster to access + * @param accessor a HelixDataAccessor for the physical properties + * @param lockProvider a lock provider + */ + public AtomicParticipantAccessor(ClusterId clusterId, HelixDataAccessor accessor, + HelixLockable lockProvider) { + super(accessor); + _clusterId = clusterId; + _accessor = accessor; + _lockProvider = lockProvider; + _participantAccessor = new ParticipantAccessor(accessor); + } + + @Override + boolean enableParticipant(ParticipantId participantId, boolean isEnabled) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participantId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _participantAccessor.enableParticipant(participantId); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public Participant readParticipant(ParticipantId participantId) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participantId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _participantAccessor.readParticipant(participantId); + } finally { + lock.unlock(); + } + } + return null; + } + + @Override + public boolean setParticipant(ParticipantConfig participantConfig) { + if (participantConfig == null) { + LOG.error("participant config cannot be null"); + return false; + } + HelixLock lock = + _lockProvider.getLock(_clusterId, Scope.participant(participantConfig.getId())); + boolean locked = lock.lock(); + if (locked) { + try { + return _participantAccessor.setParticipant(participantConfig); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public ParticipantConfig updateParticipant(ParticipantId participantId, + ParticipantConfig.Delta participantDelta) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participantId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _participantAccessor.updateParticipant(participantId, participantDelta); + } finally { + lock.unlock(); + } + } + return null; + } + + @Override + boolean dropParticipant(ParticipantId participantId) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participantId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _participantAccessor.dropParticipant(participantId); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public void insertMessagesToParticipant(ParticipantId participantId, + Map msgMap) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participantId)); + boolean locked = lock.lock(); + if (locked) { + try { + _participantAccessor.insertMessagesToParticipant(participantId, msgMap); + } finally { + lock.unlock(); + } + } + return; + } + + @Override + public void updateMessageStatus(ParticipantId participantId, Map msgMap) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participantId)); + boolean locked = lock.lock(); + if (locked) { + try { + _participantAccessor.updateMessageStatus(participantId, msgMap); + } finally { + lock.unlock(); + } + } + return; + } + + @Override + public void deleteMessagesFromParticipant(ParticipantId participantId, Set msgIdSet) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participantId)); + boolean locked = lock.lock(); + if (locked) { + try { + _participantAccessor.deleteMessagesFromParticipant(participantId, msgIdSet); + } finally { + lock.unlock(); + } + } + return; + } + + @Override + public void initParticipantStructure(ParticipantId participantId) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.participant(participantId)); + boolean locked = lock.lock(); + if (locked) { + try { + _participantAccessor.initParticipantStructure(participantId); + } finally { + lock.unlock(); + } + } + return; + } + + @Override + protected ResourceAccessor resourceAccessor() { + return new AtomicResourceAccessor(_clusterId, _accessor, _lockProvider); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java new file mode 100644 index 0000000000..6d69981de7 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java @@ -0,0 +1,150 @@ +package org.apache.helix.api.accessor; + +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.api.Resource; +import org.apache.helix.api.Scope; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.lock.HelixLock; +import org.apache.helix.lock.HelixLockable; +import org.apache.log4j.Logger; + +/* + * 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. + */ + +/** + * An atomic version of the ResourceAccessor. If atomic operations are required, use instances of + * this class. Atomicity is not guaranteed when using instances of ResourceAccessor alongside + * instances of this class. Furthermore, depending on the semantics of the lock, lock acquisition + * may fail, in which case users should handle the return value of each function if necessary.
+ *
+ * Using this class is quite expensive; it should thus be used sparingly and only in systems where + * contention on these operations is expected. For most systems running Helix, this is typically not + * the case. + */ +public class AtomicResourceAccessor extends ResourceAccessor { + private static final Logger LOG = Logger.getLogger(AtomicResourceAccessor.class); + + private final ClusterId _clusterId; + private final HelixDataAccessor _accessor; + private final HelixLockable _lockProvider; + + /** + * Non-atomic instance to protect against reentrant locking via polymorphism + */ + private final ResourceAccessor _resourceAccessor; + + /** + * Instantiate the accessor + * @param clusterId the cluster to access + * @param accessor a HelixDataAccessor for the physical properties + * @param lockProvider a lock provider + */ + public AtomicResourceAccessor(ClusterId clusterId, HelixDataAccessor accessor, + HelixLockable lockProvider) { + super(accessor); + _clusterId = clusterId; + _accessor = accessor; + _lockProvider = lockProvider; + _resourceAccessor = new ResourceAccessor(accessor); + } + + @Override + public Resource readResource(ResourceId resourceId) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.resource(resourceId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _resourceAccessor.readResource(resourceId); + } finally { + lock.unlock(); + } + } + return null; + } + + @Override + public ResourceConfig updateResource(ResourceId resourceId, ResourceConfig.Delta resourceDelta) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.resource(resourceId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _resourceAccessor.updateResource(resourceId, resourceDelta); + } finally { + lock.unlock(); + } + } + return null; + } + + @Override + public boolean setRebalancerContext(ResourceId resourceId, RebalancerContext context) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.resource(resourceId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _resourceAccessor.setRebalancerContext(resourceId, context); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public boolean setResource(ResourceConfig resourceConfig) { + if (resourceConfig == null) { + LOG.error("resource config cannot be null"); + return false; + } + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.resource(resourceConfig.getId())); + boolean locked = lock.lock(); + if (locked) { + try { + return _resourceAccessor.setResource(resourceConfig); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + public boolean generateDefaultAssignment(ResourceId resourceId, int replicaCount, + String participantGroupTag) { + HelixLock lock = _lockProvider.getLock(_clusterId, Scope.cluster(_clusterId)); + boolean locked = lock.lock(); + if (locked) { + try { + return _resourceAccessor.generateDefaultAssignment(resourceId, replicaCount, + participantGroupTag); + } finally { + lock.unlock(); + } + } + return false; + } + + @Override + protected ParticipantAccessor participantAccessor() { + return new AtomicParticipantAccessor(_clusterId, _accessor, _lockProvider); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java new file mode 100644 index 0000000000..f08d857dca --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java @@ -0,0 +1,673 @@ +package org.apache.helix.api.accessor; + +/* + * 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.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.AccessOption; +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Controller; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Resource; +import org.apache.helix.api.Scope; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ConstraintId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.model.ClusterConfiguration; +import org.apache.helix.model.ClusterConstraints; +import org.apache.helix.model.ClusterConstraints.ConstraintType; +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.Leader; +import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Message; +import org.apache.helix.model.PauseSignal; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.ResourceConfiguration; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +public class ClusterAccessor { + private static Logger LOG = Logger.getLogger(ClusterAccessor.class); + + private final HelixDataAccessor _accessor; + private final PropertyKey.Builder _keyBuilder; + private final ClusterId _clusterId; + + /** + * Instantiate a cluster accessor + * @param clusterId the cluster to access + * @param accessor HelixDataAccessor for the physical store + */ + public ClusterAccessor(ClusterId clusterId, HelixDataAccessor accessor) { + _accessor = accessor; + _keyBuilder = accessor.keyBuilder(); + _clusterId = clusterId; + } + + /** + * create a new cluster, fail if it already exists + * @return true if created, false if creation failed + */ + public boolean createCluster(ClusterConfig cluster) { + ClusterConfiguration configuration = _accessor.getProperty(_keyBuilder.clusterConfig()); + if (configuration != null && isClusterStructureValid()) { + LOG.error("Cluster already created. Aborting."); + return false; + } + clearClusterStructure(); + initClusterStructure(); + Map stateModelDefs = cluster.getStateModelMap(); + for (StateModelDefinition stateModelDef : stateModelDefs.values()) { + addStateModelDefinitionToCluster(stateModelDef); + } + Map resources = cluster.getResourceMap(); + for (ResourceConfig resource : resources.values()) { + addResourceToCluster(resource); + } + Map participants = cluster.getParticipantMap(); + for (ParticipantConfig participant : participants.values()) { + addParticipantToCluster(participant); + } + _accessor.createProperty(_keyBuilder.constraints(), null); + for (ClusterConstraints constraints : cluster.getConstraintMap().values()) { + _accessor.setProperty(_keyBuilder.constraint(constraints.getType().toString()), constraints); + } + ClusterConfiguration clusterConfig = ClusterConfiguration.from(cluster.getUserConfig()); + if (cluster.autoJoinAllowed()) { + clusterConfig.setAutoJoinAllowed(cluster.autoJoinAllowed()); + } + if (cluster.isPaused()) { + pauseCluster(); + } + _accessor.setProperty(_keyBuilder.clusterConfig(), clusterConfig); + + return true; + } + + /** + * Update the cluster configuration + * @param clusterDelta change to the cluster configuration + * @return updated ClusterConfig, or null if there was an error + */ + public ClusterConfig updateCluster(ClusterConfig.Delta clusterDelta) { + Cluster cluster = readCluster(); + if (cluster == null) { + LOG.error("Cluster does not exist, cannot be updated"); + return null; + } + ClusterConfig config = clusterDelta.mergeInto(cluster.getConfig()); + boolean status = setBasicClusterConfig(config); + return status ? config : null; + } + + /** + * Set a cluster config minus state model, participants, and resources + * @param config ClusterConfig + * @return true if correctly set, false otherwise + */ + private boolean setBasicClusterConfig(ClusterConfig config) { + if (config == null) { + return false; + } + ClusterConfiguration configuration = ClusterConfiguration.from(config.getUserConfig()); + configuration.setAutoJoinAllowed(config.autoJoinAllowed()); + _accessor.setProperty(_keyBuilder.clusterConfig(), configuration); + Map constraints = config.getConstraintMap(); + for (ConstraintType type : constraints.keySet()) { + ClusterConstraints constraint = constraints.get(type); + _accessor.setProperty(_keyBuilder.constraint(type.toString()), constraint); + } + return true; + } + + /** + * drop a cluster + * @return true if the cluster was dropped, false if there was an error + */ + public boolean dropCluster() { + LOG.info("Dropping cluster: " + _clusterId); + List liveInstanceNames = _accessor.getChildNames(_keyBuilder.liveInstances()); + if (liveInstanceNames.size() > 0) { + LOG.error("Can't drop cluster: " + _clusterId + " because there are running participant: " + + liveInstanceNames + ", shutdown participants first."); + return false; + } + + Leader leader = _accessor.getProperty(_keyBuilder.controllerLeader()); + if (leader != null) { + LOG.error("Can't drop cluster: " + _clusterId + ", because leader: " + leader.getId() + + " are running, shutdown leader first."); + return false; + } + + return _accessor.removeProperty(_keyBuilder.cluster()); + } + + /** + * read entire cluster data + * @return cluster snapshot or null + */ + public Cluster readCluster() { + if (!isClusterStructureValid()) { + LOG.error("Cluster is not fully set up"); + return null; + } + Leader leader = _accessor.getProperty(_keyBuilder.controllerLeader()); + + /** + * map of constraint-type to constraints + */ + Map constraintMap = + _accessor.getChildValuesMap(_keyBuilder.constraints()); + + // read all the resources + Map resourceMap = readResources(); + + // read all the participants + Map participantMap = readParticipants(); + + // read the controllers + Map controllerMap = new HashMap(); + ControllerId leaderId = null; + if (leader != null) { + leaderId = ControllerId.from(leader.getId()); + controllerMap.put(leaderId, new Controller(leaderId, leader, true)); + } + + // read the constraints + Map clusterConstraintMap = + new HashMap(); + for (String constraintType : constraintMap.keySet()) { + clusterConstraintMap.put(ConstraintType.valueOf(constraintType), + constraintMap.get(constraintType)); + } + + // read the pause status + PauseSignal pauseSignal = _accessor.getProperty(_keyBuilder.pause()); + boolean isPaused = pauseSignal != null; + + ClusterConfiguration clusterConfig = _accessor.getProperty(_keyBuilder.clusterConfig()); + boolean autoJoinAllowed = false; + UserConfig userConfig; + if (clusterConfig != null) { + userConfig = clusterConfig.getUserConfig(); + autoJoinAllowed = clusterConfig.autoJoinAllowed(); + } else { + userConfig = new UserConfig(Scope.cluster(_clusterId)); + } + + // read the state model definitions + Map stateModelMap = readStateModelDefinitions(); + + // create the cluster snapshot object + return new Cluster(_clusterId, resourceMap, participantMap, controllerMap, leaderId, + clusterConstraintMap, stateModelMap, userConfig, isPaused, autoJoinAllowed); + } + + /** + * Get all the state model definitions for this cluster + * @return map of state model def id to state model definition + */ + public Map readStateModelDefinitions() { + Map stateModelDefs = Maps.newHashMap(); + List stateModelList = + _accessor.getChildValues(_keyBuilder.stateModelDefs()); + for (StateModelDefinition stateModelDef : stateModelList) { + stateModelDefs.put(stateModelDef.getStateModelDefId(), stateModelDef); + } + return stateModelDefs; + } + + /** + * Read all resources in the cluster + * @return map of resource id to resource + */ + public Map readResources() { + if (!isClusterStructureValid()) { + LOG.error("Cluster is not fully set up yet!"); + return Collections.emptyMap(); + } + + /** + * map of resource-id to ideal-state + */ + Map idealStateMap = _accessor.getChildValuesMap(_keyBuilder.idealStates()); + + /** + * Map of resource id to external view + */ + Map externalViewMap = + _accessor.getChildValuesMap(_keyBuilder.externalViews()); + + /** + * Map of resource id to user configuration + */ + Map resourceConfigMap = + _accessor.getChildValuesMap(_keyBuilder.resourceConfigs()); + + /** + * Map of resource id to resource assignment + */ + Map resourceAssignmentMap = + _accessor.getChildValuesMap(_keyBuilder.resourceAssignments()); + + // read all the resources + Set allResources = Sets.newHashSet(); + allResources.addAll(idealStateMap.keySet()); + allResources.addAll(resourceConfigMap.keySet()); + Map resourceMap = Maps.newHashMap(); + for (String resourceName : allResources) { + ResourceId resourceId = ResourceId.from(resourceName); + resourceMap.put(resourceId, ResourceAccessor.createResource(resourceId, + resourceConfigMap.get(resourceName), idealStateMap.get(resourceName), + externalViewMap.get(resourceName), resourceAssignmentMap.get(resourceName))); + } + + return resourceMap; + } + + /** + * Read all participants in the cluster + * @return map of participant id to participant, or empty map + */ + public Map readParticipants() { + if (!isClusterStructureValid()) { + LOG.error("Cluster is not fully set up yet!"); + return Collections.emptyMap(); + } + + /** + * map of instance-id to instance-config + */ + Map instanceConfigMap = + _accessor.getChildValuesMap(_keyBuilder.instanceConfigs()); + + /** + * map of instance-id to live-instance + */ + Map liveInstanceMap = + _accessor.getChildValuesMap(_keyBuilder.liveInstances()); + + /** + * map of participant-id to map of message-id to message + */ + Map> messageMap = new HashMap>(); + for (String instanceName : liveInstanceMap.keySet()) { + Map instanceMsgMap = + _accessor.getChildValuesMap(_keyBuilder.messages(instanceName)); + messageMap.put(instanceName, instanceMsgMap); + } + + /** + * map of participant-id to map of resource-id to current-state + */ + Map> currentStateMap = + new HashMap>(); + for (String participantName : liveInstanceMap.keySet()) { + LiveInstance liveInstance = liveInstanceMap.get(participantName); + SessionId sessionId = liveInstance.getTypedSessionId(); + Map instanceCurStateMap = + _accessor.getChildValuesMap(_keyBuilder.currentStates(participantName, + sessionId.stringify())); + + currentStateMap.put(participantName, instanceCurStateMap); + } + + // read all the participants + Map participantMap = Maps.newHashMap(); + for (String participantName : instanceConfigMap.keySet()) { + InstanceConfig instanceConfig = instanceConfigMap.get(participantName); + UserConfig userConfig = instanceConfig.getUserConfig(); + LiveInstance liveInstance = liveInstanceMap.get(participantName); + Map instanceMsgMap = messageMap.get(participantName); + + ParticipantId participantId = ParticipantId.from(participantName); + + participantMap.put(participantId, ParticipantAccessor.createParticipant(participantId, + instanceConfig, userConfig, liveInstance, instanceMsgMap, + currentStateMap.get(participantName))); + } + + return participantMap; + } + + /** + * Get cluster constraints of a given type + * @param type ConstraintType value + * @return ClusterConstraints, or null if none present + */ + public ClusterConstraints readConstraints(ConstraintType type) { + return _accessor.getProperty(_keyBuilder.constraint(type.toString())); + } + + /** + * Remove a constraint from the cluster + * @param type the constraint type + * @param constraintId the constraint id + * @return true if removed, false otherwise + */ + public boolean removeConstraint(ConstraintType type, ConstraintId constraintId) { + ClusterConstraints constraints = _accessor.getProperty(_keyBuilder.constraint(type.toString())); + if (constraints == null || constraints.getConstraintItem(constraintId) == null) { + LOG.error("Constraint with id " + constraintId + " not present"); + return false; + } + constraints.removeConstraintItem(constraintId); + return _accessor.setProperty(_keyBuilder.constraint(type.toString()), constraints); + } + + /** + * Read the user config of the cluster + * @return UserConfig, or null + */ + public UserConfig readUserConfig() { + ClusterConfiguration clusterConfig = _accessor.getProperty(_keyBuilder.clusterConfig()); + return clusterConfig != null ? clusterConfig.getUserConfig() : null; + } + + /** + * Set the user config of the cluster, overwriting existing user configs + * @param userConfig the new user config + * @return true if the user config was set, false otherwise + */ + public boolean setUserConfig(UserConfig userConfig) { + ClusterConfig.Delta delta = new ClusterConfig.Delta(_clusterId).setUserConfig(userConfig); + return updateCluster(delta) != null; + } + + /** + * Clear any user-specified configuration from the cluster + * @return true if the config was cleared, false otherwise + */ + public boolean dropUserConfig() { + return setUserConfig(new UserConfig(Scope.cluster(_clusterId))); + } + + /** + * Add user configuration to the existing cluster user configuration. Overwrites properties with + * the same key + * @param userConfig the user config key-value pairs to add + * @return true if the user config was updated, false otherwise + */ + public boolean updateUserConfig(UserConfig userConfig) { + ClusterConfiguration clusterConfig = new ClusterConfiguration(_clusterId); + clusterConfig.addNamespacedConfig(userConfig); + return _accessor.updateProperty(_keyBuilder.clusterConfig(), clusterConfig); + } + + /** + * pause controller of cluster + * @return true if cluster was paused, false if pause failed or already paused + */ + public boolean pauseCluster() { + return _accessor.createProperty(_keyBuilder.pause(), new PauseSignal("pause")); + } + + /** + * resume controller of cluster + * @return true if resume succeeded, false otherwise + */ + public boolean resumeCluster() { + return _accessor.removeProperty(_keyBuilder.pause()); + } + + /** + * add a resource to cluster + * @param resource + * @return true if resource added, false if there was an error + */ + public boolean addResourceToCluster(ResourceConfig resource) { + if (resource == null || resource.getRebalancerConfig() == null) { + LOG.error("Resource not fully defined with a rebalancer context"); + return false; + } + + if (!isClusterStructureValid()) { + LOG.error("Cluster: " + _clusterId + " structure is not valid"); + return false; + } + RebalancerContext context = + resource.getRebalancerConfig().getRebalancerContext(RebalancerContext.class); + StateModelDefId stateModelDefId = context.getStateModelDefId(); + if (_accessor.getProperty(_keyBuilder.stateModelDef(stateModelDefId.stringify())) == null) { + LOG.error("State model: " + stateModelDefId + " not found in cluster: " + _clusterId); + return false; + } + + ResourceId resourceId = resource.getId(); + if (_accessor.getProperty(_keyBuilder.idealStates(resourceId.stringify())) != null) { + LOG.error("Skip adding resource: " + resourceId + + ", because resource ideal state already exists in cluster: " + _clusterId); + return false; + } + if (_accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify())) != null) { + LOG.error("Skip adding resource: " + resourceId + + ", because resource config already exists in cluster: " + _clusterId); + return false; + } + + // Add resource user config + if (resource.getUserConfig() != null) { + ResourceConfiguration configuration = new ResourceConfiguration(resourceId); + configuration.setType(resource.getType()); + configuration.addNamespacedConfig(resource.getUserConfig()); + configuration.addNamespacedConfig(resource.getRebalancerConfig().toNamespacedConfig()); + configuration.setBucketSize(resource.getBucketSize()); + configuration.setBatchMessageMode(resource.getBatchMessageMode()); + _accessor.setProperty(_keyBuilder.resourceConfig(resourceId.stringify()), configuration); + } + + // Create an IdealState from a RebalancerConfig (if the resource is partitioned) + RebalancerConfig rebalancerConfig = resource.getRebalancerConfig(); + IdealState idealState = + ResourceAccessor.rebalancerConfigToIdealState(rebalancerConfig, resource.getBucketSize(), + resource.getBatchMessageMode()); + if (idealState != null) { + _accessor.setProperty(_keyBuilder.idealStates(resourceId.stringify()), idealState); + } + return true; + } + + /** + * drop a resource from cluster + * @param resourceId + * @return true if removal succeeded, false otherwise + */ + public boolean dropResourceFromCluster(ResourceId resourceId) { + if (_accessor.getProperty(_keyBuilder.idealStates(resourceId.stringify())) == null) { + LOG.error("Skip removing resource: " + resourceId + + ", because resource ideal state already removed from cluster: " + _clusterId); + return false; + } + _accessor.removeProperty(_keyBuilder.idealStates(resourceId.stringify())); + _accessor.removeProperty(_keyBuilder.resourceConfig(resourceId.stringify())); + return true; + } + + /** + * check if cluster structure is valid + * @return true if valid or false otherwise + */ + public boolean isClusterStructureValid() { + List paths = getRequiredPaths(_keyBuilder); + BaseDataAccessor baseAccessor = _accessor.getBaseDataAccessor(); + if (baseAccessor != null) { + boolean[] existsResults = baseAccessor.exists(paths, 0); + for (boolean exists : existsResults) { + if (!exists) { + return false; + } + } + } + return true; + } + + /** + * Create empty persistent properties to ensure that there is a valid cluster structure + */ + public void initClusterStructure() { + BaseDataAccessor baseAccessor = _accessor.getBaseDataAccessor(); + List paths = getRequiredPaths(_keyBuilder); + for (String path : paths) { + boolean status = baseAccessor.create(path, null, AccessOption.PERSISTENT); + if (!status && LOG.isDebugEnabled()) { + LOG.debug(path + " already exists"); + } + } + } + + /** + * Remove all but the top level cluster node; intended for reconstructing the cluster + */ + private void clearClusterStructure() { + BaseDataAccessor baseAccessor = _accessor.getBaseDataAccessor(); + List paths = getRequiredPaths(_keyBuilder); + baseAccessor.remove(paths, 0); + } + + /** + * Get all property paths that must be set for a cluster structure to be valid + * @param keyBuilder a PropertyKey.Builder for the cluster + * @return list of paths as strings + */ + private static List getRequiredPaths(PropertyKey.Builder keyBuilder) { + List paths = Lists.newArrayList(); + paths.add(keyBuilder.clusterConfigs().getPath()); + paths.add(keyBuilder.instanceConfigs().getPath()); + paths.add(keyBuilder.propertyStore().getPath()); + paths.add(keyBuilder.liveInstances().getPath()); + paths.add(keyBuilder.instances().getPath()); + paths.add(keyBuilder.externalViews().getPath()); + paths.add(keyBuilder.controller().getPath()); + paths.add(keyBuilder.stateModelDefs().getPath()); + paths.add(keyBuilder.controllerMessages().getPath()); + paths.add(keyBuilder.controllerTaskErrors().getPath()); + paths.add(keyBuilder.controllerTaskStatuses().getPath()); + paths.add(keyBuilder.controllerLeaderHistory().getPath()); + return paths; + } + + /** + * add a participant to cluster + * @param participant + * @return true if participant added, false otherwise + */ + public boolean addParticipantToCluster(ParticipantConfig participant) { + if (participant == null) { + LOG.error("Participant not initialized"); + return false; + } + if (!isClusterStructureValid()) { + LOG.error("Cluster: " + _clusterId + " structure is not valid"); + return false; + } + + ParticipantAccessor participantAccessor = new ParticipantAccessor(_accessor); + ParticipantId participantId = participant.getId(); + InstanceConfig existConfig = + _accessor.getProperty(_keyBuilder.instanceConfig(participantId.stringify())); + if (existConfig != null && participantAccessor.isParticipantStructureValid(participantId)) { + LOG.error("Config for participant: " + participantId + " already exists in cluster: " + + _clusterId); + return false; + } + + // clear and rebuild the participant structure + participantAccessor.clearParticipantStructure(participantId); + participantAccessor.initParticipantStructure(participantId); + + // add the config + InstanceConfig instanceConfig = new InstanceConfig(participant.getId()); + instanceConfig.setHostName(participant.getHostName()); + instanceConfig.setPort(Integer.toString(participant.getPort())); + instanceConfig.setInstanceEnabled(participant.isEnabled()); + UserConfig userConfig = participant.getUserConfig(); + instanceConfig.addNamespacedConfig(userConfig); + Set tags = participant.getTags(); + for (String tag : tags) { + instanceConfig.addTag(tag); + } + Set disabledPartitions = participant.getDisabledPartitions(); + for (PartitionId partitionId : disabledPartitions) { + instanceConfig.setParticipantEnabledForPartition(partitionId, false); + } + _accessor.setProperty(_keyBuilder.instanceConfig(participantId.stringify()), instanceConfig); + return true; + } + + /** + * drop a participant from cluster + * @param participantId + * @return true if participant dropped, false if there was an error + */ + public boolean dropParticipantFromCluster(ParticipantId participantId) { + ParticipantAccessor accessor = new ParticipantAccessor(_accessor); + return accessor.dropParticipant(participantId); + } + + /** + * Add a state model definition. Updates the existing state model definition if it already exists. + * @param stateModelDef fully initialized state model definition + * @return true if the model is persisted, false otherwise + */ + public boolean addStateModelDefinitionToCluster(StateModelDefinition stateModelDef) { + if (!isClusterStructureValid()) { + LOG.error("Cluster: " + _clusterId + " structure is not valid"); + return false; + } + + return _accessor + .createProperty(_keyBuilder.stateModelDef(stateModelDef.getId()), stateModelDef); + } + + /** + * Remove a state model definition if it exists + * @param stateModelDefId state model definition id + * @return true if removed, false if it did not exist + */ + public boolean dropStateModelDefinitionFromCluster(StateModelDefId stateModelDefId) { + return _accessor.removeProperty(_keyBuilder.stateModelDef(stateModelDefId.stringify())); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ControllerAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ControllerAccessor.java new file mode 100644 index 0000000000..74328d7dbc --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ControllerAccessor.java @@ -0,0 +1,49 @@ +package org.apache.helix.api.accessor; + +/* + * 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.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.api.Controller; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.model.Leader; + +public class ControllerAccessor { + private final HelixDataAccessor _accessor; + private final PropertyKey.Builder _keyBuilder; + + public ControllerAccessor(HelixDataAccessor accessor) { + _accessor = accessor; + _keyBuilder = accessor.keyBuilder(); + } + + /** + * Read the leader controller if it is live + * @return Controller snapshot, or null + */ + public Controller readLeader() { + Leader leader = _accessor.getProperty(_keyBuilder.controllerLeader()); + if (leader != null) { + ControllerId leaderId = ControllerId.from(leader.getId()); + return new Controller(leaderId, leader, true); + } + return null; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java new file mode 100644 index 0000000000..2721d91328 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java @@ -0,0 +1,774 @@ +package org.apache.helix.api.accessor; + +/* + * 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.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import org.I0Itec.zkclient.DataUpdater; +import org.apache.helix.AccessOption; +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixDefinedState; +import org.apache.helix.HelixException; +import org.apache.helix.PropertyKey; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Resource; +import org.apache.helix.api.RunningInstance; +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.InstanceConfig.InstanceConfigProperty; +import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Message; +import org.apache.helix.model.Message.MessageState; +import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +public class ParticipantAccessor { + private static final Logger LOG = Logger.getLogger(ParticipantAccessor.class); + + private final HelixDataAccessor _accessor; + private final PropertyKey.Builder _keyBuilder; + + public ParticipantAccessor(HelixDataAccessor accessor) { + _accessor = accessor; + _keyBuilder = accessor.keyBuilder(); + } + + /** + * enable/disable a participant + * @param participantId + * @param isEnabled + * @return true if enable state succeeded, false otherwise + */ + boolean enableParticipant(ParticipantId participantId, boolean isEnabled) { + String participantName = participantId.stringify(); + if (_accessor.getProperty(_keyBuilder.instanceConfig(participantName)) == null) { + LOG.error("Config for participant: " + participantId + " does NOT exist in cluster"); + return false; + } + + InstanceConfig config = new InstanceConfig(participantName); + config.setInstanceEnabled(isEnabled); + return _accessor.updateProperty(_keyBuilder.instanceConfig(participantName), config); + } + + /** + * disable participant + * @param participantId + * @return true if disabled successfully, false otherwise + */ + public boolean disableParticipant(ParticipantId participantId) { + return enableParticipant(participantId, false); + } + + /** + * enable participant + * @param participantId + * @return true if enabled successfully, false otherwise + */ + public boolean enableParticipant(ParticipantId participantId) { + return enableParticipant(participantId, true); + } + + /** + * create messages for participant + * @param participantId + * @param msgMap map of message-id to message + */ + public void insertMessagesToParticipant(ParticipantId participantId, + Map msgMap) { + List msgKeys = new ArrayList(); + List msgs = new ArrayList(); + for (MessageId msgId : msgMap.keySet()) { + msgKeys.add(_keyBuilder.message(participantId.stringify(), msgId.stringify())); + msgs.add(msgMap.get(msgId)); + } + + _accessor.createChildren(msgKeys, msgs); + } + + /** + * set messages of participant + * @param participantId + * @param msgMap map of message-id to message + */ + public void updateMessageStatus(ParticipantId participantId, Map msgMap) { + String participantName = participantId.stringify(); + List msgKeys = new ArrayList(); + List msgs = new ArrayList(); + for (MessageId msgId : msgMap.keySet()) { + msgKeys.add(_keyBuilder.message(participantName, msgId.stringify())); + msgs.add(msgMap.get(msgId)); + } + _accessor.setChildren(msgKeys, msgs); + } + + /** + * delete messages from participant + * @param participantId + * @param msgIdSet + */ + public void deleteMessagesFromParticipant(ParticipantId participantId, Set msgIdSet) { + String participantName = participantId.stringify(); + List msgKeys = new ArrayList(); + for (MessageId msgId : msgIdSet) { + msgKeys.add(_keyBuilder.message(participantName, msgId.stringify())); + } + + // TODO impl batch remove + for (PropertyKey msgKey : msgKeys) { + _accessor.removeProperty(msgKey); + } + } + + /** + * enable/disable partitions on a participant + * @param enabled + * @param participantId + * @param resourceId + * @param partitionIdSet + * @return true if enable state changed successfully, false otherwise + */ + boolean enablePartitionsForParticipant(final boolean enabled, final ParticipantId participantId, + final ResourceId resourceId, final Set partitionIdSet) { + String participantName = participantId.stringify(); + String resourceName = resourceId.stringify(); + + // check instanceConfig exists + PropertyKey instanceConfigKey = _keyBuilder.instanceConfig(participantName); + if (_accessor.getProperty(instanceConfigKey) == null) { + LOG.error("Config for participant: " + participantId + " does NOT exist in cluster"); + return false; + } + + // check resource exist. warn if not + IdealState idealState = _accessor.getProperty(_keyBuilder.idealStates(resourceName)); + if (idealState == null) { + LOG.warn("Disable partitions: " + partitionIdSet + ", resource: " + resourceId + + " does NOT exist. probably disable it during ERROR->DROPPED transtition"); + + } else { + // check partitions exist. warn if not + for (PartitionId partitionId : partitionIdSet) { + if ((idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO && idealState + .getPreferenceList(partitionId) == null) + || (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED && idealState + .getParticipantStateMap(partitionId) == null)) { + LOG.warn("Resource: " + resourceId + ", partition: " + partitionId + + ", partition does NOT exist in ideal state"); + } + } + } + + BaseDataAccessor baseAccessor = _accessor.getBaseDataAccessor(); + final List partitionNames = new ArrayList(); + for (PartitionId partitionId : partitionIdSet) { + partitionNames.add(partitionId.stringify()); + } + + return baseAccessor.update(instanceConfigKey.getPath(), new DataUpdater() { + @Override + public ZNRecord update(ZNRecord currentData) { + if (currentData == null) { + throw new HelixException("Instance: " + participantId + ", participant config is null"); + } + + // TODO: merge with InstanceConfig.setInstanceEnabledForPartition + List list = + currentData.getListField(InstanceConfigProperty.HELIX_DISABLED_PARTITION.toString()); + Set disabledPartitions = new HashSet(); + if (list != null) { + disabledPartitions.addAll(list); + } + + if (enabled) { + disabledPartitions.removeAll(partitionNames); + } else { + disabledPartitions.addAll(partitionNames); + } + + list = new ArrayList(disabledPartitions); + Collections.sort(list); + currentData.setListField(InstanceConfigProperty.HELIX_DISABLED_PARTITION.toString(), list); + return currentData; + } + }, AccessOption.PERSISTENT); + } + + /** + * disable partitions on a participant + * @param participantId + * @param resourceId + * @param disablePartitionIdSet + * @return true if disabled successfully, false otherwise + */ + public boolean disablePartitionsForParticipant(ParticipantId participantId, + ResourceId resourceId, Set disablePartitionIdSet) { + return enablePartitionsForParticipant(false, participantId, resourceId, disablePartitionIdSet); + } + + /** + * enable partitions on a participant + * @param participantId + * @param resourceId + * @param enablePartitionIdSet + * @return true if enabled successfully, false otherwise + */ + public boolean enablePartitionsForParticipant(ParticipantId participantId, ResourceId resourceId, + Set enablePartitionIdSet) { + return enablePartitionsForParticipant(true, participantId, resourceId, enablePartitionIdSet); + } + + /** + * Reset partitions assigned to a set of participants + * @param resetParticipantIdSet the participants to reset + * @return true if reset, false otherwise + */ + public boolean resetParticipants(Set resetParticipantIdSet) { + List extViews = _accessor.getChildValues(_keyBuilder.externalViews()); + for (ParticipantId participantId : resetParticipantIdSet) { + for (ExternalView extView : extViews) { + Set resetPartitionIdSet = Sets.newHashSet(); + for (PartitionId partitionId : extView.getPartitionIdSet()) { + Map stateMap = extView.getStateMap(partitionId); + if (stateMap.containsKey(participantId) + && stateMap.get(participantId).equals(State.from(HelixDefinedState.ERROR))) { + resetPartitionIdSet.add(partitionId); + } + } + resetPartitionsForParticipant(participantId, extView.getResourceId(), resetPartitionIdSet); + } + } + return true; + } + + /** + * reset partitions on a participant + * @param participantId + * @param resourceId + * @param resetPartitionIdSet + * @return true if partitions reset, false otherwise + */ + public boolean resetPartitionsForParticipant(ParticipantId participantId, ResourceId resourceId, + Set resetPartitionIdSet) { + // make sure the participant is running + Participant participant = readParticipant(participantId); + if (!participant.isAlive()) { + LOG.error("Cannot reset partitions because the participant is not running"); + return false; + } + RunningInstance runningInstance = participant.getRunningInstance(); + + // check that the resource exists + ResourceAccessor resourceAccessor = resourceAccessor(); + Resource resource = resourceAccessor.readResource(resourceId); + if (resource == null || resource.getRebalancerConfig() == null) { + LOG.error("Cannot reset partitions because the resource is not present"); + return false; + } + + // need the rebalancer context for the resource + RebalancerContext context = + resource.getRebalancerConfig().getRebalancerContext(RebalancerContext.class); + if (context == null) { + LOG.error("Rebalancer context for resource does not exist"); + return false; + } + + // ensure that all partitions to reset exist + Set partitionSet = ImmutableSet.copyOf(context.getSubUnitIdSet()); + if (!partitionSet.containsAll(resetPartitionIdSet)) { + LOG.error("Not all of the specified partitions to reset exist for the resource"); + return false; + } + + // check for a valid current state that has all specified partitions in ERROR state + CurrentState currentState = participant.getCurrentStateMap().get(resourceId); + if (currentState == null) { + LOG.error("The participant does not have a current state for the resource"); + return false; + } + for (PartitionId partitionId : resetPartitionIdSet) { + if (!currentState.getState(partitionId).equals(State.from(HelixDefinedState.ERROR))) { + LOG.error("Partition " + partitionId + " is not in error state, aborting reset"); + return false; + } + } + + // make sure that there are no pending transition messages + for (Message message : participant.getMessageMap().values()) { + if (!MessageType.STATE_TRANSITION.toString().equalsIgnoreCase(message.getMsgType()) + || !runningInstance.getSessionId().equals(message.getTypedTgtSessionId()) + || !resourceId.equals(message.getResourceId()) + || !resetPartitionIdSet.contains(message.getPartitionId())) { + continue; + } + LOG.error("Cannot reset partitions because of the following pending message: " + message); + return false; + } + + // set up the source id + String adminName = null; + try { + adminName = InetAddress.getLocalHost().getCanonicalHostName() + "-ADMIN"; + } catch (UnknownHostException e) { + // can ignore it + if (LOG.isInfoEnabled()) { + LOG.info("Unable to get host name. Will set it to UNKNOWN, mostly ignorable", e); + } + adminName = "UNKNOWN"; + } + + // build messages to signal the transition + StateModelDefId stateModelDefId = context.getStateModelDefId(); + StateModelDefinition stateModelDef = + _accessor.getProperty(_keyBuilder.stateModelDef(stateModelDefId.stringify())); + Map messageMap = Maps.newHashMap(); + for (PartitionId partitionId : resetPartitionIdSet) { + // send ERROR to initialState message + MessageId msgId = MessageId.from(UUID.randomUUID().toString()); + Message message = new Message(MessageType.STATE_TRANSITION, msgId); + message.setSrcName(adminName); + message.setTgtName(participantId.stringify()); + message.setMsgState(MessageState.NEW); + message.setPartitionId(partitionId); + message.setResourceId(resourceId); + message.setTgtSessionId(runningInstance.getSessionId()); + message.setStateModelDef(stateModelDefId); + message.setFromState(State.from(HelixDefinedState.ERROR.toString())); + message.setToState(stateModelDef.getTypedInitialState()); + message.setStateModelFactoryId(context.getStateModelFactoryId()); + + messageMap.put(message.getMessageId(), message); + } + + // send the messages + insertMessagesToParticipant(participantId, messageMap); + return true; + } + + /** + * Read the user config of the participant + * @param participantId the participant to to look up + * @return UserConfig, or null + */ + public UserConfig readUserConfig(ParticipantId participantId) { + InstanceConfig instanceConfig = + _accessor.getProperty(_keyBuilder.instanceConfig(participantId.stringify())); + return instanceConfig != null ? instanceConfig.getUserConfig() : null; + } + + /** + * Set the user config of the participant, overwriting existing user configs + * @param participantId the participant to update + * @param userConfig the new user config + * @return true if the user config was set, false otherwise + */ + public boolean setUserConfig(ParticipantId participantId, UserConfig userConfig) { + ParticipantConfig.Delta delta = + new ParticipantConfig.Delta(participantId).setUserConfig(userConfig); + return updateParticipant(participantId, delta) != null; + } + + /** + * Add user configuration to the existing participant user configuration. Overwrites properties + * with + * the same key + * @param participant the participant to update + * @param userConfig the user config key-value pairs to add + * @return true if the user config was updated, false otherwise + */ + public boolean updateUserConfig(ParticipantId participantId, UserConfig userConfig) { + InstanceConfig instanceConfig = new InstanceConfig(participantId); + instanceConfig.addNamespacedConfig(userConfig); + return _accessor.updateProperty(_keyBuilder.instanceConfig(participantId.stringify()), + instanceConfig); + } + + /** + * Clear any user-specified configuration from the participant + * @param participantId the participant to update + * @return true if the config was cleared, false otherwise + */ + public boolean dropUserConfig(ParticipantId participantId) { + return setUserConfig(participantId, new UserConfig(Scope.participant(participantId))); + } + + /** + * Update a participant configuration + * @param participantId the participant to update + * @param participantDelta changes to the participant + * @return ParticipantConfig, or null if participant is not persisted + */ + public ParticipantConfig updateParticipant(ParticipantId participantId, + ParticipantConfig.Delta participantDelta) { + Participant participant = readParticipant(participantId); + if (participant == null) { + LOG.error("Participant " + participantId + " does not exist, cannot be updated"); + return null; + } + ParticipantConfig config = participantDelta.mergeInto(participant.getConfig()); + setParticipant(config); + return config; + } + + /** + * Set the configuration of an existing participant + * @param participantConfig participant configuration + * @return true if config was set, false if there was an error + */ + public boolean setParticipant(ParticipantConfig participantConfig) { + if (participantConfig == null) { + LOG.error("Participant config not initialized"); + return false; + } + InstanceConfig instanceConfig = new InstanceConfig(participantConfig.getId()); + instanceConfig.setHostName(participantConfig.getHostName()); + instanceConfig.setPort(Integer.toString(participantConfig.getPort())); + for (String tag : participantConfig.getTags()) { + instanceConfig.addTag(tag); + } + for (PartitionId partitionId : participantConfig.getDisabledPartitions()) { + instanceConfig.setParticipantEnabledForPartition(partitionId, false); + } + instanceConfig.setInstanceEnabled(participantConfig.isEnabled()); + instanceConfig.addNamespacedConfig(participantConfig.getUserConfig()); + _accessor.setProperty(_keyBuilder.instanceConfig(participantConfig.getId().stringify()), + instanceConfig); + return true; + } + + /** + * create a participant based on physical model + * @param participantId + * @param instanceConfig + * @param userConfig + * @param liveInstance + * @param instanceMsgMap map of message-id to message + * @param instanceCurStateMap map of resource-id to current-state + * @return participant + */ + static Participant createParticipant(ParticipantId participantId, InstanceConfig instanceConfig, + UserConfig userConfig, LiveInstance liveInstance, Map instanceMsgMap, + Map instanceCurStateMap) { + + String hostName = instanceConfig.getHostName(); + + int port = -1; + try { + port = Integer.parseInt(instanceConfig.getPort()); + } catch (IllegalArgumentException e) { + // keep as -1 + } + if (port < 0 || port > 65535) { + port = -1; + } + boolean isEnabled = instanceConfig.getInstanceEnabled(); + + List disabledPartitions = instanceConfig.getDisabledPartitions(); + Set disabledPartitionIdSet = Collections.emptySet(); + if (disabledPartitions != null) { + disabledPartitionIdSet = new HashSet(); + for (String partitionId : disabledPartitions) { + disabledPartitionIdSet.add(PartitionId.from(PartitionId.extractResourceId(partitionId), + PartitionId.stripResourceId(partitionId))); + } + } + + Set tags = new HashSet(instanceConfig.getTags()); + + RunningInstance runningInstance = null; + if (liveInstance != null) { + runningInstance = + new RunningInstance(liveInstance.getTypedSessionId(), + liveInstance.getTypedHelixVersion(), liveInstance.getProcessId()); + } + + Map msgMap = new HashMap(); + if (instanceMsgMap != null) { + for (String msgId : instanceMsgMap.keySet()) { + Message message = instanceMsgMap.get(msgId); + msgMap.put(MessageId.from(msgId), message); + } + } + + Map curStateMap = new HashMap(); + if (instanceCurStateMap != null) { + + for (String resourceName : instanceCurStateMap.keySet()) { + curStateMap.put(ResourceId.from(resourceName), instanceCurStateMap.get(resourceName)); + } + } + + return new Participant(participantId, hostName, port, isEnabled, disabledPartitionIdSet, tags, + runningInstance, curStateMap, msgMap, userConfig); + } + + /** + * read participant related data + * @param participantId + * @return participant, or null if participant not available + */ + public Participant readParticipant(ParticipantId participantId) { + // read physical model + String participantName = participantId.stringify(); + InstanceConfig instanceConfig = + _accessor.getProperty(_keyBuilder.instanceConfig(participantName)); + + if (instanceConfig == null) { + LOG.error("Participant " + participantId + " is not present on the cluster"); + return null; + } + + UserConfig userConfig = instanceConfig.getUserConfig(); + LiveInstance liveInstance = _accessor.getProperty(_keyBuilder.liveInstance(participantName)); + + Map instanceMsgMap = Collections.emptyMap(); + Map instanceCurStateMap = Collections.emptyMap(); + if (liveInstance != null) { + SessionId sessionId = liveInstance.getTypedSessionId(); + + instanceMsgMap = _accessor.getChildValuesMap(_keyBuilder.messages(participantName)); + instanceCurStateMap = + _accessor.getChildValuesMap(_keyBuilder.currentStates(participantName, + sessionId.stringify())); + } + + return createParticipant(participantId, instanceConfig, userConfig, liveInstance, + instanceMsgMap, instanceCurStateMap); + } + + /** + * update resource current state of a participant + * @param resourceId resource id + * @param participantId participant id + * @param sessionId session id + * @param curStateUpdate current state change delta + */ + public void updateCurrentState(ResourceId resourceId, ParticipantId participantId, + SessionId sessionId, CurrentState curStateUpdate) { + _accessor.updateProperty( + _keyBuilder.currentState(participantId.stringify(), sessionId.stringify(), + resourceId.stringify()), curStateUpdate); + } + + /** + * drop resource current state of a participant + * @param resourceId resource id + * @param participantId participant id + * @param sessionId session id + * @return true if dropped, false otherwise + */ + public boolean dropCurrentState(ResourceId resourceId, ParticipantId participantId, + SessionId sessionId) { + return _accessor.removeProperty(_keyBuilder.currentState(participantId.stringify(), + sessionId.stringify(), resourceId.stringify())); + } + + /** + * drop a participant from cluster + * @param participantId + * @return true if participant dropped, false if there was an error + */ + boolean dropParticipant(ParticipantId participantId) { + if (_accessor.getProperty(_keyBuilder.instanceConfig(participantId.stringify())) == null) { + LOG.error("Config for participant: " + participantId + " does NOT exist in cluster"); + } + + if (_accessor.getProperty(_keyBuilder.instance(participantId.stringify())) == null) { + LOG.error("Participant: " + participantId + " structure does NOT exist in cluster"); + } + + // delete participant config path + _accessor.removeProperty(_keyBuilder.instanceConfig(participantId.stringify())); + + // delete participant path + _accessor.removeProperty(_keyBuilder.instance(participantId.stringify())); + return true; + } + + /** + * Let a new participant take the place of an existing participant + * @param oldParticipantId the participant to drop + * @param newParticipantId the participant that takes its place + * @return true if swap successful, false otherwise + */ + public boolean swapParticipants(ParticipantId oldParticipantId, ParticipantId newParticipantId) { + Participant oldParticipant = readParticipant(oldParticipantId); + if (oldParticipant == null) { + LOG.error("Could not swap participants because the old participant does not exist"); + return false; + } + if (oldParticipant.isEnabled()) { + LOG.error("Could not swap participants because the old participant is still enabled"); + return false; + } + if (oldParticipant.isAlive()) { + LOG.error("Could not swap participants because the old participant is still live"); + return false; + } + Participant newParticipant = readParticipant(newParticipantId); + if (newParticipant == null) { + LOG.error("Could not swap participants because the new participant does not exist"); + return false; + } + dropParticipant(oldParticipantId); + ResourceAccessor resourceAccessor = resourceAccessor(); + Map idealStateMap = _accessor.getChildValuesMap(_keyBuilder.idealStates()); + for (String resourceName : idealStateMap.keySet()) { + IdealState idealState = idealStateMap.get(resourceName); + swapParticipantsInIdealState(idealState, oldParticipantId, newParticipantId); + PartitionedRebalancerContext context = PartitionedRebalancerContext.from(idealState); + resourceAccessor.setRebalancerContext(ResourceId.from(resourceName), context); + _accessor.setProperty(_keyBuilder.idealStates(resourceName), idealState); + } + return true; + } + + /** + * Replace occurrences of participants in preference lists and maps + * @param idealState the current ideal state + * @param oldParticipantId the participant to drop + * @param newParticipantId the participant that replaces it + */ + protected void swapParticipantsInIdealState(IdealState idealState, + ParticipantId oldParticipantId, ParticipantId newParticipantId) { + for (PartitionId partitionId : idealState.getPartitionIdSet()) { + List oldPreferenceList = idealState.getPreferenceList(partitionId); + if (oldPreferenceList != null) { + List newPreferenceList = Lists.newArrayList(); + for (ParticipantId participantId : oldPreferenceList) { + if (participantId.equals(oldParticipantId)) { + newPreferenceList.add(newParticipantId); + } else if (!participantId.equals(newParticipantId)) { + newPreferenceList.add(participantId); + } + } + idealState.setPreferenceList(partitionId, newPreferenceList); + } + Map preferenceMap = idealState.getParticipantStateMap(partitionId); + if (preferenceMap != null) { + if (preferenceMap.containsKey(oldParticipantId)) { + State state = preferenceMap.get(oldParticipantId); + preferenceMap.remove(oldParticipantId); + preferenceMap.put(newParticipantId, state); + } + idealState.setParticipantStateMap(partitionId, preferenceMap); + } + } + } + + /** + * Create empty persistent properties to ensure that there is a valid participant structure + */ + public void initParticipantStructure(ParticipantId participantId) { + List paths = getRequiredPaths(_keyBuilder, participantId); + BaseDataAccessor baseAccessor = _accessor.getBaseDataAccessor(); + for (String path : paths) { + boolean status = baseAccessor.create(path, null, AccessOption.PERSISTENT); + if (!status && LOG.isDebugEnabled()) { + LOG.debug(path + " already exists"); + } + } + } + + /** + * Clear properties for the participant + */ + void clearParticipantStructure(ParticipantId participantId) { + List paths = getRequiredPaths(_keyBuilder, participantId); + BaseDataAccessor baseAccessor = _accessor.getBaseDataAccessor(); + baseAccessor.remove(paths, 0); + } + + /** + * check if participant structure is valid + * @return true if valid or false otherwise + */ + public boolean isParticipantStructureValid(ParticipantId participantId) { + List paths = getRequiredPaths(_keyBuilder, participantId); + BaseDataAccessor baseAccessor = _accessor.getBaseDataAccessor(); + if (baseAccessor != null) { + boolean[] existsResults = baseAccessor.exists(paths, 0); + for (boolean exists : existsResults) { + if (!exists) { + return false; + } + } + } + return true; + } + + /** + * Get the paths that should be created if the participant exists + * @param keyBuilder PropertyKey.Builder for the cluster + * @param participantId the participant for which to generate paths + * @return list of required paths as strings + */ + private static List getRequiredPaths(PropertyKey.Builder keyBuilder, + ParticipantId participantId) { + List paths = Lists.newArrayList(); + paths.add(keyBuilder.instanceConfig(participantId.stringify()).getPath()); + paths.add(keyBuilder.messages(participantId.stringify()).getPath()); + paths.add(keyBuilder.currentStates(participantId.stringify()).getPath()); + paths.add(keyBuilder.participantErrors(participantId.stringify()).getPath()); + paths.add(keyBuilder.statusUpdates(participantId.stringify()).getPath()); + return paths; + } + + /** + * Get a ResourceAccessor instance + * @return ResourceAccessor + */ + protected ResourceAccessor resourceAccessor() { + return new ResourceAccessor(_accessor); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java new file mode 100644 index 0000000000..e5ac57cc94 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java @@ -0,0 +1,472 @@ +package org.apache.helix.api.accessor; + +/* + * 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.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.HelixConstants.StateModelToken; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixDefinedState; +import org.apache.helix.PropertyKey; +import org.apache.helix.api.Resource; +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.ResourceConfig.ResourceType; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.context.CustomRebalancerContext; +import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.ResourceConfiguration; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +public class ResourceAccessor { + private static final Logger LOG = Logger.getLogger(ResourceAccessor.class); + private final HelixDataAccessor _accessor; + private final PropertyKey.Builder _keyBuilder; + + public ResourceAccessor(HelixDataAccessor accessor) { + _accessor = accessor; + _keyBuilder = accessor.keyBuilder(); + } + + /** + * Read a single snapshot of a resource + * @param resourceId the resource id to read + * @return Resource or null if not present + */ + public Resource readResource(ResourceId resourceId) { + ResourceConfiguration config = + _accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify())); + IdealState idealState = _accessor.getProperty(_keyBuilder.idealStates(resourceId.stringify())); + + if (config == null && idealState == null) { + LOG.error("Resource " + resourceId + " not present on the cluster"); + return null; + } + + ExternalView externalView = + _accessor.getProperty(_keyBuilder.externalView(resourceId.stringify())); + ResourceAssignment resourceAssignment = + _accessor.getProperty(_keyBuilder.resourceAssignment(resourceId.stringify())); + return createResource(resourceId, config, idealState, externalView, resourceAssignment); + } + + /** + * Update a resource configuration + * @param resourceId the resource id to update + * @param resourceDelta changes to the resource + * @return ResourceConfig, or null if the resource is not persisted + */ + public ResourceConfig updateResource(ResourceId resourceId, ResourceConfig.Delta resourceDelta) { + Resource resource = readResource(resourceId); + if (resource == null) { + LOG.error("Resource " + resourceId + " does not exist, cannot be updated"); + return null; + } + ResourceConfig config = resourceDelta.mergeInto(resource.getConfig()); + setResource(config); + return config; + } + + /** + * save resource assignment + * @param resourceId + * @param resourceAssignment + * @return true if set, false otherwise + */ + public boolean setResourceAssignment(ResourceId resourceId, ResourceAssignment resourceAssignment) { + return _accessor.setProperty(_keyBuilder.resourceAssignment(resourceId.stringify()), + resourceAssignment); + } + + /** + * get resource assignment + * @param resourceId + * @return resource assignment or null + */ + public ResourceAssignment getResourceAssignment(ResourceId resourceId) { + return _accessor.getProperty(_keyBuilder.resourceAssignment(resourceId.stringify())); + } + + /** + * Set a physical resource configuration, which may include user-defined configuration, as well as + * rebalancer configuration + * @param resourceId + * @param configuration + * @return true if set, false otherwise + */ + private boolean setConfiguration(ResourceId resourceId, ResourceConfiguration configuration) { + boolean status = + _accessor.setProperty(_keyBuilder.resourceConfig(resourceId.stringify()), configuration); + // also set an ideal state if the resource supports it + RebalancerConfig rebalancerConfig = new RebalancerConfig(configuration); + IdealState idealState = + rebalancerConfigToIdealState(rebalancerConfig, configuration.getBucketSize(), + configuration.getBatchMessageMode()); + if (idealState != null) { + _accessor.setProperty(_keyBuilder.idealStates(resourceId.stringify()), idealState); + } + return status; + } + + /** + * Set the context of the rebalancer. This includes all properties required for rebalancing this + * resource + * @param resourceId the resource to update + * @param context the new rebalancer context + * @return true if the context was set, false otherwise + */ + public boolean setRebalancerContext(ResourceId resourceId, RebalancerContext context) { + RebalancerConfig config = new RebalancerConfig(context); + ResourceConfiguration resourceConfig = new ResourceConfiguration(resourceId); + resourceConfig.addNamespacedConfig(config.toNamespacedConfig()); + + // update the ideal state if applicable + IdealState oldIdealState = + _accessor.getProperty(_keyBuilder.idealStates(resourceId.stringify())); + if (oldIdealState != null) { + IdealState idealState = + rebalancerConfigToIdealState(config, oldIdealState.getBucketSize(), + oldIdealState.getBatchMessageMode()); + if (idealState != null) { + _accessor.setProperty(_keyBuilder.idealStates(resourceId.stringify()), idealState); + } + } + + return _accessor.updateProperty(_keyBuilder.resourceConfig(resourceId.stringify()), + resourceConfig); + } + + /** + * Read the user config of the resource + * @param resourceId the resource to to look up + * @return UserConfig, or null + */ + public UserConfig readUserConfig(ResourceId resourceId) { + ResourceConfiguration resourceConfig = + _accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify())); + return resourceConfig != null ? UserConfig.from(resourceConfig) : null; + } + + /** + * Read the rebalancer config of the resource + * @param resourceId the resource to to look up + * @return RebalancerConfig, or null + */ + public RebalancerConfig readRebalancerConfig(ResourceId resourceId) { + ResourceConfiguration resourceConfig = + _accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify())); + return resourceConfig != null ? RebalancerConfig.from(resourceConfig) : null; + } + + /** + * Set the user config of the resource, overwriting existing user configs + * @param resourceId the resource to update + * @param userConfig the new user config + * @return true if the user config was set, false otherwise + */ + public boolean setUserConfig(ResourceId resourceId, UserConfig userConfig) { + ResourceConfig.Delta delta = new ResourceConfig.Delta(resourceId).setUserConfig(userConfig); + return updateResource(resourceId, delta) != null; + } + + /** + * Add user configuration to the existing resource user configuration. Overwrites properties with + * the same key + * @param resourceId the resource to update + * @param userConfig the user config key-value pairs to add + * @return true if the user config was updated, false otherwise + */ + public boolean updateUserConfig(ResourceId resourceId, UserConfig userConfig) { + ResourceConfiguration resourceConfig = new ResourceConfiguration(resourceId); + resourceConfig.addNamespacedConfig(userConfig); + return _accessor.updateProperty(_keyBuilder.resourceConfig(resourceId.stringify()), + resourceConfig); + } + + /** + * Clear any user-specified configuration from the resource + * @param resourceId the resource to update + * @return true if the config was cleared, false otherwise + */ + public boolean dropUserConfig(ResourceId resourceId) { + return setUserConfig(resourceId, new UserConfig(Scope.resource(resourceId))); + } + + /** + * Persist an existing resource's logical configuration + * @param resourceConfig logical resource configuration + * @return true if resource is set, false otherwise + */ + public boolean setResource(ResourceConfig resourceConfig) { + if (resourceConfig == null || resourceConfig.getRebalancerConfig() == null) { + LOG.error("Resource not fully defined with a rebalancer context"); + return false; + } + ResourceId resourceId = resourceConfig.getId(); + ResourceConfiguration config = new ResourceConfiguration(resourceId); + config.addNamespacedConfig(resourceConfig.getUserConfig()); + config.addNamespacedConfig(resourceConfig.getRebalancerConfig().toNamespacedConfig()); + config.setBucketSize(resourceConfig.getBucketSize()); + config.setBatchMessageMode(resourceConfig.getBatchMessageMode()); + setConfiguration(resourceId, config); + return true; + } + + /** + * Get a resource configuration, which may include user-defined configuration, as well as + * rebalancer configuration + * @param resourceId + * @return configuration or null + */ + public ResourceConfiguration getConfiguration(ResourceId resourceId) { + return _accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify())); + } + + /** + * set external view of a resource + * @param resourceId + * @param extView + * @return true if set, false otherwise + */ + public boolean setExternalView(ResourceId resourceId, ExternalView extView) { + return _accessor.setProperty(_keyBuilder.externalView(resourceId.stringify()), extView); + } + + /** + * get the external view of a resource + * @param resourceId the resource to look up + * @return external view or null + */ + public ExternalView readExternalView(ResourceId resourceId) { + return _accessor.getProperty(_keyBuilder.externalView(resourceId.stringify())); + } + + /** + * drop external view of a resource + * @param resourceId + * @return true if dropped, false otherwise + */ + public boolean dropExternalView(ResourceId resourceId) { + return _accessor.removeProperty(_keyBuilder.externalView(resourceId.stringify())); + } + + /** + * reset resources for all participants + * @param resetResourceIdSet the resources to reset + * @return true if they were reset, false otherwise + */ + public boolean resetResources(Set resetResourceIdSet) { + ParticipantAccessor accessor = participantAccessor(); + List extViews = _accessor.getChildValues(_keyBuilder.externalViews()); + for (ExternalView extView : extViews) { + if (!resetResourceIdSet.contains(extView.getResourceId())) { + continue; + } + + Map> resetPartitionIds = Maps.newHashMap(); + for (PartitionId partitionId : extView.getPartitionIdSet()) { + Map stateMap = extView.getStateMap(partitionId); + for (ParticipantId participantId : stateMap.keySet()) { + State state = stateMap.get(participantId); + if (state.equals(State.from(HelixDefinedState.ERROR))) { + if (!resetPartitionIds.containsKey(participantId)) { + resetPartitionIds.put(participantId, new HashSet()); + } + resetPartitionIds.get(participantId).add(partitionId); + } + } + } + for (ParticipantId participantId : resetPartitionIds.keySet()) { + accessor.resetPartitionsForParticipant(participantId, extView.getResourceId(), + resetPartitionIds.get(participantId)); + } + } + return true; + } + + /** + * Generate a default assignment for partitioned resources + * @param resourceId the resource to update + * @param replicaCount the new replica count (or -1 to use the existing one) + * @param participantGroupTag the new participant group tag (or null to use the existing one) + * @return true if assignment successful, false otherwise + */ + public boolean generateDefaultAssignment(ResourceId resourceId, int replicaCount, + String participantGroupTag) { + Resource resource = readResource(resourceId); + RebalancerConfig config = resource.getRebalancerConfig(); + PartitionedRebalancerContext context = + config.getRebalancerContext(PartitionedRebalancerContext.class); + if (context == null) { + LOG.error("Only partitioned resource types are supported"); + return false; + } + if (replicaCount != -1) { + context.setReplicaCount(replicaCount); + } + if (participantGroupTag != null) { + context.setParticipantGroupTag(participantGroupTag); + } + StateModelDefinition stateModelDef = + _accessor.getProperty(_keyBuilder.stateModelDef(context.getStateModelDefId().stringify())); + List participantConfigs = + _accessor.getChildValues(_keyBuilder.instanceConfigs()); + Set participantSet = Sets.newHashSet(); + for (InstanceConfig participantConfig : participantConfigs) { + participantSet.add(participantConfig.getParticipantId()); + } + context.generateDefaultConfiguration(stateModelDef, participantSet); + setRebalancerContext(resourceId, context); + return true; + } + + /** + * Get an ideal state from a rebalancer config if the resource is partitioned + * @param config RebalancerConfig instance + * @param bucketSize bucket size to use + * @param batchMessageMode true if batch messaging allowed, false otherwise + * @return IdealState, or null + */ + static IdealState rebalancerConfigToIdealState(RebalancerConfig config, int bucketSize, + boolean batchMessageMode) { + PartitionedRebalancerContext partitionedContext = + config.getRebalancerContext(PartitionedRebalancerContext.class); + if (partitionedContext != null) { + IdealState idealState = new IdealState(partitionedContext.getResourceId()); + idealState.setRebalanceMode(partitionedContext.getRebalanceMode()); + idealState.setRebalancerRef(partitionedContext.getRebalancerRef()); + String replicas = null; + if (partitionedContext.anyLiveParticipant()) { + replicas = StateModelToken.ANY_LIVEINSTANCE.toString(); + } else { + replicas = Integer.toString(partitionedContext.getReplicaCount()); + } + idealState.setReplicas(replicas); + idealState.setNumPartitions(partitionedContext.getPartitionSet().size()); + idealState.setInstanceGroupTag(partitionedContext.getParticipantGroupTag()); + idealState.setMaxPartitionsPerInstance(partitionedContext.getMaxPartitionsPerParticipant()); + idealState.setStateModelDefId(partitionedContext.getStateModelDefId()); + idealState.setStateModelFactoryId(partitionedContext.getStateModelFactoryId()); + idealState.setBucketSize(bucketSize); + idealState.setBatchMessageMode(batchMessageMode); + if (partitionedContext.getRebalanceMode() == RebalanceMode.SEMI_AUTO) { + SemiAutoRebalancerContext semiAutoContext = + config.getRebalancerContext(SemiAutoRebalancerContext.class); + for (PartitionId partitionId : semiAutoContext.getPartitionSet()) { + idealState.setPreferenceList(partitionId, semiAutoContext.getPreferenceList(partitionId)); + } + } else if (partitionedContext.getRebalanceMode() == RebalanceMode.CUSTOMIZED) { + CustomRebalancerContext customContext = + config.getRebalancerContext(CustomRebalancerContext.class); + for (PartitionId partitionId : customContext.getPartitionSet()) { + idealState.setParticipantStateMap(partitionId, + customContext.getPreferenceMap(partitionId)); + } + } else { + for (PartitionId partitionId : partitionedContext.getPartitionSet()) { + List preferenceList = Collections.emptyList(); + idealState.setPreferenceList(partitionId, preferenceList); + Map participantStateMap = Collections.emptyMap(); + idealState.setParticipantStateMap(partitionId, participantStateMap); + } + } + return idealState; + } + return null; + } + + /** + * Create a resource snapshot instance from the physical model + * @param resourceId the resource id + * @param resourceConfiguration physical resource configuration + * @param idealState ideal state of the resource + * @param externalView external view of the resource + * @param resourceAssignment current resource assignment + * @return Resource + */ + static Resource createResource(ResourceId resourceId, + ResourceConfiguration resourceConfiguration, IdealState idealState, + ExternalView externalView, ResourceAssignment resourceAssignment) { + UserConfig userConfig; + RebalancerContext rebalancerContext = null; + ResourceType type = ResourceType.DATA; + if (resourceConfiguration != null) { + userConfig = resourceConfiguration.getUserConfig(); + type = resourceConfiguration.getType(); + } else { + userConfig = new UserConfig(Scope.resource(resourceId)); + } + int bucketSize = 0; + boolean batchMessageMode = false; + if (idealState != null) { + if (resourceConfiguration != null + && idealState.getRebalanceMode() != RebalanceMode.USER_DEFINED) { + // prefer rebalancer context for non-user_defined data rebalancing + rebalancerContext = + resourceConfiguration.getRebalancerContext(PartitionedRebalancerContext.class); + } + if (rebalancerContext == null) { + // prefer ideal state for non-user_defined data rebalancing + rebalancerContext = PartitionedRebalancerContext.from(idealState); + } + bucketSize = idealState.getBucketSize(); + batchMessageMode = idealState.getBatchMessageMode(); + idealState.updateUserConfig(userConfig); + } else if (resourceConfiguration != null) { + bucketSize = resourceConfiguration.getBucketSize(); + batchMessageMode = resourceConfiguration.getBatchMessageMode(); + RebalancerConfig rebalancerConfig = new RebalancerConfig(resourceConfiguration); + rebalancerContext = rebalancerConfig.getRebalancerContext(RebalancerContext.class); + } + if (rebalancerContext == null) { + rebalancerContext = new PartitionedRebalancerContext(); + } + return new Resource(resourceId, type, idealState, resourceAssignment, externalView, + rebalancerContext, userConfig, bucketSize, batchMessageMode); + } + + /** + * Get a ParticipantAccessor instance + * @return ParticipantAccessor + */ + protected ParticipantAccessor participantAccessor() { + return new ParticipantAccessor(_accessor); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java new file mode 100644 index 0000000000..467228053d --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java @@ -0,0 +1,741 @@ +package org.apache.helix.api.config; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ConstraintId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.model.ClusterConstraints; +import org.apache.helix.model.ClusterConstraints.ConstraintAttribute; +import org.apache.helix.model.ClusterConstraints.ConstraintType; +import org.apache.helix.model.ClusterConstraints.ConstraintValue; +import org.apache.helix.model.ConstraintItem; +import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.model.Transition; +import org.apache.helix.model.builder.ConstraintItemBuilder; +import org.apache.log4j.Logger; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +/* + * 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. + */ + +/** + * Configuration properties of a cluster + */ +public class ClusterConfig { + private static final Logger LOG = Logger.getLogger(ClusterConfig.class); + + private final ClusterId _id; + private final Map _resourceMap; + private final Map _participantMap; + private final Map _constraintMap; + private final Map _stateModelMap; + private final UserConfig _userConfig; + private final boolean _isPaused; + private final boolean _autoJoin; + + /** + * Initialize a cluster configuration. Also see ClusterConfig.Builder + * @param id cluster id + * @param resourceMap map of resource id to resource config + * @param participantMap map of participant id to participant config + * @param constraintMap map of constraint type to all constraints of that type + * @param stateModelMap map of state model id to state model definition + * @param stats statistics to watch on the cluster + * @param alerts alerts that the cluster can trigger + * @param userConfig user-defined cluster properties + * @param isPaused true if paused, false if active + * @param allowAutoJoin true if participants can join automatically, false otherwise + */ + private ClusterConfig(ClusterId id, Map resourceMap, + Map participantMap, + Map constraintMap, + Map stateModelMap, UserConfig userConfig, + boolean isPaused, boolean allowAutoJoin) { + _id = id; + _resourceMap = ImmutableMap.copyOf(resourceMap); + _participantMap = ImmutableMap.copyOf(participantMap); + _constraintMap = ImmutableMap.copyOf(constraintMap); + _stateModelMap = ImmutableMap.copyOf(stateModelMap); + _userConfig = userConfig; + _isPaused = isPaused; + _autoJoin = allowAutoJoin; + } + + /** + * Get cluster id + * @return cluster id + */ + public ClusterId getId() { + return _id; + } + + /** + * Get resources in the cluster + * @return a map of resource id to resource, or empty map if none + */ + public Map getResourceMap() { + return _resourceMap; + } + + /** + * Get all the constraints on the cluster + * @return map of constraint type to constraints + */ + public Map getConstraintMap() { + return _constraintMap; + } + + /** + * Get the maximum number of participants that can be in a state + * @param scope the scope for the bound + * @param stateModelDefId the state model of the state + * @param state the constrained state + * @return The upper bound, which can be "-1" if unspecified, a numerical upper bound, "R" for + * number of replicas, or "N" for number of participants + */ + public String getStateUpperBoundConstraint(Scope scope, StateModelDefId stateModelDefId, + State state) { + // set up attributes to match based on the scope + ClusterConstraints stateConstraints = getConstraintMap().get(ConstraintType.STATE_CONSTRAINT); + Map matchAttributes = Maps.newHashMap(); + matchAttributes.put(ConstraintAttribute.STATE, state.toString()); + matchAttributes.put(ConstraintAttribute.STATE_MODEL, stateModelDefId.toString()); + switch (scope.getType()) { + case CLUSTER: + // cluster is implicit + break; + case RESOURCE: + matchAttributes.put(ConstraintAttribute.RESOURCE, scope.getScopedId().stringify()); + break; + default: + LOG.error("Unsupported scope for state constraint: " + scope); + return "-1"; + } + Set matches = stateConstraints.match(matchAttributes); + int value = -1; + for (ConstraintItem item : matches) { + // match: if an R or N is found, always choose that one + // otherwise, take the minimum of the counts specified in the constraints + String constraintValue = item.getConstraintValue(); + if (constraintValue != null) { + if (constraintValue.equals(ConstraintValue.N.toString()) + || constraintValue.equals(ConstraintValue.R.toString())) { + return constraintValue; + } else { + try { + int current = Integer.parseInt(constraintValue); + if (value == -1 || current < value) { + value = current; + } + } catch (NumberFormatException e) { + LOG.error("Invalid state upper bound: " + constraintValue); + } + } + } + } + return Integer.toString(value); + } + + /** + * Get the limit of simultaneous execution of a transition + * @param scope the scope under which the transition is constrained + * @param stateModelDefId the state model of which the transition is a part + * @param transition the constrained transition + * @return the limit, or Integer.MAX_VALUE if there is no limit + */ + public int getTransitionConstraint(Scope scope, StateModelDefId stateModelDefId, + Transition transition) { + // set up attributes to match based on the scope + ClusterConstraints transitionConstraints = + getConstraintMap().get(ConstraintType.MESSAGE_CONSTRAINT); + Map matchAttributes = Maps.newHashMap(); + matchAttributes.put(ConstraintAttribute.STATE_MODEL, stateModelDefId.toString()); + matchAttributes.put(ConstraintAttribute.MESSAGE_TYPE, MessageType.STATE_TRANSITION.toString()); + matchAttributes.put(ConstraintAttribute.TRANSITION, transition.toString()); + switch (scope.getType()) { + case CLUSTER: + // cluster is implicit + break; + case RESOURCE: + matchAttributes.put(ConstraintAttribute.RESOURCE, scope.getScopedId().stringify()); + break; + case PARTICIPANT: + matchAttributes.put(ConstraintAttribute.INSTANCE, scope.getScopedId().stringify()); + break; + default: + LOG.error("Unsupported scope for transition constraints: " + scope); + return Integer.MAX_VALUE; + } + Set matches = transitionConstraints.match(matchAttributes); + int value = Integer.MAX_VALUE; + for (ConstraintItem item : matches) { + String constraintValue = item.getConstraintValue(); + if (constraintValue != null) { + try { + int current = Integer.parseInt(constraintValue); + if (current < value) { + value = current; + } + } catch (NumberFormatException e) { + LOG.error("Invalid in-flight transition cap: " + constraintValue); + } + } + } + return value; + } + + /** + * Get participants of the cluster + * @return a map of participant id to participant, or empty map if none + */ + public Map getParticipantMap() { + return _participantMap; + } + + /** + * Get all the state model definitions on the cluster + * @return map of state model definition id to state model definition + */ + public Map getStateModelMap() { + return _stateModelMap; + } + + /** + * Get user-specified configuration properties of this cluster + * @return UserConfig properties + */ + public UserConfig getUserConfig() { + return _userConfig; + } + + /** + * Check the paused status of the cluster + * @return true if paused, false otherwise + */ + public boolean isPaused() { + return _isPaused; + } + + /** + * Check if this cluster allows participants to join automatically + * @return true if allowed, false if disallowed + */ + public boolean autoJoinAllowed() { + return _autoJoin; + } + + /** + * Update context for a ClusterConfig + */ + public static class Delta { + private enum Fields { + USER_CONFIG, + AUTO_JOIN + } + + private Set _updateFields; + private Map> _removedConstraints; + private Builder _builder; + + /** + * Instantiate the delta for a cluster config + * @param clusterId the cluster to update + */ + public Delta(ClusterId clusterId) { + _updateFields = Sets.newHashSet(); + _removedConstraints = Maps.newHashMap(); + for (ConstraintType type : ConstraintType.values()) { + Set constraints = Sets.newHashSet(); + _removedConstraints.put(type, constraints); + } + _builder = new Builder(clusterId); + } + + /** + * Add a state upper bound constraint + * @param scope scope under which the constraint is valid + * @param stateModelDefId identifier of the state model that owns the state + * @param state the state to constrain + * @param upperBound maximum number of replicas per partition in the state + * @return Delta + */ + public Delta addStateUpperBoundConstraint(Scope scope, StateModelDefId stateModelDefId, + State state, int upperBound) { + return addStateUpperBoundConstraint(scope, stateModelDefId, state, + Integer.toString(upperBound)); + } + + /** + * Add a state upper bound constraint + * @param scope scope under which the constraint is valid + * @param stateModelDefId identifier of the state model that owns the state + * @param state the state to constrain + * @param dynamicUpperBound the upper bound of replicas per partition in the state, can be a + * number, or the currently supported special bound values:
+ * "R" - Refers to the number of replicas specified during resource + * creation. This allows having different replication factor for each + * resource without having to create a different state machine.
+ * "N" - Refers to all nodes in the cluster. Useful for resources that need + * to exist on all nodes. This way one can add/remove nodes without having + * the change the bounds. + * @return Delta + */ + public Delta addStateUpperBoundConstraint(Scope scope, StateModelDefId stateModelDefId, + State state, String dynamicUpperBound) { + _builder.addStateUpperBoundConstraint(scope, stateModelDefId, state, dynamicUpperBound); + return this; + } + + /** + * Remove state upper bound constraint + * @param scope scope under which the constraint is valid + * @param stateModelDefId identifier of the state model that owns the state + * @param state the state to constrain + * @return Delta + */ + public Delta removeStateUpperBoundConstraint(Scope scope, StateModelDefId stateModelDefId, + State state) { + _removedConstraints.get(ConstraintType.STATE_CONSTRAINT).add( + ConstraintId.from(scope, stateModelDefId, state)); + return this; + } + + /** + * Add a constraint on the maximum number of in-flight transitions of a certain type + * @param scope scope of the constraint + * @param stateModelDefId identifies the state model containing the transition + * @param transition the transition to constrain + * @param maxInFlightTransitions number of allowed in-flight transitions in the scope + * @return Delta + */ + public Delta addTransitionConstraint(Scope scope, StateModelDefId stateModelDefId, + Transition transition, int maxInFlightTransitions) { + _builder.addTransitionConstraint(scope, stateModelDefId, transition, maxInFlightTransitions); + return this; + } + + /** + * Remove a constraint on the maximum number of in-flight transitions of a certain type + * @param scope scope of the constraint + * @param stateModelDefId identifies the state model containing the transition + * @param transition the transition to constrain + * @return Delta + */ + public Delta removeTransitionConstraint(Scope scope, StateModelDefId stateModelDefId, + Transition transition) { + _removedConstraints.get(ConstraintType.MESSAGE_CONSTRAINT).add( + ConstraintId.from(scope, stateModelDefId, transition)); + return this; + } + + /** + * Add a single constraint item + * @param type type of the constraint item + * @param constraintId unique constraint id + * @param item instantiated ConstraintItem + * @return Delta + */ + public Delta addConstraintItem(ConstraintType type, ConstraintId constraintId, + ConstraintItem item) { + _builder.addConstraint(type, constraintId, item); + return this; + } + + /** + * Remove a single constraint item + * @param type type of the constraint item + * @param constraintId unique constraint id + * @return Delta + */ + public Delta removeConstraintItem(ConstraintType type, ConstraintId constraintId) { + _removedConstraints.get(type).add(constraintId); + return this; + } + + /* + * Set the user configuration + * @param userConfig user-specified properties + * @return Delta + */ + public Delta setUserConfig(UserConfig userConfig) { + _builder.userConfig(userConfig); + _updateFields.add(Fields.USER_CONFIG); + return this; + } + + /** + * Allow or disallow participants from automatically being able to join the cluster + * @param autoJoin true if allowed, false if disallowed + * @return Delta + */ + public Delta setAutoJoin(boolean autoJoin) { + _builder.autoJoin(autoJoin); + _updateFields.add(Fields.AUTO_JOIN); + return this; + } + + /** + * Create a ClusterConfig that is the combination of an existing ClusterConfig and this delta + * @param orig the original ClusterConfig + * @return updated ClusterConfig + */ + public ClusterConfig mergeInto(ClusterConfig orig) { + // copy in original and updated fields + ClusterConfig deltaConfig = _builder.build(); + Builder builder = + new Builder(orig.getId()).addResources(orig.getResourceMap().values()) + .addParticipants(orig.getParticipantMap().values()) + .addStateModelDefinitions(orig.getStateModelMap().values()) + .userConfig(orig.getUserConfig()).pausedStatus(orig.isPaused()) + .autoJoin(orig.autoJoinAllowed()); + for (Fields field : _updateFields) { + switch (field) { + case USER_CONFIG: + builder.userConfig(deltaConfig.getUserConfig()); + break; + case AUTO_JOIN: + builder.autoJoin(deltaConfig.autoJoinAllowed()); + break; + } + } + // add constraint deltas + for (ConstraintType type : ConstraintType.values()) { + ClusterConstraints constraints; + if (orig.getConstraintMap().containsKey(type)) { + constraints = orig.getConstraintMap().get(type); + } else { + constraints = new ClusterConstraints(type); + } + // add new constraints + if (deltaConfig.getConstraintMap().containsKey(type)) { + ClusterConstraints deltaConstraints = deltaConfig.getConstraintMap().get(type); + for (ConstraintId constraintId : deltaConstraints.getConstraintItems().keySet()) { + ConstraintItem constraintItem = deltaConstraints.getConstraintItem(constraintId); + constraints.addConstraintItem(constraintId, constraintItem); + } + } + // remove constraints + for (ConstraintId constraintId : _removedConstraints.get(type)) { + constraints.removeConstraintItem(constraintId); + } + builder.addConstraint(constraints); + } + + // get the result + ClusterConfig result = builder.build(); + + return result; + } + } + + /** + * Assembles a cluster configuration + */ + public static class Builder { + private final ClusterId _id; + private final Map _resourceMap; + private final Map _participantMap; + private final Map _constraintMap; + private final Map _stateModelMap; + private UserConfig _userConfig; + private boolean _isPaused; + private boolean _autoJoin; + + /** + * Initialize builder for a cluster + * @param id cluster id + */ + public Builder(ClusterId id) { + _id = id; + _resourceMap = new HashMap(); + _participantMap = new HashMap(); + _constraintMap = new HashMap(); + _stateModelMap = new HashMap(); + _isPaused = false; + _autoJoin = false; + _userConfig = new UserConfig(Scope.cluster(id)); + } + + /** + * Add a resource to the cluster + * @param resource resource configuration + * @return Builder + */ + public Builder addResource(ResourceConfig resource) { + _resourceMap.put(resource.getId(), resource); + return this; + } + + /** + * Add multiple resources to the cluster + * @param resources resource configurations + * @return Builder + */ + public Builder addResources(Collection resources) { + for (ResourceConfig resource : resources) { + addResource(resource); + } + return this; + } + + /** + * Add a participant to the cluster + * @param participant participant configuration + * @return Builder + */ + public Builder addParticipant(ParticipantConfig participant) { + _participantMap.put(participant.getId(), participant); + return this; + } + + /** + * Add multiple participants to the cluster + * @param participants participant configurations + * @return Builder + */ + public Builder addParticipants(Collection participants) { + for (ParticipantConfig participant : participants) { + addParticipant(participant); + } + return this; + } + + /** + * Add a constraint to the cluster + * @param constraint cluster constraint of a specific type + * @return Builder + */ + public Builder addConstraint(ClusterConstraints constraint) { + ClusterConstraints existConstraints = getConstraintsInstance(constraint.getType()); + for (ConstraintId constraintId : constraint.getConstraintItems().keySet()) { + existConstraints + .addConstraintItem(constraintId, constraint.getConstraintItem(constraintId)); + } + return this; + } + + /** + * Add a single constraint item + * @param type type of the constraint + * @param constraintId unique constraint identifier + * @param item instantiated ConstraintItem + * @return Builder + */ + public Builder addConstraint(ConstraintType type, ConstraintId constraintId, ConstraintItem item) { + ClusterConstraints existConstraints = getConstraintsInstance(type); + existConstraints.addConstraintItem(constraintId, item); + return this; + } + + /** + * Add multiple constraints to the cluster + * @param constraints cluster constraints of multiple distinct types + * @return Builder + */ + public Builder addConstraints(Collection constraints) { + for (ClusterConstraints constraint : constraints) { + addConstraint(constraint); + } + return this; + } + + /** + * Add a constraint on the maximum number of in-flight transitions of a certain type + * @param scope scope of the constraint + * @param stateModelDefId identifies the state model containing the transition + * @param transition the transition to constrain + * @param maxInFlightTransitions number of allowed in-flight transitions in the scope + * @return Builder + */ + public Builder addTransitionConstraint(Scope scope, StateModelDefId stateModelDefId, + Transition transition, int maxInFlightTransitions) { + Map attributes = Maps.newHashMap(); + attributes.put(ConstraintAttribute.MESSAGE_TYPE.toString(), + MessageType.STATE_TRANSITION.toString()); + attributes.put(ConstraintAttribute.CONSTRAINT_VALUE.toString(), + Integer.toString(maxInFlightTransitions)); + attributes.put(ConstraintAttribute.TRANSITION.toString(), transition.toString()); + attributes.put(ConstraintAttribute.STATE_MODEL.toString(), stateModelDefId.stringify()); + switch (scope.getType()) { + case CLUSTER: + // cluster is implicit + break; + case RESOURCE: + attributes.put(ConstraintAttribute.RESOURCE.toString(), scope.getScopedId().stringify()); + break; + case PARTICIPANT: + attributes.put(ConstraintAttribute.INSTANCE.toString(), scope.getScopedId().stringify()); + break; + default: + LOG.error("Unsupported scope for adding a transition constraint: " + scope); + return this; + } + ConstraintItem item = new ConstraintItemBuilder().addConstraintAttributes(attributes).build(); + ClusterConstraints constraints = getConstraintsInstance(ConstraintType.MESSAGE_CONSTRAINT); + constraints.addConstraintItem(ConstraintId.from(scope, stateModelDefId, transition), item); + return this; + } + + /** + * Add a state upper bound constraint + * @param scope scope under which the constraint is valid + * @param stateModelDefId identifier of the state model that owns the state + * @param state the state to constrain + * @param upperBound maximum number of replicas per partition in the state + * @return Builder + */ + public Builder addStateUpperBoundConstraint(Scope scope, StateModelDefId stateModelDefId, + State state, int upperBound) { + return addStateUpperBoundConstraint(scope, stateModelDefId, state, + Integer.toString(upperBound)); + } + + /** + * Add a state upper bound constraint + * @param scope scope under which the constraint is valid + * @param stateModelDefId identifier of the state model that owns the state + * @param state the state to constrain + * @param dynamicUpperBound the upper bound of replicas per partition in the state, can be a + * number, or the currently supported special bound values:
+ * "R" - Refers to the number of replicas specified during resource + * creation. This allows having different replication factor for each + * resource without having to create a different state machine.
+ * "N" - Refers to all nodes in the cluster. Useful for resources that need + * to exist on all nodes. This way one can add/remove nodes without having + * the change the bounds. + * @return Builder + */ + public Builder addStateUpperBoundConstraint(Scope scope, StateModelDefId stateModelDefId, + State state, String dynamicUpperBound) { + Map attributes = Maps.newHashMap(); + attributes.put(ConstraintAttribute.STATE.toString(), state.toString()); + attributes.put(ConstraintAttribute.STATE_MODEL.toString(), stateModelDefId.stringify()); + attributes.put(ConstraintAttribute.CONSTRAINT_VALUE.toString(), dynamicUpperBound); + switch (scope.getType()) { + case CLUSTER: + // cluster is implicit + break; + case RESOURCE: + attributes.put(ConstraintAttribute.RESOURCE.toString(), scope.getScopedId().stringify()); + break; + default: + LOG.error("Unsupported scope for adding a state constraint: " + scope); + return this; + } + ConstraintItem item = new ConstraintItemBuilder().addConstraintAttributes(attributes).build(); + ClusterConstraints constraints = getConstraintsInstance(ConstraintType.STATE_CONSTRAINT); + constraints.addConstraintItem(ConstraintId.from(scope, stateModelDefId, state), item); + return this; + } + + /** + * Add a state model definition to the cluster + * @param stateModelDef state model definition of the cluster + * @return Builder + */ + public Builder addStateModelDefinition(StateModelDefinition stateModelDef) { + _stateModelMap.put(stateModelDef.getStateModelDefId(), stateModelDef); + // add state constraints from the state model definition + for (State state : stateModelDef.getTypedStatesPriorityList()) { + if (!stateModelDef.getNumParticipantsPerState(state).equals("-1")) { + addStateUpperBoundConstraint(Scope.cluster(_id), stateModelDef.getStateModelDefId(), + state, stateModelDef.getNumParticipantsPerState(state)); + } + } + return this; + } + + /** + * Add multiple state model definitions + * @param stateModelDefs collection of state model definitions for the cluster + * @return Builder + */ + public Builder addStateModelDefinitions(Collection stateModelDefs) { + for (StateModelDefinition stateModelDef : stateModelDefs) { + addStateModelDefinition(stateModelDef); + } + return this; + } + + /** + * Set the paused status of the cluster + * @param isPaused true if paused, false otherwise + * @return Builder + */ + public Builder pausedStatus(boolean isPaused) { + _isPaused = isPaused; + return this; + } + + /** + * Allow or disallow participants from automatically being able to join the cluster + * @param autoJoin true if allowed, false if disallowed + * @return Builder + */ + public Builder autoJoin(boolean autoJoin) { + _autoJoin = autoJoin; + return this; + } + + /** + * Set the user configuration + * @param userConfig user-specified properties + * @return Builder + */ + public Builder userConfig(UserConfig userConfig) { + _userConfig = userConfig; + return this; + } + + /** + * Create the cluster configuration + * @return ClusterConfig + */ + public ClusterConfig build() { + return new ClusterConfig(_id, _resourceMap, _participantMap, _constraintMap, _stateModelMap, + _userConfig, _isPaused, _autoJoin); + } + + /** + * Get a valid instance of ClusterConstraints for a type + * @param type the type + * @return ClusterConstraints + */ + private ClusterConstraints getConstraintsInstance(ConstraintType type) { + ClusterConstraints constraints = _constraintMap.get(type); + if (constraints == null) { + constraints = new ClusterConstraints(type); + _constraintMap.put(type, constraints); + } + return constraints; + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java new file mode 100644 index 0000000000..4ea0ace8eb --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java @@ -0,0 +1,228 @@ +package org.apache.helix.api.config; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.helix.HelixProperty; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.Scope; + +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + +/* + * 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. + */ + +/** + * Generic configuration of Helix components prefixed with a namespace + */ +public class NamespacedConfig extends ZNRecord { + public static final char PREFIX_CHAR = '!'; + private final String _prefix; + + /** + * Instantiate a NamespacedConfig. It is intended for use only by entities that can be identified + * @param scope scope object + */ + public NamespacedConfig(Scope scope, String prefix) { + super(scope.getScopedId().stringify()); + _prefix = prefix + PREFIX_CHAR; + } + + /** + * Instantiate a NamespacedConfig from an existing HelixProperty + * @param property property wrapping a configuration + */ + public NamespacedConfig(HelixProperty property, String prefix) { + super(property.getRecord()); + _prefix = prefix + PREFIX_CHAR; + filterNonPrefixedFields(); + } + + /** + * Instantiate a NamespacedConfig as a copy of another NamedspacedConfig + * @param config populated NamespacedConfig + */ + public NamespacedConfig(NamespacedConfig config) { + super(config.getId()); + _prefix = config.getPrefix() + PREFIX_CHAR; + if (config.getRawPayload() != null && config.getRawPayload().length > 0) { + setRawPayload(config.getRawPayload()); + setPayloadSerializer(config.getPayloadSerializer()); + } + super.setSimpleFields(config.getPrefixedSimpleFields()); + super.setListFields(config.getPrefixedListFields()); + super.setMapFields(config.getPrefixedMapFields()); + } + + @Override + public void setMapField(String k, Map v) { + super.setMapField(_prefix + k, v); + } + + @Override + public Map getMapField(String k) { + return super.getMapField(_prefix + k); + } + + @Override + public void setMapFields(Map> mapFields) { + for (String k : mapFields.keySet()) { + super.setMapField(_prefix + k, mapFields.get(k)); + } + } + + /** + * Returns an immutable map of map fields + */ + @Override + public Map> getMapFields() { + return convertToPrefixlessMap(super.getMapFields(), _prefix); + } + + @Override + public void setListField(String k, List v) { + super.setListField(_prefix + k, v); + } + + @Override + public List getListField(String k) { + return super.getListField(_prefix + k); + } + + @Override + public void setListFields(Map> listFields) { + for (String k : listFields.keySet()) { + super.setListField(_prefix + k, listFields.get(k)); + } + } + + /** + * Returns an immutable map of list fields + */ + @Override + public Map> getListFields() { + return convertToPrefixlessMap(super.getListFields(), _prefix); + } + + @Override + public void setSimpleField(String k, String v) { + super.setSimpleField(_prefix + k, v); + } + + @Override + public String getSimpleField(String k) { + return super.getSimpleField(_prefix + k); + } + + @Override + public void setSimpleFields(Map simpleFields) { + for (String k : simpleFields.keySet()) { + super.setSimpleField(_prefix + k, simpleFields.get(k)); + } + } + + /** + * Returns an immutable map of simple fields + */ + @Override + public Map getSimpleFields() { + return convertToPrefixlessMap(super.getSimpleFields(), _prefix); + } + + /** + * Get the prefix used to distinguish these config properties + * @return string prefix, not including the underscore + */ + public String getPrefix() { + return _prefix.substring(0, _prefix.indexOf(PREFIX_CHAR)); + } + + /** + * Remove all fields from this config that are not prefixed + */ + private void filterNonPrefixedFields() { + // filter out any configuration that isn't user-defined + Predicate keyFilter = new Predicate() { + @Override + public boolean apply(String key) { + return key.contains(_prefix); + } + }; + super.setMapFields(Maps.filterKeys(super.getMapFields(), keyFilter)); + super.setListFields(Maps.filterKeys(super.getListFields(), keyFilter)); + super.setSimpleFields(Maps.filterKeys(super.getSimpleFields(), keyFilter)); + } + + /** + * Get all map fields with prefixed keys + * @return prefixed map fields + */ + public Map> getPrefixedMapFields() { + return super.getMapFields(); + } + + /** + * Get all list fields with prefixed keys + * @return prefixed list fields + */ + public Map> getPrefixedListFields() { + return super.getListFields(); + } + + /** + * Get all simple fields with prefixed keys + * @return prefixed simple fields + */ + public Map getPrefixedSimpleFields() { + return super.getSimpleFields(); + } + + /** + * Add user configuration to an existing helix property. + * @param property the property to update + * @param config the user config + */ + public static void addConfigToProperty(HelixProperty property, NamespacedConfig config) { + ZNRecord record = property.getRecord(); + record.getMapFields().putAll(config.getPrefixedMapFields()); + record.getListFields().putAll(config.getPrefixedListFields()); + record.getSimpleFields().putAll(config.getPrefixedSimpleFields()); + if (config.getRawPayload() != null && config.getRawPayload().length > 0) { + record.setPayloadSerializer(config.getPayloadSerializer()); + record.setRawPayload(config.getRawPayload()); + } + } + + /** + * Get a copy of a map with the key prefix stripped. The resulting map is immutable + * @param rawMap map of key, value pairs where the key is prefixed + * @return map of key, value pairs where the key is not prefixed + */ + private static Map convertToPrefixlessMap(Map rawMap, String prefix) { + Map convertedMap = new HashMap(); + for (String rawKey : rawMap.keySet()) { + String k = rawKey.substring(prefix.length()); + convertedMap.put(k, rawMap.get(rawKey)); + } + return ImmutableMap.copyOf(convertedMap); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java new file mode 100644 index 0000000000..1d7b23efcd --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java @@ -0,0 +1,382 @@ +package org.apache.helix.api.config; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.helix.api.Scope; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; + +/* + * 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. + */ + +/** + * Configuration properties of a Helix participant + */ +public class ParticipantConfig { + private final ParticipantId _id; + private final String _hostName; + private final int _port; + private final boolean _isEnabled; + private final Set _disabledPartitions; + private final Set _tags; + private final UserConfig _userConfig; + + /** + * Initialize a participant configuration. Also see ParticipantConfig.Builder + * @param id participant id + * @param hostName host where participant can be reached + * @param port port to use to contact participant + * @param isEnabled true if enabled, false if disabled + * @param disabledPartitions set of partitions, if any to disable on this participant + * @param tags tags to set for the participant + */ + public ParticipantConfig(ParticipantId id, String hostName, int port, boolean isEnabled, + Set disabledPartitions, Set tags, UserConfig userConfig) { + _id = id; + _hostName = hostName; + _port = port; + _isEnabled = isEnabled; + _disabledPartitions = ImmutableSet.copyOf(disabledPartitions); + _tags = ImmutableSet.copyOf(tags); + _userConfig = userConfig; + } + + /** + * Get the host name of the participant + * @return host name, or null if not applicable + */ + public String getHostName() { + return _hostName; + } + + /** + * Get the port of the participant + * @return port number, or -1 if not applicable + */ + public int getPort() { + return _port; + } + + /** + * Get if the participant is enabled + * @return true if enabled or false otherwise + */ + public boolean isEnabled() { + return _isEnabled; + } + + /** + * Get disabled partition id's + * @return set of disabled partition id's, or empty set if none + */ + public Set getDisabledPartitions() { + return _disabledPartitions; + } + + /** + * Get tags + * @return set of tags + */ + public Set getTags() { + return _tags; + } + + /** + * Check if participant has a tag + * @param tag tag to check + * @return true if tagged, false otherwise + */ + public boolean hasTag(String tag) { + return _tags.contains(tag); + } + + /** + * Get user-specified configuration properties of this participant + * @return UserConfig properties + */ + public UserConfig getUserConfig() { + return _userConfig; + } + + /** + * Get the participant id + * @return ParticipantId + */ + public ParticipantId getId() { + return _id; + } + + /** + * Update context for a ParticipantConfig + */ + public static class Delta { + private enum Fields { + HOST_NAME, + PORT, + ENABLED, + USER_CONFIG + } + + private Set _updateFields; + private Set _removedTags; + private Set _removedDisabledPartitions; + private Builder _builder; + + /** + * Instantiate the delta for a participant config + * @param participantId the participant to update + */ + public Delta(ParticipantId participantId) { + _updateFields = Sets.newHashSet(); + _removedTags = Sets.newHashSet(); + _removedDisabledPartitions = Sets.newHashSet(); + _builder = new Builder(participantId); + } + + /** + * Set the participant host name + * @param hostName reachable host when live + * @return Delta + */ + public Delta setHostName(String hostName) { + _builder.hostName(hostName); + _updateFields.add(Fields.HOST_NAME); + return this; + } + + /** + * Set the participant port + * @param port port number + * @return Delta + */ + public Delta setPort(int port) { + _builder.port(port); + _updateFields.add(Fields.PORT); + return this; + } + + /** + * <<<<<<< HEAD:helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java + * Set the enabled status of the participant + * @param isEnabled true if enabled, false if disabled + * ======= + * Set whether or not the participant is enabled + * @param isEnabled true if enabled, false otherwise + * >>>>>>> helix-logical-model:helix-core/src/main/java/org/apache/helix/api/ + * ParticipantConfig.java + * @return Delta + */ + public Delta setEnabled(boolean isEnabled) { + _builder.enabled(isEnabled); + _updateFields.add(Fields.ENABLED); + return this; + } + + /** + * Set the user configuration + * @param userConfig user-specified properties + * @return Delta + */ + public Delta setUserConfig(UserConfig userConfig) { + _builder.userConfig(userConfig); + _updateFields.add(Fields.USER_CONFIG); + return this; + } + + /** + * Add an new tag for this participant + * @param tag the tag to add + * @return Delta + */ + public Delta addTag(String tag) { + _builder.addTag(tag); + return this; + } + + /** + * Remove a tag for this participant + * @param tag the tag to remove + * @return Delta + */ + public Delta removeTag(String tag) { + _removedTags.add(tag); + return this; + } + + /** + * Add a partition to disable for this participant + * @param partitionId the partition to disable + * @return Delta + */ + public Delta addDisabledPartition(PartitionId partitionId) { + _builder.addDisabledPartition(partitionId); + return this; + } + + /** + * Remove a partition from the disabled set for this participant + * @param partitionId the partition to enable + * @return Delta + */ + public Delta removeDisabledPartition(PartitionId partitionId) { + _removedDisabledPartitions.add(partitionId); + return this; + } + + /** + * Create a ParticipantConfig that is the combination of an existing ParticipantConfig and this + * delta + * @param orig the original ParticipantConfig + * @return updated ParticipantConfig + */ + public ParticipantConfig mergeInto(ParticipantConfig orig) { + ParticipantConfig deltaConfig = _builder.build(); + Builder builder = + new Builder(orig.getId()).hostName(orig.getHostName()).port(orig.getPort()) + .enabled(orig.isEnabled()).userConfig(orig.getUserConfig()); + for (Fields field : _updateFields) { + switch (field) { + case HOST_NAME: + builder.hostName(deltaConfig.getHostName()); + break; + case PORT: + builder.port(deltaConfig.getPort()); + break; + case ENABLED: + builder.enabled(deltaConfig.isEnabled()); + break; + case USER_CONFIG: + builder.userConfig(deltaConfig.getUserConfig()); + break; + } + } + Set tags = Sets.newHashSet(orig.getTags()); + tags.addAll(deltaConfig.getTags()); + tags.removeAll(_removedTags); + for (String tag : tags) { + builder.addTag(tag); + } + Set disabledPartitions = Sets.newHashSet(orig.getDisabledPartitions()); + disabledPartitions.addAll(deltaConfig.getDisabledPartitions()); + disabledPartitions.removeAll(_removedDisabledPartitions); + for (PartitionId partitionId : disabledPartitions) { + builder.addDisabledPartition(partitionId); + } + return builder.build(); + } + } + + /** + * Assemble a participant + */ + public static class Builder { + private final ParticipantId _id; + private String _hostName; + private int _port; + private boolean _isEnabled; + private final Set _disabledPartitions; + private final Set _tags; + private UserConfig _userConfig; + + /** + * Build a participant with a given id + * @param id participant id + */ + public Builder(ParticipantId id) { + _id = id; + _disabledPartitions = new HashSet(); + _tags = new HashSet(); + _isEnabled = true; + _userConfig = new UserConfig(Scope.participant(id)); + } + + /** + * Set the participant host name + * @param hostName reachable host when live + * @return Builder + */ + public Builder hostName(String hostName) { + _hostName = hostName; + return this; + } + + /** + * Set the participant port + * @param port port number + * @return Builder + */ + public Builder port(int port) { + _port = port; + return this; + } + + /** + * Set whether or not the participant is enabled + * @param isEnabled true if enabled, false otherwise + * @return Builder + */ + public Builder enabled(boolean isEnabled) { + _isEnabled = isEnabled; + return this; + } + + /** + * Add a partition to disable for this participant + * @param partitionId the partition to disable + * @return Builder + */ + public Builder addDisabledPartition(PartitionId partitionId) { + _disabledPartitions.add(partitionId); + return this; + } + + /** + * Add an arbitrary tag for this participant + * @param tag the tag to add + * @return Builder + */ + public Builder addTag(String tag) { + _tags.add(tag); + return this; + } + + /** + * Set the user configuration + * @param userConfig user-specified properties + * @return Builder + */ + public Builder userConfig(UserConfig userConfig) { + _userConfig = userConfig; + return this; + } + + /** + * Assemble the participant + * @return instantiated Participant + */ + public ParticipantConfig build() { + return new ParticipantConfig(_id, _hostName, _port, _isEnabled, _disabledPartitions, _tags, + _userConfig); + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java new file mode 100644 index 0000000000..38d48ab958 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java @@ -0,0 +1,373 @@ +package org.apache.helix.api.config; + +import java.util.Map; +import java.util.Set; + +import org.apache.helix.api.Partition; +import org.apache.helix.api.Scope; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; + +import com.google.common.collect.Sets; + +/* + * 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. + */ + +/** + * Full configuration of a Helix resource. Typically used to add or modify resources on a cluster + */ +public class ResourceConfig { + /** + * Type of a resource. A resource is any entity that can be managed by Helix. + */ + public enum ResourceType { + /** + * A resource that is persistent, and potentially partitioned and replicated. + */ + DATA + } + + private final ResourceId _id; + private final RebalancerConfig _rebalancerConfig; + private final SchedulerTaskConfig _schedulerTaskConfig; + private final UserConfig _userConfig; + private final int _bucketSize; + private final boolean _batchMessageMode; + private final ResourceType _resourceType; + + /** + * Instantiate a configuration. Consider using ResourceConfig.Builder + * @param id resource id + * @param partitionMap map of partition identifiers to partition objects + * @param schedulerTaskConfig configuration for scheduler tasks associated with the resource + * @param rebalancerConfig configuration for rebalancing the resource + * @param userConfig user-defined resource properties + * @param bucketSize bucket size for this resource + * @param batchMessageMode whether or not batch messaging is allowed + */ + public ResourceConfig(ResourceId id, ResourceType resourceType, + SchedulerTaskConfig schedulerTaskConfig, RebalancerConfig rebalancerConfig, + UserConfig userConfig, int bucketSize, boolean batchMessageMode) { + _id = id; + _resourceType = resourceType; + _schedulerTaskConfig = schedulerTaskConfig; + _rebalancerConfig = rebalancerConfig; + _userConfig = userConfig; + _bucketSize = bucketSize; + _batchMessageMode = batchMessageMode; + } + + /** + * Get the subunits of the resource + * @return map of subunit id to subunit or empty map if none + */ + public Map getSubUnitMap() { + return _rebalancerConfig.getRebalancerContext(RebalancerContext.class).getSubUnitMap(); + } + + /** + * Get a subunit that the resource contains + * @param subUnitId the subunit id to look up + * @return Partition or null if none is present with the given id + */ + public Partition getSubUnit(PartitionId subUnitId) { + return getSubUnitMap().get(subUnitId); + } + + /** + * Get the set of subunit ids that the resource contains + * @return subunit id set, or empty if none + */ + public Set getSubUnitSet() { + return getSubUnitMap().keySet(); + } + + /** + * Get the resource properties configuring rebalancing + * @return RebalancerConfig properties + */ + public RebalancerConfig getRebalancerConfig() { + return _rebalancerConfig; + } + + /** + * Get the resource id + * @return ResourceId + */ + public ResourceId getId() { + return _id; + } + + /** + * Get the resource type + * @return ResourceType + */ + public ResourceType getType() { + return _resourceType; + } + + /** + * Get the properties configuring scheduler tasks + * @return SchedulerTaskConfig properties + */ + public SchedulerTaskConfig getSchedulerTaskConfig() { + return _schedulerTaskConfig; + } + + /** + * Get user-specified configuration properties of this resource + * @return UserConfig properties + */ + public UserConfig getUserConfig() { + return _userConfig; + } + + /** + * Get the bucket size for this resource + * @return bucket size + */ + public int getBucketSize() { + return _bucketSize; + } + + /** + * Get the batch message mode + * @return true if enabled, false if disabled + */ + public boolean getBatchMessageMode() { + return _batchMessageMode; + } + + @Override + public String toString() { + return getSubUnitMap().toString(); + } + + /** + * Update context for a ResourceConfig + */ + public static class Delta { + private enum Fields { + TYPE, + REBALANCER_CONTEXT, + USER_CONFIG, + BUCKET_SIZE, + BATCH_MESSAGE_MODE + } + + private Set _updateFields; + private Builder _builder; + + /** + * Instantiate the delta for a resource config + * @param resourceId the resource to update + */ + public Delta(ResourceId resourceId) { + _builder = new Builder(resourceId); + _updateFields = Sets.newHashSet(); + } + + /** + * Set the type of this resource + * @param type ResourceType + * @return Delta + */ + public Delta setType(ResourceType type) { + _builder.type(type); + _updateFields.add(Fields.TYPE); + return this; + } + + /** + * Set the rebalancer configuration + * @param context properties of interest for rebalancing + * @return Delta + */ + public Delta setRebalancerContext(RebalancerContext context) { + _builder.rebalancerContext(context); + _updateFields.add(Fields.REBALANCER_CONTEXT); + return this; + } + + /** + * Set the user configuration + * @param userConfig user-specified properties + * @return Delta + */ + public Delta setUserConfig(UserConfig userConfig) { + _builder.userConfig(userConfig); + _updateFields.add(Fields.USER_CONFIG); + return this; + } + + /** + * Set the bucket size + * @param bucketSize the size to use + * @return Delta + */ + public Delta setBucketSize(int bucketSize) { + _builder.bucketSize(bucketSize); + _updateFields.add(Fields.BUCKET_SIZE); + return this; + } + + /** + * Set the batch message mode + * @param batchMessageMode true to enable, false to disable + * @return Delta + */ + public Delta setBatchMessageMode(boolean batchMessageMode) { + _builder.batchMessageMode(batchMessageMode); + _updateFields.add(Fields.BATCH_MESSAGE_MODE); + return this; + } + + /** + * Create a ResourceConfig that is the combination of an existing ResourceConfig and this delta + * @param orig the original ResourceConfig + * @return updated ResourceConfig + */ + public ResourceConfig mergeInto(ResourceConfig orig) { + ResourceConfig deltaConfig = _builder.build(); + Builder builder = + new Builder(orig.getId()) + .type(orig.getType()) + .rebalancerContext( + orig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class)) + .schedulerTaskConfig(orig.getSchedulerTaskConfig()).userConfig(orig.getUserConfig()) + .bucketSize(orig.getBucketSize()).batchMessageMode(orig.getBatchMessageMode()); + for (Fields field : _updateFields) { + switch (field) { + case TYPE: + builder.type(deltaConfig.getType()); + break; + case REBALANCER_CONTEXT: + builder.rebalancerContext(deltaConfig.getRebalancerConfig().getRebalancerContext( + RebalancerContext.class)); + break; + case USER_CONFIG: + builder.userConfig(deltaConfig.getUserConfig()); + break; + case BUCKET_SIZE: + builder.bucketSize(deltaConfig.getBucketSize()); + break; + case BATCH_MESSAGE_MODE: + builder.batchMessageMode(deltaConfig.getBatchMessageMode()); + break; + } + } + return builder.build(); + } + } + + /** + * Assembles a ResourceConfig + */ + public static class Builder { + private final ResourceId _id; + private ResourceType _type; + private RebalancerConfig _rebalancerConfig; + private SchedulerTaskConfig _schedulerTaskConfig; + private UserConfig _userConfig; + private int _bucketSize; + private boolean _batchMessageMode; + + /** + * Build a Resource with an id + * @param id resource id + */ + public Builder(ResourceId id) { + _id = id; + _type = ResourceType.DATA; + _bucketSize = 0; + _batchMessageMode = false; + _userConfig = new UserConfig(Scope.resource(id)); + } + + /** + * Set the type of this resource + * @param type ResourceType + * @return Builder + */ + public Builder type(ResourceType type) { + _type = type; + return this; + } + + /** + * Set the rebalancer configuration + * @param rebalancerContext properties of interest for rebalancing + * @return Builder + */ + public Builder rebalancerContext(RebalancerContext rebalancerContext) { + _rebalancerConfig = new RebalancerConfig(rebalancerContext); + return this; + } + + /** + * Set the user configuration + * @param userConfig user-specified properties + * @return Builder + */ + public Builder userConfig(UserConfig userConfig) { + _userConfig = userConfig; + return this; + } + + /** + * @param schedulerTaskConfig + * @return + */ + public Builder schedulerTaskConfig(SchedulerTaskConfig schedulerTaskConfig) { + _schedulerTaskConfig = schedulerTaskConfig; + return this; + } + + /** + * Set the bucket size + * @param bucketSize the size to use + * @return Builder + */ + public Builder bucketSize(int bucketSize) { + _bucketSize = bucketSize; + return this; + } + + /** + * Set the batch message mode + * @param batchMessageMode true to enable, false to disable + * @return Builder + */ + public Builder batchMessageMode(boolean batchMessageMode) { + _batchMessageMode = batchMessageMode; + return this; + } + + /** + * Create a Resource object + * @return instantiated Resource + */ + public ResourceConfig build() { + return new ResourceConfig(_id, _type, _schedulerTaskConfig, _rebalancerConfig, _userConfig, + _bucketSize, _batchMessageMode); + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/config/SchedulerTaskConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/SchedulerTaskConfig.java new file mode 100644 index 0000000000..bb67be239f --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/config/SchedulerTaskConfig.java @@ -0,0 +1,88 @@ +package org.apache.helix.api.config; + +/* + * 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.Set; + +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.model.Message; + +import com.google.common.collect.ImmutableMap; + +public class SchedulerTaskConfig { + // TODO refactor using Transition logical model + private final Map _transitionTimeoutMap; + + private final Map _innerMessageMap; + + public SchedulerTaskConfig(Map transitionTimeoutMap, + Map innerMsgMap) { + _transitionTimeoutMap = ImmutableMap.copyOf(transitionTimeoutMap); + _innerMessageMap = ImmutableMap.copyOf(innerMsgMap); + } + + /** + * Get inner message for a partition + * @param partitionId + * @return inner message + */ + public Message getInnerMessage(PartitionId partitionId) { + return _innerMessageMap.get(partitionId); + } + + /** + * Get timeout for a transition + * @param transition + * @return timeout or -1 if not available + */ + public int getTransitionTimeout(String transition) { + Integer timeout = _transitionTimeoutMap.get(transition); + if (timeout == null) { + return -1; + } + + return timeout; + } + + /** + * Get timeout for an inner message + * @param transition + * @param partitionId + * @return timeout or -1 if not available + */ + public int getTimeout(String transition, PartitionId partitionId) { + Integer timeout = getTransitionTimeout(transition); + if (timeout == null) { + Message innerMessage = getInnerMessage(partitionId); + timeout = innerMessage.getTimeout(); + } + + return timeout; + } + + /** + * Get partition-id set + * @return partition-id set + */ + public Set getPartitionSet() { + return _innerMessageMap.keySet(); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java new file mode 100644 index 0000000000..dbf70ea45d --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java @@ -0,0 +1,53 @@ +package org.apache.helix.api.config; + +import org.apache.helix.HelixProperty; +import org.apache.helix.api.Scope; + +/* + * 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. + */ + +/** + * Generic user-defined configuration of Helix components + */ +public class UserConfig extends NamespacedConfig { + /** + * Instantiate a UserConfig. It is intended for use only by entities that can be identified + * @param scope scope of the configuration, e.g. cluster, resource, partition, participant, etc + */ + public UserConfig(Scope scope) { + super(scope, UserConfig.class.getSimpleName()); + } + + /** + * Instantiate a UserConfig from an existing HelixProperty + * @param property property wrapping a configuration + */ + private UserConfig(HelixProperty property) { + super(property, UserConfig.class.getSimpleName()); + } + + /** + * Get a UserConfig that filters out the user-specific configurations in a property + * @param property the property to extract from + * @return UserConfig + */ + public static UserConfig from(HelixProperty property) { + return new UserConfig(property); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java b/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java new file mode 100644 index 0000000000..f05bb5d552 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java @@ -0,0 +1,57 @@ +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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. + */ + +/** + * Identifies a cluster + */ +public class ClusterId extends Id { + @JsonProperty("id") + final private String _id; + + /** + * Create a cluster id + * @param id string representation of the id + */ + @JsonCreator + public ClusterId(@JsonProperty("id") String id) { + _id = id; + } + + @Override + public String stringify() { + return _id; + } + + /** + * Get a concrete cluster id for a string name + * @param clusterId string cluster identifier + * @return ClusterId + */ + public static ClusterId from(String clusterId) { + if (clusterId == null) { + return null; + } + return new ClusterId(clusterId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java b/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java new file mode 100644 index 0000000000..26a7610bd3 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java @@ -0,0 +1,80 @@ +package org.apache.helix.api.id; + +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.model.Transition; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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. + */ + +/** + * Identifies a constraint item on the cluster + */ +public class ConstraintId extends Id { + @JsonProperty("id") + private final String _id; + + /** + * Create a constraint id + * @param constraintId string representing the constraint id + */ + @JsonCreator + public ConstraintId(@JsonProperty("id") String id) { + _id = id; + } + + @Override + public String stringify() { + return _id; + } + + /** + * Get a constraint id from a string + * @param constraintId string representing the constraint id + * @return ConstraintId + */ + public static ConstraintId from(String constraintId) { + return new ConstraintId(constraintId); + } + + /** + * Get a state constraint id based on the state model definition and state + * @param scope the scope of the constraint + * @param stateModelDefId the state model + * @param state the constrained state + * @return ConstraintId + */ + public static ConstraintId from(Scope scope, StateModelDefId stateModelDefId, State state) { + return new ConstraintId(scope + "|" + stateModelDefId + "|" + state); + } + + /** + * Get a state constraint id based on the state model definition and transition + * @param scope the scope of the constraint + * @param stateModelDefId the state model + * @param transition the constrained transition + * @return ConstraintId + */ + public static ConstraintId from(Scope scope, StateModelDefId stateModelDefId, + Transition transition) { + return new ConstraintId(scope + "|" + stateModelDefId + "|" + transition); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java b/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java new file mode 100644 index 0000000000..1130afd226 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java @@ -0,0 +1,54 @@ +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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. + */ + +/** + * Identifies Helix nodes that take on the CONTROLLER role + */ +public class ControllerId extends Id { + @JsonProperty("id") + private final String _id; + + /** + * Create a controller id + * @param id string representation of a controller id + */ + @JsonCreator + public ControllerId(@JsonProperty("id") String id) { + _id = id; + } + + @Override + public String stringify() { + return _id; + } + + /** + * Get a ControllerId from a string + * @param controllerId string representing the id + * @return ControllerId + */ + public static ControllerId from(String controllerId) { + return new ControllerId(controllerId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/Id.java b/helix-core/src/main/java/org/apache/helix/api/id/Id.java new file mode 100644 index 0000000000..ea9c6cc93e --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/Id.java @@ -0,0 +1,55 @@ +package org.apache.helix.api.id; + +/* + * 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. + */ + +/** + * Generic identifier for Helix constructs + */ +public abstract class Id implements Comparable { + public abstract String stringify(); + + @Override + public String toString() { + return stringify(); + } + + @Override + public boolean equals(Object that) { + if (that instanceof Id) { + return this.stringify().equals(((Id) that).stringify()); + } else if (that instanceof String) { + return this.stringify().equals(that); + } + return false; + } + + @Override + public int hashCode() { + return this.stringify().hashCode(); + } + + @Override + public int compareTo(Id that) { + if (that instanceof Id) { + return this.stringify().compareTo(that.stringify()); + } + return -1; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java b/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java new file mode 100644 index 0000000000..a59976d09f --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java @@ -0,0 +1,54 @@ +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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 MessageId extends Id { + @JsonProperty("id") + private final String _id; + + /** + * Create a message id + * @param id string representation of a message id + */ + @JsonCreator + public MessageId(@JsonProperty("id") String id) { + _id = id; + } + + @Override + public String stringify() { + return _id; + } + + /** + * Get a concrete message id + * @param messageId string message identifier + * @return MsgId + */ + public static MessageId from(String messageId) { + if (messageId == null) { + return null; + } + return new MessageId(messageId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java b/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java new file mode 100644 index 0000000000..5fe91dd091 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java @@ -0,0 +1,54 @@ +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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 ParticipantId extends Id { + @JsonProperty("id") + private final String _id; + + /** + * Instantiate for a participant with a string name + * @param id string participant id + */ + @JsonCreator + public ParticipantId(@JsonProperty("id") String id) { + _id = id; + } + + @Override + public String stringify() { + return _id; + } + + /** + * Get a concrete participant id + * @param participantId string participant identifier + * @return ParticipantId + */ + public static ParticipantId from(String participantId) { + if (participantId == null) { + return null; + } + return new ParticipantId(participantId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java b/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java new file mode 100644 index 0000000000..dd1bc0dbe0 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java @@ -0,0 +1,112 @@ +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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 PartitionId extends Id { + @JsonProperty("resourceId") + private final ResourceId _resourceId; + @JsonProperty("partitionName") + private final String _partitionName; + + /** + * Instantiate for a resource and suffix + * @param resourceId resource that the partition belongs to + * @param partitionName name of the partition relative to the resource + */ + @JsonCreator + public PartitionId(@JsonProperty("resourceId") ResourceId resourceId, + @JsonProperty("partitionName") String partitionName) { + _resourceId = resourceId; + _partitionName = partitionName; + } + + /** + * Get the id of the resource containing this partition + * @return ResourceId + */ + public ResourceId getResourceId() { + return _resourceId; + } + + @Override + public String stringify() { + // check in case the partition name is instantiated incorrectly + if (_resourceId.stringify().equals(_partitionName)) { + return _partitionName; + } + return String.format("%s_%s", _resourceId, _partitionName); + } + + /** + * @param partitionName + * @return + */ + public static String stripResourceId(String partitionName) { + if (partitionName == null || !partitionName.contains("_")) { + return partitionName; + } + return partitionName.substring(partitionName.lastIndexOf("_") + 1); + } + + /** + * @param partitionName + * @return + */ + public static ResourceId extractResourceId(String partitionName) { + if (partitionName == null || !partitionName.contains("_")) { + return ResourceId.from(partitionName); + } + return ResourceId.from(partitionName.substring(0, partitionName.lastIndexOf("_"))); + } + + /** + * Get a concrete partition id + * @param partitionId string partition identifier + * @return PartitionId + */ + public static PartitionId from(String partitionId) { + if (partitionId == null) { + return null; + } + return new PartitionId(extractResourceId(partitionId), stripResourceId(partitionId)); + } + + /** + * Same as {@link PartitionId#from(String)}. + * @param partitionId string partition identifier + * @return PartitionId + */ + public static PartitionId valueOf(String partitionId) { + return from(partitionId); + } + + /** + * Get a concrete partition id + * @param resourceId resource identifier + * @param partitionSuffix partition identifier relative to a resource + * @return PartitionId + */ + public static PartitionId from(ResourceId resourceId, String partitionSuffix) { + return new PartitionId(resourceId, partitionSuffix); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java b/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java new file mode 100644 index 0000000000..3b8c21d8c7 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java @@ -0,0 +1,54 @@ +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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 ProcId extends Id { + @JsonProperty("id") + private final String _id; + + /** + * Create a process id + * @param id string representation of a process id + */ + @JsonCreator + public ProcId(@JsonProperty("id") String id) { + _id = id; + } + + @Override + public String stringify() { + return _id; + } + + /** + * Get a concrete process id + * @param processId string process identifier (e.g. pid@host) + * @return ProcId + */ + public static ProcId from(String processId) { + if (processId == null) { + return null; + } + return new ProcId(processId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java b/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java new file mode 100644 index 0000000000..e70fff15d5 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java @@ -0,0 +1,57 @@ +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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. + */ + +/** + * Identifies a resource + */ +public class ResourceId extends Id { + @JsonProperty("id") + private final String _id; + + @Override + public String stringify() { + return _id; + } + + /** + * Create a resource id + * @param id string representation of a resource id + */ + @JsonCreator + public ResourceId(@JsonProperty("id") String id) { + _id = id; + } + + /** + * Get a concrete resource id for a string name + * @param resourceId string resource identifier + * @return ResourceId + */ + public static ResourceId from(String resourceId) { + if (resourceId == null) { + return null; + } + return new ResourceId(resourceId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/GreaterAlertComparator.java b/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java similarity index 55% rename from helix-core/src/main/java/org/apache/helix/alerts/GreaterAlertComparator.java rename to helix-core/src/main/java/org/apache/helix/api/id/SessionId.java index 0e9c8f1bd2..17fb3a32a0 100644 --- a/helix-core/src/main/java/org/apache/helix/alerts/GreaterAlertComparator.java +++ b/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java @@ -1,4 +1,7 @@ -package org.apache.helix.alerts; +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,27 +22,33 @@ * under the License. */ -import java.util.Iterator; +public class SessionId extends Id { + @JsonProperty("id") + private final String _id; -public class GreaterAlertComparator extends AlertComparator { + /** + * Create a session id + * @param id string representing a session id + */ + @JsonCreator + public SessionId(@JsonProperty("id") String id) { + _id = id; + } @Override - /* - * Returns true if any element left tuple exceeds any element in right tuple + public String stringify() { + return _id; + } + + /** + * Get a concrete session id + * @param sessionId string session identifier + * @return SessionId */ - public boolean evaluate(Tuple leftTup, Tuple rightTup) { - Iterator leftIter = leftTup.iterator(); - while (leftIter.hasNext()) { - double leftVal = Double.parseDouble(leftIter.next()); - Iterator rightIter = rightTup.iterator(); - while (rightIter.hasNext()) { - double rightVal = Double.parseDouble(rightIter.next()); - if (leftVal > rightVal) { - return true; - } - } + public static SessionId from(String sessionId) { + if (sessionId == null) { + return null; } - return false; + return new SessionId(sessionId); } - } diff --git a/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java b/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java new file mode 100644 index 0000000000..d50390c229 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java @@ -0,0 +1,51 @@ +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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 SpectatorId extends Id { + @JsonProperty("id") + private final String _id; + + /** + * Create a spectator id + * @param id string representing a spectator id + */ + @JsonCreator + public SpectatorId(@JsonProperty("id") String id) { + _id = id; + } + + @Override + public String stringify() { + return _id; + } + + /** + * Create a spectator id from a string + * @param spectatorId string representing a spectator id + * @return SpectatorId + */ + public static SpectatorId from(String spectatorId) { + return new SpectatorId(spectatorId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java b/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java new file mode 100644 index 0000000000..7c84f0f476 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java @@ -0,0 +1,66 @@ +package org.apache.helix.api.id; + +import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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 StateModelDefId extends Id { + public static final StateModelDefId SchedulerTaskQueue = StateModelDefId + .from(DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE); + @JsonProperty("id") + private final String _id; + + /** + * Create a state model definition id + * @param id string representing a state model definition id + */ + @JsonCreator + public StateModelDefId(@JsonProperty("id") String id) { + _id = id; + } + + @Override + public String stringify() { + return _id; + } + + /** + * Check if the underlying state model definition id is equal if case is ignored + * @param that the StateModelDefId to compare + * @return true if equal ignoring case, false otherwise + */ + public boolean equalsIgnoreCase(StateModelDefId that) { + return _id.equalsIgnoreCase(that._id); + } + + /** + * Get a concrete state model definition id + * @param stateModelDefId string state model identifier + * @return StateModelDefId + */ + public static StateModelDefId from(String stateModelDefId) { + if (stateModelDefId == null) { + return null; + } + return new StateModelDefId(stateModelDefId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java b/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java new file mode 100644 index 0000000000..795c14c363 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java @@ -0,0 +1,57 @@ +package org.apache.helix.api.id; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/* + * 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. + */ + +/** + * Id representing a state model factory + */ +public class StateModelFactoryId extends Id { + @JsonProperty("id") + private final String _id; + + /** + * Create a state model factory id + * @param id string representing a state model factory + */ + @JsonCreator + public StateModelFactoryId(@JsonProperty("id") String id) { + _id = id; + } + + @Override + public String stringify() { + return _id; + } + + /** + * Get a concrete state model factory id + * @param stateModelFactoryId the string version of the id + * @return StateModelFactoryId + */ + public static StateModelFactoryId from(String stateModelFactoryId) { + if (stateModelFactoryId == null) { + return null; + } + return new StateModelFactoryId(stateModelFactoryId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/ExternalViewGenerator.java b/helix-core/src/main/java/org/apache/helix/controller/ExternalViewGenerator.java index 3d811bc3f3..c3287ce2dd 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/ExternalViewGenerator.java +++ b/helix-core/src/main/java/org/apache/helix/controller/ExternalViewGenerator.java @@ -28,8 +28,8 @@ import java.util.TreeSet; import org.apache.helix.ZNRecord; -import org.apache.helix.model.Message; import org.apache.helix.model.CurrentState.CurrentStateProperty; +import org.apache.helix.model.Message; import org.apache.log4j.Logger; /* diff --git a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java index 8e4e1ea1b3..b9527e6e5b 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java +++ b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java @@ -32,35 +32,34 @@ import org.apache.helix.ControllerChangeListener; import org.apache.helix.CurrentStateChangeListener; import org.apache.helix.ExternalViewChangeListener; -import org.apache.helix.HealthStateChangeListener; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; import org.apache.helix.IdealStateChangeListener; import org.apache.helix.LiveInstanceChangeListener; import org.apache.helix.MessageListener; import org.apache.helix.NotificationContext; -import org.apache.helix.ZNRecord; import org.apache.helix.NotificationContext.Type; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; import org.apache.helix.controller.pipeline.Pipeline; import org.apache.helix.controller.pipeline.PipelineRegistry; -import org.apache.helix.controller.stages.BestPossibleStateCalcStage; import org.apache.helix.controller.stages.ClusterEvent; +import org.apache.helix.controller.stages.BestPossibleStateCalcStage; import org.apache.helix.controller.stages.CompatibilityCheckStage; import org.apache.helix.controller.stages.CurrentStateComputationStage; import org.apache.helix.controller.stages.ExternalViewComputeStage; -import org.apache.helix.controller.stages.MessageGenerationPhase; +import org.apache.helix.controller.stages.MessageGenerationStage; import org.apache.helix.controller.stages.MessageSelectionStage; import org.apache.helix.controller.stages.MessageThrottleStage; import org.apache.helix.controller.stages.ReadClusterDataStage; -import org.apache.helix.controller.stages.RebalanceIdealStateStage; import org.apache.helix.controller.stages.ResourceComputationStage; import org.apache.helix.controller.stages.TaskAssignmentStage; +import org.apache.helix.controller.stages.PersistAssignmentStage; import org.apache.helix.model.CurrentState; import org.apache.helix.model.ExternalView; -import org.apache.helix.model.HealthStat; import org.apache.helix.model.IdealState; import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.Leader; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; import org.apache.helix.model.PauseSignal; @@ -83,7 +82,7 @@ */ public class GenericHelixController implements ConfigChangeListener, IdealStateChangeListener, LiveInstanceChangeListener, MessageListener, CurrentStateChangeListener, - ExternalViewChangeListener, ControllerChangeListener, HealthStateChangeListener { + ExternalViewChangeListener, ControllerChangeListener { private static final Logger logger = Logger.getLogger(GenericHelixController.class.getName()); volatile boolean init = false; private final PipelineRegistry _registry; @@ -179,11 +178,12 @@ private static PipelineRegistry createDefaultRegistry() { // rebalance pipeline Pipeline rebalancePipeline = new Pipeline(); + rebalancePipeline.addStage(new CompatibilityCheckStage()); rebalancePipeline.addStage(new ResourceComputationStage()); rebalancePipeline.addStage(new CurrentStateComputationStage()); - rebalancePipeline.addStage(new RebalanceIdealStateStage()); rebalancePipeline.addStage(new BestPossibleStateCalcStage()); - rebalancePipeline.addStage(new MessageGenerationPhase()); + rebalancePipeline.addStage(new PersistAssignmentStage()); + rebalancePipeline.addStage(new MessageGenerationStage()); rebalancePipeline.addStage(new MessageSelectionStage()); rebalancePipeline.addStage(new MessageThrottleStage()); rebalancePipeline.addStage(new TaskAssignmentStage()); @@ -192,15 +192,10 @@ private static PipelineRegistry createDefaultRegistry() { Pipeline externalViewPipeline = new Pipeline(); externalViewPipeline.addStage(new ExternalViewComputeStage()); - // backward compatibility check - Pipeline liveInstancePipeline = new Pipeline(); - liveInstancePipeline.addStage(new CompatibilityCheckStage()); - registry.register("idealStateChange", dataRefresh, rebalancePipeline); registry.register("currentStateChange", dataRefresh, rebalancePipeline, externalViewPipeline); registry.register("configChange", dataRefresh, rebalancePipeline); - registry.register("liveInstanceChange", dataRefresh, liveInstancePipeline, rebalancePipeline, - externalViewPipeline); + registry.register("liveInstanceChange", dataRefresh, rebalancePipeline, externalViewPipeline); registry.register("messageChange", dataRefresh, rebalancePipeline); registry.register("externalView", dataRefresh); @@ -208,13 +203,6 @@ private static PipelineRegistry createDefaultRegistry() { registry .register("periodicalRebalance", dataRefresh, rebalancePipeline, externalViewPipeline); - // health stats pipeline - // Pipeline healthStatsAggregationPipeline = new Pipeline(); - // StatsAggregationStage statsStage = new StatsAggregationStage(); - // healthStatsAggregationPipeline.addStage(new ReadHealthDataStage()); - // healthStatsAggregationPipeline.addStage(statsStage); - // registry.register("healthChange", healthStatsAggregationPipeline); - return registry; } } @@ -320,16 +308,6 @@ public void onStateChange(String instanceName, List statesInfo, logger.info("END: GenericClusterController.onStateChange()"); } - @Override - public void onHealthChange(String instanceName, List reports, - NotificationContext changeContext) { - /** - * When there are more participant ( > 20, can be in hundreds), This callback can be - * called quite frequently as each participant reports health stat every minute. Thus - * we change the health check pipeline to run in a timer callback. - */ - } - @Override public void onMessage(String instanceName, List messages, NotificationContext changeContext) { @@ -431,7 +409,7 @@ public void onControllerChange(NotificationContext changeContext) { // double check if this controller is the leader Builder keyBuilder = accessor.keyBuilder(); - LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); if (leader == null) { logger .warn("No controller exists for cluster:" + changeContext.getManager().getClusterName()); @@ -480,7 +458,7 @@ protected void checkLiveInstancesObservation(List liveInstances, Map curSessions = new HashMap(); for (LiveInstance liveInstance : liveInstances) { curInstances.put(liveInstance.getInstanceName(), liveInstance); - curSessions.put(liveInstance.getSessionId(), liveInstance); + curSessions.put(liveInstance.getTypedSessionId().stringify(), liveInstance); } Map lastInstances = _lastSeenInstances.get(); diff --git a/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java b/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java index 4aae39bf48..62f3b23109 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java +++ b/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java @@ -48,6 +48,7 @@ import org.apache.helix.HelixManagerFactory; import org.apache.helix.InstanceType; import org.apache.helix.controller.restlet.ZKPropertyTransferServer; +import org.apache.helix.manager.zk.HelixManagerShutdownHook; import org.apache.helix.participant.DistClusterControllerStateModelFactory; import org.apache.helix.participant.StateMachineEngine; import org.apache.log4j.Logger; @@ -233,6 +234,9 @@ public static void main(String[] args) throws Exception { HelixManager manager = startHelixController(zkConnectString, clusterName, controllerName, controllerMode); + + Runtime.getRuntime().addShutdownHook(new HelixManagerShutdownHook(manager)); + try { Thread.currentThread().join(); } catch (InterruptedException e) { diff --git a/helix-core/src/main/java/org/apache/helix/controller/alert/AlertAction.java b/helix-core/src/main/java/org/apache/helix/controller/alert/AlertAction.java new file mode 100644 index 0000000000..90efe2b196 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/alert/AlertAction.java @@ -0,0 +1,142 @@ +package org.apache.helix.controller.alert; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import java.util.Arrays; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.Options; +import org.apache.helix.tools.ClusterSetup; + +public class AlertAction { + private final String[] _cliArgs; + private final CommandLine _cli; + + public static class Builder { + private String _cmd = null; + private String[] _args = null; + + public Builder cmd(String cmd) { + _cmd = cmd; + return this; + } + + public Builder args(String... args) { + _args = args; + return this; + } + + public AlertAction build() { + return new AlertAction(_cmd, _args); + } + } + + public AlertAction(String cmd, String[] args) { + if (cmd == null) { + throw new NullPointerException("command is null"); + } + if (args == null) { + args = new String[0]; + } + + // if arg is in form of "{..}", make sure it's a valid + // alertScope + for (String arg : args) { + if (arg.startsWith("{") && arg.endsWith("}")) { + try { + String filedStr = arg.substring(1, arg.length() - 1); + AlertScope.AlertScopeField.valueOf(filedStr); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Invalid alertScope value: " + arg + " in " + + Arrays.asList(args)); + } + } + } + + _cliArgs = new String[args.length + 1]; + _cliArgs[0] = cmd; + System.arraycopy(args, 0, _cliArgs, 1, args.length); + _cli = parseCliArgs(); + + } + + /** + * represent alertAction in form of: (command)(arg1 arg2 ...) + */ + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("(").append(_cliArgs[0]).append(")"); + if (_cliArgs != null) { + sb.append("("); + for (int i = 1; i < _cliArgs.length; i++) { + String arg = _cliArgs[i]; + if (i > 1) { + sb.append(" "); + } + sb.append(arg); + } + sb.append(")"); + } + return sb.toString(); + } + + public static AlertAction from(String actionStr) { + if (actionStr == null) { + throw new NullPointerException("actionStr is null"); + } + + // split (command)(arg1 arg2 ..) to [ , command, , arg1 arg2 ...] + String[] parts = actionStr.split("[()]"); + + if (parts.length == 2 && parts[0].isEmpty()) { + return new AlertAction(parts[1], null); + } else if (parts.length == 4 && parts[0].isEmpty() && parts[2].isEmpty()) { + String[] args = parts[3].split("\\s+"); + return new AlertAction(parts[1], args); + } + throw new IllegalArgumentException("Invalid alerAction string, was " + actionStr); + } + + public String[] getCliArgs() { + return _cliArgs; + } + + public CommandLine getCli() { + return _cli; + } + + private CommandLine parseCliArgs() { + Options options = new Options(); + options.addOptionGroup(ClusterSetup.constructOptionGroup()); + + try { + String[] tmpCliArgs = Arrays.copyOf(_cliArgs, _cliArgs.length); + + // prepend "-" to helix-command (_cliArgs[0]), so it can be parsed + tmpCliArgs[0] = String.format("-%s", tmpCliArgs[0]); + return new GnuParser().parse(options, tmpCliArgs); + } catch (Exception e) { + throw new IllegalArgumentException( + "Invalid HelixAdmin command: " + Arrays.toString(_cliArgs), e); + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/controller/alert/AlertComparator.java similarity index 54% rename from helix-core/src/main/java/org/apache/helix/healthcheck/HealthReportProvider.java rename to helix-core/src/main/java/org/apache/helix/controller/alert/AlertComparator.java index f7afd04f12..6bb1426d0d 100644 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthReportProvider.java +++ b/helix-core/src/main/java/org/apache/helix/controller/alert/AlertComparator.java @@ -1,4 +1,4 @@ -package org.apache.helix.healthcheck; +package org.apache.helix.controller.alert; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,21 +19,36 @@ * under the License. */ +import java.util.Collections; +import java.util.HashMap; import java.util.Map; -public abstract class HealthReportProvider { - public static final String _defaultPerfCounters = "defaultPerfCounters"; +public enum AlertComparator { + LARGER(">"), + SMALLER("<"), + NOT_EQUAL("!="); - public abstract Map getRecentHealthReport(); + private static final Map _indexMap; + private final String _op; - public Map> getRecentPartitionHealthReport() { - return null; + static { + Map aMap = new HashMap(); + for (AlertComparator cmp : AlertComparator.values()) { + aMap.put(cmp._op, cmp); + } + _indexMap = Collections.unmodifiableMap(aMap); } - public abstract void resetStats(); + private AlertComparator(String op) { + _op = op; + } - public String getReportName() { - return _defaultPerfCounters; + @Override + public String toString() { + return _op; } + public static AlertComparator from(String op) { + return _indexMap.get(op); + } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/alert/AlertName.java b/helix-core/src/main/java/org/apache/helix/controller/alert/AlertName.java new file mode 100644 index 0000000000..0fdd3a109a --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/alert/AlertName.java @@ -0,0 +1,151 @@ +package org.apache.helix.controller.alert; + +/* + * 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.api.id.ClusterId; +import org.apache.helix.api.id.Id; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; + +public class AlertName { + final AlertScope _scope; + final String _metric; + + final AlertComparator _comparator; + final String _value; + + public static class Builder { + private ClusterId _clusterId = null; + private Id _tenantId = null; + private Id _nodeId = null; + private ResourceId _resourceId = null; + private PartitionId _partitionId = null; + private String _metric = null; + private AlertComparator _comparator = null; + private String _value = null; + + public Builder cluster(ClusterId clusterId) { + _clusterId = clusterId; + return this; + } + + public Builder tenant(Id tenantId) { + _tenantId = tenantId; + return this; + } + + public Builder node(Id nodeId) { + _nodeId = nodeId; + return this; + } + + public Builder resource(ResourceId resourceId) { + _resourceId = resourceId; + return this; + } + + public Builder partitionId(PartitionId partitionId) { + _partitionId = partitionId; + return this; + } + + public Builder metric(String metric) { + _metric = metric; + return this; + } + + public Builder largerThan(String value) { + _comparator = AlertComparator.LARGER; + _value = value; + return this; + } + + public Builder smallerThan(String value) { + _comparator = AlertComparator.SMALLER; + _value = value; + return this; + } + + public AlertName build() { + return new AlertName( + new AlertScope(_clusterId, _tenantId, _nodeId, _resourceId, _partitionId), _metric, + _comparator, _value); + } + } + + public AlertName(AlertScope scope, String metric, AlertComparator comparator, String value) { + if (scope == null || metric == null || comparator == null || value == null) { + throw new NullPointerException("null arguments, was scope: " + scope + ", metric: " + metric + + ", comparator: " + comparator + ", value: " + value); + } + _scope = scope; + _metric = metric; + _comparator = comparator; + _value = value; + } + + /** + * represent alertName in form of: + * (scope)(metric)comparator(value) + */ + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("(").append(_scope.toString()).append(")"); + sb.append("(").append(_metric).append(")"); + sb.append(_comparator.toString()); + sb.append("(").append(_value).append(")"); + return sb.toString(); + } + + public static AlertName from(String alertNameStr) { + if (alertNameStr == null) { + throw new NullPointerException("alertNameStr is null"); + } + + // split (alertName)(metric)cmp(value) to [ , alertName, , metric, cmp, value] + String[] parts = alertNameStr.split("[()]"); + if (parts == null || parts.length != 6 || !parts[0].isEmpty() || !parts[2].isEmpty()) { + throw new IllegalArgumentException( + "AlertName is NOT in form of (scope)(metric)comparator(value), was " + alertNameStr); + } + + String[] scopeParts = parts[1].split("\\."); + + String metric = parts[3]; + AlertComparator cmp = AlertComparator.from(parts[4]); + if (cmp == null) { + throw new IllegalArgumentException("Invalid alert comparator, was " + parts[4]); + } + String value = parts[5]; + + AlertScope scope = new AlertScope(scopeParts); + return new AlertName(scope, metric, cmp, value); + } + + public boolean match(AlertName alertName) { + return _scope.match(alertName._scope) && _metric.equals(alertName._metric) + && _comparator == alertName._comparator && _value.equals(alertName._value); + } + + public AlertScope getScope() { + return _scope; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/alert/AlertScope.java b/helix-core/src/main/java/org/apache/helix/controller/alert/AlertScope.java new file mode 100644 index 0000000000..bc32ce6f51 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/alert/AlertScope.java @@ -0,0 +1,106 @@ +package org.apache.helix.controller.alert; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import java.util.Arrays; + +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.Id; + +public class AlertScope { + public static final String WILDCARD = "%"; + + /** + * DON'T change the sequence + */ + public enum AlertScopeField { + cluster, + tenant, + node, + resource, + partition; + } + + private final String[] _scopeFields; + + public AlertScope(Id clusterId, Id tenantId, Id nodeId, Id resourceId, Id partitionId) { + this(clusterId == null ? null : clusterId.stringify(), tenantId == null ? null : tenantId + .stringify(), nodeId == null ? null : nodeId.stringify(), resourceId == null ? null + : resourceId.stringify(), partitionId == null ? null : partitionId.stringify()); + } + + public AlertScope(String... fields) { + int maxArgNum = AlertScopeField.values().length; + if (fields != null && fields.length > maxArgNum) { + throw new IllegalArgumentException("Too many arguments. Should be no more than " + maxArgNum + + " but was " + fields.length + ", fields: " + Arrays.asList(fields)); + } + // all array elements are init'ed to null + _scopeFields = new String[AlertScopeField.values().length]; + + if (fields != null) { + for (int i = 0; i < fields.length; i++) { + if (!WILDCARD.equals(fields[i])) { + _scopeFields[i] = fields[i]; + } + } + } + } + + /** + * represent AlertScope in form of: + * {cluster}.{tenant}.{node}.{resource}.{partition} using % for null field + */ + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + for (String scopeField : _scopeFields) { + if (sb.length() > 0) { + sb.append("."); + } + sb.append(scopeField == null ? WILDCARD : scopeField); + } + + return sb.toString(); + } + + public String get(AlertScopeField field) { + return _scopeFields[field.ordinal()]; + } + + public ClusterId getClusterId() { + return ClusterId.from(get(AlertScopeField.cluster)); + } + + /** + * match two alert-scopes, null means "don't care" + * @param scope + * @return + */ + public boolean match(AlertScope scope) { + for (int i = 0; i < _scopeFields.length; i++) { + if (_scopeFields[i] != null && scope._scopeFields[i] != null + && !_scopeFields[i].equals(scope._scopeFields[i])) { + return false; + } + } + return true; + } +} \ No newline at end of file diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java deleted file mode 100644 index 4dd5ea69f5..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java +++ /dev/null @@ -1,396 +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.Collections; -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.TreeMap; - -import org.apache.helix.HelixManager; -import org.apache.helix.ZNRecord; -import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; -import org.apache.helix.controller.stages.ClusterDataCache; -import org.apache.helix.controller.stages.CurrentStateOutput; -import org.apache.helix.controller.stages.ResourceMapping; -import org.apache.helix.controller.strategy.AutoRebalanceStrategy; -import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme; -import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme; -import org.apache.helix.model.CurrentState; -import org.apache.helix.model.IdealState; -import org.apache.helix.model.IdealState.RebalanceMode; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; -import org.apache.helix.model.StateModelDefinition; -import org.apache.log4j.Logger; - -/** - * This is a Rebalancer specific to full automatic mode. It is tasked with computing the ideal - * state of a resource, fully adapting to the addition or removal of instances. This includes - * computation of a new preference list and a partition to instance and state mapping based on the - * computed instance preferences. - * The input is the current assignment of partitions to instances, as well as existing instance - * preferences, if any. - * The output is a preference list and a mapping based on that preference list, i.e. partition p - * has a replica on node k with state s. - */ -public class AutoRebalancer implements Rebalancer { - // These should be final, but are initialized in init rather than a constructor - private HelixManager _manager; - private AutoRebalanceStrategy _algorithm; - - private static final Logger LOG = Logger.getLogger(AutoRebalancer.class); - - @Override - public void init(HelixManager manager) { - this._manager = manager; - this._algorithm = null; - } - - @Override - public IdealState computeNewIdealState(String resourceName, IdealState currentIdealState, - CurrentStateOutput currentStateOutput, ClusterDataCache clusterData) { - List partitions = new ArrayList(currentIdealState.getPartitionSet()); - String stateModelName = currentIdealState.getStateModelDefRef(); - StateModelDefinition stateModelDef = clusterData.getStateModelDef(stateModelName); - Map liveInstance = clusterData.getLiveInstances(); - String replicas = currentIdealState.getReplicas(); - - LinkedHashMap stateCountMap = new LinkedHashMap(); - stateCountMap = stateCount(stateModelDef, liveInstance.size(), Integer.parseInt(replicas)); - List liveNodes = new ArrayList(liveInstance.keySet()); - Map> currentMapping = - currentMapping(currentStateOutput, resourceName, partitions, stateCountMap); - - List allNodes = new ArrayList(clusterData.getInstanceConfigMap().keySet()); - int maxPartition = currentIdealState.getMaxPartitionsPerInstance(); - - if (LOG.isInfoEnabled()) { - LOG.info("currentMapping: " + currentMapping); - LOG.info("stateCountMap: " + stateCountMap); - LOG.info("liveNodes: " + liveNodes); - LOG.info("allNodes: " + allNodes); - LOG.info("maxPartition: " + maxPartition); - } - ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme(); - placementScheme.init(_manager); - _algorithm = - new AutoRebalanceStrategy(resourceName, partitions, stateCountMap, maxPartition, - placementScheme); - ZNRecord newMapping = - _algorithm.computePartitionAssignment(liveNodes, currentMapping, allNodes); - - if (LOG.isInfoEnabled()) { - LOG.info("newMapping: " + newMapping); - } - - IdealState newIdealState = new IdealState(resourceName); - newIdealState.getRecord().setSimpleFields(currentIdealState.getRecord().getSimpleFields()); - newIdealState.setRebalanceMode(RebalanceMode.FULL_AUTO); - newIdealState.getRecord().setListFields(newMapping.getListFields()); - return newIdealState; - } - - /** - * @return state count map: state->count - */ - private LinkedHashMap stateCount(StateModelDefinition stateModelDef, - int liveNodesNb, int totalReplicas) { - LinkedHashMap stateCountMap = new LinkedHashMap(); - List statesPriorityList = stateModelDef.getStatesPriorityList(); - - int replicas = totalReplicas; - for (String state : statesPriorityList) { - String num = stateModelDef.getNumInstancesPerState(state); - if ("N".equals(num)) { - stateCountMap.put(state, liveNodesNb); - } else if ("R".equals(num)) { - // wait until we get the counts for all other states - continue; - } else { - int stateCount = -1; - try { - stateCount = Integer.parseInt(num); - } catch (Exception e) { - // LOG.error("Invalid count for state: " + state + ", count: " + num + - // ", use -1 instead"); - } - - if (stateCount > 0) { - stateCountMap.put(state, stateCount); - replicas -= stateCount; - } - } - } - - // get state count for R - for (String state : statesPriorityList) { - String num = stateModelDef.getNumInstancesPerState(state); - if ("R".equals(num)) { - stateCountMap.put(state, replicas); - // should have at most one state using R - break; - } - } - return stateCountMap; - } - - private Map> currentMapping(CurrentStateOutput currentStateOutput, - String resourceName, List partitions, Map stateCountMap) { - - Map> map = new HashMap>(); - - for (String partition : partitions) { - Map curStateMap = - currentStateOutput.getCurrentStateMap(resourceName, new Partition(partition)); - map.put(partition, new HashMap()); - for (String node : curStateMap.keySet()) { - String state = curStateMap.get(node); - if (stateCountMap.containsKey(state)) { - map.get(partition).put(node, state); - } - } - - Map pendingStateMap = - currentStateOutput.getPendingStateMap(resourceName, new Partition(partition)); - for (String node : pendingStateMap.keySet()) { - String state = pendingStateMap.get(node); - if (stateCountMap.containsKey(state)) { - map.get(partition).put(node, state); - } - } - } - return map; - } - - @Override - public ResourceMapping computeBestPossiblePartitionState(ClusterDataCache cache, - IdealState idealState, Resource resource, CurrentStateOutput currentStateOutput) { - if (LOG.isDebugEnabled()) { - LOG.debug("Processing resource:" + resource.getResourceName()); - } - String stateModelDefName = idealState.getStateModelDefRef(); - StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName); - calculateAutoBalancedIdealState(cache, idealState, stateModelDef); - ResourceMapping partitionMapping = new ResourceMapping(); - for (Partition partition : resource.getPartitions()) { - Map currentStateMap = - currentStateOutput.getCurrentStateMap(resource.getResourceName(), partition); - Set disabledInstancesForPartition = - cache.getDisabledInstancesForPartition(partition.toString()); - List preferenceList = - ConstraintBasedAssignment.getPreferenceList(cache, partition, idealState, stateModelDef); - Map bestStateForPartition = - ConstraintBasedAssignment.computeAutoBestStateForPartition(cache, stateModelDef, - preferenceList, currentStateMap, disabledInstancesForPartition); - partitionMapping.addReplicaMap(partition, bestStateForPartition); - } - return partitionMapping; - } - - /** - * Compute best state for resource in AUTO_REBALANCE ideal state mode. the algorithm - * will make sure that the master partition are evenly distributed; Also when instances - * are added / removed, the amount of diff in master partitions are minimized - * @param cache - * @param idealState - * @param instancePreferenceList - * @param stateModelDef - * @param currentStateOutput - * @return - */ - private void calculateAutoBalancedIdealState(ClusterDataCache cache, IdealState idealState, - StateModelDefinition stateModelDef) { - String topStateValue = stateModelDef.getStatesPriorityList().get(0); - Set liveInstances = cache.getLiveInstances().keySet(); - Set taggedInstances = new HashSet(); - - // If there are instances tagged with resource name, use only those instances - if (idealState.getInstanceGroupTag() != null) { - for (String instanceName : liveInstances) { - if (cache.getInstanceConfigMap().get(instanceName) - .containsTag(idealState.getInstanceGroupTag())) { - taggedInstances.add(instanceName); - } - } - } - if (taggedInstances.size() > 0) { - if (LOG.isInfoEnabled()) { - LOG.info("found the following instances with tag " + idealState.getResourceName() + " " - + taggedInstances); - } - liveInstances = taggedInstances; - } - // Obtain replica number - int replicas = 1; - try { - replicas = Integer.parseInt(idealState.getReplicas()); - } catch (Exception e) { - LOG.error("", e); - } - // Init for all partitions with empty list - Map> defaultListFields = new TreeMap>(); - List emptyList = new ArrayList(0); - for (String partition : idealState.getPartitionSet()) { - defaultListFields.put(partition, emptyList); - } - idealState.getRecord().setListFields(defaultListFields); - // Return if no live instance - if (liveInstances.size() == 0) { - if (LOG.isInfoEnabled()) { - LOG.info("No live instances, return. Idealstate : " + idealState.getResourceName()); - } - return; - } - Map> masterAssignmentMap = new HashMap>(); - for (String instanceName : liveInstances) { - masterAssignmentMap.put(instanceName, new ArrayList()); - } - Set orphanedPartitions = new HashSet(); - orphanedPartitions.addAll(idealState.getPartitionSet()); - // Go through all current states and fill the assignments - for (String liveInstanceName : liveInstances) { - CurrentState currentState = - cache.getCurrentState(liveInstanceName, - cache.getLiveInstances().get(liveInstanceName).getSessionId()) - .get(idealState.getId()); - if (currentState != null) { - Map partitionStates = currentState.getPartitionStateMap(); - for (String partitionName : partitionStates.keySet()) { - String state = partitionStates.get(partitionName); - if (state.equals(topStateValue)) { - masterAssignmentMap.get(liveInstanceName).add(partitionName); - orphanedPartitions.remove(partitionName); - } - } - } - } - List orphanedPartitionsList = new ArrayList(); - orphanedPartitionsList.addAll(orphanedPartitions); - int maxPartitionsPerInstance = idealState.getMaxPartitionsPerInstance(); - normalizeAssignmentMap(masterAssignmentMap, orphanedPartitionsList, maxPartitionsPerInstance); - idealState.getRecord().setListFields( - generateListFieldFromMasterAssignment(masterAssignmentMap, replicas)); - } - - /** - * Given the current master assignment map and the partitions not hosted, generate an - * evenly distributed partition assignment map - * @param masterAssignmentMap - * current master assignment map - * @param orphanPartitions - * partitions not hosted by any instance - * @return - */ - private void normalizeAssignmentMap(Map> masterAssignmentMap, - List orphanPartitions, int maxPartitionsPerInstance) { - int totalPartitions = 0; - String[] instanceNames = new String[masterAssignmentMap.size()]; - masterAssignmentMap.keySet().toArray(instanceNames); - Arrays.sort(instanceNames); - // Find out total partition number - for (String key : masterAssignmentMap.keySet()) { - totalPartitions += masterAssignmentMap.get(key).size(); - Collections.sort(masterAssignmentMap.get(key)); - } - totalPartitions += orphanPartitions.size(); - - // Find out how many partitions an instance should host - int partitionNumber = totalPartitions / masterAssignmentMap.size(); - int leave = totalPartitions % masterAssignmentMap.size(); - - for (int i = 0; i < instanceNames.length; i++) { - int targetPartitionNo = leave > 0 ? (partitionNumber + 1) : partitionNumber; - leave--; - // For hosts that has more partitions, move those partitions to "orphaned" - while (masterAssignmentMap.get(instanceNames[i]).size() > targetPartitionNo) { - int lastElementIndex = masterAssignmentMap.get(instanceNames[i]).size() - 1; - orphanPartitions.add(masterAssignmentMap.get(instanceNames[i]).get(lastElementIndex)); - masterAssignmentMap.get(instanceNames[i]).remove(lastElementIndex); - } - } - leave = totalPartitions % masterAssignmentMap.size(); - Collections.sort(orphanPartitions); - // Assign "orphaned" partitions to hosts that do not have enough partitions - for (int i = 0; i < instanceNames.length; i++) { - int targetPartitionNo = leave > 0 ? (partitionNumber + 1) : partitionNumber; - leave--; - if (targetPartitionNo > maxPartitionsPerInstance) { - targetPartitionNo = maxPartitionsPerInstance; - } - while (masterAssignmentMap.get(instanceNames[i]).size() < targetPartitionNo) { - int lastElementIndex = orphanPartitions.size() - 1; - masterAssignmentMap.get(instanceNames[i]).add(orphanPartitions.get(lastElementIndex)); - orphanPartitions.remove(lastElementIndex); - } - } - if (orphanPartitions.size() > 0) { - LOG.warn("orphanPartitions still contains elements"); - } - } - - /** - * Generate full preference list from the master assignment map evenly distribute the - * slave partitions mastered on a host to other hosts - * @param masterAssignmentMap - * current master assignment map - * @param orphanPartitions - * partitions not hosted by any instance - * @return - */ - private Map> generateListFieldFromMasterAssignment( - Map> masterAssignmentMap, int replicas) { - Map> listFields = new HashMap>(); - int slaves = replicas - 1; - String[] instanceNames = new String[masterAssignmentMap.size()]; - masterAssignmentMap.keySet().toArray(instanceNames); - Arrays.sort(instanceNames); - - for (int i = 0; i < instanceNames.length; i++) { - String instanceName = instanceNames[i]; - List otherInstances = new ArrayList(masterAssignmentMap.size() - 1); - for (int x = 0; x < instanceNames.length - 1; x++) { - int index = (x + i + 1) % instanceNames.length; - otherInstances.add(instanceNames[index]); - } - - List partitionList = masterAssignmentMap.get(instanceName); - for (int j = 0; j < partitionList.size(); j++) { - String partitionName = partitionList.get(j); - listFields.put(partitionName, new ArrayList()); - listFields.get(partitionName).add(instanceName); - - int slavesCanAssign = Math.min(slaves, otherInstances.size()); - for (int k = 0; k < slavesCanAssign; k++) { - int index = (j + k + 1) % otherInstances.size(); - listFields.get(partitionName).add(otherInstances.get(index)); - } - } - } - return listFields; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java index 17dc5c80cc..5209e2c2fe 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java @@ -1,5 +1,21 @@ package org.apache.helix.controller.rebalancer; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.HelixManager; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.controller.rebalancer.context.CustomRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; +import org.apache.helix.controller.stages.ResourceCurrentState; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -19,117 +35,37 @@ * under the License. */ -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -import org.apache.helix.HelixDefinedState; -import org.apache.helix.HelixManager; -import org.apache.helix.controller.stages.ClusterDataCache; -import org.apache.helix.controller.stages.CurrentStateOutput; -import org.apache.helix.controller.stages.ResourceMapping; -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.StateModelDefinition; -import org.apache.log4j.Logger; - -/** - * This is a Rebalancer specific to custom mode. It is tasked with checking an existing mapping of - * partitions against the set of live instances to mark assignment states as dropped or erroneous - * as necessary. - * The input is the required current assignment of partitions to instances, as well as the required - * existing instance preferences. - * The output is a verified mapping based on that preference list, i.e. partition p has a replica - * on node k with state s, where s may be a dropped or error state if necessary. - */ -public class CustomRebalancer implements Rebalancer { +public class CustomRebalancer implements HelixRebalancer { private static final Logger LOG = Logger.getLogger(CustomRebalancer.class); @Override - public void init(HelixManager manager) { - } - - @Override - public IdealState computeNewIdealState(String resourceName, IdealState currentIdealState, - CurrentStateOutput currentStateOutput, ClusterDataCache clusterData) { - return currentIdealState; + public void init(HelixManager helixManager) { + // do nothing } @Override - public ResourceMapping computeBestPossiblePartitionState(ClusterDataCache cache, - IdealState idealState, Resource resource, CurrentStateOutput currentStateOutput) { - String stateModelDefName = idealState.getStateModelDefRef(); - StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName); + public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, + Cluster cluster, ResourceCurrentState currentState) { + CustomRebalancerContext config = + rebalancerConfig.getRebalancerContext(CustomRebalancerContext.class); + StateModelDefinition stateModelDef = + cluster.getStateModelMap().get(config.getStateModelDefId()); if (LOG.isDebugEnabled()) { - LOG.debug("Processing resource:" + resource.getResourceName()); + LOG.debug("Processing resource:" + config.getResourceId()); } - ResourceMapping partitionMapping = new ResourceMapping(); - for (Partition partition : resource.getPartitions()) { - Map currentStateMap = - currentStateOutput.getCurrentStateMap(resource.getResourceName(), partition); - Set disabledInstancesForPartition = - cache.getDisabledInstancesForPartition(partition.toString()); - Map idealStateMap = - idealState.getInstanceStateMap(partition.getPartitionName()); - Map bestStateForPartition = - computeCustomizedBestStateForPartition(cache, stateModelDef, idealStateMap, + ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId()); + for (PartitionId partition : config.getPartitionSet()) { + Map currentStateMap = + currentState.getCurrentStateMap(config.getResourceId(), partition); + Set disabledInstancesForPartition = + ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(), partition); + Map bestStateForPartition = + ConstraintBasedAssignment.computeCustomizedBestStateForPartition(cluster + .getLiveParticipantMap().keySet(), stateModelDef, config.getPreferenceMap(partition), currentStateMap, disabledInstancesForPartition); partitionMapping.addReplicaMap(partition, bestStateForPartition); } return partitionMapping; } - - /** - * compute best state for resource in CUSTOMIZED ideal state mode - * @param cache - * @param stateModelDef - * @param idealStateMap - * @param currentStateMap - * @param disabledInstancesForPartition - * @return - */ - private Map computeCustomizedBestStateForPartition(ClusterDataCache cache, - StateModelDefinition stateModelDef, Map idealStateMap, - Map currentStateMap, Set disabledInstancesForPartition) { - Map instanceStateMap = new HashMap(); - - // if the ideal state is deleted, idealStateMap will be null/empty and - // we should drop all resources. - if (currentStateMap != null) { - for (String instance : currentStateMap.keySet()) { - if ((idealStateMap == null || !idealStateMap.containsKey(instance)) - && !disabledInstancesForPartition.contains(instance)) { - // if dropped and not disabled, transit to DROPPED - instanceStateMap.put(instance, HelixDefinedState.DROPPED.toString()); - } else if ((currentStateMap.get(instance) == null || !currentStateMap.get(instance).equals( - HelixDefinedState.ERROR.toString())) - && disabledInstancesForPartition.contains(instance)) { - // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE) - instanceStateMap.put(instance, stateModelDef.getInitialState()); - } - } - } - - // ideal state is deleted - if (idealStateMap == null) { - return instanceStateMap; - } - - Map liveInstancesMap = cache.getLiveInstances(); - for (String instance : idealStateMap.keySet()) { - boolean notInErrorState = - currentStateMap == null || currentStateMap.get(instance) == null - || !currentStateMap.get(instance).equals(HelixDefinedState.ERROR.toString()); - - if (liveInstancesMap.containsKey(instance) && notInErrorState - && !disabledInstancesForPartition.contains(instance)) { - instanceStateMap.put(instance, idealStateMap.get(instance)); - } - } - - return instanceStateMap; - } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/FallbackRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/FallbackRebalancer.java new file mode 100644 index 0000000000..fc4bfa0d5d --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/FallbackRebalancer.java @@ -0,0 +1,185 @@ +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.Map; +import java.util.Set; + +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixManager; +import org.apache.helix.PropertyKey; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; +import org.apache.helix.controller.stages.ClusterDataCache; +import org.apache.helix.controller.stages.CurrentStateOutput; +import org.apache.helix.controller.stages.ResourceCurrentState; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.Partition; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.util.HelixUtil; +import org.apache.log4j.Logger; + +/** + * This class is intented for use to wrap usages of {@link Rebalancer}. It is subject to removal + * once that class is removed. + */ +@SuppressWarnings("deprecation") +public class FallbackRebalancer implements HelixRebalancer { + private static final Logger LOG = Logger.getLogger(FallbackRebalancer.class); + private HelixManager _helixManager; + + @Override + public void init(HelixManager helixManager) { + _helixManager = helixManager; + } + + @Override + public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, + Cluster cluster, ResourceCurrentState currentState) { + // make sure the manager is not null + if (_helixManager == null) { + LOG.info("HelixManager is null!"); + return null; + } + + // get the context + PartitionedRebalancerContext context = + rebalancerConfig.getRebalancerContext(PartitionedRebalancerContext.class); + if (context == null) { + LOG.info("Resource is not partitioned"); + return null; + } + + // get the ideal state and rebalancer class + ResourceId resourceId = context.getResourceId(); + StateModelDefinition stateModelDef = + cluster.getStateModelMap().get(context.getStateModelDefId()); + if (stateModelDef == null) { + LOG.info("StateModelDefinition unavailable for " + resourceId); + return null; + } + HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + IdealState idealState = accessor.getProperty(keyBuilder.idealStates(resourceId.stringify())); + if (idealState == null) { + LOG.info("No IdealState available for " + resourceId); + return null; + } + String rebalancerClassName = idealState.getRebalancerClassName(); + if (rebalancerClassName == null) { + LOG.info("No Rebalancer class available for " + resourceId); + return null; + } + + // try to instantiate the rebalancer class + Rebalancer rebalancer = null; + try { + rebalancer = + (Rebalancer) (HelixUtil.loadClass(getClass(), rebalancerClassName).newInstance()); + } catch (Exception e) { + LOG.warn("rebalancer " + rebalancerClassName + " not available", e); + } + if (rebalancer == null) { + LOG.warn("Rebalancer class " + rebalancerClassName + " could not be instantiated for " + + resourceId); + return null; + } + + // get the cluster data cache (unfortunately involves a second read of the cluster) + ClusterDataCache cache = new ClusterDataCache(); + cache.refresh(accessor); + + // adapt ResourceCurrentState to CurrentStateOutput + CurrentStateOutput currentStateOutput = new CurrentStateOutput(); + for (ResourceId resource : currentState.getResourceIds()) { + currentStateOutput.setBucketSize(resource.stringify(), currentState.getBucketSize(resource)); + currentStateOutput.setResourceStateModelDef(resource.stringify(), currentState + .getResourceStateModelDef(resource).stringify()); + Set partitions = currentState.getCurrentStateMappedPartitions(resource); + for (PartitionId partitionId : partitions) { + // set current state + Map currentStateMap = + currentState.getCurrentStateMap(resource, partitionId); + for (ParticipantId participantId : currentStateMap.keySet()) { + currentStateOutput.setCurrentState(resource.stringify(), + new Partition(partitionId.stringify()), participantId.stringify(), currentStateMap + .get(participantId).toString()); + } + + // set pending current state + Map pendingStateMap = + currentState.getPendingStateMap(resource, partitionId); + for (ParticipantId participantId : pendingStateMap.keySet()) { + currentStateOutput.setPendingState(resource.stringify(), + new Partition(partitionId.stringify()), participantId.stringify(), pendingStateMap + .get(participantId).toString()); + } + } + } + + // call the rebalancer + rebalancer.init(_helixManager); + IdealState newIdealState = + rebalancer.computeResourceMapping(resourceId.stringify(), idealState, currentStateOutput, + cache); + + // do the resource assignments + ResourceAssignment assignment = new ResourceAssignment(resourceId); + if (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED) { + // customized ideal state uses a map + for (PartitionId partitionId : newIdealState.getPartitionIdSet()) { + Set disabledParticipants = + ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(), + partitionId); + Map replicaMap = + ConstraintBasedAssignment.computeCustomizedBestStateForPartition(cluster + .getLiveParticipantMap().keySet(), stateModelDef, newIdealState + .getParticipantStateMap(partitionId), currentState.getCurrentStateMap(resourceId, + partitionId), disabledParticipants); + assignment.addReplicaMap(partitionId, replicaMap); + } + } else { + // other modes use auto assignment + Map upperBounds = + ConstraintBasedAssignment + .stateConstraints(stateModelDef, resourceId, cluster.getConfig()); + for (PartitionId partitionId : newIdealState.getPartitionIdSet()) { + Set disabledParticipants = + ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(), + partitionId); + Map replicaMap = + ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds, cluster + .getLiveParticipantMap().keySet(), stateModelDef, newIdealState + .getPreferenceList(partitionId), currentState.getCurrentStateMap(resourceId, + partitionId), disabledParticipants); + assignment.addReplicaMap(partitionId, replicaMap); + } + } + return assignment; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/FullAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/FullAutoRebalancer.java new file mode 100644 index 0000000000..6d7b0ef798 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/FullAutoRebalancer.java @@ -0,0 +1,211 @@ +package org.apache.helix.controller.rebalancer; + +import java.util.ArrayList; +import java.util.Collections; +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 org.apache.helix.HelixManager; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.controller.rebalancer.context.FullAutoRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; +import org.apache.helix.controller.stages.ResourceCurrentState; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; + +/* + * 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 FullAutoRebalancer implements HelixRebalancer { + // These should be final, but are initialized in init rather than a constructor + private AutoRebalanceStrategy _algorithm; + + private static Logger LOG = Logger.getLogger(FullAutoRebalancer.class); + + @Override + public void init(HelixManager helixManager) { + // do nothing + } + + @Override + public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, + Cluster cluster, ResourceCurrentState currentState) { + FullAutoRebalancerContext config = + rebalancerConfig.getRebalancerContext(FullAutoRebalancerContext.class); + StateModelDefinition stateModelDef = + cluster.getStateModelMap().get(config.getStateModelDefId()); + // Compute a preference list based on the current ideal state + List partitions = new ArrayList(config.getPartitionSet()); + Map liveParticipants = cluster.getLiveParticipantMap(); + Map allParticipants = cluster.getParticipantMap(); + int replicas = -1; + if (config.anyLiveParticipant()) { + replicas = liveParticipants.size(); + } else { + replicas = config.getReplicaCount(); + } + + // count how many replicas should be in each state + Map upperBounds = + ConstraintBasedAssignment.stateConstraints(stateModelDef, config.getResourceId(), + cluster.getConfig()); + LinkedHashMap stateCountMap = + ConstraintBasedAssignment.stateCount(upperBounds, stateModelDef, liveParticipants.size(), + replicas); + + // get the participant lists + List liveParticipantList = + new ArrayList(liveParticipants.keySet()); + List allParticipantList = + new ArrayList(cluster.getParticipantMap().keySet()); + + // compute the current mapping from the current state + Map> currentMapping = + currentMapping(config, currentState, stateCountMap); + + // If there are nodes tagged with resource, use only those nodes + // If there are nodes tagged with resource name, use only those nodes + Set taggedNodes = new HashSet(); + Set taggedLiveNodes = new HashSet(); + if (config.getParticipantGroupTag() != null) { + for (ParticipantId participantId : allParticipantList) { + if (cluster.getParticipantMap().get(participantId).hasTag(config.getParticipantGroupTag())) { + taggedNodes.add(participantId); + if (liveParticipants.containsKey(participantId)) { + taggedLiveNodes.add(participantId); + } + } + } + if (!taggedLiveNodes.isEmpty()) { + // live nodes exist that have this tag + if (LOG.isInfoEnabled()) { + LOG.info("found the following participants with tag " + config.getParticipantGroupTag() + + " for " + config.getResourceId() + ": " + taggedLiveNodes); + } + } else if (taggedNodes.isEmpty()) { + // no live nodes and no configured nodes have this tag + LOG.warn("Resource " + config.getResourceId() + " has tag " + + config.getParticipantGroupTag() + " but no configured participants have this tag"); + } else { + // configured nodes have this tag, but no live nodes have this tag + LOG.warn("Resource " + config.getResourceId() + " has tag " + + config.getParticipantGroupTag() + " but no live participants have this tag"); + } + allParticipantList = new ArrayList(taggedNodes); + liveParticipantList = new ArrayList(taggedLiveNodes); + } + + // determine which nodes the replicas should live on + int maxPartition = config.getMaxPartitionsPerParticipant(); + if (LOG.isInfoEnabled()) { + LOG.info("currentMapping: " + currentMapping); + LOG.info("stateCountMap: " + stateCountMap); + LOG.info("liveNodes: " + liveParticipantList); + LOG.info("allNodes: " + allParticipantList); + LOG.info("maxPartition: " + maxPartition); + } + ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme(); + _algorithm = + new AutoRebalanceStrategy(config.getResourceId(), partitions, stateCountMap, maxPartition, + placementScheme); + ZNRecord newMapping = + _algorithm.typedComputePartitionAssignment(liveParticipantList, currentMapping, + allParticipantList); + + if (LOG.isInfoEnabled()) { + LOG.info("newMapping: " + newMapping); + } + + // compute a full partition mapping for the resource + if (LOG.isDebugEnabled()) { + LOG.debug("Processing resource:" + config.getResourceId()); + } + ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId()); + for (PartitionId partition : partitions) { + Set disabledParticipantsForPartition = + ConstraintBasedAssignment.getDisabledParticipants(allParticipants, partition); + List rawPreferenceList = newMapping.getListField(partition.stringify()); + if (rawPreferenceList == null) { + rawPreferenceList = Collections.emptyList(); + } + List preferenceList = + Lists.transform(rawPreferenceList, new Function() { + @Override + public ParticipantId apply(String participantName) { + return ParticipantId.from(participantName); + } + }); + preferenceList = + ConstraintBasedAssignment.getPreferenceList(cluster, partition, preferenceList); + Map bestStateForPartition = + ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds, + liveParticipants.keySet(), stateModelDef, preferenceList, + currentState.getCurrentStateMap(config.getResourceId(), partition), + disabledParticipantsForPartition); + partitionMapping.addReplicaMap(partition, bestStateForPartition); + } + return partitionMapping; + } + + private Map> currentMapping( + FullAutoRebalancerContext config, ResourceCurrentState currentStateOutput, + Map stateCountMap) { + Map> map = + new HashMap>(); + + for (PartitionId partition : config.getPartitionSet()) { + Map curStateMap = + currentStateOutput.getCurrentStateMap(config.getResourceId(), partition); + map.put(partition, new HashMap()); + for (ParticipantId node : curStateMap.keySet()) { + State state = curStateMap.get(node); + if (stateCountMap.containsKey(state)) { + map.get(partition).put(node, state); + } + } + + Map pendingStateMap = + currentStateOutput.getPendingStateMap(config.getResourceId(), partition); + for (ParticipantId node : pendingStateMap.keySet()) { + State state = pendingStateMap.get(node); + if (stateCountMap.containsKey(state)) { + map.get(partition).put(node, state); + } + } + } + return map; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/HelixRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/HelixRebalancer.java new file mode 100644 index 0000000000..7fcbba56b1 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/HelixRebalancer.java @@ -0,0 +1,64 @@ +package org.apache.helix.controller.rebalancer; + +import org.apache.helix.HelixManager; +import org.apache.helix.api.Cluster; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.stages.ResourceCurrentState; +import org.apache.helix.model.ResourceAssignment; + +/* + * 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. + */ + +/** + * Allows one to come up with custom implementation of a rebalancer.
+ * This will be invoked on all changes that happen in the cluster.
+ * Simply return the resource assignment for a resource in this method.
+ */ +public interface HelixRebalancer { + /** + * Initialize the rebalancer with a HelixManager if necessary + * @param manager + */ + public void init(HelixManager helixManager); + + /** + * Given an ideal state for a resource and liveness of participants, compute a assignment of + * instances and states to each partition of a resource. This method provides all the relevant + * information needed to rebalance a resource. If you need additional information use + * manager.getAccessor to read and write the cluster data. This allows one to compute the + * ResourceAssignment according to app-specific requirements.
+ *
+ * Say that you have:
+ * + *
+   * class MyRebalancerContext implements RebalancerContext
+   * 
+ * + * as your rebalancer context. To extract it from a RebalancerConfig, do the following:
+ * + *
+   * MyRebalancerContext context = rebalancerConfig.getRebalancerContext(MyRebalancerContext.class);
+   * 
+ * @param rebalancerConfig the properties of the resource for which a mapping will be computed + * @param cluster complete snapshot of the cluster + * @param currentState the current states of all partitions + */ + public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, + Cluster cluster, ResourceCurrentState currentState); +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java index a0cfbb77b6..0e6cb104e2 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java @@ -22,43 +22,35 @@ import org.apache.helix.HelixManager; import org.apache.helix.controller.stages.ClusterDataCache; import org.apache.helix.controller.stages.CurrentStateOutput; -import org.apache.helix.controller.stages.ResourceMapping; import org.apache.helix.model.IdealState; -import org.apache.helix.model.Resource; /** * Allows one to come up with custom implementation of a rebalancer.
* This will be invoked on all changes that happen in the cluster.
* Simply return the newIdealState for a resource in this method.
+ *
+ * Deprecated. Use {@link HelixRebalancer} instead. */ +@Deprecated public interface Rebalancer { - void init(HelixManager manager); - /** - * This method provides all the relevant information needed to rebalance a resource. - * If you need additional information use manager.getAccessor to read the cluster data. - * This allows one to compute the newIdealState according to app specific requirement. - * @param resourceName Name of the resource to be rebalanced - * @param currentIdealState - * @param currentStateOutput - * Provides the current state and pending state transition for all - * partitions - * @param clusterData Provides additional methods to retrieve cluster data. - * @return + * Initialize the rebalancer with a HelixManager if necessary + * @param manager */ - IdealState computeNewIdealState(String resourceName, IdealState currentIdealState, - final CurrentStateOutput currentStateOutput, final ClusterDataCache clusterData); + public void init(HelixManager manager); /** - * Given an ideal state for a resource and the liveness of instances, compute the best possible - * state assignment for each partition's replicas. - * @param cache - * @param idealState - * @param resource - * @param currentStateOutput - * Provides the current state and pending state transitions for all partitions - * @return + * Given an ideal state for a resource and liveness of instances, compute a assignment of + * instances and states to each partition of a resource. This method provides all the relevant + * information needed to rebalance a resource. If you need additional information use + * manager.getAccessor to read the cluster data. This allows one to compute the newIdealState + * according to app specific requirements. + * @param resourceName the resource for which a mapping will be computed + * @param currentIdealState the IdealState that corresponds to this resource + * @param currentStateOutput the current states of all partitions + * @param clusterData cache of the cluster state */ - ResourceMapping computeBestPossiblePartitionState(ClusterDataCache cache, IdealState idealState, - Resource resource, CurrentStateOutput currentStateOutput); + public IdealState computeResourceMapping(final String resourceName, + final IdealState currentIdealState, final CurrentStateOutput currentStateOutput, + final ClusterDataCache clusterData); } diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/RebalancerRef.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/RebalancerRef.java new file mode 100644 index 0000000000..974222d1cc --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/RebalancerRef.java @@ -0,0 +1,94 @@ +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 org.apache.helix.util.HelixUtil; +import org.apache.log4j.Logger; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonIgnore; +import org.codehaus.jackson.annotate.JsonProperty; + +/** + * Reference to a class that extends {@link HelixRebalancer}. It loads the class automatically. + */ +public class RebalancerRef { + private static final Logger LOG = Logger.getLogger(RebalancerRef.class); + + @JsonProperty("rebalancerClassName") + private final String _rebalancerClassName; + + @JsonCreator + private RebalancerRef(@JsonProperty("rebalancerClassName") String rebalancerClassName) { + _rebalancerClassName = rebalancerClassName; + } + + /** + * Get an instantiated Rebalancer + * @return Rebalancer or null if instantiation failed + */ + @JsonIgnore + public HelixRebalancer getRebalancer() { + try { + return (HelixRebalancer) (HelixUtil.loadClass(getClass(), _rebalancerClassName).newInstance()); + } catch (Exception e) { + LOG.warn("Exception while invoking custom rebalancer class:" + _rebalancerClassName, e); + } + return null; + } + + @Override + public String toString() { + return _rebalancerClassName; + } + + @Override + public boolean equals(Object that) { + if (that instanceof RebalancerRef) { + return this.toString().equals(((RebalancerRef) that).toString()); + } else if (that instanceof String) { + return this.toString().equals(that); + } + return false; + } + + /** + * Get a rebalancer class reference + * @param rebalancerClassName name of the class + * @return RebalancerRef or null if name is null + */ + public static RebalancerRef from(String rebalancerClassName) { + if (rebalancerClassName == null) { + return null; + } + return new RebalancerRef(rebalancerClassName); + } + + /** + * Get a RebalancerRef from a class object + * @param rebalancerClass class that implements Rebalancer + * @return RebalancerRef + */ + public static RebalancerRef from(Class rebalancerClass) { + if (rebalancerClass == null) { + return null; + } + return RebalancerRef.from(rebalancerClass.getName()); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java index bc682ff8a6..a0ad6f3b60 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java @@ -1,5 +1,23 @@ package org.apache.helix.controller.rebalancer; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.HelixManager; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext; +import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; +import org.apache.helix.controller.stages.ResourceCurrentState; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -19,65 +37,50 @@ * under the License. */ -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.helix.HelixManager; -import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; -import org.apache.helix.controller.stages.ClusterDataCache; -import org.apache.helix.controller.stages.CurrentStateOutput; -import org.apache.helix.controller.stages.ResourceMapping; -import org.apache.helix.model.IdealState; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; -import org.apache.helix.model.StateModelDefinition; -import org.apache.log4j.Logger; - /** - * This is a Rebalancer specific to semi-automatic mode. It is tasked with computing the ideal - * state of a resource based on a predefined preference list of instances willing to accept - * replicas. - * The input is the optional current assignment of partitions to instances, as well as the required - * existing instance preferences. - * The output is a mapping based on that preference list, i.e. partition p has a replica on node k - * with state s. + * Rebalancer for the SEMI_AUTO mode. It expects a RebalancerConfig that understands the preferred + * locations of each partition replica */ -public class SemiAutoRebalancer implements Rebalancer { - +public class SemiAutoRebalancer implements HelixRebalancer { private static final Logger LOG = Logger.getLogger(SemiAutoRebalancer.class); @Override - public void init(HelixManager manager) { + public void init(HelixManager helixManager) { + // do nothing } @Override - public IdealState computeNewIdealState(String resourceName, IdealState currentIdealState, - CurrentStateOutput currentStateOutput, ClusterDataCache clusterData) { - return currentIdealState; - } - - @Override - public ResourceMapping computeBestPossiblePartitionState(ClusterDataCache cache, - IdealState idealState, Resource resource, CurrentStateOutput currentStateOutput) { - String stateModelDefName = idealState.getStateModelDefRef(); - StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName); + public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, + Cluster cluster, ResourceCurrentState currentState) { + SemiAutoRebalancerContext config = + rebalancerConfig.getRebalancerContext(SemiAutoRebalancerContext.class); + StateModelDefinition stateModelDef = + cluster.getStateModelMap().get(config.getStateModelDefId()); if (LOG.isDebugEnabled()) { - LOG.debug("Processing resource:" + resource.getResourceName()); + LOG.debug("Processing resource:" + config.getResourceId()); } - ResourceMapping partitionMapping = new ResourceMapping(); - for (Partition partition : resource.getPartitions()) { - Map currentStateMap = - currentStateOutput.getCurrentStateMap(resource.getResourceName(), partition); - Set disabledInstancesForPartition = - cache.getDisabledInstancesForPartition(partition.toString()); - List preferenceList = - ConstraintBasedAssignment.getPreferenceList(cache, partition, idealState, stateModelDef); - Map bestStateForPartition = - ConstraintBasedAssignment.computeAutoBestStateForPartition(cache, stateModelDef, - preferenceList, currentStateMap, disabledInstancesForPartition); + ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId()); + + for (PartitionId partition : config.getPartitionSet()) { + Map currentStateMap = + currentState.getCurrentStateMap(config.getResourceId(), partition); + Set disabledInstancesForPartition = + ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(), + partition); + List preferenceList = + ConstraintBasedAssignment.getPreferenceList(cluster, partition, + config.getPreferenceList(partition)); + Map upperBounds = + ConstraintBasedAssignment.stateConstraints(stateModelDef, config.getResourceId(), + cluster.getConfig()); + + Map bestStateForPartition = + ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds, cluster + .getLiveParticipantMap().keySet(), stateModelDef, preferenceList, currentStateMap, + disabledInstancesForPartition); partitionMapping.addReplicaMap(partition, bestStateForPartition); } return partitionMapping; } + } diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/BasicRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/BasicRebalancerContext.java new file mode 100644 index 0000000000..ec765d7267 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/BasicRebalancerContext.java @@ -0,0 +1,240 @@ +package org.apache.helix.controller.rebalancer.context; + +import java.util.Set; + +import org.apache.helix.api.Partition; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; +import org.apache.helix.controller.rebalancer.RebalancerRef; +import org.codehaus.jackson.annotate.JsonIgnore; + +/* + * 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. + */ + +/** + * Abstract RebalancerContext that functions for generic subunits. Use a subclass that more + * concretely defines the subunits. + */ +public abstract class BasicRebalancerContext implements RebalancerContext { + private ResourceId _resourceId; + private StateModelDefId _stateModelDefId; + private StateModelFactoryId _stateModelFactoryId; + private String _participantGroupTag; + private Class _serializer; + private RebalancerRef _rebalancerRef; + + /** + * Instantiate a basic rebalancer context + */ + public BasicRebalancerContext() { + _serializer = DefaultContextSerializer.class; + } + + @Override + public ResourceId getResourceId() { + return _resourceId; + } + + /** + * Set the resource to rebalance + * @param resourceId resource id + */ + public void setResourceId(ResourceId resourceId) { + _resourceId = resourceId; + } + + @Override + public StateModelDefId getStateModelDefId() { + return _stateModelDefId; + } + + /** + * Set the state model definition that the resource follows + * @param stateModelDefId state model definition id + */ + public void setStateModelDefId(StateModelDefId stateModelDefId) { + _stateModelDefId = stateModelDefId; + } + + @Override + public StateModelFactoryId getStateModelFactoryId() { + return _stateModelFactoryId; + } + + /** + * Set the state model factory that the resource uses + * @param stateModelFactoryId state model factory id + */ + public void setStateModelFactoryId(StateModelFactoryId stateModelFactoryId) { + _stateModelFactoryId = stateModelFactoryId; + } + + @Override + public String getParticipantGroupTag() { + return _participantGroupTag; + } + + /** + * Set a tag that participants must have in order to serve this resource + * @param participantGroupTag string group tag + */ + public void setParticipantGroupTag(String participantGroupTag) { + _participantGroupTag = participantGroupTag; + } + + /** + * Get the serializer. If none is provided, {@link DefaultContextSerializer} is used + */ + @Override + public Class getSerializerClass() { + return _serializer; + } + + /** + * Set the class that can serialize this context + * @param serializer serializer class that implements ContextSerializer + */ + public void setSerializerClass(Class serializer) { + _serializer = serializer; + } + + @Override + @JsonIgnore + public Set getSubUnitIdSet() { + return getSubUnitMap().keySet(); + } + + @Override + @JsonIgnore + public Partition getSubUnit(PartitionId subUnitId) { + return getSubUnitMap().get(subUnitId); + } + + @Override + public RebalancerRef getRebalancerRef() { + return _rebalancerRef; + } + + /** + * Set the reference to the class used to rebalance this resource + * @param rebalancerRef RebalancerRef instance + */ + public void setRebalancerRef(RebalancerRef rebalancerRef) { + _rebalancerRef = rebalancerRef; + } + + /** + * Abstract builder for the base rebalancer context + */ + public static abstract class AbstractBuilder> { + private final ResourceId _resourceId; + private StateModelDefId _stateModelDefId; + private StateModelFactoryId _stateModelFactoryId; + private String _participantGroupTag; + private Class _serializerClass; + private RebalancerRef _rebalancerRef; + + /** + * Instantiate with a resource id + * @param resourceId resource id + */ + public AbstractBuilder(ResourceId resourceId) { + _resourceId = resourceId; + _serializerClass = DefaultContextSerializer.class; + } + + /** + * Set the state model definition that the resource should follow + * @param stateModelDefId state model definition id + * @return Builder + */ + public T stateModelDefId(StateModelDefId stateModelDefId) { + _stateModelDefId = stateModelDefId; + return self(); + } + + /** + * Set the state model factory that the resource should use + * @param stateModelFactoryId state model factory id + * @return Builder + */ + public T stateModelFactoryId(StateModelFactoryId stateModelFactoryId) { + _stateModelFactoryId = stateModelFactoryId; + return self(); + } + + /** + * Set the tag that all participants require in order to serve this resource + * @param participantGroupTag the tag + * @return Builder + */ + public T participantGroupTag(String participantGroupTag) { + _participantGroupTag = participantGroupTag; + return self(); + } + + /** + * Set the serializer class for this rebalancer context + * @param serializerClass class that implements ContextSerializer + * @return Builder + */ + public T serializerClass(Class serializerClass) { + _serializerClass = serializerClass; + return self(); + } + + /** + * Specify a custom class to use for rebalancing + * @param rebalancerRef RebalancerRef instance + * @return Builder + */ + public T rebalancerRef(RebalancerRef rebalancerRef) { + _rebalancerRef = rebalancerRef; + return self(); + } + + /** + * Update an existing context with base fields + * @param context derived context + */ + protected final void update(BasicRebalancerContext context) { + context.setResourceId(_resourceId); + context.setStateModelDefId(_stateModelDefId); + context.setStateModelFactoryId(_stateModelFactoryId); + context.setParticipantGroupTag(_participantGroupTag); + context.setSerializerClass(_serializerClass); + context.setRebalancerRef(_rebalancerRef); + } + + /** + * Get a typed reference to "this" class. Final derived classes should simply return the this + * reference. + * @return this for the most specific type + */ + protected abstract T self(); + + /** + * Get the rebalancer context from the built fields + * @return RebalancerContext + */ + public abstract RebalancerContext build(); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Alert.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ContextSerializer.java similarity index 55% rename from helix-core/src/main/java/org/apache/helix/alerts/Alert.java rename to helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ContextSerializer.java index 17af0605ab..ef12a09814 100644 --- a/helix-core/src/main/java/org/apache/helix/alerts/Alert.java +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ContextSerializer.java @@ -1,4 +1,4 @@ -package org.apache.helix.alerts; +package org.apache.helix.controller.rebalancer.context; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,34 +19,19 @@ * under the License. */ -public class Alert { - - String _name; - - String _expression; - String _comparator; - Tuple _constant; - - public Alert(String name, String expression, String comparator, Tuple constant) { - _name = name; - _expression = expression; - _comparator = comparator; - _constant = constant; - } - - public String getName() { - return _name; - } - - public String getExpression() { - return _expression; - } - - public String getComparator() { - return _comparator; - } - - public Tuple getConstant() { - return _constant; - } +public interface ContextSerializer { + /** + * Convert a RebalancerContext object instance to a String + * @param data instance of the rebalancer context type + * @return String representing the object + */ + public String serialize(final T data); + + /** + * Convert raw bytes to a generic object instance + * @param clazz The class represented by the deserialized string + * @param string String representing the object + * @return instance of the generic type or null if the conversion failed + */ + public T deserialize(final Class clazz, final String string); } diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.java new file mode 100644 index 0000000000..0d2c1f24fd --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.java @@ -0,0 +1,164 @@ +package org.apache.helix.controller.rebalancer.context; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.CustomRebalancer; +import org.apache.helix.controller.rebalancer.RebalancerRef; +import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; +import org.codehaus.jackson.annotate.JsonIgnore; + +import com.google.common.collect.Maps; + +/* + * 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. + */ + +/** + * RebalancerContext for a resource that should be rebalanced in CUSTOMIZED mode. By default, it + * corresponds to {@link CustomRebalancer} + */ +public class CustomRebalancerContext extends PartitionedRebalancerContext { + private Map> _preferenceMaps; + + /** + * Instantiate a CustomRebalancerContext + */ + public CustomRebalancerContext() { + setRebalanceMode(RebalanceMode.CUSTOMIZED); + setRebalancerRef(RebalancerRef.from(CustomRebalancer.class)); + _preferenceMaps = Maps.newHashMap(); + } + + /** + * Get the preference maps of the partitions and replicas of the resource + * @return map of partition to participant and state + */ + public Map> getPreferenceMaps() { + return _preferenceMaps; + } + + /** + * Set the preference maps of the partitions and replicas of the resource + * @param preferenceMaps map of partition to participant and state + */ + public void setPreferenceMaps(Map> preferenceMaps) { + _preferenceMaps = preferenceMaps; + } + + /** + * Get the preference map of a partition + * @param partitionId the partition to look up + * @return map of participant to state + */ + @JsonIgnore + public Map getPreferenceMap(PartitionId partitionId) { + return _preferenceMaps.get(partitionId); + } + + /** + * Generate preference maps based on a default cluster setup + * @param stateModelDef the state model definition to follow + * @param participantSet the set of participant ids to configure for + */ + @Override + @JsonIgnore + public void generateDefaultConfiguration(StateModelDefinition stateModelDef, + Set participantSet) { + // compute default upper bounds + Map upperBounds = Maps.newHashMap(); + for (State state : stateModelDef.getTypedStatesPriorityList()) { + upperBounds.put(state, stateModelDef.getNumParticipantsPerState(state)); + } + + // determine the current mapping + Map> currentMapping = getPreferenceMaps(); + + // determine the preference maps + LinkedHashMap stateCounts = + ConstraintBasedAssignment.stateCount(upperBounds, stateModelDef, participantSet.size(), + getReplicaCount()); + ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme(); + List participantList = new ArrayList(participantSet); + List partitionList = new ArrayList(getPartitionSet()); + AutoRebalanceStrategy strategy = + new AutoRebalanceStrategy(ResourceId.from(""), partitionList, stateCounts, + getMaxPartitionsPerParticipant(), placementScheme); + Map> rawPreferenceMaps = + strategy.typedComputePartitionAssignment(participantList, currentMapping, participantList) + .getMapFields(); + Map> preferenceMaps = + Maps.newHashMap(ResourceAssignment.replicaMapsFromStringMaps(rawPreferenceMaps)); + setPreferenceMaps(preferenceMaps); + } + + /** + * Build a CustomRebalancerContext. By default, it corresponds to {@link CustomRebalancer} + */ + public static final class Builder extends PartitionedRebalancerContext.AbstractBuilder { + private final Map> _preferenceMaps; + + /** + * Instantiate for a resource + * @param resourceId resource id + */ + public Builder(ResourceId resourceId) { + super(resourceId); + super.rebalancerRef(RebalancerRef.from(CustomRebalancer.class)); + super.rebalanceMode(RebalanceMode.CUSTOMIZED); + _preferenceMaps = Maps.newHashMap(); + } + + /** + * Add a preference map for a partition + * @param partitionId partition to set + * @param preferenceList map of participant id to state indicating where replicas are served + * @return Builder + */ + public Builder preferenceMap(PartitionId partitionId, Map preferenceMap) { + _preferenceMaps.put(partitionId, preferenceMap); + return self(); + } + + @Override + protected Builder self() { + return this; + } + + @Override + public CustomRebalancerContext build() { + CustomRebalancerContext context = new CustomRebalancerContext(); + super.update(context); + context.setPreferenceMaps(_preferenceMaps); + return context; + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/DefaultContextSerializer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/DefaultContextSerializer.java new file mode 100644 index 0000000000..ecc93fbf37 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/DefaultContextSerializer.java @@ -0,0 +1,83 @@ +package org.apache.helix.controller.rebalancer.context; + +import java.io.ByteArrayInputStream; +import java.io.StringWriter; + +import org.apache.helix.HelixException; +import org.apache.log4j.Logger; +import org.codehaus.jackson.map.DeserializationConfig; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.SerializationConfig; + +/* + * 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. + */ + +/** + * Default serializer implementation for RebalancerContexts. Uses the Jackson JSON library to + * convert to and from strings + */ +public class DefaultContextSerializer implements ContextSerializer { + + private static Logger logger = Logger.getLogger(DefaultContextSerializer.class); + + @Override + public String serialize(final T data) { + if (data == null) { + return null; + } + + ObjectMapper mapper = new ObjectMapper(); + SerializationConfig serializationConfig = mapper.getSerializationConfig(); + serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true); + serializationConfig.set(SerializationConfig.Feature.AUTO_DETECT_FIELDS, true); + serializationConfig.set(SerializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true); + StringWriter sw = new StringWriter(); + try { + mapper.writeValue(sw, data); + } catch (Exception e) { + logger.error("Exception during payload data serialization.", e); + throw new HelixException(e); + } + return sw.toString(); + } + + @Override + public T deserialize(final Class clazz, final String string) { + if (string == null || string.length() == 0) { + return null; + } + + ObjectMapper mapper = new ObjectMapper(); + ByteArrayInputStream bais = new ByteArrayInputStream(string.getBytes()); + + DeserializationConfig deserializationConfig = mapper.getDeserializationConfig(); + deserializationConfig.set(DeserializationConfig.Feature.AUTO_DETECT_FIELDS, true); + deserializationConfig.set(DeserializationConfig.Feature.AUTO_DETECT_SETTERS, true); + deserializationConfig.set(DeserializationConfig.Feature.AUTO_DETECT_CREATORS, true); + deserializationConfig.set(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, true); + deserializationConfig.set(DeserializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true); + try { + T payload = mapper.readValue(bais, clazz); + return payload; + } catch (Exception e) { + logger.error("Exception during deserialization of payload bytes: " + string, e); + return null; + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.java new file mode 100644 index 0000000000..2400707f49 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.java @@ -0,0 +1,64 @@ +package org.apache.helix.controller.rebalancer.context; + +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.FullAutoRebalancer; +import org.apache.helix.controller.rebalancer.RebalancerRef; +import org.apache.helix.model.IdealState.RebalanceMode; + +/* + * 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. + */ + +/** + * RebalancerContext for FULL_AUTO rebalancing mode. By default, it corresponds to + * {@link FullAutoRebalancer} + */ +public class FullAutoRebalancerContext extends PartitionedRebalancerContext { + public FullAutoRebalancerContext() { + setRebalanceMode(RebalanceMode.FULL_AUTO); + setRebalancerRef(RebalancerRef.from(FullAutoRebalancer.class)); + } + + /** + * Builder for a full auto rebalancer context. By default, it corresponds to + * {@link FullAutoRebalancer} + */ + public static final class Builder extends PartitionedRebalancerContext.AbstractBuilder { + /** + * Instantiate with a resource + * @param resourceId resource id + */ + public Builder(ResourceId resourceId) { + super(resourceId); + super.rebalancerRef(RebalancerRef.from(FullAutoRebalancer.class)); + super.rebalanceMode(RebalanceMode.FULL_AUTO); + } + + @Override + protected Builder self() { + return this; + } + + @Override + public FullAutoRebalancerContext build() { + FullAutoRebalancerContext context = new FullAutoRebalancerContext(); + super.update(context); + return context; + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.java new file mode 100644 index 0000000000..15fcf9ced5 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.java @@ -0,0 +1,393 @@ +package org.apache.helix.controller.rebalancer.context; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.HelixConstants.StateModelToken; +import org.apache.helix.api.Partition; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.RebalancerRef; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.StateModelDefinition; +import org.codehaus.jackson.annotate.JsonIgnore; +import org.codehaus.jackson.annotate.JsonIgnoreProperties; + +import com.google.common.collect.Maps; + +/* + * 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. + */ + +/** + * RebalancerContext for a resource whose subunits are partitions. In addition, these partitions can + * be replicated. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class PartitionedRebalancerContext extends BasicRebalancerContext implements + ReplicatedRebalancerContext { + private Map _partitionMap; + private boolean _anyLiveParticipant; + private int _replicaCount; + private int _maxPartitionsPerParticipant; + private RebalanceMode _rebalanceMode; + + /** + * Instantiate a DataRebalancerContext + */ + public PartitionedRebalancerContext() { + _partitionMap = Collections.emptyMap(); + _replicaCount = 1; + _anyLiveParticipant = false; + _maxPartitionsPerParticipant = Integer.MAX_VALUE; + _rebalanceMode = RebalanceMode.USER_DEFINED; + } + + /** + * Get a map from partition id to partition + * @return partition map (mutable) + */ + public Map getPartitionMap() { + return _partitionMap; + } + + /** + * Set a map of partition id to partition + * @param partitionMap partition map + */ + public void setPartitionMap(Map partitionMap) { + _partitionMap = Maps.newHashMap(partitionMap); + } + + /** + * Get the set of partitions for this resource + * @return set of partition ids + */ + @JsonIgnore + public Set getPartitionSet() { + return _partitionMap.keySet(); + } + + /** + * Get a partition + * @param partitionId id of the partition to get + * @return Partition object, or null if not present + */ + @JsonIgnore + public Partition getPartition(PartitionId partitionId) { + return _partitionMap.get(partitionId); + } + + @Override + public boolean anyLiveParticipant() { + return _anyLiveParticipant; + } + + /** + * Indicate if this resource should be assigned to any live participant + * @param anyLiveParticipant true if any live participant expected, false otherwise + */ + public void setAnyLiveParticipant(boolean anyLiveParticipant) { + _anyLiveParticipant = anyLiveParticipant; + } + + @Override + public int getReplicaCount() { + return _replicaCount; + } + + /** + * Set the number of replicas that each partition should have + * @param replicaCount + */ + public void setReplicaCount(int replicaCount) { + _replicaCount = replicaCount; + } + + /** + * Get the maximum number of partitions that a participant can serve + * @return maximum number of partitions per participant + */ + public int getMaxPartitionsPerParticipant() { + return _maxPartitionsPerParticipant; + } + + /** + * Set the maximum number of partitions that a participant can serve + * @param maxPartitionsPerParticipant maximum number of partitions per participant + */ + public void setMaxPartitionsPerParticipant(int maxPartitionsPerParticipant) { + _maxPartitionsPerParticipant = maxPartitionsPerParticipant; + } + + /** + * Set the rebalancer mode of the partitioned resource + * @param rebalanceMode {@link RebalanceMode} enum value + */ + public void setRebalanceMode(RebalanceMode rebalanceMode) { + _rebalanceMode = rebalanceMode; + } + + /** + * Get the rebalancer mode of the resource + * @return RebalanceMode + */ + public RebalanceMode getRebalanceMode() { + return _rebalanceMode; + } + + @Override + @JsonIgnore + public Map getSubUnitMap() { + return getPartitionMap(); + } + + /** + * Generate a default configuration given the state model and a participant. + * @param stateModelDef the state model definition to follow + * @param participantSet the set of participant ids to configure for + */ + @JsonIgnore + public void generateDefaultConfiguration(StateModelDefinition stateModelDef, + Set participantSet) { + // the base context does not understand enough to know do to anything + } + + /** + * Convert a physically-stored IdealState into a rebalancer context for a partitioned resource + * @param idealState populated IdealState + * @return PartitionedRebalancerContext + */ + public static PartitionedRebalancerContext from(IdealState idealState) { + PartitionedRebalancerContext context; + switch (idealState.getRebalanceMode()) { + case FULL_AUTO: + FullAutoRebalancerContext.Builder fullAutoBuilder = + new FullAutoRebalancerContext.Builder(idealState.getResourceId()); + populateContext(fullAutoBuilder, idealState); + context = fullAutoBuilder.build(); + break; + case SEMI_AUTO: + SemiAutoRebalancerContext.Builder semiAutoBuilder = + new SemiAutoRebalancerContext.Builder(idealState.getResourceId()); + for (PartitionId partitionId : idealState.getPartitionIdSet()) { + semiAutoBuilder.preferenceList(partitionId, idealState.getPreferenceList(partitionId)); + } + populateContext(semiAutoBuilder, idealState); + context = semiAutoBuilder.build(); + break; + case CUSTOMIZED: + CustomRebalancerContext.Builder customBuilder = + new CustomRebalancerContext.Builder(idealState.getResourceId()); + for (PartitionId partitionId : idealState.getPartitionIdSet()) { + customBuilder.preferenceMap(partitionId, idealState.getParticipantStateMap(partitionId)); + } + populateContext(customBuilder, idealState); + context = customBuilder.build(); + break; + default: + Builder baseBuilder = new Builder(idealState.getResourceId()); + populateContext(baseBuilder, idealState); + context = baseBuilder.build(); + break; + } + return context; + } + + /** + * Update a builder subclass with all the fields of the ideal state + * @param builder builder that extends AbstractBuilder + * @param idealState populated IdealState + */ + private static > void populateContext(T builder, + IdealState idealState) { + String replicas = idealState.getReplicas(); + int replicaCount = 0; + boolean anyLiveParticipant = false; + if (replicas.equals(StateModelToken.ANY_LIVEINSTANCE.toString())) { + anyLiveParticipant = true; + } else { + replicaCount = Integer.parseInt(replicas); + } + if (idealState.getNumPartitions() > 0 && idealState.getPartitionIdSet().size() == 0) { + // backwards compatibility: partition sets were based on pref lists/maps previously + builder.addPartitions(idealState.getNumPartitions()); + } else { + for (PartitionId partitionId : idealState.getPartitionIdSet()) { + builder.addPartition(new Partition(partitionId)); + } + } + builder.anyLiveParticipant(anyLiveParticipant).replicaCount(replicaCount) + .maxPartitionsPerParticipant(idealState.getMaxPartitionsPerInstance()) + .participantGroupTag(idealState.getInstanceGroupTag()) + .stateModelDefId(idealState.getStateModelDefId()) + .stateModelFactoryId(idealState.getStateModelFactoryId()); + RebalancerRef rebalancerRef = idealState.getRebalancerRef(); + if (rebalancerRef != null) { + builder.rebalancerRef(rebalancerRef); + } + } + + /** + * Builder for a basic data rebalancer context + */ + public static final class Builder extends AbstractBuilder { + /** + * Instantiate with a resource + * @param resourceId resource id + */ + public Builder(ResourceId resourceId) { + super(resourceId); + } + + @Override + protected Builder self() { + return this; + } + + @Override + public PartitionedRebalancerContext build() { + PartitionedRebalancerContext context = new PartitionedRebalancerContext(); + super.update(context); + return context; + } + } + + /** + * Abstract builder for a generic partitioned resource rebalancer context + */ + public static abstract class AbstractBuilder> + extends BasicRebalancerContext.AbstractBuilder { + private final ResourceId _resourceId; + private final Map _partitionMap; + private RebalanceMode _rebalanceMode; + private boolean _anyLiveParticipant; + private int _replicaCount; + private int _maxPartitionsPerParticipant; + + /** + * Instantiate with a resource + * @param resourceId resource id + */ + public AbstractBuilder(ResourceId resourceId) { + super(resourceId); + _resourceId = resourceId; + _partitionMap = Maps.newHashMap(); + _rebalanceMode = RebalanceMode.USER_DEFINED; + _anyLiveParticipant = false; + _replicaCount = 1; + _maxPartitionsPerParticipant = Integer.MAX_VALUE; + } + + /** + * Set the rebalance mode for a partitioned rebalancer context + * @param rebalanceMode {@link RebalanceMode} enum value + * @return Builder + */ + public T rebalanceMode(RebalanceMode rebalanceMode) { + _rebalanceMode = rebalanceMode; + return self(); + } + + /** + * Add a partition that the resource serves + * @param partition fully-qualified partition + * @return Builder + */ + public T addPartition(Partition partition) { + _partitionMap.put(partition.getId(), partition); + return self(); + } + + /** + * Add a collection of partitions + * @param partitions any collection of Partition objects + * @return Builder + */ + public T addPartitions(Collection partitions) { + for (Partition partition : partitions) { + addPartition(partition); + } + return self(); + } + + /** + * Add a specified number of partitions with a default naming scheme, namely + * resourceId_partitionNumber where partitionNumber starts at 0 + * @param partitionCount number of partitions to add + * @return Builder + */ + public T addPartitions(int partitionCount) { + for (int i = 0; i < partitionCount; i++) { + addPartition(new Partition(PartitionId.from(_resourceId, Integer.toString(i)))); + } + return self(); + } + + /** + * Set whether any live participant should be used in rebalancing + * @param anyLiveParticipant true if any live participant can be used, false otherwise + * @return Builder + */ + public T anyLiveParticipant(boolean anyLiveParticipant) { + _anyLiveParticipant = anyLiveParticipant; + return self(); + } + + /** + * Set the number of replicas + * @param replicaCount number of replicas + * @return Builder + */ + public T replicaCount(int replicaCount) { + _replicaCount = replicaCount; + return self(); + } + + /** + * Set the maximum number of partitions to assign to any participant + * @param maxPartitionsPerParticipant the maximum + * @return Builder + */ + public T maxPartitionsPerParticipant(int maxPartitionsPerParticipant) { + _maxPartitionsPerParticipant = maxPartitionsPerParticipant; + return self(); + } + + /** + * Update a DataRebalancerContext with fields from this builder level + * @param context DataRebalancerContext + */ + protected final void update(PartitionedRebalancerContext context) { + super.update(context); + // enforce at least one partition + if (_partitionMap.isEmpty()) { + addPartitions(1); + } + context.setRebalanceMode(_rebalanceMode); + context.setPartitionMap(_partitionMap); + context.setAnyLiveParticipant(_anyLiveParticipant); + context.setMaxPartitionsPerParticipant(_maxPartitionsPerParticipant); + context.setReplicaCount(_replicaCount); + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerConfig.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerConfig.java new file mode 100644 index 0000000000..aa872c4389 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerConfig.java @@ -0,0 +1,182 @@ +package org.apache.helix.controller.rebalancer.context; + +import org.apache.helix.api.Scope; +import org.apache.helix.api.config.NamespacedConfig; +import org.apache.helix.controller.rebalancer.HelixRebalancer; +import org.apache.helix.model.ResourceConfiguration; +import org.apache.helix.util.HelixUtil; +import org.apache.log4j.Logger; + +/* + * 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. + */ + +/** + * Configuration for a resource rebalancer. This contains a RebalancerContext, which contains + * information specific to each rebalancer. + */ +public final class RebalancerConfig { + private enum Fields { + SERIALIZER_CLASS, + REBALANCER_CONTEXT, + REBALANCER_CONTEXT_CLASS + } + + private static final Logger LOG = Logger.getLogger(RebalancerConfig.class); + private ContextSerializer _serializer; + private HelixRebalancer _rebalancer; + private final RebalancerContext _context; + private final NamespacedConfig _config; + + /** + * Instantiate a RebalancerConfig + * @param context rebalancer context + * @param rebalancerRef reference to the rebalancer class that will be used + */ + public RebalancerConfig(RebalancerContext context) { + _config = + new NamespacedConfig(Scope.resource(context.getResourceId()), + RebalancerConfig.class.getSimpleName()); + _config.setSimpleField(Fields.SERIALIZER_CLASS.toString(), context.getSerializerClass() + .getName()); + _config + .setSimpleField(Fields.REBALANCER_CONTEXT_CLASS.toString(), context.getClass().getName()); + _context = context; + try { + _serializer = context.getSerializerClass().newInstance(); + _config.setSimpleField(Fields.REBALANCER_CONTEXT.toString(), _serializer.serialize(context)); + } catch (InstantiationException e) { + LOG.error("Error initializing the configuration", e); + } catch (IllegalAccessException e) { + LOG.error("Error initializing the configuration", e); + } + } + + /** + * Instantiate from a physical ResourceConfiguration + * @param resourceConfiguration populated ResourceConfiguration + */ + public RebalancerConfig(ResourceConfiguration resourceConfiguration) { + _config = new NamespacedConfig(resourceConfiguration, RebalancerConfig.class.getSimpleName()); + _serializer = getSerializer(); + _context = getContext(); + } + + /** + * Get the class that can serialize and deserialize the rebalancer context + * @return ContextSerializer + */ + private ContextSerializer getSerializer() { + String serializerClassName = _config.getSimpleField(Fields.SERIALIZER_CLASS.toString()); + if (serializerClassName != null) { + try { + return (ContextSerializer) HelixUtil.loadClass(getClass(), serializerClassName) + .newInstance(); + } catch (InstantiationException e) { + LOG.error("Error getting the serializer", e); + } catch (IllegalAccessException e) { + LOG.error("Error getting the serializer", e); + } catch (ClassNotFoundException e) { + LOG.error("Error getting the serializer", e); + } + } + return null; + } + + private RebalancerContext getContext() { + String className = _config.getSimpleField(Fields.REBALANCER_CONTEXT_CLASS.toString()); + if (className != null) { + try { + Class contextClass = + HelixUtil.loadClass(getClass(), className).asSubclass(RebalancerContext.class); + String serialized = _config.getSimpleField(Fields.REBALANCER_CONTEXT.toString()); + return _serializer.deserialize(contextClass, serialized); + } catch (ClassNotFoundException e) { + LOG.error(className + " is not a valid class"); + } catch (ClassCastException e) { + LOG.error(className + " does not implement RebalancerContext"); + } + } + return null; + } + + /** + * Get a rebalancer class instance + * @return Rebalancer + */ + public HelixRebalancer getRebalancer() { + // cache the rebalancer to avoid loading and instantiating it excessively + if (_rebalancer == null) { + if (_context == null || _context.getRebalancerRef() == null) { + return null; + } + _rebalancer = _context.getRebalancerRef().getRebalancer(); + } + return _rebalancer; + } + + /** + * Get the instantiated RebalancerContext + * @param contextClass specific class of the RebalancerContext + * @return RebalancerContext subclass instance, or null if conversion is not possible + */ + public T getRebalancerContext(Class contextClass) { + if (_context != null) { + try { + return contextClass.cast(_context); + } catch (ClassCastException e) { + LOG.warn(contextClass + " is incompatible with context class: " + _context.getClass()); + } + } + return null; + } + + /** + * Get the rebalancer context serialized as a string + * @return string representing the context + */ + public String getSerializedContext() { + return _config.getSimpleField(Fields.REBALANCER_CONTEXT.toString()); + } + + /** + * Convert this to a namespaced config + * @return NamespacedConfig + */ + public NamespacedConfig toNamespacedConfig() { + return _config; + } + + /** + * Get a RebalancerConfig from a physical resource config + * @param resourceConfiguration physical resource config + * @return RebalancerConfig + */ + public static RebalancerConfig from(ResourceConfiguration resourceConfiguration) { + return new RebalancerConfig(resourceConfiguration); + } + + /** + * Get a RebalancerConfig from a RebalancerContext + * @param context instantiated RebalancerContext + * @return RebalancerConfig + */ + public static RebalancerConfig from(RebalancerContext context) { + return new RebalancerConfig(context); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerContext.java new file mode 100644 index 0000000000..981891b7cc --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerContext.java @@ -0,0 +1,94 @@ +package org.apache.helix.controller.rebalancer.context; + +import java.util.Map; +import java.util.Set; + +import org.apache.helix.api.Partition; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; +import org.apache.helix.controller.rebalancer.RebalancerRef; + +/* + * 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. + */ + +/** + * Defines the state available to a rebalancer. The most common use case is to use a + * {@link PartitionedRebalancerContext} or a subclass and set up a resource with it. A rebalancer + * configuration, at a minimum, is aware of subunits of a resource, the state model to follow, and + * how the configuration should be serialized. + */ +public interface RebalancerContext { + /** + * Get a map of resource partition identifiers to partitions. A partition is a subunit of a + * resource, e.g. a subtask of a task + * @return map of (subunit id, subunit) pairs + */ + public Map getSubUnitMap(); + + /** + * Get the subunits of the resource (e.g. partitions) + * @return set of subunit ids + */ + public Set getSubUnitIdSet(); + + /** + * Get a specific subunit + * @param subUnitId the id of the subunit + * @return SubUnit + */ + public Partition getSubUnit(PartitionId subUnitId); + + /** + * Get the resource to rebalance + * @return resource id + */ + public ResourceId getResourceId(); + + /** + * Get the state model definition that the resource follows + * @return state model definition id + */ + public StateModelDefId getStateModelDefId(); + + /** + * Get the state model factory of this resource + * @return state model factory id + */ + public StateModelFactoryId getStateModelFactoryId(); + + /** + * Get the tag, if any, that participants must have in order to serve this resource + * @return participant group tag, or null + */ + public String getParticipantGroupTag(); + + /** + * Get the serializer for this context + * @return ContextSerializer class object + */ + public Class getSerializerClass(); + + /** + * Get a reference to the class used to rebalance this resource + * @return RebalancerRef + */ + public RebalancerRef getRebalancerRef(); +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ReplicatedRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ReplicatedRebalancerContext.java new file mode 100644 index 0000000000..525931df39 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ReplicatedRebalancerContext.java @@ -0,0 +1,40 @@ +package org.apache.helix.controller.rebalancer.context; + +/* + * 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. + */ + +/** + * Methods specifying a rebalancer context that allows replicas. For instance, a rebalancer context + * with partitions may accept state model definitions that support multiple replicas per partition, + * and it's possible that the policy is that each live participant in the system should have a + * replica. + */ +public interface ReplicatedRebalancerContext extends RebalancerContext { + /** + * Check if this resource should be assigned to any live participant + * @return true if any live participant expected, false otherwise + */ + public boolean anyLiveParticipant(); + + /** + * Get the number of replicas that each resource subunit should have + * @return replica count + */ + public int getReplicaCount(); +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.java new file mode 100644 index 0000000000..afa81e2fce --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.java @@ -0,0 +1,178 @@ +package org.apache.helix.controller.rebalancer.context; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.RebalancerRef; +import org.apache.helix.controller.rebalancer.SemiAutoRebalancer; +import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.StateModelDefinition; +import org.codehaus.jackson.annotate.JsonIgnore; +import org.codehaus.jackson.annotate.JsonProperty; + +import com.google.common.collect.Maps; + +/* + * 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. + */ + +/** + * RebalancerContext for SEMI_AUTO rebalancer mode. It indicates the preferred locations of each + * partition replica. By default, it corresponds to {@link SemiAutoRebalancer} + */ +public final class SemiAutoRebalancerContext extends PartitionedRebalancerContext { + @JsonProperty("preferenceLists") + private Map> _preferenceLists; + + /** + * Instantiate a SemiAutoRebalancerContext + */ + public SemiAutoRebalancerContext() { + setRebalanceMode(RebalanceMode.SEMI_AUTO); + setRebalancerRef(RebalancerRef.from(SemiAutoRebalancer.class)); + _preferenceLists = Maps.newHashMap(); + } + + /** + * Get the preference lists of all partitions of the resource + * @return map of partition id to list of participant ids + */ + public Map> getPreferenceLists() { + return _preferenceLists; + } + + /** + * Set the preference lists of all partitions of the resource + * @param preferenceLists + */ + public void setPreferenceLists(Map> preferenceLists) { + _preferenceLists = preferenceLists; + } + + /** + * Get the preference list of a partition + * @param partitionId the partition to look up + * @return list of participant ids + */ + @JsonIgnore + public List getPreferenceList(PartitionId partitionId) { + return _preferenceLists.get(partitionId); + } + + /** + * Generate preference lists based on a default cluster setup + * @param stateModelDef the state model definition to follow + * @param participantSet the set of participant ids to configure for + */ + @Override + @JsonIgnore + public void generateDefaultConfiguration(StateModelDefinition stateModelDef, + Set participantSet) { + // compute default upper bounds + Map upperBounds = Maps.newHashMap(); + for (State state : stateModelDef.getTypedStatesPriorityList()) { + upperBounds.put(state, stateModelDef.getNumParticipantsPerState(state)); + } + + // determine the current mapping + Map> currentMapping = Maps.newHashMap(); + for (PartitionId partitionId : getPartitionSet()) { + List preferenceList = getPreferenceList(partitionId); + if (preferenceList != null && !preferenceList.isEmpty()) { + Set disabledParticipants = Collections.emptySet(); + Map emptyCurrentState = Collections.emptyMap(); + Map initialMap = + ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds, participantSet, + stateModelDef, preferenceList, emptyCurrentState, disabledParticipants); + currentMapping.put(partitionId, initialMap); + } + } + + // determine the preference + LinkedHashMap stateCounts = + ConstraintBasedAssignment.stateCount(upperBounds, stateModelDef, participantSet.size(), + getReplicaCount()); + ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme(); + List participantList = new ArrayList(participantSet); + List partitionList = new ArrayList(getPartitionSet()); + AutoRebalanceStrategy strategy = + new AutoRebalanceStrategy(ResourceId.from(""), partitionList, stateCounts, + getMaxPartitionsPerParticipant(), placementScheme); + Map> rawPreferenceLists = + strategy.typedComputePartitionAssignment(participantList, currentMapping, participantList) + .getListFields(); + Map> preferenceLists = + Maps.newHashMap(IdealState.preferenceListsFromStringLists(rawPreferenceLists)); + setPreferenceLists(preferenceLists); + } + + /** + * Build a SemiAutoRebalancerContext. By default, it corresponds to {@link SemiAutoRebalancer} + */ + public static final class Builder extends PartitionedRebalancerContext.AbstractBuilder { + private final Map> _preferenceLists; + + /** + * Instantiate for a resource + * @param resourceId resource id + */ + public Builder(ResourceId resourceId) { + super(resourceId); + super.rebalancerRef(RebalancerRef.from(SemiAutoRebalancer.class)); + super.rebalanceMode(RebalanceMode.SEMI_AUTO); + _preferenceLists = Maps.newHashMap(); + } + + /** + * Add a preference list for a partition + * @param partitionId partition to set + * @param preferenceList ordered list of participants who can serve the partition + * @return Builder + */ + public Builder preferenceList(PartitionId partitionId, List preferenceList) { + _preferenceLists.put(partitionId, preferenceList); + return self(); + } + + @Override + protected Builder self() { + return this; + } + + @Override + public SemiAutoRebalancerContext build() { + SemiAutoRebalancerContext context = new SemiAutoRebalancerContext(); + super.update(context); + context.setPreferenceLists(_preferenceLists); + return context; + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java index 3fd52f4d96..79517841d3 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java +++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java @@ -20,95 +20,165 @@ */ import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; 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 org.apache.helix.HelixDefinedState; import org.apache.helix.HelixConstants.StateModelToken; -import org.apache.helix.controller.stages.ClusterDataCache; -import org.apache.helix.model.IdealState; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.Partition; +import org.apache.helix.HelixDefinedState; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; import org.apache.helix.model.StateModelDefinition; import org.apache.log4j.Logger; +import com.google.common.base.Predicate; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + /** - * Collection of functions that will compute the best possible states given the live instances and - * an ideal state. + * Collection of functions that will compute the best possible state based on the participants and + * the rebalancer configuration of a resource. */ public class ConstraintBasedAssignment { private static Logger logger = Logger.getLogger(ConstraintBasedAssignment.class); - public static List getPreferenceList(ClusterDataCache cache, Partition resource, - IdealState idealState, StateModelDefinition stateModelDef) { - List listField = idealState.getPreferenceList(resource.getPartitionName()); + /** + * Get a set of disabled participants for a partition + * @param participantMap map of all participants + * @param partitionId the partition to check + * @return a set of all participants that are disabled for the partition + */ + public static Set getDisabledParticipants( + final Map participantMap, final PartitionId partitionId) { + Set participantSet = new HashSet(participantMap.keySet()); + Set disabledParticipantsForPartition = + Sets.filter(participantSet, new Predicate() { + @Override + public boolean apply(ParticipantId participantId) { + Participant participant = participantMap.get(participantId); + return !participant.isEnabled() + || participant.getDisabledPartitionIds().contains(partitionId); + } + }); + return disabledParticipantsForPartition; + } - if (listField != null && listField.size() == 1 - && StateModelToken.ANY_LIVEINSTANCE.toString().equals(listField.get(0))) { - Map liveInstances = cache.getLiveInstances(); - List prefList = new ArrayList(liveInstances.keySet()); + /** + * Get an ordered list of participants that can serve a partition + * @param cluster cluster snapshot + * @param partitionId the partition to look up + * @param config rebalancing constraints + * @return list with most preferred participants first + */ + public static List getPreferenceList(Cluster cluster, PartitionId partitionId, + List prefList) { + if (prefList != null && prefList.size() == 1 + && StateModelToken.ANY_LIVEINSTANCE.toString().equals(prefList.get(0).stringify())) { + prefList = new ArrayList(cluster.getLiveParticipantMap().keySet()); Collections.sort(prefList); - return prefList; - } else { - return listField; } + return prefList; } /** - * compute best state for resource in AUTO ideal state mode - * @param cache - * @param stateModelDef - * @param instancePreferenceList - * @param currentStateMap - * : instance->state for each partition - * @param disabledInstancesForPartition - * @return + * Get a map of state to upper bound constraint given a cluster + * @param stateModelDef the state model definition to check + * @param resourceId the resource that is constraint + * @param cluster the cluster the resource belongs to + * @return map of state to upper bound */ - public static Map computeAutoBestStateForPartition(ClusterDataCache cache, - StateModelDefinition stateModelDef, List instancePreferenceList, - Map currentStateMap, Set disabledInstancesForPartition) { - Map instanceStateMap = new HashMap(); + public static Map stateConstraints(StateModelDefinition stateModelDef, + ResourceId resourceId, ClusterConfig cluster) { + Map stateMap = Maps.newHashMap(); + for (State state : stateModelDef.getTypedStatesPriorityList()) { + String num = + cluster.getStateUpperBoundConstraint(Scope.resource(resourceId), + stateModelDef.getStateModelDefId(), state); + stateMap.put(state, num); + } + return stateMap; + } - // if the ideal state is deleted, instancePreferenceList will be empty and + /** + * Get a mapping for a partition for the current state participants who have been dropped or + * disabled for a given partition. + * @param currentStateMap current map of participant id to state for a partition + * @param participants participants selected to serve the partition + * @param disabledParticipants participants that have been disabled for this partition + * @param initialState the initial state of the resource state model + * @return map of participant id to state of dropped and disabled partitions + */ + public static Map dropAndDisablePartitions( + Map currentStateMap, Collection participants, + Set disabledParticipants, State initialState) { + Map participantStateMap = new HashMap(); + // if the resource is deleted, instancePreferenceList will be empty and // we should drop all resources. if (currentStateMap != null) { - for (String instance : currentStateMap.keySet()) { - if ((instancePreferenceList == null || !instancePreferenceList.contains(instance)) - && !disabledInstancesForPartition.contains(instance)) { + for (ParticipantId participantId : currentStateMap.keySet()) { + if ((participants == null || !participants.contains(participantId)) + && !disabledParticipants.contains(participantId)) { // if dropped and not disabled, transit to DROPPED - instanceStateMap.put(instance, HelixDefinedState.DROPPED.toString()); - } else if ((currentStateMap.get(instance) == null || !currentStateMap.get(instance).equals( - HelixDefinedState.ERROR.toString())) - && disabledInstancesForPartition.contains(instance)) { + participantStateMap.put(participantId, State.from(HelixDefinedState.DROPPED)); + } else if ((currentStateMap.get(participantId) == null || !currentStateMap.get( + participantId).equals(State.from(HelixDefinedState.ERROR))) + && disabledParticipants.contains(participantId)) { // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE) - instanceStateMap.put(instance, stateModelDef.getInitialState()); + participantStateMap.put(participantId, initialState); } } } + return participantStateMap; + } - // ideal state is deleted - if (instancePreferenceList == null) { - return instanceStateMap; - } + /** + * compute best state for resource in SEMI_AUTO and FULL_AUTO modes + * @param upperBounds map of state to upper bound + * @param liveParticipantSet set of live participant ids + * @param stateModelDef + * @param participantPreferenceList + * @param currentStateMap + * : participant->state for each partition + * @param disabledParticipantsForPartition + * @return + */ + public static Map computeAutoBestStateForPartition( + Map upperBounds, Set liveParticipantSet, + StateModelDefinition stateModelDef, List participantPreferenceList, + Map currentStateMap, Set disabledParticipantsForPartition) { + // drop and disable participants if necessary + Map participantStateMap = + dropAndDisablePartitions(currentStateMap, participantPreferenceList, + disabledParticipantsForPartition, stateModelDef.getTypedInitialState()); - List statesPriorityList = stateModelDef.getStatesPriorityList(); - boolean assigned[] = new boolean[instancePreferenceList.size()]; + // resource is deleted + if (participantPreferenceList == null) { + return participantStateMap; + } - Map liveInstancesMap = cache.getLiveInstances(); + List statesPriorityList = stateModelDef.getTypedStatesPriorityList(); + boolean assigned[] = new boolean[participantPreferenceList.size()]; - for (String state : statesPriorityList) { - String num = stateModelDef.getNumInstancesPerState(state); + for (State state : statesPriorityList) { + String num = upperBounds.get(state); int stateCount = -1; if ("N".equals(num)) { - Set liveAndEnabled = new HashSet(liveInstancesMap.keySet()); - liveAndEnabled.removeAll(disabledInstancesForPartition); + Set liveAndEnabled = new HashSet(liveParticipantSet); + liveAndEnabled.removeAll(disabledParticipantsForPartition); stateCount = liveAndEnabled.size(); } else if ("R".equals(num)) { - stateCount = instancePreferenceList.size(); + stateCount = participantPreferenceList.size(); } else { try { stateCount = Integer.parseInt(num); @@ -118,16 +188,18 @@ public static Map computeAutoBestStateForPartition(ClusterDataCa } if (stateCount > -1) { int count = 0; - for (int i = 0; i < instancePreferenceList.size(); i++) { - String instanceName = instancePreferenceList.get(i); + for (int i = 0; i < participantPreferenceList.size(); i++) { + ParticipantId participantId = participantPreferenceList.get(i); boolean notInErrorState = - currentStateMap == null || currentStateMap.get(instanceName) == null - || !currentStateMap.get(instanceName).equals(HelixDefinedState.ERROR.toString()); + currentStateMap == null + || currentStateMap.get(participantId) == null + || !currentStateMap.get(participantId) + .equals(State.from(HelixDefinedState.ERROR)); - if (liveInstancesMap.containsKey(instanceName) && !assigned[i] && notInErrorState - && !disabledInstancesForPartition.contains(instanceName)) { - instanceStateMap.put(instanceName, state); + if (liveParticipantSet.contains(participantId) && !assigned[i] && notInErrorState + && !disabledParticipantsForPartition.contains(participantId)) { + participantStateMap.put(participantId, state); count = count + 1; assigned[i] = true; if (count == stateCount) { @@ -137,6 +209,105 @@ public static Map computeAutoBestStateForPartition(ClusterDataCa } } } - return instanceStateMap; + return participantStateMap; + } + + /** + * Get the number of replicas that should be in each state for a partition + * @param upperBounds map of state to upper bound + * @param stateModelDef StateModelDefinition object + * @param liveNodesNb number of live nodes + * @param total number of replicas + * @return state count map: state->count + */ + public static LinkedHashMap stateCount(Map upperBounds, + StateModelDefinition stateModelDef, int liveNodesNb, int totalReplicas) { + LinkedHashMap stateCountMap = new LinkedHashMap(); + List statesPriorityList = stateModelDef.getTypedStatesPriorityList(); + + int replicas = totalReplicas; + for (State state : statesPriorityList) { + String num = upperBounds.get(state); + if ("N".equals(num)) { + stateCountMap.put(state, liveNodesNb); + } else if ("R".equals(num)) { + // wait until we get the counts for all other states + continue; + } else { + int stateCount = -1; + try { + stateCount = Integer.parseInt(num); + } catch (Exception e) { + // LOG.error("Invalid count for state: " + state + ", count: " + num + + // ", use -1 instead"); + } + + if (stateCount > 0) { + stateCountMap.put(state, stateCount); + replicas -= stateCount; + } + } + } + + // get state count for R + for (State state : statesPriorityList) { + String num = upperBounds.get(state); + if ("R".equals(num)) { + stateCountMap.put(state, replicas); + // should have at most one state using R + break; + } + } + return stateCountMap; + } + + /** + * compute best state for resource in CUSTOMIZED rebalancer mode + * @param liveParticipantMap + * @param stateModelDef + * @param preferenceMap + * @param currentStateMap + * @param disabledParticipantsForPartition + * @return + */ + public static Map computeCustomizedBestStateForPartition( + Set liveParticipantSet, StateModelDefinition stateModelDef, + Map preferenceMap, Map currentStateMap, + Set disabledParticipantsForPartition) { + Map participantStateMap = new HashMap(); + + // if the resource is deleted, idealStateMap will be null/empty and + // we should drop all resources. + if (currentStateMap != null) { + for (ParticipantId participantId : currentStateMap.keySet()) { + if ((preferenceMap == null || !preferenceMap.containsKey(participantId)) + && !disabledParticipantsForPartition.contains(participantId)) { + // if dropped and not disabled, transit to DROPPED + participantStateMap.put(participantId, State.from(HelixDefinedState.DROPPED)); + } else if ((currentStateMap.get(participantId) == null || !currentStateMap.get( + participantId).equals(State.from(HelixDefinedState.ERROR))) + && disabledParticipantsForPartition.contains(participantId)) { + // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE) + participantStateMap.put(participantId, stateModelDef.getTypedInitialState()); + } + } + } + + // ideal state is deleted + if (preferenceMap == null) { + return participantStateMap; + } + + for (ParticipantId participantId : preferenceMap.keySet()) { + boolean notInErrorState = + currentStateMap == null || currentStateMap.get(participantId) == null + || !currentStateMap.get(participantId).equals(State.from(HelixDefinedState.ERROR)); + + if (liveParticipantSet.contains(participantId) && notInErrorState + && !disabledParticipantsForPartition.contains(participantId)) { + participantStateMap.put(participantId, preferenceMap.get(participantId)); + } + } + return participantStateMap; } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/restlet/ZKPropertyTransferServer.java b/helix-core/src/main/java/org/apache/helix/controller/restlet/ZKPropertyTransferServer.java index 96277cf727..66bd257ac7 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/restlet/ZKPropertyTransferServer.java +++ b/helix-core/src/main/java/org/apache/helix/controller/restlet/ZKPropertyTransferServer.java @@ -26,10 +26,10 @@ import java.util.TimerTask; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; +import java.util.logging.Level; import org.I0Itec.zkclient.DataUpdater; import org.apache.helix.AccessOption; -import org.apache.helix.BaseDataAccessor; import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkBaseDataAccessor; @@ -73,6 +73,10 @@ public class ZKPropertyTransferServer { Component _component = null; Timer _timer = null; + static { + org.restlet.engine.Engine.setLogLevel(Level.SEVERE); + } + /** * Timertask for zookeeper batched writes */ diff --git a/helix-core/src/main/java/org/apache/helix/controller/restlet/ZNRecordUpdateResource.java b/helix-core/src/main/java/org/apache/helix/controller/restlet/ZNRecordUpdateResource.java index f704f54ba7..387768693c 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/restlet/ZNRecordUpdateResource.java +++ b/helix-core/src/main/java/org/apache/helix/controller/restlet/ZNRecordUpdateResource.java @@ -2,19 +2,19 @@ /* * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file + * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file + * 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 - * + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -29,39 +29,26 @@ import org.restlet.data.Form; import org.restlet.data.MediaType; import org.restlet.data.Status; -import org.restlet.resource.Representation; -import org.restlet.resource.Resource; +import org.restlet.representation.Representation; +import org.restlet.representation.Variant; +import org.restlet.resource.ServerResource; /** * REST resource for ZkPropertyTransfer server to receive PUT requests * that submits ZNRecordUpdates */ -public class ZNRecordUpdateResource extends Resource { +public class ZNRecordUpdateResource extends ServerResource { public static final String UPDATEKEY = "ZNRecordUpdate"; private static Logger LOG = Logger.getLogger(ZNRecordUpdateResource.class); - @Override - public boolean allowGet() { - return false; - } - - @Override - public boolean allowPost() { - return false; + public ZNRecordUpdateResource() { + getVariants().add(new Variant(MediaType.TEXT_PLAIN)); + getVariants().add(new Variant(MediaType.APPLICATION_JSON)); + setNegotiated(false); } @Override - public boolean allowPut() { - return true; - } - - @Override - public boolean allowDelete() { - return false; - } - - @Override - public void storeRepresentation(Representation entity) { + public Representation put(Representation entity) { try { ZKPropertyTransferServer server = ZKPropertyTransferServer.getInstance(); @@ -86,5 +73,7 @@ public void storeRepresentation(Representation entity) { LOG.error("", e); getResponse().setStatus(Status.SERVER_ERROR_INTERNAL); } + return null; } + } diff --git a/helix-core/src/main/java/org/apache/helix/controller/restlet/ZkPropertyTransferApplication.java b/helix-core/src/main/java/org/apache/helix/controller/restlet/ZkPropertyTransferApplication.java index f345fe213c..68d35cb3d6 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/restlet/ZkPropertyTransferApplication.java +++ b/helix-core/src/main/java/org/apache/helix/controller/restlet/ZkPropertyTransferApplication.java @@ -22,7 +22,7 @@ import org.restlet.Application; import org.restlet.Context; import org.restlet.Restlet; -import org.restlet.Router; +import org.restlet.routing.Router; /** * Restlet application for ZkPropertyTransfer server @@ -37,7 +37,7 @@ public ZkPropertyTransferApplication(Context context) { } @Override - public Restlet createRoot() { + public Restlet createInboundRoot() { Router router = new Router(getContext()); router.attach("/" + ZKPropertyTransferServer.RESTRESOURCENAME, ZNRecordUpdateResource.class); return router; diff --git a/helix-core/src/main/java/org/apache/helix/controller/restlet/ZkPropertyTransferClient.java b/helix-core/src/main/java/org/apache/helix/controller/restlet/ZkPropertyTransferClient.java index ed1a1140d9..092d845c90 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/restlet/ZkPropertyTransferClient.java +++ b/helix-core/src/main/java/org/apache/helix/controller/restlet/ZkPropertyTransferClient.java @@ -30,17 +30,18 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.logging.Level; import org.apache.helix.ZNRecord; import org.apache.log4j.Logger; import org.codehaus.jackson.map.ObjectMapper; import org.restlet.Client; +import org.restlet.Request; +import org.restlet.Response; import org.restlet.data.MediaType; import org.restlet.data.Method; import org.restlet.data.Protocol; import org.restlet.data.Reference; -import org.restlet.data.Request; -import org.restlet.data.Response; import org.restlet.data.Status; public class ZkPropertyTransferClient { @@ -61,6 +62,10 @@ public class ZkPropertyTransferClient { Timer _timer; volatile String _webServiceUrl = ""; + static { + org.restlet.engine.Engine.setLogLevel(Level.SEVERE); + } + public ZkPropertyTransferClient(int maxConcurrentTasks) { _maxConcurrentTasks = maxConcurrentTasks; _executorService = Executors.newFixedThreadPool(_maxConcurrentTasks); diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java index 598c3183d8..7b143bd1ae 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java @@ -20,106 +20,182 @@ */ import java.util.Map; - +import java.util.Set; + +import org.apache.helix.HelixDefinedState; +import org.apache.helix.HelixManager; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.pipeline.AbstractBaseStage; import org.apache.helix.controller.pipeline.StageException; -import org.apache.helix.controller.rebalancer.AutoRebalancer; -import org.apache.helix.controller.rebalancer.CustomRebalancer; -import org.apache.helix.controller.rebalancer.Rebalancer; -import org.apache.helix.controller.rebalancer.SemiAutoRebalancer; -import org.apache.helix.model.IdealState; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; -import org.apache.helix.model.IdealState.RebalanceMode; -import org.apache.helix.util.HelixUtil; +import org.apache.helix.controller.rebalancer.FallbackRebalancer; +import org.apache.helix.controller.rebalancer.HelixRebalancer; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; import org.apache.log4j.Logger; +import com.google.common.collect.Sets; + /** * For partition compute best possible (instance,state) pair based on * IdealState,StateModel,LiveInstance */ public class BestPossibleStateCalcStage extends AbstractBaseStage { - private static final Logger logger = Logger.getLogger(BestPossibleStateCalcStage.class.getName()); + private static final Logger LOG = Logger.getLogger(BestPossibleStateCalcStage.class.getName()); @Override public void process(ClusterEvent event) throws Exception { long startTime = System.currentTimeMillis(); - logger.info("START BestPossibleStateCalcStage.process()"); + if (LOG.isInfoEnabled()) { + LOG.info("START BestPossibleStateCalcStage.process()"); + } - CurrentStateOutput currentStateOutput = + ResourceCurrentState currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.toString()); - Map resourceMap = event.getAttribute(AttributeName.RESOURCES.toString()); - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); + Map resourceMap = + event.getAttribute(AttributeName.RESOURCES.toString()); + Cluster cluster = event.getAttribute("ClusterDataCache"); - if (currentStateOutput == null || resourceMap == null || cache == null) { + if (currentStateOutput == null || resourceMap == null || cluster == null) { throw new StageException("Missing attributes in event:" + event + ". Requires CURRENT_STATE|RESOURCES|DataCache"); } BestPossibleStateOutput bestPossibleStateOutput = - compute(event, resourceMap, currentStateOutput); + compute(cluster, event, resourceMap, currentStateOutput); event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.toString(), bestPossibleStateOutput); long endTime = System.currentTimeMillis(); - logger.info("END BestPossibleStateCalcStage.process(). took: " + (endTime - startTime) + " ms"); + if (LOG.isInfoEnabled()) { + LOG.info("END BestPossibleStateCalcStage.process(). took: " + (endTime - startTime) + " ms"); + } } - private BestPossibleStateOutput compute(ClusterEvent event, Map resourceMap, - CurrentStateOutput currentStateOutput) { - // for each ideal state - // read the state model def - // for each resource - // get the preference list - // for each instanceName check if its alive then assign a state - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); + /** + * Fallback for cases when the resource has been dropped, but current state exists + * @param cluster cluster snapshot + * @param resourceId the resource for which to generate an assignment + * @param currentStateOutput full snapshot of the current state + * @param stateModelDef state model the resource follows + * @return assignment for the dropped resource + */ + private ResourceAssignment mapDroppedResource(Cluster cluster, ResourceId resourceId, + ResourceCurrentState currentStateOutput, StateModelDefinition stateModelDef) { + ResourceAssignment partitionMapping = new ResourceAssignment(resourceId); + Set mappedPartitions = + currentStateOutput.getCurrentStateMappedPartitions(resourceId); + if (mappedPartitions == null) { + return partitionMapping; + } + for (PartitionId partitionId : mappedPartitions) { + Set disabledParticipantsForPartition = + ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(), + partitionId); + Map upperBounds = + ConstraintBasedAssignment + .stateConstraints(stateModelDef, resourceId, cluster.getConfig()); + partitionMapping.addReplicaMap(partitionId, ConstraintBasedAssignment + .computeAutoBestStateForPartition(upperBounds, cluster.getLiveParticipantMap().keySet(), + stateModelDef, null, currentStateOutput.getCurrentStateMap(resourceId, partitionId), + disabledParticipantsForPartition)); + } + return partitionMapping; + } - BestPossibleStateOutput output = new BestPossibleStateOutput(); + /** + * Update a ResourceAssignment with dropped and disabled participants for partitions + * @param cluster cluster snapshot + * @param resourceAssignment current resource assignment + * @param currentStateOutput aggregated current state + * @param stateModelDef state model definition for the resource + */ + private void mapDroppedAndDisabledPartitions(Cluster cluster, + ResourceAssignment resourceAssignment, ResourceCurrentState currentStateOutput, + StateModelDefinition stateModelDef) { + // get the total partition set: mapped and current state + ResourceId resourceId = resourceAssignment.getResourceId(); + Set mappedPartitions = Sets.newHashSet(); + mappedPartitions.addAll(currentStateOutput.getCurrentStateMappedPartitions(resourceId)); + mappedPartitions.addAll(resourceAssignment.getMappedPartitionIds()); + for (PartitionId partitionId : mappedPartitions) { + // for each partition, get the dropped and disabled mappings + Set disabledParticipants = + ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(), + partitionId); + + // get the error participants + Map currentStateMap = + currentStateOutput.getCurrentStateMap(resourceId, partitionId); + Set errorParticipants = Sets.newHashSet(); + for (ParticipantId participantId : currentStateMap.keySet()) { + State state = currentStateMap.get(participantId); + if (state.equals(State.from(HelixDefinedState.ERROR))) { + errorParticipants.add(participantId); + } + } - for (String resourceName : resourceMap.keySet()) { - logger.debug("Processing resource:" + resourceName); + // get the dropped and disabled map + State initialState = stateModelDef.getTypedInitialState(); + Map participantStateMap = resourceAssignment.getReplicaMap(partitionId); + Set participants = participantStateMap.keySet(); + Map droppedAndDisabledMap = + ConstraintBasedAssignment.dropAndDisablePartitions(currentStateMap, participants, + disabledParticipants, initialState); + + // don't map error participants + for (ParticipantId participantId : errorParticipants) { + droppedAndDisabledMap.remove(participantId); + } + // save the mappings, overwriting as necessary + participantStateMap.putAll(droppedAndDisabledMap); - Resource resource = resourceMap.get(resourceName); - // Ideal state may be gone. In that case we need to get the state model name - // from the current state - IdealState idealState = cache.getIdealState(resourceName); + // include this add step in case the resource assignment did not already map this partition + resourceAssignment.addReplicaMap(partitionId, participantStateMap); + } + } - if (idealState == null) { - // if ideal state is deleted, use an empty one - logger.info("resource:" + resourceName + " does not exist anymore"); - idealState = new IdealState(resourceName); - } + private BestPossibleStateOutput compute(Cluster cluster, ClusterEvent event, + Map resourceMap, ResourceCurrentState currentStateOutput) { + BestPossibleStateOutput output = new BestPossibleStateOutput(); + Map stateModelDefs = cluster.getStateModelMap(); - Rebalancer rebalancer = null; - if (idealState.getRebalanceMode() == RebalanceMode.USER_DEFINED - && idealState.getRebalancerClassName() != null) { - String rebalancerClassName = idealState.getRebalancerClassName(); - logger - .info("resource " + resourceName + " use idealStateRebalancer " + rebalancerClassName); - try { - rebalancer = - (Rebalancer) (HelixUtil.loadClass(getClass(), rebalancerClassName).newInstance()); - } catch (Exception e) { - logger.warn("Exception while invoking custom rebalancer class:" + rebalancerClassName, e); - } + for (ResourceId resourceId : resourceMap.keySet()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Processing resource:" + resourceId); } - if (rebalancer == null) { - if (idealState.getRebalanceMode() == RebalanceMode.FULL_AUTO) { - rebalancer = new AutoRebalancer(); - } else if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO) { - rebalancer = new SemiAutoRebalancer(); - } else { - rebalancer = new CustomRebalancer(); + ResourceConfig resourceConfig = resourceMap.get(resourceId); + RebalancerConfig rebalancerConfig = resourceConfig.getRebalancerConfig(); + RebalancerContext context = rebalancerConfig.getRebalancerContext(RebalancerContext.class); + StateModelDefinition stateModelDef = stateModelDefs.get(context.getStateModelDefId()); + ResourceAssignment resourceAssignment = null; + if (rebalancerConfig != null) { + HelixRebalancer rebalancer = rebalancerConfig.getRebalancer(); + HelixManager manager = event.getAttribute("helixmanager"); + if (rebalancer == null) { + rebalancer = new FallbackRebalancer(); } + rebalancer.init(manager); + resourceAssignment = + rebalancer.computeResourceMapping(rebalancerConfig, cluster, currentStateOutput); } - - ResourceMapping partitionStateAssignment = - rebalancer.computeBestPossiblePartitionState(cache, idealState, resource, - currentStateOutput); - for (Partition partition : resource.getPartitions()) { - Map newStateMap = partitionStateAssignment.getInstanceStateMap(partition); - output.setState(resourceName, partition, newStateMap); + if (resourceAssignment == null) { + resourceAssignment = + mapDroppedResource(cluster, resourceId, currentStateOutput, stateModelDef); + } else { + mapDroppedAndDisabledPartitions(cluster, resourceAssignment, currentStateOutput, + stateModelDef); } + output.setResourceAssignment(resourceId, resourceAssignment); } + return output; } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java index 3da9bef4f5..50c722baa1 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java @@ -19,51 +19,50 @@ * under the License. */ -import java.util.Collections; -import java.util.HashMap; import java.util.Map; +import java.util.Set; -import org.apache.helix.model.Partition; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.ResourceAssignment; + +import com.google.common.collect.Maps; public class BestPossibleStateOutput { - // resource->partition->instance->state - Map>> _dataMap; - public BestPossibleStateOutput() { - _dataMap = new HashMap>>(); - } + Map _resourceAssignmentMap; - public void setState(String resourceName, Partition resource, - Map bestInstanceStateMappingForResource) { - if (!_dataMap.containsKey(resourceName)) { - _dataMap.put(resourceName, new HashMap>()); - } - Map> map = _dataMap.get(resourceName); - map.put(resource, bestInstanceStateMappingForResource); + public BestPossibleStateOutput() { + _resourceAssignmentMap = Maps.newHashMap(); } - public Map getInstanceStateMap(String resourceName, Partition resource) { - Map> map = _dataMap.get(resourceName); - if (map != null) { - return map.get(resource); - } - return Collections.emptyMap(); + /** + * Set the computed resource assignment for a resource + * @param resourceId the resource to set + * @param resourceAssignment the computed assignment + */ + public void setResourceAssignment(ResourceId resourceId, ResourceAssignment resourceAssignment) { + _resourceAssignmentMap.put(resourceId, resourceAssignment); } - public Map> getResourceMap(String resourceName) { - Map> map = _dataMap.get(resourceName); - if (map != null) { - return map; - } - return Collections.emptyMap(); + /** + * Get the resource assignment computed for a resource + * @param resourceId resource to look up + * @return ResourceAssignment computed by the best possible state calculation + */ + public ResourceAssignment getResourceAssignment(ResourceId resourceId) { + return _resourceAssignmentMap.get(resourceId); } - public Map>> getStateMap() { - return _dataMap; + /** + * Get all of the resources currently assigned + * @return set of assigned resource ids + */ + public Set getAssignedResources() { + return _resourceAssignmentMap.keySet(); } @Override public String toString() { - return _dataMap.toString(); + return _resourceAssignmentMap.toString(); } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java index b90880eb73..ac1cef4ff8 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java @@ -25,23 +25,24 @@ import java.util.Map; import java.util.Set; -import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixConstants.StateModelToken; +import org.apache.helix.HelixDataAccessor; import org.apache.helix.PropertyKey.Builder; import org.apache.helix.model.ClusterConstraints; +import org.apache.helix.model.ClusterConstraints.ConstraintType; import org.apache.helix.model.CurrentState; 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.StateModelDefinition; -import org.apache.helix.model.ClusterConstraints.ConstraintType; import org.apache.log4j.Logger; /** * Reads the data from the cluster using data accessor. This output ClusterData which * provides useful methods to search/lookup properties */ +@Deprecated public class ClusterDataCache { Map _liveInstanceMap; @@ -72,7 +73,7 @@ public boolean refresh(HelixDataAccessor accessor) { _liveInstanceMap = accessor.getChildValuesMap(keyBuilder.liveInstances()); for (LiveInstance instance : _liveInstanceMap.values()) { - LOG.trace("live instance: " + instance.getInstanceName() + " " + instance.getSessionId()); + LOG.trace("live instance: " + instance.getParticipantId() + " " + instance.getTypedSessionId()); } _stateModelDefMap = accessor.getChildValuesMap(keyBuilder.stateModelDefs()); @@ -90,7 +91,7 @@ public boolean refresh(HelixDataAccessor accessor) { new HashMap>>(); for (String instanceName : _liveInstanceMap.keySet()) { LiveInstance liveInstance = _liveInstanceMap.get(instanceName); - String sessionId = liveInstance.getSessionId(); + String sessionId = liveInstance.getTypedSessionId().stringify(); if (!allCurStateMap.containsKey(instanceName)) { allCurStateMap.put(instanceName, new HashMap>()); } diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java index d8f98ed7b3..532ecb5746 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java @@ -23,9 +23,12 @@ import org.apache.helix.HelixManager; import org.apache.helix.HelixManagerProperties; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.HelixVersion; +import org.apache.helix.api.Participant; +import org.apache.helix.api.id.ParticipantId; import org.apache.helix.controller.pipeline.AbstractBaseStage; import org.apache.helix.controller.pipeline.StageException; -import org.apache.helix.model.LiveInstance; import org.apache.log4j.Logger; /** @@ -37,23 +40,25 @@ public class CompatibilityCheckStage extends AbstractBaseStage { @Override public void process(ClusterEvent event) throws Exception { HelixManager manager = event.getAttribute("helixmanager"); - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); - if (manager == null || cache == null) { + Cluster cluster = event.getAttribute("ClusterDataCache"); + if (manager == null || cluster == null) { throw new StageException("Missing attributes in event:" + event + ". Requires HelixManager | DataCache"); } HelixManagerProperties properties = manager.getProperties(); - Map liveInstanceMap = cache.getLiveInstances(); - for (LiveInstance liveInstance : liveInstanceMap.values()) { - String participantVersion = liveInstance.getHelixVersion(); + // Map liveInstanceMap = cache.getLiveInstances(); + Map liveParticipants = cluster.getLiveParticipantMap(); + for (Participant liveParticipant : liveParticipants.values()) { + HelixVersion version = liveParticipant.getRunningInstance().getVersion(); + String participantVersion = (version != null) ? version.toString() : null; if (!properties.isParticipantCompatible(participantVersion)) { String errorMsg = "incompatible participant. pipeline will not continue. " + "controller: " + manager.getInstanceName() + ", controllerVersion: " + properties.getVersion() + ", minimumSupportedParticipantVersion: " - + properties.getProperty("miminum_supported_version.participant") - + ", participant: " + liveInstance.getInstanceName() + ", participantVersion: " + + properties.getProperty("minimum_supported_version.participant") + + ", participant: " + liveParticipant.getId() + ", participantVersion: " + participantVersion; LOG.error(errorMsg); throw new StageException(errorMsg); 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 6097432517..c036b14822 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 @@ -22,13 +22,21 @@ import java.util.List; import java.util.Map; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Partition; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.pipeline.AbstractBaseStage; import org.apache.helix.controller.pipeline.StageException; import org.apache.helix.model.CurrentState; -import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; import org.apache.helix.model.Message.MessageType; /** @@ -39,50 +47,55 @@ public class CurrentStateComputationStage extends AbstractBaseStage { @Override public void process(ClusterEvent event) throws Exception { - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); - Map resourceMap = event.getAttribute(AttributeName.RESOURCES.toString()); + Cluster cluster = event.getAttribute("ClusterDataCache"); + Map resourceMap = + event.getAttribute(AttributeName.RESOURCES.toString()); - if (cache == null || resourceMap == null) { + if (cluster == null || resourceMap == null) { throw new StageException("Missing attributes in event:" + event + ". Requires DataCache|RESOURCE"); } - Map liveInstances = cache.getLiveInstances(); - CurrentStateOutput currentStateOutput = new CurrentStateOutput(); + ResourceCurrentState currentStateOutput = new ResourceCurrentState(); - for (LiveInstance instance : liveInstances.values()) { - String instanceName = instance.getInstanceName(); - Map instanceMessages = cache.getMessages(instanceName); - for (Message message : instanceMessages.values()) { + for (Participant liveParticipant : cluster.getLiveParticipantMap().values()) { + ParticipantId participantId = liveParticipant.getId(); + + // add pending messages + Map instanceMsgs = liveParticipant.getMessageMap(); + for (Message message : instanceMsgs.values()) { if (!MessageType.STATE_TRANSITION.toString().equalsIgnoreCase(message.getMsgType())) { continue; } - if (!instance.getSessionId().equals(message.getTgtSessionId())) { + + if (!liveParticipant.getRunningInstance().getSessionId() + .equals(message.getTypedTgtSessionId())) { continue; } - String resourceName = message.getResourceName(); - Resource resource = resourceMap.get(resourceName); + + ResourceId resourceId = message.getResourceId(); + ResourceConfig resource = resourceMap.get(resourceId); if (resource == null) { continue; } if (!message.getBatchMessageMode()) { - String partitionName = message.getPartitionName(); - Partition partition = resource.getPartition(partitionName); + PartitionId partitionId = message.getPartitionId(); + Partition partition = resource.getSubUnit(partitionId); if (partition != null) { - currentStateOutput.setPendingState(resourceName, partition, instanceName, - message.getToState()); + currentStateOutput.setPendingState(resourceId, partitionId, participantId, + message.getTypedToState()); } else { // log } } else { - List partitionNames = message.getPartitionNames(); + List partitionNames = message.getPartitionIds(); if (!partitionNames.isEmpty()) { - for (String partitionName : partitionNames) { - Partition partition = resource.getPartition(partitionName); + for (PartitionId partitionId : partitionNames) { + Partition partition = resource.getSubUnit(partitionId); if (partition != null) { - currentStateOutput.setPendingState(resourceName, partition, instanceName, - message.getToState()); + currentStateOutput.setPendingState(resourceId, partitionId, participantId, + message.getTypedToState()); } else { // log } @@ -90,40 +103,32 @@ public void process(ClusterEvent event) throws Exception { } } } - } - for (LiveInstance instance : liveInstances.values()) { - String instanceName = instance.getInstanceName(); - - String clientSessionId = instance.getSessionId(); - Map currentStateMap = - cache.getCurrentState(instanceName, clientSessionId); - for (CurrentState currentState : currentStateMap.values()) { - if (!instance.getSessionId().equals(currentState.getSessionId())) { + // add current state + SessionId sessionId = liveParticipant.getRunningInstance().getSessionId(); + Map curStateMap = liveParticipant.getCurrentStateMap(); + for (CurrentState curState : curStateMap.values()) { + if (!sessionId.equals(curState.getTypedSessionId())) { continue; } - String resourceName = currentState.getResourceName(); - String stateModelDefName = currentState.getStateModelDefRef(); - Resource resource = resourceMap.get(resourceName); + + ResourceId resourceId = curState.getResourceId(); + StateModelDefId stateModelDefId = curState.getStateModelDefId(); + ResourceConfig resource = resourceMap.get(resourceId); if (resource == null) { continue; } - if (stateModelDefName != null) { - currentStateOutput.setResourceStateModelDef(resourceName, stateModelDefName); - } - currentStateOutput.setBucketSize(resourceName, currentState.getBucketSize()); + if (stateModelDefId != null) { + currentStateOutput.setResourceStateModelDef(resourceId, stateModelDefId); + } - Map partitionStateMap = currentState.getPartitionStateMap(); - for (String partitionName : partitionStateMap.keySet()) { - Partition partition = resource.getPartition(partitionName); - if (partition != null) { - currentStateOutput.setCurrentState(resourceName, partition, instanceName, - currentState.getState(partitionName)); + currentStateOutput.setBucketSize(resourceId, curState.getBucketSize()); - } else { - // log - } + Map partitionStateMap = curState.getTypedPartitionStateMap(); + for (PartitionId partitionId : partitionStateMap.keySet()) { + currentStateOutput.setCurrentState(resourceId, partitionId, participantId, + curState.getState(partitionId)); } } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateOutput.java index b41f14b6cc..61873da31d 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateOutput.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateOutput.java @@ -26,6 +26,7 @@ import org.apache.helix.model.CurrentState; import org.apache.helix.model.Partition; +@Deprecated public class CurrentStateOutput { private final Map>> _currentStateMap; private final Map>> _pendingStateMap; diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java index 35ef177a8b..edceed6124 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java @@ -29,38 +29,44 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixDefinedState; import org.apache.helix.HelixManager; -import org.apache.helix.NotificationContext; import org.apache.helix.PropertyKey; import org.apache.helix.PropertyKey.Builder; import org.apache.helix.ZNRecord; import org.apache.helix.ZNRecordDelta; import org.apache.helix.ZNRecordDelta.MergeOperation; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.SchedulerTaskConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.pipeline.AbstractBaseStage; import org.apache.helix.controller.pipeline.StageException; -import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageType; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; import org.apache.helix.model.StatusUpdate; -import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor; import org.apache.log4j.Logger; public class ExternalViewComputeStage extends AbstractBaseStage { - private static Logger log = Logger.getLogger(ExternalViewComputeStage.class); + private static Logger LOG = Logger.getLogger(ExternalViewComputeStage.class); @Override public void process(ClusterEvent event) throws Exception { long startTime = System.currentTimeMillis(); - log.info("START ExternalViewComputeStage.process()"); + LOG.info("START ExternalViewComputeStage.process()"); HelixManager manager = event.getAttribute("helixmanager"); - Map resourceMap = event.getAttribute(AttributeName.RESOURCES.toString()); - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); + Map resourceMap = + event.getAttribute(AttributeName.RESOURCES.toString()); + Cluster cluster = event.getAttribute("ClusterDataCache"); - if (manager == null || resourceMap == null || cache == null) { + if (manager == null || resourceMap == null || cluster == null) { throw new StageException("Missing attributes in event:" + event + ". Requires ClusterManager|RESOURCES|DataCache"); } @@ -68,58 +74,64 @@ public void process(ClusterEvent event) throws Exception { HelixDataAccessor dataAccessor = manager.getHelixDataAccessor(); PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder(); - CurrentStateOutput currentStateOutput = + ResourceCurrentState currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.toString()); List newExtViews = new ArrayList(); List keys = new ArrayList(); + // TODO use external-view accessor Map curExtViews = dataAccessor.getChildValuesMap(keyBuilder.externalViews()); - for (String resourceName : resourceMap.keySet()) { - ExternalView view = new ExternalView(resourceName); + for (ResourceId resourceId : resourceMap.keySet()) { + ExternalView view = new ExternalView(resourceId.stringify()); // view.setBucketSize(currentStateOutput.getBucketSize(resourceName)); // if resource ideal state has bucket size, set it // otherwise resource has been dropped, use bucket size from current state instead - Resource resource = resourceMap.get(resourceName); + ResourceConfig resource = resourceMap.get(resourceId); + RebalancerConfig rebalancerConfig = resource.getRebalancerConfig(); + SchedulerTaskConfig schedulerTaskConfig = resource.getSchedulerTaskConfig(); + if (resource.getBucketSize() > 0) { view.setBucketSize(resource.getBucketSize()); } else { - view.setBucketSize(currentStateOutput.getBucketSize(resourceName)); + view.setBucketSize(currentStateOutput.getBucketSize(resourceId)); } - - for (Partition partition : resource.getPartitions()) { - Map currentStateMap = - currentStateOutput.getCurrentStateMap(resourceName, partition); + for (PartitionId partitionId : resource.getSubUnitMap().keySet()) { + Map currentStateMap = + currentStateOutput.getCurrentStateMap(resourceId, partitionId); if (currentStateMap != null && currentStateMap.size() > 0) { // Set disabledInstances // = cache.getDisabledInstancesForResource(resource.toString()); - for (String instance : currentStateMap.keySet()) { + for (ParticipantId participantId : currentStateMap.keySet()) { // if (!disabledInstances.contains(instance)) // { - view.setState(partition.getPartitionName(), instance, currentStateMap.get(instance)); + view.setState(partitionId.stringify(), participantId.stringify(), + currentStateMap.get(participantId).toString()); // } } } } + + // TODO fix this // Update cluster status monitor mbean - ClusterStatusMonitor clusterStatusMonitor = - (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor"); - IdealState idealState = cache._idealStateMap.get(view.getResourceName()); - if (idealState != null) { - if (clusterStatusMonitor != null - && !idealState.getStateModelDefRef().equalsIgnoreCase( - DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) { - clusterStatusMonitor.onExternalViewChange(view, - cache._idealStateMap.get(view.getResourceName())); - } - } + // ClusterStatusMonitor clusterStatusMonitor = + // (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor"); + // IdealState idealState = cache._idealStateMap.get(view.getResourceName()); + // if (idealState != null) { + // if (clusterStatusMonitor != null + // && !idealState.getStateModelDefRef().equalsIgnoreCase( + // DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) { + // clusterStatusMonitor.onExternalViewChange(view, + // cache._idealStateMap.get(view.getResourceName())); + // } + // } // compare the new external view with current one, set only on different - ExternalView curExtView = curExtViews.get(resourceName); + ExternalView curExtView = curExtViews.get(resourceId.stringify()); if (curExtView == null || !curExtView.getRecord().equals(view.getRecord())) { - keys.add(keyBuilder.externalView(resourceName)); + keys.add(keyBuilder.externalView(resourceId.stringify())); newExtViews.add(view); // For SCHEDULER_TASK_RESOURCE resource group (helix task queue), we need to find out which @@ -127,10 +139,13 @@ public void process(ClusterEvent event) throws Exception { // partitions are finished (COMPLETED or ERROR), update the status update of the original // scheduler // message, and then remove the partitions from the ideal state - if (idealState != null - && idealState.getStateModelDefRef().equalsIgnoreCase( - DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) { - updateScheduledTaskStatus(view, manager, idealState); + RebalancerContext rebalancerContext = + (rebalancerConfig != null) ? rebalancerConfig + .getRebalancerContext(RebalancerContext.class) : null; + if (rebalancerContext != null + && rebalancerContext.getStateModelDefId().equalsIgnoreCase( + StateModelDefId.SchedulerTaskQueue)) { + updateScheduledTaskStatus(resourceId, view, manager, schedulerTaskConfig); } } } @@ -144,18 +159,21 @@ public void process(ClusterEvent event) throws Exception { // remove dead external-views for (String resourceName : curExtViews.keySet()) { - if (!resourceMap.keySet().contains(resourceName)) { + if (!resourceMap.containsKey(ResourceId.from(resourceName))) { dataAccessor.removeProperty(keyBuilder.externalView(resourceName)); } } long endTime = System.currentTimeMillis(); - log.info("END ExternalViewComputeStage.process(). took: " + (endTime - startTime) + " ms"); + LOG.info("END ExternalViewComputeStage.process(). took: " + (endTime - startTime) + " ms"); } - private void updateScheduledTaskStatus(ExternalView ev, HelixManager manager, - IdealState taskQueueIdealState) { + // TODO fix it + private void updateScheduledTaskStatus(ResourceId resourceId, ExternalView ev, + HelixManager manager, SchedulerTaskConfig schedulerTaskConfig) { HelixDataAccessor accessor = manager.getHelixDataAccessor(); + Builder keyBuilder = accessor.keyBuilder(); + ZNRecord finishedTasks = new ZNRecord(ev.getResourceName()); // Place holder for finished partitions @@ -166,23 +184,21 @@ private void updateScheduledTaskStatus(ExternalView ev, HelixManager manager, Map> controllerMsgUpdates = new HashMap>(); - Builder keyBuilder = accessor.keyBuilder(); - for (String taskPartitionName : ev.getPartitionSet()) { for (String taskState : ev.getStateMap(taskPartitionName).values()) { if (taskState.equalsIgnoreCase(HelixDefinedState.ERROR.toString()) || taskState.equalsIgnoreCase("COMPLETED")) { - log.info(taskPartitionName + " finished as " + taskState); - finishedTasks.getListFields().put(taskPartitionName, emptyList); - finishedTasks.getMapFields().put(taskPartitionName, emptyMap); + LOG.info(taskPartitionName + " finished as " + taskState); + finishedTasks.setListField(taskPartitionName, emptyList); + finishedTasks.setMapField(taskPartitionName, emptyMap); // Update original scheduler message status update - if (taskQueueIdealState.getRecord().getMapField(taskPartitionName) != null) { - String controllerMsgId = - taskQueueIdealState.getRecord().getMapField(taskPartitionName) - .get(DefaultSchedulerMessageHandlerFactory.CONTROLLER_MSG_ID); + Message innerMessage = + schedulerTaskConfig.getInnerMessage(PartitionId.from(taskPartitionName)); + if (innerMessage != null) { + String controllerMsgId = innerMessage.getControllerMessageId(); if (controllerMsgId != null) { - log.info(taskPartitionName + " finished with controllerMsg " + controllerMsgId); + LOG.info(taskPartitionName + " finished with controllerMsg " + controllerMsgId); if (!controllerMsgUpdates.containsKey(controllerMsgId)) { controllerMsgUpdates.put(controllerMsgId, new HashMap()); } @@ -193,16 +209,16 @@ private void updateScheduledTaskStatus(ExternalView ev, HelixManager manager, } } // fill the controllerMsgIdCountMap - for (String taskId : taskQueueIdealState.getPartitionSet()) { - String controllerMsgId = - taskQueueIdealState.getRecord().getMapField(taskId) - .get(DefaultSchedulerMessageHandlerFactory.CONTROLLER_MSG_ID); + for (PartitionId taskId : schedulerTaskConfig.getPartitionSet()) { + Message innerMessage = schedulerTaskConfig.getInnerMessage(taskId); + String controllerMsgId = innerMessage.getControllerMessageId(); + if (controllerMsgId != null) { - if (!controllerMsgIdCountMap.containsKey(controllerMsgId)) { - controllerMsgIdCountMap.put(controllerMsgId, 0); + Integer curCnt = controllerMsgIdCountMap.get(controllerMsgId); + if (curCnt == null) { + curCnt = 0; } - controllerMsgIdCountMap.put(controllerMsgId, - (controllerMsgIdCountMap.get(controllerMsgId) + 1)); + controllerMsgIdCountMap.put(controllerMsgId, curCnt + 1); } } @@ -212,18 +228,16 @@ private void updateScheduledTaskStatus(ExternalView ev, HelixManager manager, keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), controllerMsgId); StatusUpdate controllerStatusUpdate = accessor.getProperty(controllerStatusUpdateKey); for (String taskPartitionName : controllerMsgUpdates.get(controllerMsgId).keySet()) { + Message innerMessage = + schedulerTaskConfig.getInnerMessage(PartitionId.from(taskPartitionName)); + Map result = new HashMap(); result.put("Result", controllerMsgUpdates.get(controllerMsgId).get(taskPartitionName)); controllerStatusUpdate.getRecord().setMapField( - "MessageResult " - + taskQueueIdealState.getRecord().getMapField(taskPartitionName) - .get(Message.Attributes.TGT_NAME.toString()) - + " " - + taskPartitionName - + " " - + taskQueueIdealState.getRecord().getMapField(taskPartitionName) - .get(Message.Attributes.MSG_ID.toString()), result); + "MessageResult " + innerMessage.getTgtName() + " " + taskPartitionName + " " + + innerMessage.getMessageId(), result); } + // All done for the scheduled tasks that came from controllerMsgId, add summary for it if (controllerMsgUpdates.get(controllerMsgId).size() == controllerMsgIdCountMap.get( controllerMsgId).intValue()) { @@ -255,12 +269,12 @@ private void updateScheduledTaskStatus(ExternalView ev, HelixManager manager, ZNRecordDelta znDelta = new ZNRecordDelta(finishedTasks, MergeOperation.SUBTRACT); List deltaList = new LinkedList(); deltaList.add(znDelta); - IdealState delta = new IdealState(taskQueueIdealState.getResourceName()); + IdealState delta = new IdealState(resourceId); delta.setDeltaList(deltaList); // Remove the finished (COMPLETED or ERROR) tasks from the SCHEDULER_TASK_RESOURCE idealstate keyBuilder = accessor.keyBuilder(); - accessor.updateProperty(keyBuilder.idealStates(taskQueueIdealState.getResourceName()), delta); + accessor.updateProperty(keyBuilder.idealStates(resourceId.stringify()), delta); } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/HealthDataCache.java b/helix-core/src/main/java/org/apache/helix/controller/stages/HealthDataCache.java deleted file mode 100644 index 3ab833620c..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/HealthDataCache.java +++ /dev/null @@ -1,97 +0,0 @@ -package org.apache.helix.controller.stages; - -/* - * 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.Collections; -import java.util.HashMap; -import java.util.Map; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixProperty; -import org.apache.helix.PropertyType; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.model.AlertStatus; -import org.apache.helix.model.Alerts; -import org.apache.helix.model.HealthStat; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.PersistentStats; - -public class HealthDataCache { - Map _liveInstanceMap; - - Map> _healthStatMap; - HealthStat _globalStats; // DON'T THINK I WILL USE THIS ANYMORE - PersistentStats _persistentStats; - Alerts _alerts; - AlertStatus _alertStatus; - - public HealthStat getGlobalStats() { - return _globalStats; - } - - public PersistentStats getPersistentStats() { - return _persistentStats; - } - - public Alerts getAlerts() { - return _alerts; - } - - public AlertStatus getAlertStatus() { - return _alertStatus; - } - - public Map getHealthStats(String instanceName) { - Map map = _healthStatMap.get(instanceName); - if (map != null) { - return map; - } else { - return Collections.emptyMap(); - } - } - - public Map getLiveInstances() { - return _liveInstanceMap; - } - - public boolean refresh(HelixDataAccessor accessor) { - Builder keyBuilder = accessor.keyBuilder(); - _liveInstanceMap = accessor.getChildValuesMap(keyBuilder.liveInstances()); - - Map> hsMap = new HashMap>(); - - for (String instanceName : _liveInstanceMap.keySet()) { - // xxx clearly getting znodes for the instance here...so get the - // timestamp! - - Map childValuesMap = - accessor.getChildValuesMap(keyBuilder.healthReports(instanceName)); - hsMap.put(instanceName, childValuesMap); - } - _healthStatMap = Collections.unmodifiableMap(hsMap); - _persistentStats = accessor.getProperty(keyBuilder.persistantStat()); - _alerts = accessor.getProperty(keyBuilder.alerts()); - _alertStatus = accessor.getProperty(keyBuilder.alertStatus()); - - return true; - - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationOutput.java deleted file mode 100644 index 359a95935b..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationOutput.java +++ /dev/null @@ -1,65 +0,0 @@ -package org.apache.helix.controller.stages; - -/* - * 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.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; - -public class MessageGenerationOutput { - - private final Map>> _messagesMap; - - public MessageGenerationOutput() { - _messagesMap = new HashMap>>(); - - } - - public void addMessage(String resourceName, Partition partition, Message message) { - if (!_messagesMap.containsKey(resourceName)) { - _messagesMap.put(resourceName, new HashMap>()); - } - if (!_messagesMap.get(resourceName).containsKey(partition)) { - _messagesMap.get(resourceName).put(partition, new ArrayList()); - - } - _messagesMap.get(resourceName).get(partition).add(message); - - } - - public List getMessages(String resourceName, Partition resource) { - Map> map = _messagesMap.get(resourceName); - if (map != null) { - return map.get(resource); - } - return Collections.emptyList(); - - } - - @Override - public String toString() { - return _messagesMap.toString(); - } -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java deleted file mode 100644 index 92964e93f7..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ /dev/null @@ -1,207 +0,0 @@ -package org.apache.helix.controller.stages; - -/* - * 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 java.util.UUID; - -import org.apache.helix.HelixManager; -import org.apache.helix.controller.pipeline.AbstractBaseStage; -import org.apache.helix.controller.pipeline.StageException; -import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory; -import org.apache.helix.model.IdealState; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; -import org.apache.helix.model.StateModelDefinition; -import org.apache.helix.model.Message.MessageState; -import org.apache.helix.model.Message.MessageType; -import org.apache.log4j.Logger; - -/** - * Compares the currentState, pendingState with IdealState and generate messages - */ -public class MessageGenerationPhase extends AbstractBaseStage { - private static Logger logger = Logger.getLogger(MessageGenerationPhase.class); - - @Override - public void process(ClusterEvent event) throws Exception { - HelixManager manager = event.getAttribute("helixmanager"); - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); - Map resourceMap = event.getAttribute(AttributeName.RESOURCES.toString()); - CurrentStateOutput currentStateOutput = - event.getAttribute(AttributeName.CURRENT_STATE.toString()); - BestPossibleStateOutput bestPossibleStateOutput = - event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); - if (manager == null || cache == null || resourceMap == null || currentStateOutput == null - || bestPossibleStateOutput == null) { - throw new StageException("Missing attributes in event:" + event - + ". Requires HelixManager|DataCache|RESOURCES|CURRENT_STATE|BEST_POSSIBLE_STATE"); - } - - Map liveInstances = cache.getLiveInstances(); - Map sessionIdMap = new HashMap(); - - for (LiveInstance liveInstance : liveInstances.values()) { - sessionIdMap.put(liveInstance.getInstanceName(), liveInstance.getSessionId()); - } - MessageGenerationOutput output = new MessageGenerationOutput(); - - for (String resourceName : resourceMap.keySet()) { - Resource resource = resourceMap.get(resourceName); - int bucketSize = resource.getBucketSize(); - - StateModelDefinition stateModelDef = cache.getStateModelDef(resource.getStateModelDefRef()); - - for (Partition partition : resource.getPartitions()) { - Map instanceStateMap = - bestPossibleStateOutput.getInstanceStateMap(resourceName, partition); - - // we should generate message based on the desired-state priority - // so keep generated messages in a temp map keyed by state - // desired-state->list of generated-messages - Map> messageMap = new HashMap>(); - - for (String instanceName : instanceStateMap.keySet()) { - String desiredState = instanceStateMap.get(instanceName); - - String currentState = - currentStateOutput.getCurrentState(resourceName, partition, instanceName); - if (currentState == null) { - currentState = stateModelDef.getInitialState(); - } - - if (desiredState.equalsIgnoreCase(currentState)) { - continue; - } - - String pendingState = - currentStateOutput.getPendingState(resourceName, partition, instanceName); - - String nextState = stateModelDef.getNextStateForTransition(currentState, desiredState); - if (nextState == null) { - logger.error("Unable to find a next state for partition: " - + partition.getPartitionName() + " from stateModelDefinition" - + stateModelDef.getClass() + " from:" + currentState + " to:" + desiredState); - continue; - } - - if (pendingState != null) { - if (nextState.equalsIgnoreCase(pendingState)) { - logger.debug("Message already exists for " + instanceName + " to transit " - + partition.getPartitionName() + " from " + currentState + " to " + nextState); - } else if (currentState.equalsIgnoreCase(pendingState)) { - logger.info("Message hasn't been removed for " + instanceName + " to transit" - + partition.getPartitionName() + " to " + pendingState + ", desiredState: " - + desiredState); - } else { - logger.info("IdealState changed before state transition completes for " - + partition.getPartitionName() + " on " + instanceName + ", pendingState: " - + pendingState + ", currentState: " + currentState + ", nextState: " + nextState); - } - } else { - Message message = - createMessage(manager, resourceName, partition.getPartitionName(), instanceName, - currentState, nextState, sessionIdMap.get(instanceName), stateModelDef.getId(), - resource.getStateModelFactoryname(), bucketSize); - IdealState idealState = cache.getIdealState(resourceName); - if (idealState != null - && idealState.getStateModelDefRef().equalsIgnoreCase( - DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) { - if (idealState.getRecord().getMapField(partition.getPartitionName()) != null) { - message.getRecord().setMapField(Message.Attributes.INNER_MESSAGE.toString(), - idealState.getRecord().getMapField(partition.getPartitionName())); - } - } - // Set timeout of needed - String stateTransition = - currentState + "-" + nextState + "_" + Message.Attributes.TIMEOUT; - if (idealState != null) { - String timeOutStr = idealState.getRecord().getSimpleField(stateTransition); - if (timeOutStr == null - && idealState.getStateModelDefRef().equalsIgnoreCase( - DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) { - // scheduled task queue - if (idealState.getRecord().getMapField(partition.getPartitionName()) != null) { - timeOutStr = - idealState.getRecord().getMapField(partition.getPartitionName()) - .get(Message.Attributes.TIMEOUT.toString()); - } - } - if (timeOutStr != null) { - try { - int timeout = Integer.parseInt(timeOutStr); - if (timeout > 0) { - message.setExecutionTimeout(timeout); - } - } catch (Exception e) { - logger.error("", e); - } - } - } - message.getRecord().setSimpleField("ClusterEventName", event.getName()); - // output.addMessage(resourceName, partition, message); - if (!messageMap.containsKey(desiredState)) { - messageMap.put(desiredState, new ArrayList()); - } - messageMap.get(desiredState).add(message); - } - } - - // add generated messages to output according to state priority - List statesPriorityList = stateModelDef.getStatesPriorityList(); - for (String state : statesPriorityList) { - if (messageMap.containsKey(state)) { - for (Message message : messageMap.get(state)) { - output.addMessage(resourceName, partition, message); - } - } - } - - } // end of for-each-partition - } - event.addAttribute(AttributeName.MESSAGES_ALL.toString(), output); - } - - private Message createMessage(HelixManager manager, String resourceName, String partitionName, - String instanceName, String currentState, String nextState, String sessionId, - String stateModelDefName, String stateModelFactoryName, int bucketSize) { - String uuid = UUID.randomUUID().toString(); - Message message = new Message(MessageType.STATE_TRANSITION, uuid); - message.setSrcName(manager.getInstanceName()); - message.setTgtName(instanceName); - message.setMsgState(MessageState.NEW); - message.setPartitionName(partitionName); - message.setResourceName(resourceName); - message.setFromState(currentState); - message.setToState(nextState); - message.setTgtSessionId(sessionId); - message.setSrcSessionId(manager.getSessionId()); - message.setStateModelDef(stateModelDefName); - message.setStateModelFactoryName(stateModelFactoryName); - message.setBucketSize(bucketSize); - - return message; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java new file mode 100644 index 0000000000..08e67994a5 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java @@ -0,0 +1,213 @@ +package org.apache.helix.controller.stages; + +/* + * 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 java.util.UUID; + +import org.apache.helix.HelixManager; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.SchedulerTaskConfig; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; +import org.apache.helix.controller.pipeline.AbstractBaseStage; +import org.apache.helix.controller.pipeline.StageException; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.model.Message; +import org.apache.helix.model.Message.MessageState; +import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; + +/** + * Compares the currentState, pendingState with IdealState and generate messages + */ +public class MessageGenerationStage extends AbstractBaseStage { + private static Logger LOG = Logger.getLogger(MessageGenerationStage.class); + + @Override + public void process(ClusterEvent event) throws Exception { + HelixManager manager = event.getAttribute("helixmanager"); + Cluster cluster = event.getAttribute("ClusterDataCache"); + Map stateModelDefMap = cluster.getStateModelMap(); + Map resourceMap = + event.getAttribute(AttributeName.RESOURCES.toString()); + ResourceCurrentState currentStateOutput = + event.getAttribute(AttributeName.CURRENT_STATE.toString()); + BestPossibleStateOutput bestPossibleStateOutput = + event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); + if (manager == null || cluster == null || resourceMap == null || currentStateOutput == null + || bestPossibleStateOutput == null) { + throw new StageException("Missing attributes in event:" + event + + ". Requires HelixManager|DataCache|RESOURCES|CURRENT_STATE|BEST_POSSIBLE_STATE"); + } + + MessageOutput output = new MessageOutput(); + + for (ResourceId resourceId : resourceMap.keySet()) { + ResourceConfig resourceConfig = resourceMap.get(resourceId); + int bucketSize = resourceConfig.getBucketSize(); + + RebalancerContext rebalancerCtx = + resourceConfig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class); + StateModelDefinition stateModelDef = stateModelDefMap.get(rebalancerCtx.getStateModelDefId()); + + ResourceAssignment resourceAssignment = + bestPossibleStateOutput.getResourceAssignment(resourceId); + for (PartitionId subUnitId : resourceAssignment.getMappedPartitionIds()) { + Map instanceStateMap = resourceAssignment.getReplicaMap(subUnitId); + + // we should generate message based on the desired-state priority + // so keep generated messages in a temp map keyed by state + // desired-state->list of generated-messages + Map> messageMap = new HashMap>(); + + for (ParticipantId participantId : instanceStateMap.keySet()) { + State desiredState = instanceStateMap.get(participantId); + + State currentState = + currentStateOutput.getCurrentState(resourceId, subUnitId, participantId); + if (currentState == null) { + currentState = stateModelDef.getTypedInitialState(); + } + + if (desiredState.equals(currentState)) { + continue; + } + + State pendingState = + currentStateOutput.getPendingState(resourceId, subUnitId, participantId); + + // TODO fix it + State nextState = stateModelDef.getNextStateForTransition(currentState, desiredState); + if (nextState == null) { + LOG.error("Unable to find a next state for partition: " + subUnitId + + " from stateModelDefinition" + stateModelDef.getClass() + " from:" + currentState + + " to:" + desiredState); + continue; + } + + if (pendingState != null) { + if (nextState.equals(pendingState)) { + LOG.debug("Message already exists for " + participantId + " to transit " + subUnitId + + " from " + currentState + " to " + nextState); + } else if (currentState.equals(pendingState)) { + LOG.info("Message hasn't been removed for " + participantId + " to transit" + + subUnitId + " to " + pendingState + ", desiredState: " + desiredState); + } else { + LOG.info("IdealState changed before state transition completes for " + subUnitId + + " on " + participantId + ", pendingState: " + pendingState + ", currentState: " + + currentState + ", nextState: " + nextState); + } + } else { + // TODO check if instance is alive + SessionId sessionId = + cluster.getLiveParticipantMap().get(participantId).getRunningInstance() + .getSessionId(); + RebalancerContext rebalancerContext = + resourceConfig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class); + Message message = + createMessage(manager, resourceId, subUnitId, participantId, currentState, + nextState, sessionId, StateModelDefId.from(stateModelDef.getId()), + rebalancerContext.getStateModelFactoryId(), bucketSize); + + // TODO refactor get/set timeout/inner-message + if (rebalancerContext != null + && rebalancerContext.getStateModelDefId().equalsIgnoreCase( + StateModelDefId.SchedulerTaskQueue)) { + if (resourceConfig.getSubUnitMap().size() > 0) { + // TODO refactor it -- we need a way to read in scheduler tasks a priori + Message innerMsg = + resourceConfig.getSchedulerTaskConfig().getInnerMessage(subUnitId); + if (innerMsg != null) { + message.setInnerMessage(innerMsg); + } + } + } + + // Set timeout if needed + String stateTransition = + String.format("%s-%s_%s", currentState, nextState, + Message.Attributes.TIMEOUT.name()); + SchedulerTaskConfig schedulerTaskConfig = resourceConfig.getSchedulerTaskConfig(); + if (schedulerTaskConfig != null) { + int timeout = schedulerTaskConfig.getTimeout(stateTransition, subUnitId); + if (timeout > 0) { + message.setExecutionTimeout(timeout); + } + } + message.setClusterEvent(event); + + if (!messageMap.containsKey(desiredState)) { + messageMap.put(desiredState, new ArrayList()); + } + messageMap.get(desiredState).add(message); + } + } + + // add generated messages to output according to state priority + List statesPriorityList = stateModelDef.getTypedStatesPriorityList(); + for (State state : statesPriorityList) { + if (messageMap.containsKey(state)) { + for (Message message : messageMap.get(state)) { + output.addMessage(resourceId, subUnitId, message); + } + } + } + + } // end of for-each-partition + } + event.addAttribute(AttributeName.MESSAGES_ALL.toString(), output); + // System.out.println("output: " + output); + } + + private Message createMessage(HelixManager manager, ResourceId resourceId, + PartitionId partitionId, ParticipantId participantId, State currentState, State nextState, + SessionId participantSessionId, StateModelDefId stateModelDefId, + StateModelFactoryId stateModelFactoryId, int bucketSize) { + MessageId uuid = MessageId.from(UUID.randomUUID().toString()); + Message message = new Message(MessageType.STATE_TRANSITION, uuid); + message.setSrcName(manager.getInstanceName()); + message.setTgtName(participantId.stringify()); + message.setMsgState(MessageState.NEW); + message.setPartitionId(partitionId); + message.setResourceId(resourceId); + message.setFromState(currentState); + message.setToState(nextState); + message.setTgtSessionId(participantSessionId); + message.setSrcSessionId(SessionId.from(manager.getSessionId())); + message.setStateModelDef(stateModelDefId); + message.setStateModelFactoryId(stateModelFactoryId); + message.setBucketSize(bucketSize); + + return message; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java new file mode 100644 index 0000000000..9c8c15465e --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java @@ -0,0 +1,79 @@ +package org.apache.helix.controller.stages; + +/* + * 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.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.Message; + +public class MessageOutput { + + private final Map>> _messagesMap; + + public MessageOutput() { + _messagesMap = new HashMap>>(); + + } + + public void addMessage(ResourceId resourceId, PartitionId partitionId, Message message) { + if (!_messagesMap.containsKey(resourceId)) { + _messagesMap.put(resourceId, new HashMap>()); + } + if (!_messagesMap.get(resourceId).containsKey(partitionId)) { + _messagesMap.get(resourceId).put(partitionId, new ArrayList()); + + } + _messagesMap.get(resourceId).get(partitionId).add(message); + + } + + public void setMessages(ResourceId resourceId, PartitionId partitionId, + List selectedMessages) { + if (!_messagesMap.containsKey(resourceId)) { + _messagesMap.put(resourceId, new HashMap>()); + } + _messagesMap.get(resourceId).put(partitionId, selectedMessages); + + } + + public List getMessages(ResourceId resourceId, PartitionId partitionId) { + Map> map = _messagesMap.get(resourceId); + if (map != null) { + return map.get(partitionId); + } + return Collections.emptyList(); + + } + + public Map> getMessages(ResourceId resourceId) { + return _messagesMap.get(resourceId); + } + + @Override + public String toString() { + return _messagesMap.toString(); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java index 9a420aa5f5..bbbf5c6710 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java @@ -26,13 +26,22 @@ import java.util.Map; import java.util.TreeMap; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Resource; +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.pipeline.AbstractBaseStage; import org.apache.helix.controller.pipeline.StageException; -import org.apache.helix.model.IdealState; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.controller.rebalancer.context.ReplicatedRebalancerContext; import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; import org.apache.helix.model.StateModelDefinition; import org.apache.log4j.Logger; @@ -71,41 +80,57 @@ public int getLowerBound() { public int getUpperBound() { return upper; } + + @Override + public String toString() { + return String.format("%d-%d", lower, upper); + } } @Override public void process(ClusterEvent event) throws Exception { - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); - Map resourceMap = event.getAttribute(AttributeName.RESOURCES.toString()); - CurrentStateOutput currentStateOutput = + Cluster cluster = event.getAttribute("ClusterDataCache"); + Map stateModelDefMap = cluster.getStateModelMap(); + Map resourceMap = + event.getAttribute(AttributeName.RESOURCES.toString()); + ResourceCurrentState currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.toString()); - MessageGenerationOutput messageGenOutput = - event.getAttribute(AttributeName.MESSAGES_ALL.toString()); - if (cache == null || resourceMap == null || currentStateOutput == null - || messageGenOutput == null) { + BestPossibleStateOutput bestPossibleStateOutput = + event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); + MessageOutput messageGenOutput = event.getAttribute(AttributeName.MESSAGES_ALL.toString()); + if (cluster == null || resourceMap == null || currentStateOutput == null + || messageGenOutput == null || bestPossibleStateOutput == null) { throw new StageException("Missing attributes in event:" + event - + ". Requires DataCache|RESOURCES|CURRENT_STATE|MESSAGES_ALL"); + + ". Requires DataCache|RESOURCES|CURRENT_STATE|BEST_POSSIBLE_STATE|MESSAGES_ALL"); } - MessageSelectionStageOutput output = new MessageSelectionStageOutput(); + MessageOutput output = new MessageOutput(); - for (String resourceName : resourceMap.keySet()) { - Resource resource = resourceMap.get(resourceName); - StateModelDefinition stateModelDef = cache.getStateModelDef(resource.getStateModelDefRef()); + for (ResourceId resourceId : resourceMap.keySet()) { + ResourceConfig resource = resourceMap.get(resourceId); + StateModelDefinition stateModelDef = + stateModelDefMap.get(resource.getRebalancerConfig() + .getRebalancerContext(RebalancerContext.class).getStateModelDefId()); + // TODO have a logical model for transition Map stateTransitionPriorities = getStateTransitionPriorityMap(stateModelDef); - IdealState idealState = cache.getIdealState(resourceName); - Map stateConstraints = - computeStateConstraints(stateModelDef, idealState, cache); + Resource configResource = cluster.getResource(resourceId); + + // if configResource == null, the resource has been dropped + Map stateConstraints = + computeStateConstraints(stateModelDef, + configResource == null ? null : configResource.getRebalancerConfig(), cluster); - for (Partition partition : resource.getPartitions()) { - List messages = messageGenOutput.getMessages(resourceName, partition); + // TODO fix it + for (PartitionId partitionId : bestPossibleStateOutput.getResourceAssignment(resourceId) + .getMappedPartitionIds()) { + List messages = messageGenOutput.getMessages(resourceId, partitionId); List selectedMessages = - selectMessages(cache.getLiveInstances(), - currentStateOutput.getCurrentStateMap(resourceName, partition), - currentStateOutput.getPendingStateMap(resourceName, partition), messages, - stateConstraints, stateTransitionPriorities, stateModelDef.getInitialState()); - output.addMessages(resourceName, partition, selectedMessages); + selectMessages(cluster.getLiveParticipantMap(), + currentStateOutput.getCurrentStateMap(resourceId, partitionId), + currentStateOutput.getPendingStateMap(resourceId, partitionId), messages, + stateConstraints, stateTransitionPriorities, stateModelDef.getTypedInitialState()); + output.setMessages(resourceId, partitionId, selectedMessages); } } event.addAttribute(AttributeName.MESSAGES_SELECTED.toString(), output); @@ -130,22 +155,22 @@ public void process(ClusterEvent event) throws Exception { * : FROME_STATE-TO_STATE -> priority * @return: selected messages */ - List selectMessages(Map liveInstances, - Map currentStates, Map pendingStates, List messages, - Map stateConstraints, final Map stateTransitionPriorities, - String initialState) { + List selectMessages(Map liveParticipants, + Map currentStates, Map pendingStates, + List messages, Map stateConstraints, + final Map stateTransitionPriorities, State initialState) { if (messages == null || messages.isEmpty()) { return Collections.emptyList(); } List selectedMessages = new ArrayList(); - Map bounds = new HashMap(); + Map bounds = new HashMap(); // count currentState, if no currentState, count as in initialState - for (String instance : liveInstances.keySet()) { - String state = initialState; - if (currentStates.containsKey(instance)) { - state = currentStates.get(instance); + for (ParticipantId liveParticipantId : liveParticipants.keySet()) { + State state = initialState; + if (currentStates.containsKey(liveParticipantId)) { + state = currentStates.get(liveParticipantId); } if (!bounds.containsKey(state)) { @@ -156,8 +181,8 @@ List selectMessages(Map liveInstances, } // count pendingStates - for (String instance : pendingStates.keySet()) { - String state = pendingStates.get(instance); + for (ParticipantId participantId : pendingStates.keySet()) { + State state = pendingStates.get(participantId); if (!bounds.containsKey(state)) { bounds.put(state, new Bounds(0, 0)); } @@ -169,9 +194,9 @@ List selectMessages(Map liveInstances, Map> messagesGroupByStateTransitPriority = new TreeMap>(); for (Message message : messages) { - String fromState = message.getFromState(); - String toState = message.getToState(); - String transition = fromState + "-" + toState; + State fromState = message.getTypedFromState(); + State toState = message.getTypedToState(); + String transition = fromState.toString() + "-" + toState.toString(); int priority = Integer.MAX_VALUE; if (stateTransitionPriorities.containsKey(transition)) { @@ -187,8 +212,8 @@ List selectMessages(Map liveInstances, // select messages for (List messageList : messagesGroupByStateTransitPriority.values()) { for (Message message : messageList) { - String fromState = message.getFromState(); - String toState = message.getToState(); + State fromState = message.getTypedFromState(); + State toState = message.getTypedToState(); if (!bounds.containsKey(fromState)) { LOG.error("Message's fromState is not in currentState. message: " + message); @@ -234,22 +259,35 @@ List selectMessages(Map liveInstances, * TODO: This code is duplicate in multiple places. Can we do it in to one place in the * beginning and compute the stateConstraint instance once and re use at other places. * Each IdealState must have a constraint object associated with it + * @param stateModelDefinition + * @param rebalancerConfig if rebalancerConfig == null, we can't evaluate R thus no constraints + * @param cluster + * @return */ - private Map computeStateConstraints(StateModelDefinition stateModelDefinition, - IdealState idealState, ClusterDataCache cache) { - Map stateConstraints = new HashMap(); - - List statePriorityList = stateModelDefinition.getStatesPriorityList(); - for (String state : statePriorityList) { - String numInstancesPerState = stateModelDefinition.getNumInstancesPerState(state); + private Map computeStateConstraints(StateModelDefinition stateModelDefinition, + RebalancerConfig rebalancerConfig, Cluster cluster) { + ReplicatedRebalancerContext context = + (rebalancerConfig != null) ? rebalancerConfig + .getRebalancerContext(ReplicatedRebalancerContext.class) : null; + Map stateConstraints = new HashMap(); + + List statePriorityList = stateModelDefinition.getTypedStatesPriorityList(); + for (State state : statePriorityList) { + String numInstancesPerState = + cluster.getStateUpperBoundConstraint(Scope.cluster(cluster.getId()), + stateModelDefinition.getStateModelDefId(), state); int max = -1; if ("N".equals(numInstancesPerState)) { - max = cache.getLiveInstances().size(); + max = cluster.getLiveParticipantMap().size(); } else if ("R".equals(numInstancesPerState)) { // idealState is null when resource has been dropped, // R can't be evaluated and ignore state constraints - if (idealState != null) { - max = cache.getReplicas(idealState.getResourceName()); + if (context != null) { + if (context.anyLiveParticipant()) { + max = cluster.getLiveParticipantMap().size(); + } else { + max = context.getReplicaCount(); + } } } else { try { diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStageOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStageOutput.java deleted file mode 100644 index 7ea545cc9a..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStageOutput.java +++ /dev/null @@ -1,59 +0,0 @@ -package org.apache.helix.controller.stages; - -/* - * 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.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; - -public class MessageSelectionStageOutput { - private final Map>> _messagesMap; - - public MessageSelectionStageOutput() { - _messagesMap = new HashMap>>(); - } - - public void addMessages(String resourceName, Partition partition, List selectedMessages) { - if (!_messagesMap.containsKey(resourceName)) { - _messagesMap.put(resourceName, new HashMap>()); - } - _messagesMap.get(resourceName).put(partition, selectedMessages); - - } - - public List getMessages(String resourceName, Partition partition) { - Map> map = _messagesMap.get(resourceName); - if (map != null) { - return map.get(partition); - } - return Collections.emptyList(); - - } - - @Override - public String toString() { - return _messagesMap.toString(); - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java index 9eea7f1d3b..764b422dd0 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java @@ -26,16 +26,20 @@ import java.util.Map; import java.util.Set; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; import org.apache.helix.controller.pipeline.AbstractBaseStage; import org.apache.helix.controller.pipeline.StageException; import org.apache.helix.model.ClusterConstraints; -import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; import org.apache.helix.model.ClusterConstraints.ConstraintAttribute; -import org.apache.helix.model.ConstraintItem; import org.apache.helix.model.ClusterConstraints.ConstraintType; import org.apache.helix.model.ClusterConstraints.ConstraintValue; +import org.apache.helix.model.ConstraintItem; +import org.apache.helix.model.Message; import org.apache.log4j.Logger; public class MessageThrottleStage extends AbstractBaseStage { @@ -112,39 +116,46 @@ Set selectConstraints(Set items, @Override public void process(ClusterEvent event) throws Exception { - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); - MessageSelectionStageOutput msgSelectionOutput = + Cluster cluster = event.getAttribute("ClusterDataCache"); + MessageOutput msgSelectionOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); - Map resourceMap = event.getAttribute(AttributeName.RESOURCES.toString()); + Map resourceMap = + event.getAttribute(AttributeName.RESOURCES.toString()); + BestPossibleStateOutput bestPossibleStateOutput = + event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); - if (cache == null || resourceMap == null || msgSelectionOutput == null) { + if (cluster == null || resourceMap == null || msgSelectionOutput == null + || bestPossibleStateOutput == null) { throw new StageException("Missing attributes in event: " + event - + ". Requires ClusterDataCache|RESOURCES|MESSAGES_SELECTED"); + + ". Requires ClusterDataCache|RESOURCES|BEST_POSSIBLE_STATE|MESSAGES_SELECTED"); } - MessageThrottleStageOutput output = new MessageThrottleStageOutput(); + MessageOutput output = new MessageOutput(); - ClusterConstraints constraint = cache.getConstraint(ConstraintType.MESSAGE_CONSTRAINT); + // TODO fix it + ClusterConstraints constraint = cluster.getConstraint(ConstraintType.MESSAGE_CONSTRAINT); Map throttleCounterMap = new HashMap(); if (constraint != null) { // go through all pending messages, they should be counted but not throttled - for (String instance : cache.getLiveInstances().keySet()) { - throttle(throttleCounterMap, constraint, new ArrayList(cache.getMessages(instance) - .values()), false); + for (ParticipantId participantId : cluster.getLiveParticipantMap().keySet()) { + Participant liveParticipant = cluster.getLiveParticipantMap().get(participantId); + throttle(throttleCounterMap, constraint, new ArrayList(liveParticipant + .getMessageMap().values()), false); } } // go through all new messages, throttle if necessary // assume messages should be sorted by state transition priority in messageSelection stage - for (String resourceName : resourceMap.keySet()) { - Resource resource = resourceMap.get(resourceName); - for (Partition partition : resource.getPartitions()) { - List messages = msgSelectionOutput.getMessages(resourceName, partition); + for (ResourceId resourceId : resourceMap.keySet()) { + // TODO fix it + for (PartitionId partitionId : bestPossibleStateOutput.getResourceAssignment(resourceId) + .getMappedPartitionIds()) { + List messages = msgSelectionOutput.getMessages(resourceId, partitionId); if (constraint != null && messages != null && messages.size() > 0) { messages = throttle(throttleCounterMap, constraint, messages, true); } - output.addMessages(resourceName, partition, messages); + output.setMessages(resourceId, partitionId, messages); } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStageOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStageOutput.java deleted file mode 100644 index 74159445d8..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStageOutput.java +++ /dev/null @@ -1,52 +0,0 @@ -package org.apache.helix.controller.stages; - -/* - * 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.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; - -public class MessageThrottleStageOutput { - private final Map>> _messagesMap; - - public MessageThrottleStageOutput() { - _messagesMap = new HashMap>>(); - } - - public void addMessages(String resourceName, Partition partition, List selectedMessages) { - if (!_messagesMap.containsKey(resourceName)) { - _messagesMap.put(resourceName, new HashMap>()); - } - _messagesMap.get(resourceName).put(partition, selectedMessages); - - } - - public List getMessages(String resourceName, Partition partition) { - Map> map = _messagesMap.get(resourceName); - if (map != null) { - return map.get(partition); - } - return Collections.emptyList(); - } -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/PersistAssignmentStage.java similarity index 55% rename from helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java rename to helix-core/src/main/java/org/apache/helix/controller/stages/PersistAssignmentStage.java index ae873c70f2..31dbb08c1a 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/PersistAssignmentStage.java @@ -1,5 +1,12 @@ package org.apache.helix.controller.stages; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixManager; +import org.apache.helix.api.accessor.ResourceAccessor; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.pipeline.AbstractBaseStage; +import org.apache.helix.model.ResourceAssignment; + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -19,35 +26,20 @@ * under the License. */ -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.controller.pipeline.AbstractBaseStage; -import org.apache.helix.controller.pipeline.StageException; -import org.apache.log4j.Logger; - -public class ReadHealthDataStage extends AbstractBaseStage { - private static final Logger LOG = Logger.getLogger(ReadHealthDataStage.class.getName()); - HealthDataCache _cache; - - public ReadHealthDataStage() { - _cache = new HealthDataCache(); - } - +/** + * Persist the ResourceAssignment of each resource that went through rebalancing + */ +public class PersistAssignmentStage extends AbstractBaseStage { @Override public void process(ClusterEvent event) throws Exception { - long startTime = System.currentTimeMillis(); - - HelixManager manager = event.getAttribute("helixmanager"); - if (manager == null) { - throw new StageException("HelixManager attribute value is null"); + HelixManager helixManager = event.getAttribute("helixmanager"); + HelixDataAccessor accessor = helixManager.getHelixDataAccessor(); + ResourceAccessor resourceAccessor = new ResourceAccessor(accessor); + BestPossibleStateOutput assignments = + event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); + for (ResourceId resourceId : assignments.getAssignedResources()) { + ResourceAssignment assignment = assignments.getResourceAssignment(resourceId); + resourceAccessor.setResourceAssignment(resourceId, assignment); } - // DataAccessor dataAccessor = manager.getDataAccessor(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - _cache.refresh(accessor); - - event.addAttribute("HealthDataCache", _cache); - - long processLatency = System.currentTimeMillis() - startTime; - addLatencyToMonitor(event, processLatency); } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java index f077d29908..44fddb6e76 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java @@ -21,52 +21,53 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.id.ClusterId; import org.apache.helix.controller.pipeline.AbstractBaseStage; import org.apache.helix.controller.pipeline.StageException; -import org.apache.helix.model.InstanceConfig; import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor; import org.apache.log4j.Logger; public class ReadClusterDataStage extends AbstractBaseStage { - private static final Logger logger = Logger.getLogger(ReadClusterDataStage.class.getName()); - ClusterDataCache _cache; - - public ReadClusterDataStage() { - _cache = new ClusterDataCache(); - } + private static final Logger LOG = Logger.getLogger(ReadClusterDataStage.class.getName()); @Override public void process(ClusterEvent event) throws Exception { long startTime = System.currentTimeMillis(); - logger.info("START ReadClusterDataStage.process()"); + LOG.info("START ReadClusterDataStage.process()"); HelixManager manager = event.getAttribute("helixmanager"); if (manager == null) { throw new StageException("HelixManager attribute value is null"); } - HelixDataAccessor dataAccessor = manager.getHelixDataAccessor(); - _cache.refresh(dataAccessor); + HelixDataAccessor accessor = manager.getHelixDataAccessor(); + ClusterId clusterId = ClusterId.from(manager.getClusterName()); + ClusterAccessor clusterAccessor = new ClusterAccessor(clusterId, accessor); + + Cluster cluster = clusterAccessor.readCluster(); ClusterStatusMonitor clusterStatusMonitor = (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor"); if (clusterStatusMonitor != null) { - int disabledInstances = 0; - int disabledPartitions = 0; - for (InstanceConfig config : _cache._instanceConfigMap.values()) { - if (config.getInstanceEnabled() == false) { - disabledInstances++; - } - if (config.getDisabledPartitions() != null) { - disabledPartitions += config.getDisabledPartitions().size(); - } - } - clusterStatusMonitor.setClusterStatusCounters(_cache._liveInstanceMap.size(), - _cache._instanceConfigMap.size(), disabledInstances, disabledPartitions); + // TODO fix it + // int disabledInstances = 0; + // int disabledPartitions = 0; + // for (InstanceConfig config : _cache._instanceConfigMap.values()) { + // if (config.getInstanceEnabled() == false) { + // disabledInstances++; + // } + // if (config.getDisabledPartitions() != null) { + // disabledPartitions += config.getDisabledPartitions().size(); + // } + // } + // clusterStatusMonitor.setClusterStatusCounters(_cache._liveInstanceMap.size(), + // _cache._instanceConfigMap.size(), disabledInstances, disabledPartitions); } - event.addAttribute("ClusterDataCache", _cache); + event.addAttribute("ClusterDataCache", cluster); long endTime = System.currentTimeMillis(); - logger.info("END ReadClusterDataStage.process(). took: " + (endTime - startTime) + " ms"); + LOG.info("END ReadClusterDataStage.process(). took: " + (endTime - startTime) + " ms"); } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/RebalanceIdealStateStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/RebalanceIdealStateStage.java deleted file mode 100644 index cf1633cd49..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/RebalanceIdealStateStage.java +++ /dev/null @@ -1,73 +0,0 @@ -package org.apache.helix.controller.stages; - -/* - * 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.apache.helix.HelixManager; -import org.apache.helix.controller.pipeline.AbstractBaseStage; -import org.apache.helix.controller.rebalancer.Rebalancer; -import org.apache.helix.model.IdealState; -import org.apache.helix.model.IdealState.RebalanceMode; -import org.apache.helix.util.HelixUtil; -import org.apache.log4j.Logger; - -/** - * Check and invoke custom implementation idealstate rebalancers.
- * If the resourceConfig has specified className of the customized rebalancer,
- * the rebalancer will be invoked to re-write the idealstate of the resource
- */ -public class RebalanceIdealStateStage extends AbstractBaseStage { - private static final Logger LOG = Logger.getLogger(RebalanceIdealStateStage.class.getName()); - - @Override - public void process(ClusterEvent event) throws Exception { - HelixManager manager = event.getAttribute("helixmanager"); - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); - Map idealStateMap = cache.getIdealStates(); - CurrentStateOutput currentStateOutput = - event.getAttribute(AttributeName.CURRENT_STATE.toString()); - - Map updatedIdealStates = new HashMap(); - for (String resourceName : idealStateMap.keySet()) { - IdealState currentIdealState = idealStateMap.get(resourceName); - if (currentIdealState.getRebalanceMode() == RebalanceMode.USER_DEFINED - && currentIdealState.getRebalancerClassName() != null) { - String rebalancerClassName = currentIdealState.getRebalancerClassName(); - LOG.info("resource " + resourceName + " use idealStateRebalancer " + rebalancerClassName); - try { - Rebalancer balancer = - (Rebalancer) (HelixUtil.loadClass(getClass(), rebalancerClassName).newInstance()); - balancer.init(manager); - IdealState newIdealState = - balancer.computeNewIdealState(resourceName, idealStateMap.get(resourceName), - currentStateOutput, cache); - updatedIdealStates.put(resourceName, newIdealState); - } catch (Exception e) { - LOG.error("Exception while invoking custom rebalancer class:" + rebalancerClassName, e); - } - } - } - if (updatedIdealStates.size() > 0) { - cache.getIdealStates().putAll(updatedIdealStates); - } - } -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java index 51f0ec1cdf..1fdd892213 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java @@ -19,16 +19,23 @@ * under the License. */ -import java.util.LinkedHashMap; +import java.util.HashMap; import java.util.Map; -import java.util.Set; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Partition; +import org.apache.helix.api.Resource; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelFactoryId; import org.apache.helix.controller.pipeline.AbstractBaseStage; import org.apache.helix.controller.pipeline.StageException; +import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; import org.apache.helix.model.CurrentState; -import org.apache.helix.model.IdealState; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.Resource; import org.apache.log4j.Logger; /** @@ -41,97 +48,91 @@ public class ResourceComputationStage extends AbstractBaseStage { private static Logger LOG = Logger.getLogger(ResourceComputationStage.class); @Override - public void process(ClusterEvent event) throws Exception { - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); - if (cache == null) { - throw new StageException("Missing attributes in event:" + event + ". Requires DataCache"); + public void process(ClusterEvent event) throws StageException { + Cluster cluster = event.getAttribute("ClusterDataCache"); + if (cluster == null) { + throw new StageException("Missing attributes in event: " + event + ". Requires Cluster"); } - Map idealStates = cache.getIdealStates(); + Map resCfgMap = new HashMap(); + Map csResCfgMap = getCurStateResourceCfgMap(cluster); - Map resourceMap = new LinkedHashMap(); + // ideal-state may be removed, add all resource config in current-state but not in ideal-state + for (ResourceId resourceId : csResCfgMap.keySet()) { + if (!cluster.getResourceMap().keySet().contains(resourceId)) { + resCfgMap.put(resourceId, csResCfgMap.get(resourceId)); + } + } - if (idealStates != null && idealStates.size() > 0) { - for (IdealState idealState : idealStates.values()) { - Set partitionSet = idealState.getPartitionSet(); - String resourceName = idealState.getResourceName(); + for (ResourceId resourceId : cluster.getResourceMap().keySet()) { + Resource resource = cluster.getResource(resourceId); + RebalancerConfig rebalancerCfg = resource.getRebalancerConfig(); - for (String partition : partitionSet) { - addPartition(partition, resourceName, resourceMap); - Resource resource = resourceMap.get(resourceName); - resource.setStateModelDefRef(idealState.getStateModelDefRef()); - resource.setStateModelFactoryName(idealState.getStateModelFactoryName()); - resource.setBucketSize(idealState.getBucketSize()); - resource.setBatchMessageMode(idealState.getBatchMessageMode()); - } - } + ResourceConfig.Builder resCfgBuilder = new ResourceConfig.Builder(resourceId); + resCfgBuilder.bucketSize(resource.getBucketSize()); + resCfgBuilder.batchMessageMode(resource.getBatchMessageMode()); + resCfgBuilder.schedulerTaskConfig(resource.getSchedulerTaskConfig()); + resCfgBuilder.rebalancerContext(rebalancerCfg.getRebalancerContext(RebalancerContext.class)); + resCfgMap.put(resourceId, resCfgBuilder.build()); } - // It's important to get partitions from CurrentState as well since the - // idealState might be removed. - Map availableInstances = cache.getLiveInstances(); + event.addAttribute(AttributeName.RESOURCES.toString(), resCfgMap); + } - if (availableInstances != null && availableInstances.size() > 0) { - for (LiveInstance instance : availableInstances.values()) { - String instanceName = instance.getInstanceName(); - String clientSessionId = instance.getSessionId(); + /** + * Get resource config's from current-state + * @param cluster + * @return resource config map or empty map if not available + * @throws StageException + */ + Map getCurStateResourceCfgMap(Cluster cluster) throws StageException { + Map resCfgBuilderMap = + new HashMap(); + + Map rebCtxBuilderMap = + new HashMap(); + + for (Participant liveParticipant : cluster.getLiveParticipantMap().values()) { + for (ResourceId resourceId : liveParticipant.getCurrentStateMap().keySet()) { + CurrentState currentState = liveParticipant.getCurrentStateMap().get(resourceId); + + if (currentState.getStateModelDefRef() == null) { + LOG.error("state model def is null." + "resource:" + currentState.getResourceId() + + ", partitions: " + currentState.getPartitionStateMap().keySet() + + ", states: " + currentState.getPartitionStateMap().values()); + throw new StageException("State model def is null for resource:" + + currentState.getResourceId()); + } - Map currentStateMap = - cache.getCurrentState(instanceName, clientSessionId); - if (currentStateMap == null || currentStateMap.size() == 0) { - continue; + if (!resCfgBuilderMap.containsKey(resourceId)) { + PartitionedRebalancerContext.Builder rebCtxBuilder = + new PartitionedRebalancerContext.Builder(resourceId); + rebCtxBuilder.stateModelDefId(currentState.getStateModelDefId()); + rebCtxBuilder.stateModelFactoryId(StateModelFactoryId.from(currentState + .getStateModelFactoryName())); + rebCtxBuilderMap.put(resourceId, rebCtxBuilder); + + ResourceConfig.Builder resCfgBuilder = new ResourceConfig.Builder(resourceId); + resCfgBuilder.bucketSize(currentState.getBucketSize()); + resCfgBuilder.batchMessageMode(currentState.getBatchMessageMode()); + resCfgBuilderMap.put(resourceId, resCfgBuilder); } - for (CurrentState currentState : currentStateMap.values()) { - - String resourceName = currentState.getResourceName(); - Map resourceStateMap = currentState.getPartitionStateMap(); - - // don't overwrite ideal state settings - if (!resourceMap.containsKey(resourceName)) { - addResource(resourceName, resourceMap); - Resource resource = resourceMap.get(resourceName); - resource.setStateModelDefRef(currentState.getStateModelDefRef()); - resource.setStateModelFactoryName(currentState.getStateModelFactoryName()); - resource.setBucketSize(currentState.getBucketSize()); - resource.setBatchMessageMode(currentState.getBatchMessageMode()); - } - - if (currentState.getStateModelDefRef() == null) { - LOG.error("state model def is null." + "resource:" + currentState.getResourceName() - + ", partitions: " + currentState.getPartitionStateMap().keySet() + ", states: " - + currentState.getPartitionStateMap().values()); - throw new StageException("State model def is null for resource:" - + currentState.getResourceName()); - } - - for (String partition : resourceStateMap.keySet()) { - addPartition(partition, resourceName, resourceMap); - } + + PartitionedRebalancerContext.Builder rebCtxBuilder = rebCtxBuilderMap.get(resourceId); + for (PartitionId partitionId : currentState.getTypedPartitionStateMap().keySet()) { + rebCtxBuilder.addPartition(new Partition(partitionId)); } } } - event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap); - } - - private void addResource(String resource, Map resourceMap) { - if (resource == null || resourceMap == null) { - return; - } - if (!resourceMap.containsKey(resource)) { - resourceMap.put(resource, new Resource(resource)); - } - } - - private void addPartition(String partition, String resourceName, Map resourceMap) { - if (resourceName == null || partition == null || resourceMap == null) { - return; - } - if (!resourceMap.containsKey(resourceName)) { - resourceMap.put(resourceName, new Resource(resourceName)); + Map resCfgMap = new HashMap(); + for (ResourceId resourceId : resCfgBuilderMap.keySet()) { + ResourceConfig.Builder resCfgBuilder = resCfgBuilderMap.get(resourceId); + PartitionedRebalancerContext.Builder rebCtxBuilder = rebCtxBuilderMap.get(resourceId); + resCfgBuilder.rebalancerContext(rebCtxBuilder.build()); + resCfgMap.put(resourceId, resCfgBuilder.build()); } - Resource resource = resourceMap.get(resourceName); - resource.addPartition(partition); + return resCfgMap; } } diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java new file mode 100644 index 0000000000..f04afd0998 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java @@ -0,0 +1,273 @@ +package org.apache.helix.controller.stages; + +/* + * 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.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.model.CurrentState; + +import com.google.common.collect.Sets; + +public class ResourceCurrentState { + /** + * map of resource-id to map of partition-id to map of participant-id to state + * represent current-state for the participant + */ + private final Map>> _currentStateMap; + + /** + * map of resource-id to map of partition-id to map of participant-id to state + * represent pending messages for the participant + */ + private final Map>> _pendingStateMap; + + /** + * map of resource-id to state model definition id + */ + private final Map _resourceStateModelMap; + + /** + * map of resource-id to current-state config's + */ + private final Map _curStateMetaMap; + + /** + * construct + */ + public ResourceCurrentState() { + _currentStateMap = new HashMap>>(); + _pendingStateMap = new HashMap>>(); + _resourceStateModelMap = new HashMap(); + _curStateMetaMap = new HashMap(); + + } + + /** + * Get all the resources seen in the aggregated current state + * @return set of ResourceId + */ + public Set getResourceIds() { + Set allResources = Sets.newHashSet(); + allResources.addAll(_currentStateMap.keySet()); + allResources.addAll(_pendingStateMap.keySet()); + return allResources; + } + + /** + * @param resourceId + * @param stateModelDefId + */ + public void setResourceStateModelDef(ResourceId resourceId, StateModelDefId stateModelDefId) { + _resourceStateModelMap.put(resourceId, stateModelDefId); + } + + /** + * @param resourceId + * @return + */ + public StateModelDefId getResourceStateModelDef(ResourceId resourceId) { + return _resourceStateModelMap.get(resourceId); + } + + /** + * @param resourceId + * @param bucketSize + */ + public void setBucketSize(ResourceId resourceId, int bucketSize) { + CurrentState curStateMeta = _curStateMetaMap.get(resourceId); + if (curStateMeta == null) { + curStateMeta = new CurrentState(resourceId); + _curStateMetaMap.put(resourceId, curStateMeta); + } + curStateMeta.setBucketSize(bucketSize); + } + + /** + * @param resourceId + * @return + */ + public int getBucketSize(ResourceId resourceId) { + int bucketSize = 0; + CurrentState curStateMeta = _curStateMetaMap.get(resourceId); + if (curStateMeta != null) { + bucketSize = curStateMeta.getBucketSize(); + } + + return bucketSize; + } + + /** + * @param stateMap + * @param resourceId + * @param partitionId + * @param participantId + * @param state + */ + static void setStateMap(Map>> stateMap, + ResourceId resourceId, PartitionId partitionId, ParticipantId participantId, State state) { + if (!stateMap.containsKey(resourceId)) { + stateMap.put(resourceId, new HashMap>()); + } + + if (!stateMap.get(resourceId).containsKey(partitionId)) { + stateMap.get(resourceId).put(partitionId, new HashMap()); + } + stateMap.get(resourceId).get(partitionId).put(participantId, state); + } + + /** + * @param stateMap + * @param resourceId + * @param partitionId + * @param participantId + * @return state + */ + static State getState(Map>> stateMap, + ResourceId resourceId, PartitionId partitionId, ParticipantId participantId) { + Map> map = stateMap.get(resourceId); + if (map != null) { + Map instanceStateMap = map.get(partitionId); + if (instanceStateMap != null) { + return instanceStateMap.get(participantId); + } + } + return null; + + } + + /** + * @param stateMap + * @param resourceId + * @param partitionId + * @return + */ + static Map getStateMap( + Map>> stateMap, ResourceId resourceId, + PartitionId partitionId) { + if (stateMap.containsKey(resourceId)) { + Map> map = stateMap.get(resourceId); + if (map.containsKey(partitionId)) { + return map.get(partitionId); + } + } + return Collections.emptyMap(); + } + + /** + * @param resourceId + * @param partitionId + * @param participantId + * @param state + */ + public void setCurrentState(ResourceId resourceId, PartitionId partitionId, + ParticipantId participantId, State state) { + setStateMap(_currentStateMap, resourceId, partitionId, participantId, state); + } + + /** + * @param resourceId + * @param partitionId + * @param participantId + * @param state + */ + public void setPendingState(ResourceId resourceId, PartitionId partitionId, + ParticipantId participantId, State state) { + setStateMap(_pendingStateMap, resourceId, partitionId, participantId, state); + } + + /** + * given (resource, partition, instance), returns currentState + * @param resourceName + * @param partition + * @param instanceName + * @return + */ + public State getCurrentState(ResourceId resourceId, PartitionId partitionId, + ParticipantId participantId) { + return getState(_currentStateMap, resourceId, partitionId, participantId); + } + + /** + * given (resource, partition, instance), returns toState + * @param resourceName + * @param partition + * @param instanceName + * @return + */ + public State getPendingState(ResourceId resourceId, PartitionId partitionId, + ParticipantId participantId) { + return getState(_pendingStateMap, resourceId, partitionId, participantId); + } + + /** + * @param resourceId + * @param partitionId + * @return + */ + public Map getCurrentStateMap(ResourceId resourceId, PartitionId partitionId) { + return getStateMap(_currentStateMap, resourceId, partitionId); + } + + /** + * Get the partitions mapped in the current state + * @param resourceId resource to look up + * @return set of mapped partitions, or empty set if there are none + */ + public Set getCurrentStateMappedPartitions(ResourceId resourceId) { + Map> currentStateMap = _currentStateMap.get(resourceId); + Map> pendingStateMap = _pendingStateMap.get(resourceId); + Set partitionSet = Sets.newHashSet(); + if (currentStateMap != null) { + partitionSet.addAll(currentStateMap.keySet()); + } + if (pendingStateMap != null) { + partitionSet.addAll(pendingStateMap.keySet()); + } + return partitionSet; + } + + /** + * @param resourceId + * @param partitionId + * @return + */ + public Map getPendingStateMap(ResourceId resourceId, PartitionId partitionId) { + return getStateMap(_pendingStateMap, resourceId, partitionId); + + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("current state= ").append(_currentStateMap); + sb.append(", pending state= ").append(_pendingStateMap); + return sb.toString(); + + } + +} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceMapping.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceMapping.java deleted file mode 100644 index 2609791846..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceMapping.java +++ /dev/null @@ -1,58 +0,0 @@ -package org.apache.helix.controller.stages; - -/* - * 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.Collections; -import java.util.HashMap; -import java.util.Map; - -import org.apache.helix.model.Partition; - -/** - * Represents the assignments of replicas for an entire resource, keyed on partitions of the - * resource. Each partition has its replicas assigned to a node, and each replica is in a state. - */ -public class ResourceMapping { - - private final Map> _resourceMap; - - public ResourceMapping() { - this(new HashMap>()); - } - - public ResourceMapping(Map> resourceMap) { - _resourceMap = resourceMap; - } - - public Map> getResourceMap() { - return _resourceMap; - } - - public Map getInstanceStateMap(Partition partition) { - if (_resourceMap.containsKey(partition)) { - return _resourceMap.get(partition); - } - return Collections.emptyMap(); - } - - public void addReplicaMap(Partition partition, Map replicaMap) { - _resourceMap.put(partition, replicaMap); - } -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java deleted file mode 100644 index e531c88494..0000000000 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java +++ /dev/null @@ -1,399 +0,0 @@ -package org.apache.helix.controller.stages; - -/* - * 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.text.SimpleDateFormat; -import java.util.Arrays; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.HelixProperty; -import org.apache.helix.PropertyType; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.alerts.AlertParser; -import org.apache.helix.alerts.AlertProcessor; -import org.apache.helix.alerts.AlertValueAndStatus; -import org.apache.helix.alerts.AlertsHolder; -import org.apache.helix.alerts.ExpressionParser; -import org.apache.helix.alerts.StatsHolder; -import org.apache.helix.alerts.Tuple; -import org.apache.helix.controller.pipeline.AbstractBaseStage; -import org.apache.helix.controller.pipeline.StageContext; -import org.apache.helix.controller.pipeline.StageException; -import org.apache.helix.healthcheck.StatHealthReportProvider; -import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory.ActionOnError; -import org.apache.helix.model.AlertHistory; -import org.apache.helix.model.HealthStat; -import org.apache.helix.model.IdealState; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.PersistentStats; -import org.apache.helix.monitoring.mbeans.ClusterAlertMBeanCollection; -import org.apache.log4j.Logger; - -/** - * For each LiveInstances select currentState and message whose sessionId matches - * sessionId from LiveInstance Get Partition,State for all the resources computed in - * previous State [ResourceComputationStage] - */ -public class StatsAggregationStage extends AbstractBaseStage { - - public static final int ALERT_HISTORY_SIZE = 30; - - private static final Logger logger = Logger.getLogger(StatsAggregationStage.class.getName()); - - StatsHolder _statsHolder = null; - AlertsHolder _alertsHolder = null; - Map> _alertStatus; - Map> _statStatus; - ClusterAlertMBeanCollection _alertBeanCollection = new ClusterAlertMBeanCollection(); - Map _alertActionTaken = new HashMap(); - - public final String PARTICIPANT_STAT_REPORT_NAME = StatHealthReportProvider.REPORT_NAME; - public final String ESPRESSO_STAT_REPORT_NAME = "RestQueryStats"; - public final String REPORT_NAME = "AggStats"; - // public final String DEFAULT_AGG_TYPE = "decay"; - // public final String DEFAULT_DECAY_PARAM = "0.1"; - // public final String DEFAULT_AGG_TYPE = "window"; - // public final String DEFAULT_DECAY_PARAM = "5"; - - public StatHealthReportProvider _aggStatsProvider; - - // public AggregationType _defaultAggType; - - public Map> getAlertStatus() { - return _alertStatus; - } - - public Map> getStatStatus() { - return _statStatus; - } - - public void persistAggStats(HelixManager manager) { - Map report = _aggStatsProvider.getRecentHealthReport(); - Map> partitionReport = - _aggStatsProvider.getRecentPartitionHealthReport(); - ZNRecord record = new ZNRecord(_aggStatsProvider.getReportName()); - if (report != null) { - record.setSimpleFields(report); - } - if (partitionReport != null) { - record.setMapFields(partitionReport); - } - - // DataAccessor accessor = manager.getDataAccessor(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - // boolean retVal = accessor.setProperty(PropertyType.PERSISTENTSTATS, record); - Builder keyBuilder = accessor.keyBuilder(); - boolean retVal = accessor.setProperty(keyBuilder.persistantStat(), new PersistentStats(record)); - if (retVal == false) { - logger.error("attempt to persist derived stats failed"); - } - } - - @Override - public void init(StageContext context) { - } - - public String getAgeStatName(String instance) { - return instance + ExpressionParser.statFieldDelim + "reportingage"; - } - - // currTime in seconds - public void reportAgeStat(LiveInstance instance, long modifiedTime, long currTime) { - String statName = getAgeStatName(instance.getInstanceName()); - long age = (currTime - modifiedTime) / 1000; // XXX: ensure this is in - // seconds - Map ageStatMap = new HashMap(); - ageStatMap.put(StatsHolder.TIMESTAMP_NAME, String.valueOf(currTime)); - ageStatMap.put(StatsHolder.VALUE_NAME, String.valueOf(age)); - // note that applyStat will only work if alert already added - _statsHolder.applyStat(statName, ageStatMap); - } - - @Override - public void process(ClusterEvent event) throws Exception { - long startTime = System.currentTimeMillis(); - // String aggTypeName = - // DEFAULT_AGG_TYPE+AggregationType.DELIM+DEFAULT_DECAY_PARAM; - // _defaultAggType = AggregationTypeFactory.getAggregationType(aggTypeName); - - HelixManager manager = event.getAttribute("helixmanager"); - HealthDataCache cache = event.getAttribute("HealthDataCache"); - - if (manager == null || cache == null) { - throw new StageException("helixmanager|HealthDataCache attribute value is null"); - } - if (_alertsHolder == null) { - _statsHolder = new StatsHolder(manager, cache); - _alertsHolder = new AlertsHolder(manager, cache, _statsHolder); - } else { - _statsHolder.updateCache(cache); - _alertsHolder.updateCache(cache); - } - if (_statsHolder.getStatsList().size() == 0) { - if (logger.isTraceEnabled()) { - logger.trace("stat holder is empty"); - } - return; - } - - // init agg stats from cache - // initAggStats(cache); - - Map liveInstances = cache.getLiveInstances(); - - long currTime = System.currentTimeMillis(); - // for each live node, read node's stats - long readInstancesStart = System.currentTimeMillis(); - for (LiveInstance instance : liveInstances.values()) { - String instanceName = instance.getInstanceName(); - logger.debug("instanceName: " + instanceName); - // XXX: now have map of HealthStats, so no need to traverse them...verify - // correctness - Map stats; - stats = cache.getHealthStats(instanceName); - // find participants stats - long modTime = -1; - // TODO: get healthreport child node modified time and reportAgeStat based on that - boolean reportedAge = false; - for (HealthStat participantStat : stats.values()) { - if (participantStat != null && !reportedAge) { - // generate and report stats for how old this node's report is - modTime = participantStat.getLastModifiedTimeStamp(); - reportAgeStat(instance, modTime, currTime); - reportedAge = true; - } - // System.out.println(modTime); - // XXX: need to convert participantStat to a better format - // need to get instanceName in here - - if (participantStat != null) { - // String timestamp = String.valueOf(instance.getModifiedTime()); WANT - // REPORT LEVEL TS - Map> statMap = participantStat.getHealthFields(instanceName); - for (String key : statMap.keySet()) { - _statsHolder.applyStat(key, statMap.get(key)); - } - } - } - } - // Call _statsHolder.persistStats() once per pipeline. This will - // write the updated persisted stats into zookeeper - _statsHolder.persistStats(); - logger.info("Done processing stats: " + (System.currentTimeMillis() - readInstancesStart)); - // populate _statStatus - _statStatus = _statsHolder.getStatsMap(); - - for (String statKey : _statStatus.keySet()) { - logger.debug("Stat key, value: " + statKey + ": " + _statStatus.get(statKey)); - } - - long alertExecuteStartTime = System.currentTimeMillis(); - // execute alerts, populate _alertStatus - _alertStatus = - AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - logger.info("done executing alerts: " + (System.currentTimeMillis() - alertExecuteStartTime)); - for (String originAlertName : _alertStatus.keySet()) { - _alertBeanCollection.setAlerts(originAlertName, _alertStatus.get(originAlertName), - manager.getClusterName()); - } - - executeAlertActions(manager); - // Write alert fire history to zookeeper - updateAlertHistory(manager); - long writeAlertStartTime = System.currentTimeMillis(); - // write out alert status (to zk) - _alertsHolder.addAlertStatusSet(_alertStatus); - logger.info("done writing alerts: " + (System.currentTimeMillis() - writeAlertStartTime)); - - // TODO: access the 2 status variables from somewhere to populate graphs - - long logAlertStartTime = System.currentTimeMillis(); - // logging alert status - for (String alertOuterKey : _alertStatus.keySet()) { - logger.debug("Alert Outer Key: " + alertOuterKey); - Map alertInnerMap = _alertStatus.get(alertOuterKey); - if (alertInnerMap == null) { - logger.debug(alertOuterKey + " has no alerts to report."); - continue; - } - for (String alertInnerKey : alertInnerMap.keySet()) { - logger.debug(" " + alertInnerKey + " value: " - + alertInnerMap.get(alertInnerKey).getValue() + ", status: " - + alertInnerMap.get(alertInnerKey).isFired()); - } - } - - logger.info("done logging alerts: " + (System.currentTimeMillis() - logAlertStartTime)); - - long processLatency = System.currentTimeMillis() - startTime; - addLatencyToMonitor(event, processLatency); - logger.info("process end: " + processLatency); - } - - /** - * Go through the _alertStatus, and call executeAlertAction for those actual alerts that - * has been fired - */ - - void executeAlertActions(HelixManager manager) { - _alertActionTaken.clear(); - // Go through the original alert strings - for (String originAlertName : _alertStatus.keySet()) { - Map alertFields = _alertsHolder.getAlertsMap().get(originAlertName); - if (alertFields != null && alertFields.containsKey(AlertParser.ACTION_NAME)) { - String actionValue = alertFields.get(AlertParser.ACTION_NAME); - Map alertResultMap = _alertStatus.get(originAlertName); - if (alertResultMap == null) { - logger.info("Alert " + originAlertName + " does not have alert status map"); - continue; - } - // For each original alert, iterate all actual alerts that it expands into - for (String actualStatName : alertResultMap.keySet()) { - // if the actual alert is fired, execute the action - if (alertResultMap.get(actualStatName).isFired()) { - logger.warn("Alert " + originAlertName + " action " + actionValue + " is triggered by " - + actualStatName); - _alertActionTaken.put(actualStatName, actionValue); - // move functionalities into a seperate class - executeAlertAction(actualStatName, actionValue, manager); - } - } - } - } - } - - /** - * Execute the action if an alert is fired, and the alert has an action associated with it. - * NOTE: consider unify this with DefaultParticipantErrorMessageHandler.handleMessage() - */ - void executeAlertAction(String actualStatName, String actionValue, HelixManager manager) { - if (actionValue.equals(ActionOnError.DISABLE_INSTANCE.toString())) { - String instanceName = parseInstanceName(actualStatName, manager); - if (instanceName != null) { - logger.info("Disabling instance " + instanceName); - manager.getClusterManagmentTool().enableInstance(manager.getClusterName(), instanceName, - false); - } - } else if (actionValue.equals(ActionOnError.DISABLE_PARTITION.toString())) { - String instanceName = parseInstanceName(actualStatName, manager); - String resourceName = parseResourceName(actualStatName, manager); - String partitionName = parsePartitionName(actualStatName, manager); - if (instanceName != null && resourceName != null && partitionName != null) { - logger.info("Disabling partition " + partitionName + " instanceName " + instanceName); - manager.getClusterManagmentTool().enablePartition(false, manager.getClusterName(), - instanceName, resourceName, Arrays.asList(partitionName)); - } - } else if (actionValue.equals(ActionOnError.DISABLE_RESOURCE.toString())) { - String instanceName = parseInstanceName(actualStatName, manager); - String resourceName = parseResourceName(actualStatName, manager); - logger.info("Disabling resource " + resourceName + " instanceName " + instanceName - + " not implemented"); - - } - } - - public static String parseResourceName(String actualStatName, HelixManager manager) { - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - Builder kb = accessor.keyBuilder(); - List idealStates = accessor.getChildValues(kb.idealStates()); - for (IdealState idealState : idealStates) { - String resourceName = idealState.getResourceName(); - if (actualStatName.contains("=" + resourceName + ".") - || actualStatName.contains("=" + resourceName + ";")) { - return resourceName; - } - } - return null; - } - - public static String parsePartitionName(String actualStatName, HelixManager manager) { - String resourceName = parseResourceName(actualStatName, manager); - if (resourceName != null) { - String partitionKey = "=" + resourceName + "_"; - if (actualStatName.contains(partitionKey)) { - int pos = actualStatName.indexOf(partitionKey); - int nextDotPos = actualStatName.indexOf('.', pos + partitionKey.length()); - int nextCommaPos = actualStatName.indexOf(';', pos + partitionKey.length()); - if (nextCommaPos > 0 && nextCommaPos < nextDotPos) { - nextDotPos = nextCommaPos; - } - - String partitionName = actualStatName.substring(pos + 1, nextDotPos); - return partitionName; - } - } - return null; - } - - public static String parseInstanceName(String actualStatName, HelixManager manager) { - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - Builder kb = accessor.keyBuilder(); - List liveInstances = accessor.getChildValues(kb.liveInstances()); - for (LiveInstance instance : liveInstances) { - String instanceName = instance.getInstanceName(); - if (actualStatName.startsWith(instanceName)) { - return instanceName; - } - } - return null; - } - - void updateAlertHistory(HelixManager manager) { - // Write alert fire history to zookeeper - _alertBeanCollection.refreshAlertDelta(manager.getClusterName()); - Map delta = _alertBeanCollection.getRecentAlertDelta(); - // Update history only when some beans has changed - if (delta.size() > 0) { - delta.putAll(_alertActionTaken); - SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd-hh:mm:ss:SSS"); - String date = dateFormat.format(new Date()); - - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - HelixProperty property = accessor.getProperty(keyBuilder.alertHistory()); - ZNRecord alertFiredHistory; - if (property == null) { - alertFiredHistory = new ZNRecord(PropertyType.ALERT_HISTORY.toString()); - } else { - alertFiredHistory = property.getRecord(); - } - while (alertFiredHistory.getMapFields().size() >= ALERT_HISTORY_SIZE) { - // ZNRecord uses TreeMap which is sorted ascending internally - String firstKey = (String) (alertFiredHistory.getMapFields().keySet().toArray()[0]); - alertFiredHistory.getMapFields().remove(firstKey); - } - alertFiredHistory.setMapField(date, delta); - // manager.getDataAccessor().setProperty(PropertyType.ALERT_HISTORY, alertFiredHistory); - accessor.setProperty(keyBuilder.alertHistory(), new AlertHistory(alertFiredHistory)); - _alertBeanCollection.setAlertHistory(alertFiredHistory); - } - } - - public ClusterAlertMBeanCollection getClusterAlertMBeanCollection() { - return _alertBeanCollection; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java index 192a645d02..bc2ee502d7 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java @@ -30,12 +30,15 @@ import org.apache.helix.HelixManagerProperties; import org.apache.helix.PropertyKey; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; import org.apache.helix.controller.pipeline.AbstractBaseStage; import org.apache.helix.controller.pipeline.StageException; -import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; import org.apache.log4j.Logger; public class TaskAssignmentStage extends AbstractBaseStage { @@ -47,30 +50,34 @@ public void process(ClusterEvent event) throws Exception { logger.info("START TaskAssignmentStage.process()"); HelixManager manager = event.getAttribute("helixmanager"); - Map resourceMap = event.getAttribute(AttributeName.RESOURCES.toString()); - MessageThrottleStageOutput messageOutput = - event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString()); - ClusterDataCache cache = event.getAttribute("ClusterDataCache"); - Map liveInstanceMap = cache.getLiveInstances(); - - if (manager == null || resourceMap == null || messageOutput == null || cache == null - || liveInstanceMap == null) { - throw new StageException("Missing attributes in event:" + event - + ". Requires HelixManager|RESOURCES|MESSAGES_THROTTLE|DataCache|liveInstanceMap"); + Map resourceMap = + event.getAttribute(AttributeName.RESOURCES.toString()); + MessageOutput messageOutput = event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString()); + BestPossibleStateOutput bestPossibleStateOutput = + event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); + Cluster cluster = event.getAttribute("ClusterDataCache"); + Map liveParticipantMap = cluster.getLiveParticipantMap(); + + if (manager == null || resourceMap == null || messageOutput == null || cluster == null + || liveParticipantMap == null) { + throw new StageException( + "Missing attributes in event:" + + event + + ". Requires HelixManager|RESOURCES|MESSAGES_THROTTLE|BEST_POSSIBLE_STATE|DataCache|liveInstanceMap"); } HelixDataAccessor dataAccessor = manager.getHelixDataAccessor(); List messagesToSend = new ArrayList(); - for (String resourceName : resourceMap.keySet()) { - Resource resource = resourceMap.get(resourceName); - for (Partition partition : resource.getPartitions()) { - List messages = messageOutput.getMessages(resourceName, partition); + for (ResourceId resourceId : resourceMap.keySet()) { + for (PartitionId partitionId : bestPossibleStateOutput.getResourceAssignment(resourceId) + .getMappedPartitionIds()) { + List messages = messageOutput.getMessages(resourceId, partitionId); messagesToSend.addAll(messages); } } List outputMessages = - batchMessage(dataAccessor.keyBuilder(), messagesToSend, resourceMap, liveInstanceMap, + batchMessage(dataAccessor.keyBuilder(), messagesToSend, resourceMap, liveParticipantMap, manager.getProperties()); sendMessages(dataAccessor, outputMessages); @@ -80,8 +87,8 @@ public void process(ClusterEvent event) throws Exception { } List batchMessage(Builder keyBuilder, List messages, - Map resourceMap, Map liveInstanceMap, - HelixManagerProperties properties) { + Map resourceMap, + Map liveParticipantMap, HelixManagerProperties properties) { // group messages by its CurrentState path + "/" + fromState + "/" + toState Map batchMessages = new HashMap(); List outputMessages = new ArrayList(); @@ -89,14 +96,14 @@ List batchMessage(Builder keyBuilder, List messages, Iterator iter = messages.iterator(); while (iter.hasNext()) { Message message = iter.next(); - String resourceName = message.getResourceName(); - Resource resource = resourceMap.get(resourceName); + ResourceId resourceId = message.getResourceId(); + ResourceConfig resource = resourceMap.get(resourceId); - String instanceName = message.getTgtName(); - LiveInstance liveInstance = liveInstanceMap.get(instanceName); + ParticipantId participantId = ParticipantId.from(message.getTgtName()); + Participant liveParticipant = liveParticipantMap.get(participantId); String participantVersion = null; - if (liveInstance != null) { - participantVersion = liveInstance.getHelixVersion(); + if (liveParticipant != null) { + participantVersion = liveParticipant.getRunningInstance().getVersion().toString(); } if (resource == null || !resource.getBatchMessageMode() || participantVersion == null @@ -106,9 +113,9 @@ List batchMessage(Builder keyBuilder, List messages, } String key = - keyBuilder.currentState(message.getTgtName(), message.getTgtSessionId(), - message.getResourceName()).getPath() - + "/" + message.getFromState() + "/" + message.getToState(); + keyBuilder.currentState(message.getTgtName(), message.getTypedTgtSessionId().stringify(), + message.getResourceId().stringify()).getPath() + + "/" + message.getTypedFromState() + "/" + message.getTypedToState(); if (!batchMessages.containsKey(key)) { Message batchMessage = new Message(message.getRecord()); @@ -116,7 +123,7 @@ List batchMessage(Builder keyBuilder, List messages, outputMessages.add(batchMessage); batchMessages.put(key, batchMessage); } - batchMessages.get(key).addPartitionName(message.getPartitionName()); + batchMessages.get(key).addPartitionName(message.getPartitionId().stringify()); } return outputMessages; @@ -131,14 +138,14 @@ protected void sendMessages(HelixDataAccessor dataAccessor, List messag List keys = new ArrayList(); for (Message message : messages) { - logger.info("Sending Message " + message.getMsgId() + " to " + message.getTgtName() - + " transit " + message.getPartitionName() + "|" + message.getPartitionNames() + " from:" - + message.getFromState() + " to:" + message.getToState()); - - // System.out.println("[dbg] Sending Message " + message.getMsgId() + " to " + - // message.getTgtName() - // + " transit " + message.getPartitionName() + "|" + message.getPartitionNames() - // + " from: " + message.getFromState() + " to: " + message.getToState()); + logger.info("Sending Message " + message.getMessageId() + " to " + message.getTgtName() + + " transit " + message.getPartitionId() + "|" + message.getPartitionIds() + " from:" + + message.getTypedFromState() + " to:" + message.getTypedToState()); + + // System.out.println("[dbg] Sending Message " + message.getMsgId() + " to " + // + message.getTgtName() + " transit " + message.getPartitionId() + "|" + // + message.getPartitionIds() + " from: " + message.getFromState() + " to: " + // + message.getToState()); keys.add(keyBuilder.message(message.getTgtName(), message.getId())); } diff --git a/helix-core/src/main/java/org/apache/helix/controller/strategy/AutoRebalanceStrategy.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/AutoRebalanceStrategy.java index 76560a4b82..bff7e46f92 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/strategy/AutoRebalanceStrategy.java +++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/AutoRebalanceStrategy.java @@ -24,17 +24,26 @@ import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; -import java.util.Map.Entry; import org.apache.helix.HelixManager; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.ResourceAssignment; import org.apache.log4j.Logger; +import com.google.common.base.Functions; +import com.google.common.collect.Lists; + public class AutoRebalanceStrategy { private static Logger logger = Logger.getLogger(AutoRebalanceStrategy.class); @@ -54,6 +63,15 @@ public class AutoRebalanceStrategy { private Map _existingNonPreferredAssignment; private Set _orphaned; + /** + * Initialize this strategy for a resource + * @param resourceName the resource for which an assignment will be computed + * @param partitions the partition names for the resource + * @param states the states and the number of replicas that should be in each state + * @param maximumPerNode the maximum number of replicas any note can hold + * @param placementScheme the scheme to use for preferred replica locations. If null, this is + * {@link DefaultPlacementScheme} + */ public AutoRebalanceStrategy(String resourceName, final List partitions, final LinkedHashMap states, int maximumPerNode, ReplicaPlacementScheme placementScheme) { @@ -68,30 +86,91 @@ public AutoRebalanceStrategy(String resourceName, final List partitions, } } + /** + * Initialize the strategy with a default placement scheme + * @see #AutoRebalanceStrategy(String, List, LinkedHashMap, int, ReplicaPlacementScheme) + */ public AutoRebalanceStrategy(String resourceName, final List partitions, final LinkedHashMap states) { this(resourceName, partitions, states, Integer.MAX_VALUE, new DefaultPlacementScheme()); } + /** + * Constructor to support logically-typed Helix components + * @param resourceId the resource for which to compute an assignment + * @param partitions the partitions of the resource + * @param states the states and counts for each state + * @param maximumPerNode the maximum number of replicas per node + * @param placementScheme the scheme to use for preferred replica locations. If null, this is + * {@link DefaultPlacementScheme} + */ + public AutoRebalanceStrategy(ResourceId resourceId, final List partitions, + final LinkedHashMap states, int maximumPerNode, + ReplicaPlacementScheme placementScheme) { + LinkedHashMap rawStateCountMap = new LinkedHashMap(); + for (State state : states.keySet()) { + rawStateCountMap.put(state.toString(), states.get(state)); + } + List partitionNames = Lists.transform(partitions, Functions.toStringFunction()); + _resourceName = resourceId.stringify(); + _partitions = partitionNames; + _states = rawStateCountMap; + _maximumPerNode = maximumPerNode; + if (placementScheme != null) { + _placementScheme = placementScheme; + } else { + _placementScheme = new DefaultPlacementScheme(); + } + } + + /** + * Wrap {@link #computePartitionAssignment(List, Map, List)} with a function that takes concrete + * types + * @param liveNodes list of live participant ids + * @param currentMapping map of partition id to map of participant id to state + * @param allNodes list of all participant ids + * @return the preference list and replica mapping + */ + public ZNRecord typedComputePartitionAssignment(final List liveNodes, + final Map> currentMapping, + final List allNodes) { + final List rawLiveNodes = Lists.transform(liveNodes, Functions.toStringFunction()); + final List rawAllNodes = Lists.transform(allNodes, Functions.toStringFunction()); + final Map> rawCurrentMapping = + ResourceAssignment.stringMapsFromReplicaMaps(currentMapping); + return computePartitionAssignment(rawLiveNodes, rawCurrentMapping, rawAllNodes); + } + + /** + * Determine a preference list and mapping of partitions to nodes for all replicas + * @param liveNodes the current list of live participants + * @param currentMapping the current assignment of replicas to nodes + * @param allNodes the full list of known nodes in the system + * @return the preference list and replica mapping + */ public ZNRecord computePartitionAssignment(final List liveNodes, final Map> currentMapping, final List allNodes) { + List sortedLiveNodes = new ArrayList(liveNodes); + Collections.sort(sortedLiveNodes); + List sortedAllNodes = new ArrayList(allNodes); + Collections.sort(sortedAllNodes); int numReplicas = countStateReplicas(); ZNRecord znRecord = new ZNRecord(_resourceName); - if (liveNodes.size() == 0) { + if (sortedLiveNodes.size() == 0) { return znRecord; } - int distRemainder = (numReplicas * _partitions.size()) % liveNodes.size(); - int distFloor = (numReplicas * _partitions.size()) / liveNodes.size(); + int distRemainder = (numReplicas * _partitions.size()) % sortedLiveNodes.size(); + int distFloor = (numReplicas * _partitions.size()) / sortedLiveNodes.size(); _nodeMap = new HashMap(); _liveNodesList = new ArrayList(); - for (String id : allNodes) { + for (String id : sortedAllNodes) { Node node = new Node(id); node.capacity = 0; node.hasCeilingCapacity = false; _nodeMap.put(id, node); } - for (int i = 0; i < liveNodes.size(); i++) { + for (int i = 0; i < sortedLiveNodes.size(); i++) { boolean usingCeiling = false; int targetSize = (_maximumPerNode > 0) ? Math.min(distFloor, _maximumPerNode) : distFloor; if (distRemainder > 0 && targetSize < _maximumPerNode) { @@ -99,7 +178,7 @@ public ZNRecord computePartitionAssignment(final List liveNodes, distRemainder = distRemainder - 1; usingCeiling = true; } - Node node = _nodeMap.get(liveNodes.get(i)); + Node node = _nodeMap.get(sortedLiveNodes.get(i)); node.isAlive = true; node.capacity = targetSize; node.hasCeilingCapacity = usingCeiling; @@ -110,7 +189,7 @@ public ZNRecord computePartitionAssignment(final List liveNodes, _stateMap = generateStateMap(); // compute the preferred mapping if all nodes were up - _preferredAssignment = computePreferredPlacement(allNodes); + _preferredAssignment = computePreferredPlacement(sortedAllNodes); // logger.info("preferred mapping:"+ preferredAssignment); // from current mapping derive the ones in preferred location @@ -141,8 +220,8 @@ public ZNRecord computePartitionAssignment(final List liveNodes, * and its preferred node is under capacity. */ private void moveNonPreferredReplicasToPreferred() { - // iterate through non preferred and see if we can move them to - // preferredlocation if the donor has more than it should and stealer has + // iterate through non preferred and see if we can move them to the + // preferred location if the donor has more than it should and stealer has // enough capacity Iterator> iterator = _existingNonPreferredAssignment.entrySet().iterator(); while (iterator.hasNext()) { @@ -156,6 +235,8 @@ private void moveNonPreferredReplicasToPreferred() { receiver.currentlyAssigned = receiver.currentlyAssigned + 1; donor.nonPreferred.remove(replica); receiver.preferred.add(replica); + donor.newReplicas.remove(replica); + receiver.newReplicas.add(replica); iterator.remove(); } } @@ -178,6 +259,7 @@ private void assignOrphans() { if (receiver.capacity > receiver.currentlyAssigned && receiver.canAdd(replica)) { receiver.currentlyAssigned = receiver.currentlyAssigned + 1; receiver.nonPreferred.add(replica); + receiver.newReplicas.add(replica); added = true; break; } @@ -272,36 +354,136 @@ private void prepareResult(ZNRecord znRecord) { // The list fields are also keyed on partition and list all the nodes serving that partition. // This is useful to verify that there is no node serving multiple replicas of the same // partition. + Map> newPreferences = new TreeMap>(); for (String partition : _partitions) { znRecord.setMapField(partition, new TreeMap()); znRecord.setListField(partition, new ArrayList()); + newPreferences.put(partition, new ArrayList()); } + + // for preference lists, the rough priority that we want is: + // [existing preferred, existing non-preferred, non-existing preferred, non-existing + // non-preferred] for (Node node : _liveNodesList) { for (Replica replica : node.preferred) { - znRecord.getMapField(replica.partition).put(node.id, _stateMap.get(replica.replicaId)); + if (node.newReplicas.contains(replica)) { + newPreferences.get(replica.partition).add(node.id); + } else { + znRecord.getListField(replica.partition).add(node.id); + } } + } + for (Node node : _liveNodesList) { for (Replica replica : node.nonPreferred) { - znRecord.getMapField(replica.partition).put(node.id, _stateMap.get(replica.replicaId)); + if (node.newReplicas.contains(replica)) { + newPreferences.get(replica.partition).add(node.id); + } else { + znRecord.getListField(replica.partition).add(node.id); + } } } + normalizePreferenceLists(znRecord.getListFields(), newPreferences); - int count = countStateReplicas(); - for (int replicaId = 0; replicaId < count; replicaId++) { - for (Node node : _liveNodesList) { - for (Replica replica : node.preferred) { - if (replicaId == replica.replicaId) { - znRecord.getListField(replica.partition).add(node.id); - } - } - for (Replica replica : node.nonPreferred) { - if (replicaId == replica.replicaId) { - znRecord.getListField(replica.partition).add(node.id); - } - } + // generate preference maps based on the preference lists + for (String partition : _partitions) { + List preferenceList = znRecord.getListField(partition); + int i = 0; + for (String participant : preferenceList) { + znRecord.getMapField(partition).put(participant, _stateMap.get(i)); + i++; } } } + /** + * Adjust preference lists to reduce the number of same replicas on an instance. This will + * separately normalize two sets of preference lists, and then append the results of the second + * set to those of the first. This basically ensures that existing replicas are automatically + * preferred. + * @param preferenceLists map of (partition --> list of nodes) + * @param newPreferences map containing node preferences not consistent with the current + * assignment + */ + private void normalizePreferenceLists(Map> preferenceLists, + Map> newPreferences) { + Map> nodeReplicaCounts = + new HashMap>(); + for (String partition : preferenceLists.keySet()) { + normalizePreferenceList(preferenceLists.get(partition), nodeReplicaCounts); + } + for (String partition : newPreferences.keySet()) { + normalizePreferenceList(newPreferences.get(partition), nodeReplicaCounts); + preferenceLists.get(partition).addAll(newPreferences.get(partition)); + } + } + + /** + * Adjust a single preference list for replica assignment imbalance + * @param preferenceList list of node names + * @param nodeReplicaCounts map of (node --> state --> count) + */ + private void normalizePreferenceList(List preferenceList, + Map> nodeReplicaCounts) { + // make this a LinkedHashSet to preserve iteration order + Set notAssigned = new LinkedHashSet(preferenceList); + List newPreferenceList = new ArrayList(); + int replicas = Math.min(countStateReplicas(), preferenceList.size()); + for (int i = 0; i < replicas; i++) { + String state = _stateMap.get(i); + String node = getMinimumNodeForReplica(state, notAssigned, nodeReplicaCounts); + newPreferenceList.add(node); + notAssigned.remove(node); + Map counts = nodeReplicaCounts.get(node); + counts.put(state, counts.get(state) + 1); + } + preferenceList.clear(); + preferenceList.addAll(newPreferenceList); + } + + /** + * Get the node which hosts the fewest of a given replica + * @param state the state + * @param nodes nodes to check + * @param nodeReplicaCounts current assignment of replicas + * @return the node most willing to accept the replica + */ + private String getMinimumNodeForReplica(String state, Set nodes, + Map> nodeReplicaCounts) { + String minimalNode = null; + int minimalCount = Integer.MAX_VALUE; + for (String node : nodes) { + int count = getReplicaCountForNode(state, node, nodeReplicaCounts); + if (count < minimalCount) { + minimalCount = count; + minimalNode = node; + } + } + return minimalNode; + } + + /** + * Safe check for the number of replicas of a given id assiged to a node + * @param state the state to assign + * @param node the node to check + * @param nodeReplicaCounts a map of node to replica id and counts + * @return the number of currently assigned replicas of the given id + */ + private int getReplicaCountForNode(String state, String node, + Map> nodeReplicaCounts) { + if (!nodeReplicaCounts.containsKey(node)) { + Map replicaCounts = new HashMap(); + replicaCounts.put(state, 0); + nodeReplicaCounts.put(node, replicaCounts); + return 0; + } + Map replicaCounts = nodeReplicaCounts.get(node); + if (!replicaCounts.containsKey(state)) { + replicaCounts.put(state, 0); + return 0; + } + return replicaCounts.get(state); + } + /** * Compute the subset of the current mapping where replicas are not mapped according to their * preferred assignment. @@ -463,7 +645,6 @@ private Map generateStateMap() { * of replicas assigned to it, so it can decide if it can receive additional replicas. */ class Node { - public int currentlyAssigned; public int capacity; public boolean hasCeilingCapacity; @@ -471,10 +652,12 @@ class Node { boolean isAlive; private List preferred; private List nonPreferred; + private Set newReplicas; public Node(String id) { preferred = new ArrayList(); nonPreferred = new ArrayList(); + newReplicas = new TreeSet(); currentlyAssigned = 0; isAlive = false; this.id = id; @@ -527,6 +710,7 @@ public void steal(Node donor, Replica replica) { capacity++; currentlyAssigned++; nonPreferred.add(replica); + newReplicas.add(replica); } @Override @@ -543,7 +727,6 @@ public String toString() { * and an identifier signifying a specific replica of a given partition and state. */ class Replica implements Comparable { - private String partition; private int replicaId; // this is a partition-relative id private String format; @@ -551,7 +734,7 @@ class Replica implements Comparable { public Replica(String partition, int replicaId) { this.partition = partition; this.replicaId = replicaId; - this.format = partition + "|" + replicaId; + this.format = this.partition + "|" + this.replicaId; } @Override @@ -575,7 +758,7 @@ public int hashCode() { @Override public int compareTo(Replica that) { if (that instanceof Replica) { - return this.format.compareTo(((Replica) that).format); + return this.format.compareTo(that.format); } return -1; } diff --git a/helix-core/src/main/java/org/apache/helix/tools/IdealCalculatorByConsistentHashing.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/ConsistentHashingMasterSlaveStrategy.java similarity index 97% rename from helix-core/src/main/java/org/apache/helix/tools/IdealCalculatorByConsistentHashing.java rename to helix-core/src/main/java/org/apache/helix/controller/strategy/ConsistentHashingMasterSlaveStrategy.java index 1101a6df84..017d3d8c83 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/IdealCalculatorByConsistentHashing.java +++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/ConsistentHashingMasterSlaveStrategy.java @@ -1,4 +1,4 @@ -package org.apache.helix.tools; +package org.apache.helix.controller.strategy; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -32,7 +32,7 @@ import org.apache.helix.ZNRecord; import org.apache.helix.model.IdealState.IdealStateProperty; -public class IdealCalculatorByConsistentHashing { +public class ConsistentHashingMasterSlaveStrategy { /** * Interface to calculate the hash function value of a string */ @@ -456,8 +456,8 @@ public static void printHashRingStat(int[] hashRing) { static int[] getFnvHashArray(List strings) { int[] result = new int[strings.size()]; int i = 0; - IdealCalculatorByConsistentHashing.FnvHash hashfunc = - new IdealCalculatorByConsistentHashing.FnvHash(); + ConsistentHashingMasterSlaveStrategy.FnvHash hashfunc = + new ConsistentHashingMasterSlaveStrategy.FnvHash(); for (String s : strings) { int val = hashfunc.getHashValue(s) % 65536; if (val < 0) @@ -498,8 +498,8 @@ public static void main(String args[]) throws Exception { String dbName = "espressoDB1"; ZNRecord result = - IdealCalculatorByConsistentHashing.calculateIdealState(instanceNames, partitions, replicas, - dbName, new IdealCalculatorByConsistentHashing.FnvHash()); + ConsistentHashingMasterSlaveStrategy.calculateIdealState(instanceNames, partitions, + replicas, dbName, new ConsistentHashingMasterSlaveStrategy.FnvHash()); System.out.println("\nMaster :"); printIdealStateStats(result, "MASTER"); diff --git a/helix-core/src/main/java/org/apache/helix/tools/DefaultIdealStateCalculator.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/DefaultTwoStateStrategy.java similarity index 51% rename from helix-core/src/main/java/org/apache/helix/tools/DefaultIdealStateCalculator.java rename to helix-core/src/main/java/org/apache/helix/controller/strategy/DefaultTwoStateStrategy.java index dea8e0ae52..c965748ba0 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/DefaultIdealStateCalculator.java +++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/DefaultTwoStateStrategy.java @@ -1,4 +1,4 @@ -package org.apache.helix.tools; +package org.apache.helix.controller.strategy; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -31,17 +31,17 @@ import org.apache.helix.model.IdealState.IdealStateProperty; /** - * DefaultIdealStateCalculator tries to optimally allocate master/slave partitions among - * espresso storage nodes. + * DefaultIdealStateCalculator tries to optimally allocate two state partitions among + * storage nodes. * Given a batch of storage nodes, the partition and replication factor, the algorithm first given a * initial state * When new batches of storage nodes are added, the algorithm will calculate the new ideal state * such that the total * partition movements are minimized. */ -public class DefaultIdealStateCalculator { - static final String _MasterAssignmentMap = "MasterAssignmentMap"; - static final String _SlaveAssignmentMap = "SlaveAssignmentMap"; +public class DefaultTwoStateStrategy { + static final String _PrimaryAssignmentMap = "PrimaryAssignmentMap"; + static final String _SecondaryAssignmentMap = "SecondaryAssignmentMap"; static final String _partitions = "partitions"; static final String _replicas = "replicas"; @@ -49,49 +49,49 @@ public class DefaultIdealStateCalculator { * Calculate the initial ideal state given a batch of storage instances, the replication factor * and * number of partitions - * 1. Calculate the master assignment by random shuffling - * 2. for each storage instance, calculate the 1st slave assignment map, by another random - * shuffling - * 3. for each storage instance, calculate the i-th slave assignment map - * 4. Combine the i-th slave assignment maps together + * 1. Calculate the primary state assignment by random shuffling + * 2. for each storage instance, calculate the 1st secondary state assignment map, by another + * random shuffling + * 3. for each storage instance, calculate the i-th secondary state assignment map + * 4. Combine the i-th secondary state assignment maps together * @param instanceNames * list of storage node instances * @param partitions * number of partitions * @param replicas - * The number of replicas (slave partitions) per master partition - * @param masterStateValue - * master state value: e.g. "MASTER" or "LEADER" - * @param slaveStateValue - * slave state value: e.g. "SLAVE" or "STANDBY" + * The number of replicas (secondary partitions) per primary partition + * @param primaryStateValue + * primary state value: e.g. "MASTER" or "LEADER" + * @param secondaryStateValue + * secondary state value: e.g. "SLAVE" or "STANDBY" * @param resourceName * @return a ZNRecord that contain the idealstate info */ public static ZNRecord calculateIdealState(List instanceNames, int partitions, - int replicas, String resourceName, String masterStateValue, String slaveStateValue) { + int replicas, String resourceName, String primaryStateValue, String secondaryStateValue) { Collections.sort(instanceNames); if (instanceNames.size() < replicas + 1) { throw new HelixException("Number of instances must not be less than replicas + 1. " + "instanceNr:" + instanceNames.size() + ", replicas:" + replicas); } else if (partitions < instanceNames.size()) { ZNRecord idealState = - IdealStateCalculatorByShuffling.calculateIdealState(instanceNames, partitions, replicas, - resourceName, 12345, masterStateValue, slaveStateValue); + ShufflingTwoStateStrategy.calculateIdealState(instanceNames, partitions, replicas, + resourceName, 12345, primaryStateValue, secondaryStateValue); int i = 0; for (String partitionId : idealState.getMapFields().keySet()) { Map partitionAssignmentMap = idealState.getMapField(partitionId); List partitionAssignmentPriorityList = new ArrayList(); - String masterInstance = ""; + String primaryInstance = ""; for (String instanceName : partitionAssignmentMap.keySet()) { - if (partitionAssignmentMap.get(instanceName).equalsIgnoreCase(masterStateValue) - && masterInstance.equals("")) { - masterInstance = instanceName; + if (partitionAssignmentMap.get(instanceName).equalsIgnoreCase(primaryStateValue) + && primaryInstance.equals("")) { + primaryInstance = instanceName; } else { partitionAssignmentPriorityList.add(instanceName); } } Collections.shuffle(partitionAssignmentPriorityList, new Random(i++)); - partitionAssignmentPriorityList.add(0, masterInstance); + partitionAssignmentPriorityList.add(0, primaryInstance); idealState.setListField(partitionId, partitionAssignmentPriorityList); } return idealState; @@ -99,12 +99,12 @@ public static ZNRecord calculateIdealState(List instanceNames, int parti Map result = calculateInitialIdealState(instanceNames, partitions, replicas); - return convertToZNRecord(result, resourceName, masterStateValue, slaveStateValue); + return convertToZNRecord(result, resourceName, primaryStateValue, secondaryStateValue); } public static ZNRecord calculateIdealStateBatch(List> instanceBatches, - int partitions, int replicas, String resourceName, String masterStateValue, - String slaveStateValue) { + int partitions, int replicas, String resourceName, String primaryStateValue, + String secondaryStateValue) { Map result = calculateInitialIdealState(instanceBatches.get(0), partitions, replicas); @@ -112,18 +112,18 @@ public static ZNRecord calculateIdealStateBatch(List> instanceBatch result = calculateNextIdealState(instanceBatches.get(i), result); } - return convertToZNRecord(result, resourceName, masterStateValue, slaveStateValue); + return convertToZNRecord(result, resourceName, primaryStateValue, secondaryStateValue); } /** * Convert the internal result (stored as a Map) into ZNRecord. */ public static ZNRecord convertToZNRecord(Map result, String resourceName, - String masterStateValue, String slaveStateValue) { - Map> nodeMasterAssignmentMap = - (Map>) (result.get(_MasterAssignmentMap)); - Map>> nodeSlaveAssignmentMap = - (Map>>) (result.get(_SlaveAssignmentMap)); + String primaryStateValue, String secondaryStateValue) { + Map> nodePrimaryAssignmentMap = + (Map>) (result.get(_PrimaryAssignmentMap)); + Map>> nodeSecondaryAssignmentMap = + (Map>>) (result.get(_SecondaryAssignmentMap)); int partitions = (Integer) (result.get("partitions")); @@ -131,44 +131,45 @@ public static ZNRecord convertToZNRecord(Map result, String reso idealState.setSimpleField(IdealStateProperty.NUM_PARTITIONS.toString(), String.valueOf(partitions)); - for (String instanceName : nodeMasterAssignmentMap.keySet()) { - for (Integer partitionId : nodeMasterAssignmentMap.get(instanceName)) { + for (String instanceName : nodePrimaryAssignmentMap.keySet()) { + for (Integer partitionId : nodePrimaryAssignmentMap.get(instanceName)) { String partitionName = resourceName + "_" + partitionId; if (!idealState.getMapFields().containsKey(partitionName)) { idealState.setMapField(partitionName, new TreeMap()); } - idealState.getMapField(partitionName).put(instanceName, masterStateValue); + idealState.getMapField(partitionName).put(instanceName, primaryStateValue); } } - for (String instanceName : nodeSlaveAssignmentMap.keySet()) { - Map> slaveAssignmentMap = nodeSlaveAssignmentMap.get(instanceName); + for (String instanceName : nodeSecondaryAssignmentMap.keySet()) { + Map> secondaryAssignmentMap = + nodeSecondaryAssignmentMap.get(instanceName); - for (String slaveNode : slaveAssignmentMap.keySet()) { - List slaveAssignment = slaveAssignmentMap.get(slaveNode); - for (Integer partitionId : slaveAssignment) { + for (String secondaryNode : secondaryAssignmentMap.keySet()) { + List secondaryAssignment = secondaryAssignmentMap.get(secondaryNode); + for (Integer partitionId : secondaryAssignment) { String partitionName = resourceName + "_" + partitionId; - idealState.getMapField(partitionName).put(slaveNode, slaveStateValue); + idealState.getMapField(partitionName).put(secondaryNode, secondaryStateValue); } } } - // generate the priority list of instances per partition. Master should be at front and slave - // follows. + // generate the priority list of instances per partition. the primary should be at front + // and the secondaries follow. for (String partitionId : idealState.getMapFields().keySet()) { Map partitionAssignmentMap = idealState.getMapField(partitionId); List partitionAssignmentPriorityList = new ArrayList(); - String masterInstance = ""; + String primaryInstance = ""; for (String instanceName : partitionAssignmentMap.keySet()) { - if (partitionAssignmentMap.get(instanceName).equalsIgnoreCase(masterStateValue) - && masterInstance.equals("")) { - masterInstance = instanceName; + if (partitionAssignmentMap.get(instanceName).equalsIgnoreCase(primaryStateValue) + && primaryInstance.equals("")) { + primaryInstance = instanceName; } else { partitionAssignmentPriorityList.add(instanceName); } } Collections.shuffle(partitionAssignmentPriorityList); - partitionAssignmentPriorityList.add(0, masterInstance); + partitionAssignmentPriorityList.add(0, primaryInstance); idealState.setListField(partitionId, partitionAssignmentPriorityList); } assert (result.containsKey("replicas")); @@ -181,11 +182,11 @@ public static ZNRecord convertToZNRecord(Map result, String reso * Calculate the initial ideal state given a batch of storage instances, the replication factor * and * number of partitions - * 1. Calculate the master assignment by random shuffling - * 2. for each storage instance, calculate the 1st slave assignment map, by another random - * shuffling - * 3. for each storage instance, calculate the i-th slave assignment map - * 4. Combine the i-th slave assignment maps together + * 1. Calculate the primary assignment by random shuffling + * 2. for each storage instance, calculate the 1st secondary state assignment map, by another + * random shuffling + * 3. for each storage instance, calculate the i-th secondary state assignment map + * 4. Combine the i-th secondary state assignment maps together * @param instanceNames * list of storage node instances * @param weight @@ -193,108 +194,109 @@ public static ZNRecord convertToZNRecord(Map result, String reso * @param partitions * number of partitions * @param replicas - * The number of replicas (slave partitions) per master partition + * The number of replicas (secondary partitions) per primary partition * @return a map that contain the idealstate info */ public static Map calculateInitialIdealState(List instanceNames, int partitions, int replicas) { Random r = new Random(54321); assert (replicas <= instanceNames.size() - 1); - - ArrayList masterPartitionAssignment = new ArrayList(); + ArrayList primaryPartitionAssignment = new ArrayList(); for (int i = 0; i < partitions; i++) { - masterPartitionAssignment.add(i); + primaryPartitionAssignment.add(i); } // shuffle the partition id array - Collections.shuffle(masterPartitionAssignment, new Random(r.nextInt())); + Collections.shuffle(primaryPartitionAssignment, new Random(r.nextInt())); - // 1. Generate the random master partition assignment - // instanceName -> List of master partitions on that instance - Map> nodeMasterAssignmentMap = new TreeMap>(); - for (int i = 0; i < masterPartitionAssignment.size(); i++) { + // 1. Generate the random primary partition assignment + // instanceName -> List of primary partitions on that instance + Map> nodePrimaryAssignmentMap = new TreeMap>(); + for (int i = 0; i < primaryPartitionAssignment.size(); i++) { String instanceName = instanceNames.get(i % instanceNames.size()); - if (!nodeMasterAssignmentMap.containsKey(instanceName)) { - nodeMasterAssignmentMap.put(instanceName, new ArrayList()); + if (!nodePrimaryAssignmentMap.containsKey(instanceName)) { + nodePrimaryAssignmentMap.put(instanceName, new ArrayList()); } - nodeMasterAssignmentMap.get(instanceName).add(masterPartitionAssignment.get(i)); + nodePrimaryAssignmentMap.get(instanceName).add(primaryPartitionAssignment.get(i)); } - // instanceName -> slave assignment for its master partitions - // slave assignment: instanceName -> list of slave partitions on it - List>>> nodeSlaveAssignmentMapsList = + // instanceName -> secondary assignment for its primary partitions + // secondary assignment: instanceName -> list of secondary partitions on it + List>>> nodeSecondaryAssignmentMapsList = new ArrayList>>>(replicas); - Map>> firstNodeSlaveAssignmentMap = + Map>> firstNodeSecondaryAssignmentMap = new TreeMap>>(); - Map>> combinedNodeSlaveAssignmentMap = + Map>> combinedNodeSecondaryAssignmentMap = new TreeMap>>(); if (replicas > 0) { - // 2. For each node, calculate the evenly distributed slave as the first slave assignment - // We will figure out the 2nd ...replicas-th slave assignment based on the first level slave - // assignment + // 2. For each node, calculate the evenly distributed secondary state as the first secondary + // state assignment + // We will figure out the 2nd ...replicas-th secondary state assignment based on the first + // level secondary state assignment for (int i = 0; i < instanceNames.size(); i++) { - List slaveInstances = new ArrayList(); - ArrayList slaveAssignment = new ArrayList(); - TreeMap> slaveAssignmentMap = new TreeMap>(); + List secondaryInstances = new ArrayList(); + ArrayList secondaryAssignment = new ArrayList(); + TreeMap> secondaryAssignmentMap = + new TreeMap>(); for (int j = 0; j < instanceNames.size(); j++) { if (j != i) { - slaveInstances.add(instanceNames.get(j)); - slaveAssignmentMap.put(instanceNames.get(j), new ArrayList()); + secondaryInstances.add(instanceNames.get(j)); + secondaryAssignmentMap.put(instanceNames.get(j), new ArrayList()); } } - // Get the number of master partitions on instanceName - List masterAssignment = nodeMasterAssignmentMap.get(instanceNames.get(i)); - // do a random shuffling as in step 1, so that the first-level slave are distributed among - // rest instances + // Get the number of primary partitions on instanceName + List primaryAssignment = nodePrimaryAssignmentMap.get(instanceNames.get(i)); + // do a random shuffling as in step 1, so that the first-level secondary states are + // distributed among rest instances - for (int j = 0; j < masterAssignment.size(); j++) { - slaveAssignment.add(j); + for (int j = 0; j < primaryAssignment.size(); j++) { + secondaryAssignment.add(j); } - Collections.shuffle(slaveAssignment, new Random(r.nextInt())); - - Collections.shuffle(slaveInstances, new Random(instanceNames.get(i).hashCode())); - - // Get the slave assignment map of node instanceName - for (int j = 0; j < masterAssignment.size(); j++) { - String slaveInstanceName = - slaveInstances.get(slaveAssignment.get(j) % slaveInstances.size()); - if (!slaveAssignmentMap.containsKey(slaveInstanceName)) { - slaveAssignmentMap.put(slaveInstanceName, new ArrayList()); + Collections.shuffle(secondaryAssignment, new Random(r.nextInt())); + + Collections.shuffle(secondaryInstances, new Random(instanceNames.get(i).hashCode())); + // Get the secondary assignment map of node instanceName + for (int j = 0; j < primaryAssignment.size(); j++) { + String secondaryInstanceName = + secondaryInstances.get(secondaryAssignment.get(j) % secondaryInstances.size()); + if (!secondaryAssignmentMap.containsKey(secondaryInstanceName)) { + secondaryAssignmentMap.put(secondaryInstanceName, new ArrayList()); } - slaveAssignmentMap.get(slaveInstanceName).add(masterAssignment.get(j)); + secondaryAssignmentMap.get(secondaryInstanceName).add(primaryAssignment.get(j)); } - firstNodeSlaveAssignmentMap.put(instanceNames.get(i), slaveAssignmentMap); + firstNodeSecondaryAssignmentMap.put(instanceNames.get(i), secondaryAssignmentMap); } - nodeSlaveAssignmentMapsList.add(firstNodeSlaveAssignmentMap); - // From the first slave assignment map, calculate the rest slave assignment maps + nodeSecondaryAssignmentMapsList.add(firstNodeSecondaryAssignmentMap); + // From the first secondary assignment map, calculate the rest secondary assignment maps for (int replicaOrder = 1; replicaOrder < replicas; replicaOrder++) { - // calculate the next slave partition assignment map - Map>> nextNodeSlaveAssignmentMap = - calculateNextSlaveAssignemntMap(firstNodeSlaveAssignmentMap, replicaOrder); - nodeSlaveAssignmentMapsList.add(nextNodeSlaveAssignmentMap); + // calculate the next secondary partition assignment map + Map>> nextNodeSecondaryAssignmentMap = + calculateNextSecondaryAssignemntMap(firstNodeSecondaryAssignmentMap, replicaOrder); + nodeSecondaryAssignmentMapsList.add(nextNodeSecondaryAssignmentMap); } - // Combine the calculated 1...replicas-th slave assignment map together - - for (String instanceName : nodeMasterAssignmentMap.keySet()) { - Map> combinedSlaveAssignmentMap = + // Combine the calculated 1...replicas-th secondary assignment map together + for (String instanceName : nodePrimaryAssignmentMap.keySet()) { + Map> combinedSecondaryAssignmentMap = new TreeMap>(); - for (Map>> slaveNodeAssignmentMap : nodeSlaveAssignmentMapsList) { - Map> slaveAssignmentMap = slaveNodeAssignmentMap.get(instanceName); + for (Map>> secondaryNodeAssignmentMap : nodeSecondaryAssignmentMapsList) { + Map> secondaryAssignmentMap = + secondaryNodeAssignmentMap.get(instanceName); - for (String slaveInstance : slaveAssignmentMap.keySet()) { - if (!combinedSlaveAssignmentMap.containsKey(slaveInstance)) { - combinedSlaveAssignmentMap.put(slaveInstance, new ArrayList()); + for (String secondaryInstance : secondaryAssignmentMap.keySet()) { + if (!combinedSecondaryAssignmentMap.containsKey(secondaryInstance)) { + combinedSecondaryAssignmentMap.put(secondaryInstance, new ArrayList()); } - combinedSlaveAssignmentMap.get(slaveInstance).addAll( - slaveAssignmentMap.get(slaveInstance)); + combinedSecondaryAssignmentMap.get(secondaryInstance).addAll( + secondaryAssignmentMap.get(secondaryInstance)); } } - migrateSlaveAssignMapToNewInstances(combinedSlaveAssignmentMap, new ArrayList()); - combinedNodeSlaveAssignmentMap.put(instanceName, combinedSlaveAssignmentMap); + migrateSecondaryAssignMapToNewInstances(combinedSecondaryAssignmentMap, + new ArrayList()); + combinedNodeSecondaryAssignmentMap.put(instanceName, combinedSecondaryAssignmentMap); } } /* @@ -340,40 +342,41 @@ public static Map calculateInitialIdealState(List instan * } */ Map result = new TreeMap(); - result.put("MasterAssignmentMap", nodeMasterAssignmentMap); - result.put("SlaveAssignmentMap", combinedNodeSlaveAssignmentMap); + result.put("PrimaryAssignmentMap", nodePrimaryAssignmentMap); + result.put("SecondaryAssignmentMap", combinedNodeSecondaryAssignmentMap); result.put("replicas", new Integer(replicas)); result.put("partitions", new Integer(partitions)); return result; } /** - * In the case there are more than 1 slave, we use the following algorithm to calculate the n-th - * slave - * assignment map based on the first level slave assignment map. - * @param firstInstanceSlaveAssignmentMap the first slave assignment map for all instances - * @param order of the slave - * @return the n-th slave assignment map for all the instances + * In the case there are more than 1 secondary, we use the following algorithm to calculate the + * n-th secondary + * assignment map based on the first level secondary assignment map. + * @param firstInstanceSecondaryAssignmentMap the first secondary assignment map for all instances + * @param order of the secondary state + * @return the n-th secondary assignment map for all the instances */ - static Map>> calculateNextSlaveAssignemntMap( - Map>> firstInstanceSlaveAssignmentMap, int replicaOrder) { + static Map>> calculateNextSecondaryAssignemntMap( + Map>> firstInstanceSecondaryAssignmentMap, int replicaOrder) { Map>> result = new TreeMap>>(); - for (String currentInstance : firstInstanceSlaveAssignmentMap.keySet()) { + for (String currentInstance : firstInstanceSecondaryAssignmentMap.keySet()) { Map> resultAssignmentMap = new TreeMap>(); result.put(currentInstance, resultAssignmentMap); } - for (String currentInstance : firstInstanceSlaveAssignmentMap.keySet()) { - Map> previousSlaveAssignmentMap = - firstInstanceSlaveAssignmentMap.get(currentInstance); + for (String currentInstance : firstInstanceSecondaryAssignmentMap.keySet()) { + Map> previousSecondaryAssignmentMap = + firstInstanceSecondaryAssignmentMap.get(currentInstance); Map> resultAssignmentMap = result.get(currentInstance); int offset = replicaOrder - 1; - for (String instance : previousSlaveAssignmentMap.keySet()) { - List otherInstances = new ArrayList(previousSlaveAssignmentMap.size() - 1); + for (String instance : previousSecondaryAssignmentMap.keySet()) { + List otherInstances = + new ArrayList(previousSecondaryAssignmentMap.size() - 1); // Obtain an array of other instances - for (String otherInstance : previousSlaveAssignmentMap.keySet()) { + for (String otherInstance : previousSecondaryAssignmentMap.keySet()) { otherInstances.add(otherInstance); } Collections.sort(otherInstances); @@ -387,12 +390,12 @@ static Map>> calculateNextSlaveAssignemntMap( if (instanceIndex == otherInstances.size() - 1) { instanceIndex--; } - // Since we need to evenly distribute the slaves on "instance" to other partitions, we + // Since we need to evenly distribute the secondaries on "instance" to other partitions, we // need to remove "instance" from the array. otherInstances.remove(instance); - // distribute previous slave assignment to other instances. - List previousAssignmentList = previousSlaveAssignmentMap.get(instance); + // distribute previous secondary assignment to other instances. + List previousAssignmentList = previousSecondaryAssignmentMap.get(instance); for (int i = 0; i < previousAssignmentList.size(); i++) { // Evenly distribute the previousAssignmentList to the remaining other instances @@ -411,20 +414,21 @@ static Map>> calculateNextSlaveAssignemntMap( /** * Given the current idealState, and the list of new Instances needed to be added, calculate the * new Ideal state. - * 1. Calculate how many master partitions should be moved to the new cluster of instances - * 2. assign the number of master partitions px to be moved to each previous node + * 1. Calculate how many primary partitions should be moved to the new cluster of instances + * 2. assign the number of primary partitions px to be moved to each previous node * 3. for each previous node, * 3.1 randomly choose px nodes, move them to temp list - * 3.2 for each px nodes, remove them from the slave assignment map; record the map position of + * 3.2 for each px nodes, remove them from the secondary assignment map; record the map position + * of * the partition; - * 3.3 calculate # of new nodes that should be put in the slave assignment map - * 3.4 even-fy the slave assignment map; + * 3.3 calculate # of new nodes that should be put in the secondary assignment map + * 3.4 even-fy the secondary assignment map; * 3.5 randomly place # of new nodes that should be placed in - * 4. from all the temp master node list get from 3.1, + * 4. from all the temp primary node list get from 3.1, * 4.1 randomly assign them to nodes in the new cluster * 5. for each node in the new cluster, - * 5.1 assemble the slave assignment map - * 5.2 even-fy the slave assignment map + * 5.1 assemble the secondary assignment map + * 5.2 even-fy the secondary assignment map * @param newInstances * list of new added storage node instances * @param weight @@ -435,122 +439,120 @@ static Map>> calculateNextSlaveAssignemntMap( */ public static Map calculateNextIdealState(List newInstances, Map previousIdealState) { - // Obtain the master / slave assignment info maps + // Obtain the primary / secondary assignment info maps Collections.sort(newInstances); - Map> previousMasterAssignmentMap = - (Map>) (previousIdealState.get("MasterAssignmentMap")); - Map>> nodeSlaveAssignmentMap = - (Map>>) (previousIdealState.get("SlaveAssignmentMap")); + Map> previousPrimaryAssignmentMap = + (Map>) (previousIdealState.get("PrimaryAssignmentMap")); + Map>> nodeSecondaryAssignmentMap = + (Map>>) (previousIdealState.get("SecondaryAssignmentMap")); List oldInstances = new ArrayList(); - for (String oldInstance : previousMasterAssignmentMap.keySet()) { + for (String oldInstance : previousPrimaryAssignmentMap.keySet()) { oldInstances.add(oldInstance); } - int previousInstanceNum = previousMasterAssignmentMap.size(); + int previousInstanceNum = previousPrimaryAssignmentMap.size(); int partitions = (Integer) (previousIdealState.get("partitions")); // TODO: take weight into account when calculate this - int totalMasterParitionsToMove = + int totalPrimaryParitionsToMove = partitions * (newInstances.size()) / (previousInstanceNum + newInstances.size()); - int numMastersFromEachNode = totalMasterParitionsToMove / previousInstanceNum; - int remain = totalMasterParitionsToMove % previousInstanceNum; + int numPrimariesFromEachNode = totalPrimaryParitionsToMove / previousInstanceNum; + int remain = totalPrimaryParitionsToMove % previousInstanceNum; - // Note that when remain > 0, we should make [remain] moves with (numMastersFromEachNode + 1) + // Note that when remain > 0, we should make [remain] moves with (numPrimariesFromEachNode + 1) // partitions. - // And we should first choose those (numMastersFromEachNode + 1) moves from the instances that + // And we should first choose those (numPrimariesFromEachNode + 1) moves from the instances that // has more - // master partitions - List masterPartitionListToMove = new ArrayList(); + // primary partitions + List primaryPartitionListToMove = new ArrayList(); - // For corresponding moved slave partitions, keep track of their original location; the new node - // does not + // For corresponding moved secondary partitions, keep track of their original location; the new + // node does not // need to migrate all of them. - Map> slavePartitionsToMoveMap = new TreeMap>(); + Map> secondaryPartitionsToMoveMap = new TreeMap>(); - // Make sure that the instances that holds more master partitions are put in front + // Make sure that the instances that holds more primary partitions are put in front List bigList = new ArrayList(), smallList = new ArrayList(); - for (String oldInstance : previousMasterAssignmentMap.keySet()) { - List masterAssignmentList = previousMasterAssignmentMap.get(oldInstance); - if (masterAssignmentList.size() > numMastersFromEachNode) { + for (String oldInstance : previousPrimaryAssignmentMap.keySet()) { + List primaryAssignmentList = previousPrimaryAssignmentMap.get(oldInstance); + if (primaryAssignmentList.size() > numPrimariesFromEachNode) { bigList.add(oldInstance); } else { smallList.add(oldInstance); } } - // "sort" the list, such that the nodes that has more master partitions moves more partitions to - // the + // "sort" the list, such that the nodes that has more primary partitions moves more partitions + // to the // new added batch of instances. bigList.addAll(smallList); - int totalSlaveMoves = 0; for (String oldInstance : bigList) { - List masterAssignmentList = previousMasterAssignmentMap.get(oldInstance); - int numToChoose = numMastersFromEachNode; + List primaryAssignmentList = previousPrimaryAssignmentMap.get(oldInstance); + int numToChoose = numPrimariesFromEachNode; if (remain > 0) { - numToChoose = numMastersFromEachNode + 1; + numToChoose = numPrimariesFromEachNode + 1; remain--; } - // randomly remove numToChoose of master partitions to the new added nodes - ArrayList masterPartionsMoved = new ArrayList(); - randomSelect(masterAssignmentList, masterPartionsMoved, numToChoose); - - masterPartitionListToMove.addAll(masterPartionsMoved); - Map> slaveAssignmentMap = nodeSlaveAssignmentMap.get(oldInstance); - removeFromSlaveAssignmentMap(slaveAssignmentMap, masterPartionsMoved, - slavePartitionsToMoveMap); - - // Make sure that for old instances, the slave placement map is evenly distributed - // Trace the "local slave moves", which should together contribute to most of the slave - // migrations - int movesWithinInstance = - migrateSlaveAssignMapToNewInstances(slaveAssignmentMap, newInstances); + // randomly remove numToChoose of primary partitions to the new added nodes + ArrayList primaryPartionsMoved = new ArrayList(); + randomSelect(primaryAssignmentList, primaryPartionsMoved, numToChoose); + + primaryPartitionListToMove.addAll(primaryPartionsMoved); + Map> secondaryAssignmentMap = + nodeSecondaryAssignmentMap.get(oldInstance); + removeFromSecondaryAssignmentMap(secondaryAssignmentMap, primaryPartionsMoved, + secondaryPartitionsToMoveMap); + + // Make sure that for old instances, the secondary placement map is evenly distributed + // Trace the "local secondary moves", which should together contribute to most of the + // secondary migrations + migrateSecondaryAssignMapToNewInstances(secondaryAssignmentMap, newInstances); // System.out.println("local moves: "+ movesWithinInstance); - totalSlaveMoves += movesWithinInstance; } // System.out.println("local slave moves total: "+ totalSlaveMoves); - // calculate the master /slave assignment for the new added nodes + // calculate the primary /secondary assignment for the new added nodes - // We already have the list of master partitions that will migrate to new batch of instances, + // We already have the list of primary partitions that will migrate to new batch of instances, // shuffle the partitions and assign them to new instances - Collections.shuffle(masterPartitionListToMove, new Random(12345)); + Collections.shuffle(primaryPartitionListToMove, new Random(12345)); for (int i = 0; i < newInstances.size(); i++) { String newInstance = newInstances.get(i); - List masterPartitionList = new ArrayList(); - for (int j = 0; j < masterPartitionListToMove.size(); j++) { + List primaryPartitionList = new ArrayList(); + for (int j = 0; j < primaryPartitionListToMove.size(); j++) { if (j % newInstances.size() == i) { - masterPartitionList.add(masterPartitionListToMove.get(j)); + primaryPartitionList.add(primaryPartitionListToMove.get(j)); } } - Map> slavePartitionMap = new TreeMap>(); + Map> secondaryPartitionMap = new TreeMap>(); for (String oldInstance : oldInstances) { - slavePartitionMap.put(oldInstance, new ArrayList()); + secondaryPartitionMap.put(oldInstance, new ArrayList()); } - // Build the slave assignment map for the new instance, based on the saved information - // about those slave partition locations in slavePartitionsToMoveMap - for (Integer x : masterPartitionList) { - for (String oldInstance : slavePartitionsToMoveMap.keySet()) { - List slaves = slavePartitionsToMoveMap.get(oldInstance); - if (slaves.contains(x)) { - slavePartitionMap.get(oldInstance).add(x); + // Build the secondary assignment map for the new instance, based on the saved information + // about those secondary partition locations in secondaryPartitionsToMoveMap + for (Integer x : primaryPartitionList) { + for (String oldInstance : secondaryPartitionsToMoveMap.keySet()) { + List secondaries = secondaryPartitionsToMoveMap.get(oldInstance); + if (secondaries.contains(x)) { + secondaryPartitionMap.get(oldInstance).add(x); } } } - // add entry for other new instances into the slavePartitionMap + // add entry for other new instances into the secondaryPartitionMap List otherNewInstances = new ArrayList(); for (String instance : newInstances) { if (!instance.equalsIgnoreCase(newInstance)) { otherNewInstances.add(instance); } } - // Make sure that slave partitions are evenly distributed - migrateSlaveAssignMapToNewInstances(slavePartitionMap, otherNewInstances); + // Make sure that secondary partitions are evenly distributed + migrateSecondaryAssignMapToNewInstances(secondaryPartitionMap, otherNewInstances); // Update the result in the result map. We can reuse the input previousIdealState map as // the result. - previousMasterAssignmentMap.put(newInstance, masterPartitionList); - nodeSlaveAssignmentMap.put(newInstance, slavePartitionMap); + previousPrimaryAssignmentMap.put(newInstance, primaryPartitionList); + nodeSecondaryAssignmentMap.put(newInstance, secondaryPartitionMap); } /* @@ -597,27 +599,28 @@ public static Map calculateNextIdealState(List newInstan } public ZNRecord calculateNextIdealState(List newInstances, - Map previousIdealState, String resourceName, String masterStateValue, - String slaveStateValue) { + Map previousIdealState, String resourceName, String primaryStateValue, + String secondaryStateValue) { return convertToZNRecord(calculateNextIdealState(newInstances, previousIdealState), - resourceName, masterStateValue, slaveStateValue); + resourceName, primaryStateValue, secondaryStateValue); } /** - * Given the list of master partition that will be migrated away from the storage instance, - * Remove their entries from the local instance slave assignment map. - * @param slaveAssignmentMap the local instance slave assignment map - * @param masterPartionsMoved the list of master partition ids that will be migrated away - * @param removedAssignmentMap keep track of the removed slave assignment info. The info can be + * Given the list of primary partitions that will be migrated away from the storage instance, + * Remove their entries from the local instance secondary assignment map. + * @param secondaryAssignmentMap the local instance secondary assignment map + * @param primaryPartionsMoved the list of primary partition ids that will be migrated away + * @param removedAssignmentMap keep track of the removed secondary assignment info. The info can + * be * used by new added storage nodes. */ - static void removeFromSlaveAssignmentMap(Map> slaveAssignmentMap, - List masterPartionsMoved, Map> removedAssignmentMap) { - for (String instanceName : slaveAssignmentMap.keySet()) { - List slaveAssignment = slaveAssignmentMap.get(instanceName); - for (Integer partitionId : masterPartionsMoved) { - if (slaveAssignment.contains(partitionId)) { - slaveAssignment.remove(partitionId); + static void removeFromSecondaryAssignmentMap(Map> secondaryAssignmentMap, + List primaryPartionsMoved, Map> removedAssignmentMap) { + for (String instanceName : secondaryAssignmentMap.keySet()) { + List secondaryAssignment = secondaryAssignmentMap.get(instanceName); + for (Integer partitionId : primaryPartionsMoved) { + if (secondaryAssignment.contains(partitionId)) { + secondaryAssignment.remove(partitionId); if (!removedAssignmentMap.containsKey(instanceName)) { removedAssignmentMap.put(instanceName, new ArrayList()); } @@ -629,38 +632,40 @@ static void removeFromSlaveAssignmentMap(Map> slaveAssignm /** * Since some new storage instances are added, each existing storage instance should migrate some - * slave partitions to the new added instances. - * The algorithm keeps moving one partition to from the instance that hosts most slave partitions + * secondary partitions to the new added instances. + * The algorithm keeps moving one partition to from the instance that hosts most secondary + * partitions * to the instance that hosts least number of partitions, until max-min <= 1. - * In this way we can guarantee that all instances hosts almost same number of slave partitions, - * also - * slave partitions are evenly distributed. - * @param slaveAssignmentMap the local instance slave assignment map - * @param masterPartionsMoved the list of master partition ids that will be migrated away - * @param removedAssignmentMap keep track of the removed slave assignment info. The info can be + * In this way we can guarantee that all instances hosts almost same number of secondary + * partitions, also + * secondary partitions are evenly distributed. + * @param secondaryAssignmentMap the local instance secondary assignment map + * @param primaryPartionsMoved the list of primary partition ids that will be migrated away + * @param removedAssignmentMap keep track of the removed secondary assignment info. The info can + * be * used by new added storage nodes. */ - static int migrateSlaveAssignMapToNewInstances(Map> nodeSlaveAssignmentMap, - List newInstances) { + static int migrateSecondaryAssignMapToNewInstances( + Map> secondaryAssignmentMap, List newInstances) { int moves = 0; boolean done = false; for (String newInstance : newInstances) { - nodeSlaveAssignmentMap.put(newInstance, new ArrayList()); + secondaryAssignmentMap.put(newInstance, new ArrayList()); } while (!done) { List maxAssignment = null, minAssignment = null; int minCount = Integer.MAX_VALUE, maxCount = Integer.MIN_VALUE; String minInstance = ""; - for (String instanceName : nodeSlaveAssignmentMap.keySet()) { - List slaveAssignment = nodeSlaveAssignmentMap.get(instanceName); - if (minCount > slaveAssignment.size()) { - minCount = slaveAssignment.size(); - minAssignment = slaveAssignment; + for (String instanceName : secondaryAssignmentMap.keySet()) { + List secondaryAssignment = secondaryAssignmentMap.get(instanceName); + if (minCount > secondaryAssignment.size()) { + minCount = secondaryAssignment.size(); + minAssignment = secondaryAssignment; minInstance = instanceName; } - if (maxCount < slaveAssignment.size()) { - maxCount = slaveAssignment.size(); - maxAssignment = slaveAssignment; + if (maxCount < secondaryAssignment.size()) { + maxCount = secondaryAssignment.size(); + maxAssignment = secondaryAssignment; } } if (maxCount - minCount <= 1) { @@ -688,7 +693,7 @@ static int migrateSlaveAssignMapToNewInstances(Map> nodeSl /** * Randomly select a number of elements from original list and put them in the selectedList - * The algorithm is used to select master partitions to be migrated when new instances are added. + * The algorithm is used to select primary partitions to be migrated when new instances are added. * @param originalList the original list * @param selectedList the list that contain selected elements * @param num number of elements to be selected @@ -716,7 +721,7 @@ public static void main(String args[]) { } int partitions = 48 * 3, replicas = 3; Map resultOriginal = - DefaultIdealStateCalculator.calculateInitialIdealState(instanceNames, partitions, replicas); + DefaultTwoStateStrategy.calculateInitialIdealState(instanceNames, partitions, replicas); } } diff --git a/helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorForEspressoRelay.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/EspressoRelayStrategy.java similarity index 95% rename from helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorForEspressoRelay.java rename to helix-core/src/main/java/org/apache/helix/controller/strategy/EspressoRelayStrategy.java index 25ae625fe8..06bcbede04 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorForEspressoRelay.java +++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/EspressoRelayStrategy.java @@ -1,4 +1,4 @@ -package org.apache.helix.tools; +package org.apache.helix.controller.strategy; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -27,9 +27,10 @@ import java.util.TreeMap; import org.apache.helix.HelixException; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.model.IdealState; -public class IdealStateCalculatorForEspressoRelay { +public class EspressoRelayStrategy { public static IdealState calculateRelayIdealState(List partitions, List instances, String resultRecordName, int replica, String firstValue, String restValue, String stateModelName) { @@ -42,7 +43,7 @@ public static IdealState calculateRelayIdealState(List partitions, IdealState result = new IdealState(resultRecordName); result.setNumPartitions(partitions.size()); result.setReplicas("" + replica); - result.setStateModelDefRef(stateModelName); + result.setStateModelDefId(StateModelDefId.from(stateModelName)); int groups = instances.size() / replica; int remainder = instances.size() % replica; diff --git a/helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorByRush.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHMasterSlaveStrategy.java similarity index 97% rename from helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorByRush.java rename to helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHMasterSlaveStrategy.java index 7677b42b58..39561ae50b 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorByRush.java +++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHMasterSlaveStrategy.java @@ -1,4 +1,4 @@ -package org.apache.helix.tools; +package org.apache.helix.controller.strategy; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -29,7 +29,7 @@ import org.apache.helix.ZNRecord; import org.apache.helix.model.IdealState.IdealStateProperty; -public class IdealStateCalculatorByRush { +public class RUSHMasterSlaveStrategy { /** * Build the config map for RUSH algorithm. The input of RUSH algorithm groups * nodes into "cluster"s, and different clusters can be assigned with @@ -263,7 +263,7 @@ public static void main(String args[]) throws Exception { List weights1 = new ArrayList(); weights1.add(1); ZNRecord result = - IdealStateCalculatorByRush.calculateIdealState(instanceCluster1, weights1, partitions, + RUSHMasterSlaveStrategy.calculateIdealState(instanceCluster1, weights1, partitions, replicas, resourceName); printIdealStateStats(result); @@ -275,7 +275,7 @@ public static void main(String args[]) throws Exception { instanceCluster1.add(instanceNames2); weights1.add(1); ZNRecord result2 = - IdealStateCalculatorByRush.calculateIdealState(instanceCluster1, weights1, partitions, + RUSHMasterSlaveStrategy.calculateIdealState(instanceCluster1, weights1, partitions, replicas, resourceName); printDiff(result, result2); diff --git a/helix-core/src/main/java/org/apache/helix/tools/RUSHrHash.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHrHash.java similarity index 98% rename from helix-core/src/main/java/org/apache/helix/tools/RUSHrHash.java rename to helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHrHash.java index fb9c594e4e..2314dede36 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/RUSHrHash.java +++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHrHash.java @@ -1,4 +1,4 @@ -package org.apache.helix.tools; +package org.apache.helix.controller.strategy; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,7 +19,9 @@ * under the License. */ -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Random; import java.util.zip.CRC32; public class RUSHrHash { @@ -299,7 +301,7 @@ public int drawWHG(int replicas, int disksInCurrentCluster, int totalDisks, int for (int i = 0; i < replicas; i++) { if (totalDisks != 0) { ranInt = ran.nextInt((int) (ranMax + 1)); - z = ((float) ranInt / ranMax); + z = (ranInt / ranMax); prob = ((float) disksInCurrentCluster / (float) totalDisks); if (z <= prob) { found++; diff --git a/helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorByShuffling.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/ShufflingTwoStateStrategy.java similarity index 74% rename from helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorByShuffling.java rename to helix-core/src/main/java/org/apache/helix/controller/strategy/ShufflingTwoStateStrategy.java index d4764eff57..7b4ce739f7 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorByShuffling.java +++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/ShufflingTwoStateStrategy.java @@ -1,4 +1,4 @@ -package org.apache.helix.tools; +package org.apache.helix.controller.strategy; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -31,7 +31,8 @@ /* * Ideal state calculator for the cluster manager V1. The ideal state is - * calculated by randomly assign master partitions to storage nodes. + * calculated by randomly assign primary partitions to storage nodes. This is intended for a + * two-state scheme where one is primary and the other is secondary. * * Note that the following code is a native strategy and is for cluster manager V1 only. We will * use the other algorithm to calculate the ideal state in future milestones. @@ -39,19 +40,30 @@ * * */ -public class IdealStateCalculatorByShuffling { +public class ShufflingTwoStateStrategy { /* * Given the number of nodes, partitions and replica number, calculate the - * ideal state in the following manner: For the master partition assignment, + * ideal state in the following manner: For the primary partition assignment, * 1. construct Arraylist partitionList, with partitionList[i] = i; 2. Shuffle * the partitions array 3. Scan the shuffled array, then assign * partitionList[i] to node (i % nodes) * for the slave partitions, simply put them in the node after the node that + * <<<<<<< + * HEAD:helix-core/src/main/java/org/apache/helix/controller/strategy/ShufflingTwoStateStrategy + * .java + * contains the primary partition. + * ======= * contains the master partition. + * >>>>>>> + * master:helix-core/src/main/java/org/apache/helix/tools/IdealStateCalculatorByShuffling.java * The result of the method is a ZNRecord, which contains a list of maps; each - * map is from the name of nodes to either "MASTER" or "SLAVE". + * map is from the name of nodes to either state name ("MASTER" or "SLAVE" for + * MasterSlave). */ + /** + * Calculate an ideal state for a MasterSlave configuration + */ public static ZNRecord calculateIdealState(List instanceNames, int partitions, int replicas, String resourceName, long randomSeed) { return calculateIdealState(instanceNames, partitions, replicas, resourceName, randomSeed, @@ -59,9 +71,9 @@ public static ZNRecord calculateIdealState(List instanceNames, int parti } public static ZNRecord calculateIdealState(List instanceNames, int partitions, - int replicas, String resourceName, long randomSeed, String masterValue, String slaveValue) { + int replicas, String resourceName, long randomSeed, String primaryValue, String secondaryValue) { if (instanceNames.size() <= replicas) { - throw new IllegalArgumentException("Replicas must be less than number of storage nodes"); + throw new IllegalArgumentException("Replicas must be less than number of nodes"); } Collections.sort(instanceNames); @@ -79,18 +91,18 @@ public static ZNRecord calculateIdealState(List instanceNames, int parti for (int i = 0; i < partitionList.size(); i++) { int partitionId = partitionList.get(i); Map partitionAssignment = new TreeMap(); - int masterNode = i % instanceNames.size(); - // the first in the list is the node that contains the master - partitionAssignment.put(instanceNames.get(masterNode), masterValue); + int primaryNode = i % instanceNames.size(); + // the first in the list is the node that contains the primary + partitionAssignment.put(instanceNames.get(primaryNode), primaryValue); - // for the jth replica, we put it on (masterNode + j) % nodes-th + // for the jth replica, we put it on (primaryNode + j) % nodes-th // node for (int j = 1; j <= replicas; j++) { - int index = (masterNode + j * partitionList.size()) % instanceNames.size(); + int index = (primaryNode + j * partitionList.size()) % instanceNames.size(); while (partitionAssignment.keySet().contains(instanceNames.get(index))) { index = (index + 1) % instanceNames.size(); } - partitionAssignment.put(instanceNames.get(index), slaveValue); + partitionAssignment.put(instanceNames.get(index), secondaryValue); } String partitionName = resourceName + "_" + partitionId; result.setMapField(partitionName, partitionAssignment); diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationTypeFactory.java b/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationTypeFactory.java deleted file mode 100644 index d9466411b7..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationTypeFactory.java +++ /dev/null @@ -1,69 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.StringTokenizer; - -import org.apache.log4j.Logger; - -public class AggregationTypeFactory { - private static final Logger logger = Logger.getLogger(AggregationTypeFactory.class); - - public AggregationTypeFactory() { - } - - // TODO: modify this function so that it takes a single string, but can parse - // apart params from type - public static AggregationType getAggregationType(String input) { - if (input == null) { - logger.error("AggregationType name is null"); - return null; - } - StringTokenizer tok = new StringTokenizer(input, AggregationType.DELIM); - String type = tok.nextToken(); - int numParams = tok.countTokens(); - String[] params = new String[numParams]; - for (int i = 0; i < numParams; i++) { - if (!tok.hasMoreTokens()) { - logger.error("Trying to parse non-existent params"); - return null; - } - params[i] = tok.nextToken(); - } - - if (type.equals(AccumulateAggregationType.TYPE_NAME)) { - return new AccumulateAggregationType(); - } else if (type.equals(DecayAggregationType.TYPE_NAME)) { - if (params.length < 1) { - logger.error("DecayAggregationType must contain parameter"); - return null; - } - return new DecayAggregationType(Double.parseDouble(params[0])); - } else if (type.equals(WindowAggregationType.TYPE_NAME)) { - if (params.length < 1) { - logger.error("WindowAggregationType must contain parameter"); - } - return new WindowAggregationType(Integer.parseInt(params[0])); - } else { - logger.error("Unknown AggregationType " + type); - return null; - } - } -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java b/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java deleted file mode 100644 index 34e449a261..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java +++ /dev/null @@ -1,58 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.log4j.Logger; - -public class DecayAggregationType implements AggregationType { - - private static final Logger logger = Logger.getLogger(DecayAggregationType.class); - - public final static String TYPE_NAME = "decay"; - - double _decayFactor = 0.1; - - public DecayAggregationType(double df) { - super(); - _decayFactor = df; - } - - @Override - public String getName() { - StringBuilder sb = new StringBuilder(); - sb.append(TYPE_NAME); - sb.append(DELIM); - sb.append(_decayFactor); - return sb.toString(); - } - - @Override - public String merge(String iv, String ev, long prevTimestamp) { - double incomingVal = Double.parseDouble(iv); - double existingVal = Double.parseDouble(ev); - long currTimestamp = System.currentTimeMillis(); - double minutesOld = (currTimestamp - prevTimestamp) / 60000.0; - // come up with decay coeff for old value. More time passed, the more it - // decays - double oldDecayCoeff = Math.pow((1 - _decayFactor), minutesOld); - return String - .valueOf((double) (oldDecayCoeff * existingVal + (1 - oldDecayCoeff) * incomingVal)); - } -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java deleted file mode 100644 index b7502f4509..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java +++ /dev/null @@ -1,87 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.lang.management.ManagementFactory; -import java.lang.management.OperatingSystemMXBean; -import java.util.HashMap; -import java.util.Map; -import java.util.TreeMap; - -import org.apache.helix.ZNRecord; -import org.apache.log4j.Logger; - -class DefaultHealthReportProvider extends HealthReportProvider { - private static final Logger _logger = Logger.getLogger(DefaultHealthReportProvider.class); - - public final static String _availableCPUs = "availableCPUs"; - public final static String _freePhysicalMemory = "freePhysicalMemory"; - public final static String _totalJvmMemory = "totalJvmMemory"; - public final static String _freeJvmMemory = "freeJvmMemory"; - public final static String _averageSystemLoad = "averageSystemLoad"; - - public DefaultHealthReportProvider() { - } - - @Override - public Map getRecentHealthReport() { - OperatingSystemMXBean osMxBean = ManagementFactory.getOperatingSystemMXBean(); - long freeJvmMemory = Runtime.getRuntime().freeMemory(); - long totalJvmMemory = Runtime.getRuntime().totalMemory(); - int availableCPUs = osMxBean.getAvailableProcessors(); - double avgSystemLoad = osMxBean.getSystemLoadAverage(); - long freePhysicalMemory = Long.MAX_VALUE; - - try { - // if( osMxBean instanceof com.sun.management.OperatingSystemMXBean) - // { - // com.sun.management.OperatingSystemMXBean sunOsMxBean - // = (com.sun.management.OperatingSystemMXBean) osMxBean; - // freePhysicalMemory = sunOsMxBean.getFreePhysicalMemorySize(); - // } - } catch (Throwable t) { - _logger.error(t); - } - - Map result = new TreeMap(); - - result.put(_availableCPUs, "" + availableCPUs); - result.put(_freePhysicalMemory, "" + freePhysicalMemory); - result.put(_freeJvmMemory, "" + freeJvmMemory); - result.put(_totalJvmMemory, "" + totalJvmMemory); - result.put(_averageSystemLoad, "" + avgSystemLoad); - - return result; - } - - @Override - public Map> getRecentPartitionHealthReport() { - Map> result = new HashMap>(); - - result.put(getReportName(), getRecentHealthReport()); - return result; - } - - @Override - public void resetStats() { - // TODO Auto-generated method stub - - } -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultPerfCounters.java b/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultPerfCounters.java deleted file mode 100644 index e1afd5cfa9..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultPerfCounters.java +++ /dev/null @@ -1,95 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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 org.apache.helix.ZNRecord; -import org.apache.log4j.Logger; - -@Deprecated -public class DefaultPerfCounters extends ZNRecord { - private static final Logger _logger = Logger.getLogger(DefaultPerfCounters.class); - - public final static String _availableCPUs = "availableCPUs"; - public final static String _freePhysicalMemory = "freePhysicalMemory"; - public final static String _totalJvmMemory = "totalJvmMemory"; - public final static String _freeJvmMemory = "freeJvmMemory"; - public final static String _averageSystemLoad = "averageSystemLoad"; - - public DefaultPerfCounters(String instanceName, long availableCPUs, long freePhysicalMemory, - long freeJvmMemory, long totalJvmMemory, double averageSystemLoad) { - super("DefaultPerfCounters"); - setSimpleField("instanceName", instanceName); - setSimpleField("createTime", new Date().toString()); - - setSimpleField(_availableCPUs, "" + availableCPUs); - setSimpleField(_freePhysicalMemory, "" + freePhysicalMemory); - setSimpleField(_freeJvmMemory, "" + freeJvmMemory); - setSimpleField(_totalJvmMemory, "" + totalJvmMemory); - setSimpleField(_averageSystemLoad, "" + averageSystemLoad); - } - - public long getAvailableCpus() { - return getSimpleLongVal(_availableCPUs); - } - - public double getAverageSystemLoad() { - return getSimpleDoubleVal(_averageSystemLoad); - } - - public long getTotalJvmMemory() { - return getSimpleLongVal(_totalJvmMemory); - } - - public long getFreeJvmMemory() { - return getSimpleLongVal(_freeJvmMemory); - } - - public long getFreePhysicalMemory() { - return getSimpleLongVal(_freePhysicalMemory); - } - - long getSimpleLongVal(String key) { - String strVal = getSimpleField(key); - if (strVal == null) { - return 0; - } - try { - return Long.parseLong(strVal); - } catch (Exception e) { - _logger.warn(e); - return 0; - } - } - - double getSimpleDoubleVal(String key) { - String strVal = getSimpleField(key); - if (strVal == null) { - return 0; - } - try { - return Double.parseDouble(strVal); - } catch (Exception e) { - _logger.warn(e); - return 0; - } - } -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregationTask.java b/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregationTask.java deleted file mode 100644 index 05ffaef85d..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregationTask.java +++ /dev/null @@ -1,89 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.Random; -import java.util.Timer; -import java.util.TimerTask; - -import org.apache.helix.HelixTimerTask; -import org.apache.log4j.Logger; - -public class HealthStatsAggregationTask extends HelixTimerTask { - private static final Logger LOG = Logger.getLogger(HealthStatsAggregationTask.class); - public final static int DEFAULT_HEALTH_CHECK_LATENCY = 30 * 1000; - - final HealthStatsAggregator _healthStatsAggregator; - - class HealthStatsAggregationTaskTimer extends TimerTask { - - @Override - public void run() { - _healthStatsAggregator.aggregate(); - } - - } - - private Timer _timer; - private final int _delay; - private final int _period; - - public HealthStatsAggregationTask(HealthStatsAggregator healthStatsAggregator, int delay, - int period) { - _healthStatsAggregator = healthStatsAggregator; - - _delay = delay; - _period = period; - } - - public HealthStatsAggregationTask(HealthStatsAggregator healthStatsAggregator) { - this(healthStatsAggregator, DEFAULT_HEALTH_CHECK_LATENCY, DEFAULT_HEALTH_CHECK_LATENCY); - } - - @Override - public void start() { - - if (_timer == null) { - LOG.info("START HealthStatsAggregationTimerTask"); - - // Remove all the previous health check values, if any - _healthStatsAggregator.init(); - - _timer = new Timer("HealthStatsAggregationTimerTask", true); - _timer.scheduleAtFixedRate(new HealthStatsAggregationTaskTimer(), - new Random().nextInt(_delay), _period); - } else { - LOG.warn("HealthStatsAggregationTimerTask already started"); - } - } - - @Override - public synchronized void stop() { - if (_timer != null) { - LOG.info("Stop HealthStatsAggregationTimerTask"); - _timer.cancel(); - _healthStatsAggregator.reset(); - _timer = null; - } else { - LOG.warn("HealthStatsAggregationTimerTask already stopped"); - } - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregator.java b/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregator.java deleted file mode 100644 index bc95e6dd02..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregator.java +++ /dev/null @@ -1,141 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.List; -import java.util.Map; - -import org.apache.helix.ConfigAccessor; -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.controller.pipeline.Pipeline; -import org.apache.helix.controller.pipeline.Stage; -import org.apache.helix.controller.stages.ClusterEvent; -import org.apache.helix.controller.stages.ReadHealthDataStage; -import org.apache.helix.controller.stages.StatsAggregationStage; -import org.apache.helix.model.ConfigScope; -import org.apache.helix.model.builder.ConfigScopeBuilder; -import org.apache.helix.monitoring.mbeans.ClusterAlertMBeanCollection; -import org.apache.helix.monitoring.mbeans.HelixStageLatencyMonitor; -import org.apache.log4j.Logger; - -public class HealthStatsAggregator { - private static final Logger LOG = Logger.getLogger(HealthStatsAggregator.class); - - public final static int DEFAULT_HEALTH_CHECK_LATENCY = 30 * 1000; - - private final HelixManager _manager; - private final Pipeline _healthStatsAggregationPipeline; - private final ClusterAlertMBeanCollection _alertItemCollection; - private final Map _stageLatencyMonitorMap = - new HashMap(); - - public HealthStatsAggregator(HelixManager manager) { - _manager = manager; - - // health stats pipeline - _healthStatsAggregationPipeline = new Pipeline(); - _healthStatsAggregationPipeline.addStage(new ReadHealthDataStage()); - StatsAggregationStage statAggregationStage = new StatsAggregationStage(); - _healthStatsAggregationPipeline.addStage(statAggregationStage); - _alertItemCollection = statAggregationStage.getClusterAlertMBeanCollection(); - - registerStageLatencyMonitor(_healthStatsAggregationPipeline); - } - - private void registerStageLatencyMonitor(Pipeline pipeline) { - for (Stage stage : pipeline.getStages()) { - String stgName = stage.getStageName(); - if (!_stageLatencyMonitorMap.containsKey(stgName)) { - try { - _stageLatencyMonitorMap.put(stage.getStageName(), - new HelixStageLatencyMonitor(_manager.getClusterName(), stgName)); - } catch (Exception e) { - LOG.error("Couldn't create StageLatencyMonitor mbean for stage: " + stgName, e); - } - } else { - LOG.error("StageLatencyMonitor for stage: " + stgName + " already exists. Skip register it"); - } - } - } - - public synchronized void aggregate() { - if (!isEnabled()) { - LOG.info("HealthAggregationTask is disabled."); - return; - } - - if (!_manager.isLeader()) { - LOG.error("Cluster manager: " + _manager.getInstanceName() - + " is not leader. Pipeline will not be invoked"); - return; - } - - try { - ClusterEvent event = new ClusterEvent("healthChange"); - event.addAttribute("helixmanager", _manager); - event.addAttribute("HelixStageLatencyMonitorMap", _stageLatencyMonitorMap); - - _healthStatsAggregationPipeline.handle(event); - _healthStatsAggregationPipeline.finish(); - } catch (Exception e) { - LOG.error("Exception while executing pipeline: " + _healthStatsAggregationPipeline, e); - } - } - - private boolean isEnabled() { - ConfigAccessor configAccessor = _manager.getConfigAccessor(); - boolean enabled = true; - if (configAccessor != null) { - // zk-based cluster manager - ConfigScope scope = new ConfigScopeBuilder().forCluster(_manager.getClusterName()).build(); - String isEnabled = configAccessor.get(scope, "healthChange.enabled"); - if (isEnabled != null) { - enabled = new Boolean(isEnabled); - } - } else { - LOG.debug("File-based cluster manager doesn't support disable healthChange"); - } - return enabled; - } - - public void init() { - // Remove all the previous health check values, if any - HelixDataAccessor accessor = _manager.getHelixDataAccessor(); - List existingHealthRecordNames = - accessor.getChildNames(accessor.keyBuilder().healthReports(_manager.getInstanceName())); - for (String healthReportName : existingHealthRecordNames) { - LOG.info("Removing old healthrecord " + healthReportName); - accessor.removeProperty(accessor.keyBuilder().healthReport(_manager.getInstanceName(), - healthReportName)); - } - - } - - public void reset() { - _alertItemCollection.reset(); - - for (HelixStageLatencyMonitor stgLatencyMonitor : _stageLatencyMonitorMap.values()) { - stgLatencyMonitor.reset(); - } - - } -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollector.java b/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollector.java deleted file mode 100644 index 266ed8b203..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollector.java +++ /dev/null @@ -1,32 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.ZNRecord; - -public interface ParticipantHealthReportCollector { - public abstract void addHealthReportProvider(HealthReportProvider provider); - - public abstract void removeHealthReportProvider(HealthReportProvider provider); - - public abstract void reportHealthReportMessage(ZNRecord healthReport); - - public abstract void transmitHealthReports(); -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java b/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java deleted file mode 100644 index a300a02204..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java +++ /dev/null @@ -1,115 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.LinkedList; -import java.util.Map; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.alerts.StatsHolder; -import org.apache.helix.model.HealthStat; -import org.apache.log4j.Logger; - -public class ParticipantHealthReportCollectorImpl implements ParticipantHealthReportCollector { - private final LinkedList _healthReportProviderList = - new LinkedList(); - private static final Logger _logger = Logger - .getLogger(ParticipantHealthReportCollectorImpl.class); - private final HelixManager _helixManager; - String _instanceName; - - public ParticipantHealthReportCollectorImpl(HelixManager helixManager, String instanceName) { - _helixManager = helixManager; - _instanceName = instanceName; - addDefaultHealthCheckInfoProvider(); - } - - private void addDefaultHealthCheckInfoProvider() { - addHealthReportProvider(new DefaultHealthReportProvider()); - } - - @Override - public void addHealthReportProvider(HealthReportProvider provider) { - try { - synchronized (_healthReportProviderList) { - if (!_healthReportProviderList.contains(provider)) { - _healthReportProviderList.add(provider); - } else { - _logger.warn("Skipping a duplicated HealthCheckInfoProvider"); - } - } - } catch (Exception e) { - _logger.error(e); - } - } - - @Override - public void removeHealthReportProvider(HealthReportProvider provider) { - synchronized (_healthReportProviderList) { - if (_healthReportProviderList.contains(provider)) { - _healthReportProviderList.remove(provider); - } else { - _logger.warn("Skip removing a non-exist HealthCheckInfoProvider"); - } - } - } - - @Override - public void reportHealthReportMessage(ZNRecord healthCheckInfoUpdate) { - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - accessor.setProperty(keyBuilder.healthReport(_instanceName, healthCheckInfoUpdate.getId()), - new HealthStat(healthCheckInfoUpdate)); - - } - - @Override - public synchronized void transmitHealthReports() { - synchronized (_healthReportProviderList) { - for (HealthReportProvider provider : _healthReportProviderList) { - try { - Map report = provider.getRecentHealthReport(); - Map> partitionReport = - provider.getRecentPartitionHealthReport(); - ZNRecord record = new ZNRecord(provider.getReportName()); - if (report != null) { - record.setSimpleFields(report); - } - if (partitionReport != null) { - record.setMapFields(partitionReport); - } - record.setSimpleField(StatsHolder.TIMESTAMP_NAME, "" + System.currentTimeMillis()); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - accessor.setProperty(keyBuilder.healthReport(_instanceName, record.getId()), - new HealthStat(record)); - - provider.resetStats(); - } catch (Exception e) { - _logger.error("fail to transmit health report", e); - } - } - } - } -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportTask.java b/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportTask.java deleted file mode 100644 index 59d74c7dbd..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportTask.java +++ /dev/null @@ -1,71 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.Random; -import java.util.Timer; -import java.util.TimerTask; - -import org.apache.helix.HelixTimerTask; -import org.apache.log4j.Logger; - -public class ParticipantHealthReportTask extends HelixTimerTask { - private static final Logger LOG = Logger.getLogger(ParticipantHealthReportTask.class); - public final static int DEFAULT_REPORT_LATENCY = 60 * 1000; - - Timer _timer; - final ParticipantHealthReportCollectorImpl _healthReportCollector; - - class ParticipantHealthReportTimerTask extends TimerTask { - - @Override - public void run() { - _healthReportCollector.transmitHealthReports(); - } - } - - public ParticipantHealthReportTask(ParticipantHealthReportCollectorImpl healthReportCollector) { - _healthReportCollector = healthReportCollector; - } - - @Override - public void start() { - if (_timer == null) { - LOG.info("Start HealthCheckInfoReportingTask"); - _timer = new Timer("ParticipantHealthReportTimerTask", true); - _timer.scheduleAtFixedRate(new ParticipantHealthReportTimerTask(), - new Random().nextInt(DEFAULT_REPORT_LATENCY), DEFAULT_REPORT_LATENCY); - } else { - LOG.warn("ParticipantHealthReportTimerTask already started"); - } - } - - @Override - public void stop() { - if (_timer != null) { - LOG.info("Stop ParticipantHealthReportTimerTask"); - _timer.cancel(); - _timer = null; - } else { - LOG.warn("ParticipantHealthReportTimerTask already stopped"); - } - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java deleted file mode 100644 index 3a151a7547..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java +++ /dev/null @@ -1,138 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.TreeMap; - -import org.apache.log4j.Logger; - -public class PerformanceHealthReportProvider extends HealthReportProvider { - - private static final Logger _logger = Logger.getLogger(PerformanceHealthReportProvider.class); - - public final static String _testStat = "testStat"; - public final static String _readLatencyStat = "readLatencyStat"; - public final static String _requestCountStat = "requestCountStat"; - public final static String _partitionRequestCountStat = "partitionRequestCountStat"; - - public static final String _performanceCounters = "performanceCounters"; - - public int readLatencyCount = 0; - public double readLatencySum = 0; - - public int requestCount = 0; - - // private final Map _partitionCountsMap = new HashMap(); - - private final Map> _partitionStatMaps = - new HashMap>(); - - public PerformanceHealthReportProvider() { - } - - @Override - public Map getRecentHealthReport() { - long testStat = 10; - - Map result = new TreeMap(); - - result.put(_testStat, "" + testStat); - result.put(_readLatencyStat, "" + readLatencySum / (double) readLatencyCount); - result.put(_requestCountStat, "" + requestCount); - - return result; - } - - @Override - public Map> getRecentPartitionHealthReport() { - Map> result = new TreeMap>(); - for (String statName : _partitionStatMaps.keySet()) { - result.put(statName, _partitionStatMaps.get(statName)); - } - return result; - } - - HashMap getStatMap(String statName, boolean createIfMissing) { - // check if map for this stat exists. if not, create it - HashMap statMap; - if (!_partitionStatMaps.containsKey(statName)) { - if (!createIfMissing) { - return null; - } - statMap = new HashMap(); - _partitionStatMaps.put(statName, statMap); - } else { - statMap = _partitionStatMaps.get(statName); - } - return statMap; - } - - // TODO: - // Currently participant is source of truth and updates ZK. We want ZK to be - // source of truth. - // Revise this approach the participant sends deltas of stats to controller - // (ZK?) and have controller do aggregation - // and update ZK. Make sure to wipe the participant between uploads. - String getPartitionStat(HashMap partitionMap, String partitionName) { - return partitionMap.get(partitionName); - } - - void setPartitionStat(HashMap partitionMap, String partitionName, String value) { - partitionMap.put(partitionName, value); - } - - public void incrementPartitionStat(String statName, String partitionName) { - HashMap statMap = getStatMap(statName, true); - String currValStr = getPartitionStat(statMap, partitionName); - double currVal; - if (currValStr == null) { - currVal = 1.0; - } else { - currVal = Double.parseDouble(getPartitionStat(statMap, partitionName)); - currVal++; - } - setPartitionStat(statMap, partitionName, String.valueOf(currVal)); - } - - public void submitPartitionStat(String statName, String partitionName, String value) { - HashMap statMap = getStatMap(statName, true); - setPartitionStat(statMap, partitionName, value); - } - - public String getPartitionStat(String statName, String partitionName) { - HashMap statMap = getStatMap(statName, false); - if (statMap == null) { - return null; - } else { - return statMap.get(partitionName); - } - } - - public void resetStats() { - _partitionStatMaps.clear(); - } - - public String getReportName() { - return _performanceCounters; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java b/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java deleted file mode 100644 index d5a31d4a7b..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java +++ /dev/null @@ -1,125 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.log4j.Logger; - -import java.util.Map; - -public class Stat { - - private static final Logger _logger = Logger.getLogger(Stat.class); - - public final static String OP_TYPE = "HTTP_OP"; - public final static String MEASUREMENT_TYPE = "MEASUREMENT"; - public final static String RESOURCE_NAME = "RESOURCE_NAME"; - public final static String PARTITION_NAME = "PARTITION_NAME"; - public final static String NODE_NAME = "NODE_NAME"; - public final static String TIMESTAMP = "TIMESTAMP"; - public final static String RETURN_STATUS = "RETURN_STATUS"; - public final static String METRIC_NAME = "METRIC_NAME"; - public final static String AGG_TYPE = "AGG_TYPE"; - - public String _opType; - public String _measurementType; - public String _resourceName; - public String _partitionName; - public String _nodeName; - public String _returnStatus; - public String _metricName; - public String _aggTypeName; - public String _timestamp; - - public Stat(String opType, String measurementType, String resourceName, String partitionName, - String nodeName) { - // this(opType, measurementType, resourceName, partitionName, nodeName, - // null, null, null); - this(opType, measurementType, resourceName, partitionName, nodeName, null, null, null); - } - - public Stat(String opType, String measurementType, String resourceName, String partitionName, - String nodeName, String returnStatus, String metricName, AggregationType aggType) { - this._opType = opType; - this._measurementType = measurementType; - this._resourceName = resourceName; - this._partitionName = partitionName; - this._nodeName = nodeName; - this._returnStatus = returnStatus; - this._metricName = metricName; - this._aggTypeName = null; - if (aggType != null) { - this._aggTypeName = aggType.getName(); - } - - _timestamp = String.valueOf(System.currentTimeMillis()); - } - - public Stat(Map in) { - _opType = in.get(OP_TYPE); - _measurementType = in.get(MEASUREMENT_TYPE); - _resourceName = in.get(RESOURCE_NAME); - _partitionName = in.get(PARTITION_NAME); - _nodeName = in.get(NODE_NAME); - _timestamp = String.valueOf(System.currentTimeMillis()); - } - - public void setAggType(AggregationType aggType) { - this._aggTypeName = aggType.getName(); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof Stat)) { - return false; - } - Stat other = (Stat) obj; - if (!_partitionName.equals(other._partitionName)) { - return false; - } - if (!_opType.equals(other._opType)) { - return false; - } - if (!_measurementType.equals(other._measurementType)) { - return false; - } - if (!_resourceName.equals(other._resourceName)) { - return false; - } - if (!_nodeName.equals(other._nodeName)) { - return false; - } - return true; - } - - @Override - public int hashCode() { - return (_partitionName + _opType + _measurementType + _resourceName + _nodeName).hashCode(); - } - - public void addAlert(long value) { - // TODO Auto-generated method stub - - } - - public String toString() { - return _nodeName + "." + _resourceName + "." + _partitionName + "." + _opType + "." - + _measurementType + "." + _returnStatus + "." + _metricName + "." + _aggTypeName; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java deleted file mode 100644 index 04bb9437aa..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java +++ /dev/null @@ -1,159 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.log4j.Logger; - -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -public class StatHealthReportProvider extends HealthReportProvider { - - private static final Logger _logger = Logger.getLogger(StatHealthReportProvider.class); - - /* - * public final static String _testStat = "testStat"; public final static - * String _readLatencyStat = "readLatencyStat"; public final static String - * _requestCountStat = "requestCountStat"; public final static String - * _partitionRequestCountStat = "partitionRequestCountStat"; - */ - - public static final String REPORT_NAME = "ParticipantStats"; - public String _reportName = REPORT_NAME; - - public static final String STAT_VALUE = "value"; - public static final String TIMESTAMP = "timestamp"; - - public int readLatencyCount = 0; - public double readLatencySum = 0; - - public int requestCount = 0; - - // private final Map _partitionCountsMap = new HashMap(); - - // private final Map> _partitionStatMaps = new - // HashMap>(); - private final ConcurrentHashMap _statsToValues = - new ConcurrentHashMap(); - private final ConcurrentHashMap _statsToTimestamps = - new ConcurrentHashMap(); - - public StatHealthReportProvider() { - } - - @Override - public Map getRecentHealthReport() { - return null; - } - - // TODO: function is misnamed, but return type is what I want - @Override - public Map> getRecentPartitionHealthReport() { - Map> result = new HashMap>(); - for (String stat : _statsToValues.keySet()) { - Map currStat = new HashMap(); - /* - * currStat.put(Stat.OP_TYPE, stat._opType); - * currStat.put(Stat.MEASUREMENT_TYPE, stat._measurementType); - * currStat.put(Stat.NODE_NAME, stat._nodeName); - * currStat.put(Stat.PARTITION_NAME, stat._partitionName); - * currStat.put(Stat.RESOURCE_NAME, stat._resourceName); - * currStat.put(Stat.RETURN_STATUS, stat._returnStatus); - * currStat.put(Stat.METRIC_NAME, stat._metricName); - * currStat.put(Stat.AGG_TYPE, stat._aggTypeName); - */ - currStat.put(TIMESTAMP, _statsToTimestamps.get(stat)); - currStat.put(STAT_VALUE, _statsToValues.get(stat)); - result.put(stat, currStat); - } - return result; - } - - public boolean contains(Stat inStat) { - return _statsToValues.containsKey(inStat); - } - - public Set keySet() { - return _statsToValues.keySet(); - } - - public String getStatValue(Stat inStat) { - return _statsToValues.get(inStat); - } - - public long getStatTimestamp(Stat inStat) { - return Long.parseLong(_statsToTimestamps.get(inStat)); - } - - /* - * public String getStatValue(String opType, String measurementType, String - * resourceName, String partitionName, String nodeName, boolean - * createIfMissing) { Stat rs = new Stat(opType, measurementType, - * resourceName, partitionName, nodeName); String val = - * _statsToValues.get(rs); if (val == null && createIfMissing) { val = "0"; - * _statsToValues.put(rs, val); } return val; } - */ - - public void writeStat(String statName, String val, String timestamp) { - _statsToValues.put(statName, val); - _statsToTimestamps.put(statName, timestamp); - } - - /* - * public void setStat(Stat es, String val, String timestamp) { writeStat(es, - * val, timestamp); } - * public void setStat(String opType, String measurementType, String - * resourceName, String partitionName, String nodeName, double val, String - * timestamp) { Stat rs = new Stat(opType, measurementType, resourceName, - * partitionName, nodeName); writeStat(rs, String.valueOf(val), timestamp); } - */ - - public void incrementStat(String statName, String timestamp) { - // Stat rs = new Stat(opType, measurementType, resourceName, partitionName, - // nodeName); - String val = _statsToValues.get(statName); - if (val == null) { - val = "0"; - } else { - val = String.valueOf(Double.parseDouble(val) + 1); - } - writeStat(statName, val, timestamp); - } - - public int size() { - return _statsToValues.size(); - } - - public void resetStats() { - _statsToValues.clear(); - _statsToTimestamps.clear(); - } - - public void setReportName(String name) { - _reportName = name; - } - - public String getReportName() { - return _reportName; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/WindowAggregationType.java b/helix-core/src/main/java/org/apache/helix/healthcheck/WindowAggregationType.java deleted file mode 100644 index 77161afe65..0000000000 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/WindowAggregationType.java +++ /dev/null @@ -1,67 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.log4j.Logger; - -public class WindowAggregationType implements AggregationType { - - private static final Logger logger = Logger.getLogger(WindowAggregationType.class); - - public final String WINDOW_DELIM = "#"; - - public final static String TYPE_NAME = "window"; - - int _windowSize = 1; - - public WindowAggregationType(int ws) { - super(); - _windowSize = ws; - } - - @Override - public String getName() { - StringBuilder sb = new StringBuilder(); - sb.append(TYPE_NAME); - sb.append(DELIM); - sb.append(_windowSize); - return sb.toString(); - } - - @Override - public String merge(String incomingVal, String existingVal, long prevTimestamp) { - String[] windowVals; - if (existingVal == null) { - return incomingVal; - } else { - windowVals = existingVal.split(WINDOW_DELIM); - int currLength = windowVals.length; - // window not full - if (currLength < _windowSize) { - return existingVal + WINDOW_DELIM + incomingVal; - } - // evict oldest - else { - int firstDelim = existingVal.indexOf(WINDOW_DELIM); - return existingVal.substring(firstDelim + 1) + WINDOW_DELIM + incomingVal; - } - } - } -} diff --git a/helix-core/src/main/java/org/apache/helix/josql/ClusterJosqlQueryProcessor.java b/helix-core/src/main/java/org/apache/helix/josql/ClusterJosqlQueryProcessor.java index 8f11de5f46..bac2b15d8c 100644 --- a/helix-core/src/main/java/org/apache/helix/josql/ClusterJosqlQueryProcessor.java +++ b/helix-core/src/main/java/org/apache/helix/josql/ClusterJosqlQueryProcessor.java @@ -28,9 +28,9 @@ import org.apache.helix.HelixException; import org.apache.helix.HelixManager; import org.apache.helix.HelixProperty; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.PropertyType; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; import org.apache.helix.model.LiveInstance.LiveInstanceProperty; import org.apache.log4j.Logger; diff --git a/helix-core/src/main/java/org/apache/helix/HealthStateChangeListener.java b/helix-core/src/main/java/org/apache/helix/lock/HelixLock.java similarity index 60% rename from helix-core/src/main/java/org/apache/helix/HealthStateChangeListener.java rename to helix-core/src/main/java/org/apache/helix/lock/HelixLock.java index 92023307b7..a567a5c112 100644 --- a/helix-core/src/main/java/org/apache/helix/HealthStateChangeListener.java +++ b/helix-core/src/main/java/org/apache/helix/lock/HelixLock.java @@ -1,4 +1,4 @@ -package org.apache.helix; +package org.apache.helix.lock; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,21 +19,25 @@ * under the License. */ -import java.util.List; - -import org.apache.helix.model.HealthStat; - /** - * Interface to implement to listen for when a health status event is triggered. + * Generic (distributed) lock for Helix-related persisted updates */ -public interface HealthStateChangeListener { +public interface HelixLock { + /** + * Synchronously acquire a lock + * @return true if the lock was acquired, false if could not be acquired + */ + public boolean lock(); + + /** + * Release a lock + * @return true if the lock was released, false if it could not be released + */ + public boolean unlock(); /** - * Invoked when health stats change - * @param instanceName the instance where the health status changed - * @param reports the health statuses - * @param changeContext event properties and state + * Check if this object is blocked waiting on the lock + * @return true if blocked, false otherwise */ - public void onHealthChange(String instanceName, List reports, - NotificationContext changeContext); + public boolean isBlocked(); } diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertValueAndStatus.java b/helix-core/src/main/java/org/apache/helix/lock/HelixLockable.java similarity index 62% rename from helix-core/src/main/java/org/apache/helix/alerts/AlertValueAndStatus.java rename to helix-core/src/main/java/org/apache/helix/lock/HelixLockable.java index 15823126dc..fdb2ca5ee4 100644 --- a/helix-core/src/main/java/org/apache/helix/alerts/AlertValueAndStatus.java +++ b/helix-core/src/main/java/org/apache/helix/lock/HelixLockable.java @@ -1,4 +1,7 @@ -package org.apache.helix.alerts; +package org.apache.helix.lock; + +import org.apache.helix.api.Scope; +import org.apache.helix.api.id.ClusterId; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,24 +22,15 @@ * under the License. */ -public class AlertValueAndStatus { - public final static String VALUE_NAME = "value"; - public final static String FIRED_NAME = "fired"; - - private Tuple value; - private boolean fired; - - public AlertValueAndStatus(Tuple value, boolean fired) { - this.value = value; - this.fired = fired; - } - - public Tuple getValue() { - return value; - } - - public boolean isFired() { - return fired; - } - +/** + * Implemented by any Helix construct that is lockable and is able to return a HelixLock instance + */ +public interface HelixLockable { + /** + * Get a lock object on a scope + * @param clusterId cluster to lock + * @param scope scope relative to the cluster that the lock protects + * @return HelixLock instance + */ + HelixLock getLock(ClusterId clusterId, Scope scope); } diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Aggregator.java b/helix-core/src/main/java/org/apache/helix/lock/zk/LockListener.java similarity index 68% rename from helix-core/src/main/java/org/apache/helix/alerts/Aggregator.java rename to helix-core/src/main/java/org/apache/helix/lock/zk/LockListener.java index 65a4285113..bb2118c911 100644 --- a/helix-core/src/main/java/org/apache/helix/alerts/Aggregator.java +++ b/helix-core/src/main/java/org/apache/helix/lock/zk/LockListener.java @@ -1,4 +1,4 @@ -package org.apache.helix.alerts; +package org.apache.helix.lock.zk; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,22 +19,21 @@ * under the License. */ -public abstract class Aggregator { - - int _numArgs; - - public Aggregator() { - - } - - /* - * Take curr and new values. Update curr. +/** + * This class has two methods which are call + * back methods when a lock is acquired and + * when the lock is released. + */ +interface LockListener { + /** + * call back called when the lock + * is acquired */ - public abstract void merge(Tuple currVal, Tuple newVal, Tuple currTime, - Tuple newTime, String... args); - - public int getRequiredNumArgs() { - return _numArgs; - } + public void lockAcquired(); + /** + * call back called when the lock is + * released. + */ + public void lockReleased(); } diff --git a/helix-core/src/main/java/org/apache/helix/lock/zk/ProtocolSupport.java b/helix-core/src/main/java/org/apache/helix/lock/zk/ProtocolSupport.java new file mode 100644 index 0000000000..23bef6a2cb --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/lock/zk/ProtocolSupport.java @@ -0,0 +1,191 @@ +package org.apache.helix.lock.zk; + +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.log4j.Logger; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Stat; + +/* + * 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. + */ + +/** + * A base class for protocol implementations which provides a number of higher + * level helper methods for working with ZooKeeper along with retrying synchronous + * operations if the connection to ZooKeeper closes such as + * {@link #retryOperation(ZooKeeperOperation)} + */ +class ProtocolSupport { + private static final Logger LOG = Logger.getLogger(ProtocolSupport.class); + + protected final ZooKeeper zookeeper; + private AtomicBoolean closed = new AtomicBoolean(false); + private long retryDelay = 500L; + private int retryCount = 10; + private List acl = ZooDefs.Ids.OPEN_ACL_UNSAFE; + + public ProtocolSupport(ZooKeeper zookeeper) { + this.zookeeper = zookeeper; + } + + /** + * Closes this strategy and releases any ZooKeeper resources; but keeps the + * ZooKeeper instance open + */ + public void close() { + if (closed.compareAndSet(false, true)) { + doClose(); + } + } + + /** + * return zookeeper client instance + * @return zookeeper client instance + */ + public ZooKeeper getZookeeper() { + return zookeeper; + } + + /** + * return the acl its using + * @return the acl. + */ + public List getAcl() { + return acl; + } + + /** + * set the acl + * @param acl the acl to set to + */ + public void setAcl(List acl) { + this.acl = acl; + } + + /** + * get the retry delay in milliseconds + * @return the retry delay + */ + public long getRetryDelay() { + return retryDelay; + } + + /** + * Sets the time waited between retry delays + * @param retryDelay the retry delay + */ + public void setRetryDelay(long retryDelay) { + this.retryDelay = retryDelay; + } + + /** + * Allow derived classes to perform + * some custom closing operations to release resources + */ + protected void doClose() { + } + + /** + * Perform the given operation, retrying if the connection fails + * @return object. it needs to be cast to the callee's expected + * return type. + */ + protected Object retryOperation(ZooKeeperOperation operation) throws KeeperException, + InterruptedException { + KeeperException exception = null; + for (int i = 0; i < retryCount; i++) { + try { + return operation.execute(); + } catch (KeeperException.SessionExpiredException e) { + LOG.warn("Session expired for: " + zookeeper + " so reconnecting due to: " + e, e); + throw e; + } catch (KeeperException.ConnectionLossException e) { + if (exception == null) { + exception = e; + } + LOG.debug("Attempt " + i + " failed with connection loss so " + "attempting to reconnect: " + + e, e); + retryDelay(i); + } + } + throw exception; + } + + /** + * Ensures that the given path exists with no data, the current + * ACL and no flags + * @param path + */ + protected void ensurePathExists(String path) { + ensureExists(path, null, acl, CreateMode.PERSISTENT); + } + + /** + * Ensures that the given path exists with the given data, ACL and flags + * @param path + * @param acl + * @param flags + */ + protected void ensureExists(final String path, final byte[] data, final List acl, + final CreateMode flags) { + try { + retryOperation(new ZooKeeperOperation() { + public boolean execute() throws KeeperException, InterruptedException { + Stat stat = zookeeper.exists(path, false); + if (stat != null) { + return true; + } + zookeeper.create(path, data, acl, flags); + return true; + } + }); + } catch (KeeperException e) { + LOG.warn("Caught: " + e, e); + } catch (InterruptedException e) { + LOG.warn("Caught: " + e, e); + } + } + + /** + * Returns true if this protocol has been closed + * @return true if this protocol is closed + */ + protected boolean isClosed() { + return closed.get(); + } + + /** + * Performs a retry delay if this is not the first attempt + * @param attemptCount the number of the attempts performed so far + */ + protected void retryDelay(int attemptCount) { + if (attemptCount > 0) { + try { + Thread.sleep(attemptCount * retryDelay); + } catch (InterruptedException e) { + LOG.debug("Failed to sleep: " + e, e); + } + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/lock/zk/WriteLock.java b/helix-core/src/main/java/org/apache/helix/lock/zk/WriteLock.java new file mode 100644 index 0000000000..aef76188df --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/lock/zk/WriteLock.java @@ -0,0 +1,294 @@ +package org.apache.helix.lock.zk; + +import static org.apache.zookeeper.CreateMode.EPHEMERAL_SEQUENTIAL; + +import java.util.List; +import java.util.SortedSet; +import java.util.TreeSet; + +import org.apache.log4j.Logger; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Stat; + +/* + * 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. + */ + +/** + * A protocol to implement an exclusive + * write lock or to elect a leader. + *

+ * You invoke {@link #lock()} to start the process of grabbing the lock; you may get the lock then + * or it may be some time later. + *

+ * You can register a listener so that you are invoked when you get the lock; otherwise you can ask + * if you have the lock by calling {@link #isOwner()} + */ +class WriteLock extends ProtocolSupport { + private static final Logger LOG = Logger.getLogger(WriteLock.class); + + private final String dir; + private String id; + private ZNodeName idName; + private String ownerId; + private String lastChildId; + private byte[] data = { + 0x12, 0x34 + }; + private LockListener callback; + private LockZooKeeperOperation zop; + + /** + * zookeeper contructor for writelock + * @param zookeeper zookeeper client instance + * @param dir the parent path you want to use for locking + * @param acls the acls that you want to use for all the paths, + * if null world read/write is used. + */ + public WriteLock(ZooKeeper zookeeper, String dir, List acl) { + super(zookeeper); + this.dir = dir; + if (acl != null) { + setAcl(acl); + } + this.zop = new LockZooKeeperOperation(); + } + + /** + * zookeeper contructor for writelock with callback + * @param zookeeper the zookeeper client instance + * @param dir the parent path you want to use for locking + * @param acl the acls that you want to use for all the paths + * @param callback the call back instance + */ + public WriteLock(ZooKeeper zookeeper, String dir, List acl, LockListener callback) { + this(zookeeper, dir, acl); + this.callback = callback; + } + + /** + * return the current locklistener + * @return the locklistener + */ + public LockListener getLockListener() { + return this.callback; + } + + /** + * register a different call back listener + * @param callback the call back instance + */ + public void setLockListener(LockListener callback) { + this.callback = callback; + } + + /** + * Removes the lock or associated znode if + * you no longer require the lock. this also + * removes your request in the queue for locking + * in case you do not already hold the lock. + * @throws RuntimeException throws a runtime exception + * if it cannot connect to zookeeper. + */ + public synchronized void unlock() throws RuntimeException { + + if (!isClosed() && id != null) { + // we don't need to retry this operation in the case of failure + // as ZK will remove ephemeral files and we don't wanna hang + // this process when closing if we cannot reconnect to ZK + try { + + ZooKeeperOperation zopdel = new ZooKeeperOperation() { + public boolean execute() throws KeeperException, InterruptedException { + zookeeper.delete(id, -1); + return Boolean.TRUE; + } + }; + zopdel.execute(); + } catch (InterruptedException e) { + LOG.warn("Caught: " + e, e); + // set that we have been interrupted. + Thread.currentThread().interrupt(); + } catch (KeeperException.NoNodeException e) { + // do nothing + } catch (KeeperException e) { + LOG.warn("Caught: " + e, e); + throw (RuntimeException) new RuntimeException(e.getMessage()).initCause(e); + } finally { + if (callback != null) { + callback.lockReleased(); + } + id = null; + } + } + } + + /** + * the watcher called on + * getting watch while watching + * my predecessor + */ + private class LockWatcher implements Watcher { + public void process(WatchedEvent event) { + // lets either become the leader or watch the new/updated node + LOG.debug("Watcher fired on path: " + event.getPath() + " state: " + event.getState() + + " type " + event.getType()); + try { + lock(); + } catch (Exception e) { + LOG.warn("Failed to acquire lock: " + e, e); + } + } + } + + /** + * a zoookeeper operation that is mainly responsible + * for all the magic required for locking. + */ + private class LockZooKeeperOperation implements ZooKeeperOperation { + + /** + * find if we have been created earler if not create our node + * @param prefix the prefix node + * @param zookeeper teh zookeeper client + * @param dir the dir paretn + * @throws KeeperException + * @throws InterruptedException + */ + private void findPrefixInChildren(String prefix, ZooKeeper zookeeper, String dir) + throws KeeperException, InterruptedException { + List names = zookeeper.getChildren(dir, false); + for (String name : names) { + if (name.startsWith(prefix)) { + id = name; + if (LOG.isDebugEnabled()) { + LOG.debug("Found id created last time: " + id); + } + break; + } + } + if (id == null) { + id = zookeeper.create(dir + "/" + prefix, data, getAcl(), EPHEMERAL_SEQUENTIAL); + + if (LOG.isDebugEnabled()) { + LOG.debug("Created id: " + id); + } + } + + } + + /** + * the command that is run and retried for actually + * obtaining the lock + * @return if the command was successful or not + */ + public boolean execute() throws KeeperException, InterruptedException { + do { + if (id == null) { + long sessionId = zookeeper.getSessionId(); + String prefix = "x-" + sessionId + "-"; + // lets try look up the current ID if we failed + // in the middle of creating the znode + findPrefixInChildren(prefix, zookeeper, dir); + idName = new ZNodeName(id); + } + if (id != null) { + List names = zookeeper.getChildren(dir, false); + if (names.isEmpty()) { + LOG.warn("No children in: " + dir + " when we've just " + + "created one! Lets recreate it..."); + // lets force the recreation of the id + id = null; + } else { + // lets sort them explicitly (though they do seem to come back in order ususally :) + SortedSet sortedNames = new TreeSet(); + for (String name : names) { + sortedNames.add(new ZNodeName(dir + "/" + name)); + } + ownerId = sortedNames.first().getName(); + SortedSet lessThanMe = sortedNames.headSet(idName); + if (!lessThanMe.isEmpty()) { + ZNodeName lastChildName = lessThanMe.last(); + lastChildId = lastChildName.getName(); + if (LOG.isDebugEnabled()) { + LOG.debug("watching less than me node: " + lastChildId); + } + Stat stat = zookeeper.exists(lastChildId, new LockWatcher()); + if (stat != null) { + return Boolean.FALSE; + } else { + LOG.warn("Could not find the" + " stats for less than me: " + + lastChildName.getName()); + } + } else { + if (isOwner()) { + if (callback != null) { + callback.lockAcquired(); + } + return Boolean.TRUE; + } + } + } + } + } while (id == null); + return Boolean.FALSE; + } + }; + + /** + * Attempts to acquire the exclusive write lock returning whether or not it was + * acquired. Note that the exclusive lock may be acquired some time later after + * this method has been invoked due to the current lock owner going away. + */ + public synchronized boolean lock() throws KeeperException, InterruptedException { + if (isClosed()) { + return false; + } + ensurePathExists(dir); + + return (Boolean) retryOperation(zop); + } + + /** + * return the parent dir for lock + * @return the parent dir used for locks. + */ + public String getDir() { + return dir; + } + + /** + * Returns true if this node is the owner of the + * lock (or the leader) + */ + public boolean isOwner() { + return id != null && ownerId != null && id.equals(ownerId); + } + + /** + * return the id for this lock + * @return the id for this lock + */ + public String getId() { + return this.id; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/lock/zk/ZKHelixLock.java b/helix-core/src/main/java/org/apache/helix/lock/zk/ZKHelixLock.java new file mode 100644 index 0000000000..1d56f1359d --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/lock/zk/ZKHelixLock.java @@ -0,0 +1,154 @@ +package org.apache.helix.lock.zk; + +import org.I0Itec.zkclient.ZkConnection; +import org.apache.helix.AccessOption; +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.Scope; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.lock.HelixLock; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.manager.zk.ZkClient; +import org.apache.log4j.Logger; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; + +/* + * 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. + */ + +/** + * Locking scheme for Helix that uses the ZooKeeper exclusive lock implementation + * Please use the following lock order convention: Cluster, Participant, Resource, Partition + * WARNING: this is not a reentrant lock + */ +public class ZKHelixLock implements HelixLock { + private static final Logger LOG = Logger.getLogger(ZKHelixLock.class); + + private static final String LOCK_ROOT = "LOCKS"; + private final String _rootPath; + private final WriteLock _writeLock; + private final ZkClient _zkClient; + private volatile boolean _locked; + private volatile boolean _canceled; + private volatile boolean _blocked; + + private final LockListener _listener = new LockListener() { + @Override + public void lockReleased() { + } + + @Override + public void lockAcquired() { + synchronized (ZKHelixLock.this) { + if (!_canceled) { + _locked = true; + } else { + unlock(); + } + ZKHelixLock.this.notify(); + } + } + }; + + /** + * Initialize for a cluster and scope + * @param clusterId the cluster under which the lock will live + * @param scope the scope to lock + * @param zkClient an active ZK client + */ + public ZKHelixLock(ClusterId clusterId, Scope scope, ZkClient zkClient) { + _zkClient = zkClient; + _rootPath = + '/' + clusterId.stringify() + '/' + LOCK_ROOT + '/' + scope.getType() + '_' + + scope.getScopedId(); + ZooKeeper zookeeper = ((ZkConnection) zkClient.getConnection()).getZookeeper(); + _writeLock = new WriteLock(zookeeper, _rootPath, null, _listener); + _locked = false; + _canceled = false; + _blocked = false; + } + + /** + * Try to synchronously lock the scope + * @return true if the lock succeeded, false if it failed, as is the case if the connection to ZK + * is lost + */ + @Override + public synchronized boolean lock() { + _canceled = false; + if (_locked || isBlocked()) { + // no need to proceed if the lock is already acquired or already waiting + return false; + } + try { + // create the root path if it doesn't exist + BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_zkClient); + baseAccessor.create(_rootPath, null, AccessOption.PERSISTENT); + + // try to acquire the lock + boolean acquired = _writeLock.lock(); + if (acquired) { + _locked = true; + } else { + setBlocked(true); + wait(); + } + } catch (KeeperException e) { + LOG.error("Error acquiring a lock on " + _rootPath, e); + _canceled = true; + } catch (InterruptedException e) { + LOG.error("Interrupted while acquiring a lock on " + _rootPath); + _canceled = true; + } + setBlocked(false); + return _locked; + } + + /** + * Unlock the scope + * @return true if unlock executed, false otherwise + */ + @Override + public synchronized boolean unlock() { + try { + _writeLock.unlock(); + } catch (IllegalArgumentException e) { + if (LOG.isInfoEnabled()) { + LOG.info("Unlock skipped because lock node was not present"); + } + } catch (RuntimeException e) { + LOG.error("Error connecting to release the lock"); + } + _locked = false; + return true; + } + + @Override + public boolean isBlocked() { + return _blocked; + } + + /** + * Set if this the lock method is currently blocked + * @param isBlocked true if blocked, false otherwise + */ + protected void setBlocked(boolean isBlocked) { + _blocked = isBlocked; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/lock/zk/ZNodeName.java b/helix-core/src/main/java/org/apache/helix/lock/zk/ZNodeName.java new file mode 100644 index 0000000000..47253e6fb2 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/lock/zk/ZNodeName.java @@ -0,0 +1,113 @@ +package org.apache.helix.lock.zk; + +import org.apache.log4j.Logger; + +/* + * 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. + */ + +/** + * Represents an ephemeral znode name which has an ordered sequence number + * and can be sorted in order + */ +class ZNodeName implements Comparable { + private final String name; + private String prefix; + private int sequence = -1; + private static final Logger LOG = Logger.getLogger(ZNodeName.class); + + public ZNodeName(String name) { + if (name == null) { + throw new NullPointerException("id cannot be null"); + } + this.name = name; + this.prefix = name; + int idx = name.lastIndexOf('-'); + if (idx >= 0) { + this.prefix = name.substring(0, idx); + try { + this.sequence = Integer.parseInt(name.substring(idx + 1)); + // If an exception occurred we misdetected a sequence suffix, + // so return -1. + } catch (NumberFormatException e) { + LOG.info("Number format exception for " + idx, e); + } catch (ArrayIndexOutOfBoundsException e) { + LOG.info("Array out of bounds for " + idx, e); + } + } + } + + @Override + public String toString() { + return name.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + ZNodeName sequence = (ZNodeName) o; + + if (!name.equals(sequence.name)) + return false; + + return true; + } + + @Override + public int hashCode() { + return name.hashCode() + 37; + } + + public int compareTo(ZNodeName that) { + int answer = this.prefix.compareTo(that.prefix); + if (answer == 0) { + int s1 = this.sequence; + int s2 = that.sequence; + if (s1 == -1 && s2 == -1) { + return this.name.compareTo(that.name); + } + answer = s1 == -1 ? 1 : s2 == -1 ? -1 : s1 - s2; + } + return answer; + } + + /** + * Returns the name of the znode + */ + public String getName() { + return name; + } + + /** + * Returns the sequence number + */ + public int getZNodeName() { + return sequence; + } + + /** + * Returns the text prefix before the sequence number + */ + public String getPrefix() { + return prefix; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/lock/zk/ZooKeeperOperation.java b/helix-core/src/main/java/org/apache/helix/lock/zk/ZooKeeperOperation.java new file mode 100644 index 0000000000..58b9fe35f0 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/lock/zk/ZooKeeperOperation.java @@ -0,0 +1,38 @@ +package org.apache.helix.lock.zk; + +import org.apache.zookeeper.KeeperException; + +/* + * 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. + */ + +/** + * A callback object which can be used for implementing retry-able operations in the + * {@link org.apache.helix.lock.zk.recipes.lock.ProtocolSupport} class + */ +interface ZooKeeperOperation { + + /** + * Performs the operation - which may be involved multiple times if the connection + * to ZooKeeper closes during this operation + * @return the result of the operation or null + * @throws KeeperException + * @throws InterruptedException + */ + public boolean execute() throws KeeperException, InterruptedException; +} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java deleted file mode 100644 index 4f549e4ac6..0000000000 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java +++ /dev/null @@ -1,693 +0,0 @@ -package org.apache.helix.manager.zk; - -/* - * 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.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.concurrent.TimeUnit; - -import org.I0Itec.zkclient.IZkStateListener; -import org.I0Itec.zkclient.ZkConnection; -import org.apache.helix.BaseDataAccessor; -import org.apache.helix.ClusterMessagingService; -import org.apache.helix.ConfigAccessor; -import org.apache.helix.ConfigChangeListener; -import org.apache.helix.ControllerChangeListener; -import org.apache.helix.CurrentStateChangeListener; -import org.apache.helix.ExternalViewChangeListener; -import org.apache.helix.HealthStateChangeListener; -import org.apache.helix.HelixAdmin; -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixException; -import org.apache.helix.HelixManager; -import org.apache.helix.HelixManagerProperties; -import org.apache.helix.HelixTimerTask; -import org.apache.helix.IdealStateChangeListener; -import org.apache.helix.InstanceConfigChangeListener; -import org.apache.helix.InstanceType; -import org.apache.helix.LiveInstanceChangeListener; -import org.apache.helix.LiveInstanceInfoProvider; -import org.apache.helix.MessageListener; -import org.apache.helix.PreConnectCallback; -import org.apache.helix.PropertyKey; -import org.apache.helix.PropertyPathConfig; -import org.apache.helix.PropertyType; -import org.apache.helix.ScopedConfigChangeListener; -import org.apache.helix.ZNRecord; -import org.apache.helix.HelixConstants.ChangeType; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.healthcheck.ParticipantHealthReportCollector; -import org.apache.helix.messaging.DefaultMessagingService; -import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; -import org.apache.helix.participant.StateMachineEngine; -import org.apache.helix.store.zk.ZkHelixPropertyStore; -import org.apache.log4j.Logger; -import org.apache.zookeeper.Watcher.Event.EventType; -import org.apache.zookeeper.Watcher.Event.KeeperState; -import org.apache.zookeeper.ZooKeeper.States; - -public abstract class AbstractManager implements HelixManager, IZkStateListener { - private static Logger LOG = Logger.getLogger(AbstractManager.class); - - final String _zkAddress; - final String _clusterName; - final String _instanceName; - final InstanceType _instanceType; - final int _sessionTimeout; - final List _preConnectCallbacks; - protected final List _handlers; - final HelixManagerProperties _properties; - - /** - * helix version# - */ - final String _version; - - protected ZkClient _zkclient = null; - final DefaultMessagingService _messagingService; - - BaseDataAccessor _baseDataAccessor; - ZKHelixDataAccessor _dataAccessor; - final Builder _keyBuilder; - ConfigAccessor _configAccessor; - ZkHelixPropertyStore _helixPropertyStore; - LiveInstanceInfoProvider _liveInstanceInfoProvider = null; - final List _timerTasks = new ArrayList(); - - volatile String _sessionId; - - /** - * Keep track of timestamps that zk State has become Disconnected - * If in a _timeWindowLengthMs window zk State has become Disconnected - * for more than_maxDisconnectThreshold times disconnect the zkHelixManager - */ - final List _disconnectTimeHistory = new LinkedList(); - - final int _flappingTimeWindowMs; - final int _maxDisconnectThreshold; - - public AbstractManager(String zkAddress, String clusterName, String instanceName, - InstanceType instanceType) { - - LOG.info("Create a zk-based cluster manager. zkSvr: " + zkAddress + ", clusterName: " - + clusterName + ", instanceName: " + instanceName + ", type: " + instanceType); - - _zkAddress = zkAddress; - _clusterName = clusterName; - _instanceType = instanceType; - _instanceName = instanceName; - _preConnectCallbacks = new LinkedList(); - _handlers = new ArrayList(); - _properties = new HelixManagerProperties("cluster-manager-version.properties"); - _version = _properties.getVersion(); - - _keyBuilder = new Builder(clusterName); - _messagingService = new DefaultMessagingService(this); - - /** - * use system property if available - */ - _flappingTimeWindowMs = - getSystemPropertyAsInt("helixmanager.flappingTimeWindow", - ZKHelixManager.FLAPPING_TIME_WINDIOW); - - _maxDisconnectThreshold = - getSystemPropertyAsInt("helixmanager.maxDisconnectThreshold", - ZKHelixManager.MAX_DISCONNECT_THRESHOLD); - - _sessionTimeout = - getSystemPropertyAsInt("zk.session.timeout", ZkClient.DEFAULT_SESSION_TIMEOUT); - - } - - private int getSystemPropertyAsInt(String propertyKey, int propertyDefaultValue) { - String valueString = System.getProperty(propertyKey, "" + propertyDefaultValue); - - try { - int value = Integer.parseInt(valueString); - if (value > 0) { - return value; - } - } catch (NumberFormatException e) { - LOG.warn("Exception while parsing property: " + propertyKey + ", string: " + valueString - + ", using default value: " + propertyDefaultValue); - } - - return propertyDefaultValue; - } - - /** - * different types of helix manager should impl its own handle new session logic - */ - // public abstract void handleNewSession(); - - @Override - public void connect() throws Exception { - LOG.info("ClusterManager.connect()"); - if (isConnected()) { - LOG.warn("Cluster manager: " + _instanceName + " for cluster: " + _clusterName - + " already connected. skip connect"); - return; - } - - try { - createClient(); - _messagingService.onConnected(); - } catch (Exception e) { - LOG.error("fail to connect " + _instanceName, e); - disconnect(); - throw e; - } - } - - @Override - public boolean isConnected() { - if (_zkclient == null) { - return false; - } - ZkConnection zkconnection = (ZkConnection) _zkclient.getConnection(); - if (zkconnection != null) { - States state = zkconnection.getZookeeperState(); - return state == States.CONNECTED; - } - return false; - } - - /** - * specific disconnect logic for each helix-manager type - */ - abstract void doDisconnect(); - - /** - * This function can be called when the connection are in bad state(e.g. flapping), - * in which isConnected() could be false and we want to disconnect from cluster. - */ - @Override - public void disconnect() { - LOG.info("disconnect " + _instanceName + "(" + _instanceType + ") from " + _clusterName); - - try { - /** - * stop all timer tasks - */ - stopTimerTasks(); - - /** - * shutdown thread pool first to avoid reset() being invoked in the middle of state - * transition - */ - _messagingService.getExecutor().shutdown(); - - // TODO reset user defined handlers only - resetHandlers(); - - _dataAccessor.shutdown(); - - doDisconnect(); - - _zkclient.unsubscribeAll(); - } finally { - _zkclient.close(); - LOG.info("Cluster manager: " + _instanceName + " disconnected"); - } - } - - @Override - public void addIdealStateChangeListener(IdealStateChangeListener listener) throws Exception { - addListener(listener, new Builder(_clusterName).idealStates(), ChangeType.IDEAL_STATE, - new EventType[] { - EventType.NodeDataChanged, EventType.NodeDeleted, EventType.NodeCreated - }); - } - - @Override - public void addLiveInstanceChangeListener(LiveInstanceChangeListener listener) throws Exception { - addListener(listener, new Builder(_clusterName).liveInstances(), ChangeType.LIVE_INSTANCE, - new EventType[] { - EventType.NodeDataChanged, EventType.NodeChildrenChanged, EventType.NodeDeleted, - EventType.NodeCreated - }); - } - - @Override - public void addConfigChangeListener(ConfigChangeListener listener) throws Exception { - addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG, - new EventType[] { - EventType.NodeChildrenChanged - }); - } - - @Override - public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener) - throws Exception { - addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG, - new EventType[] { - EventType.NodeChildrenChanged - }); - } - - @Override - public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope) - throws Exception { - Builder keyBuilder = new Builder(_clusterName); - - PropertyKey propertyKey = null; - switch (scope) { - case CLUSTER: - propertyKey = keyBuilder.clusterConfigs(); - break; - case PARTICIPANT: - propertyKey = keyBuilder.instanceConfigs(); - break; - case RESOURCE: - propertyKey = keyBuilder.resourceConfigs(); - break; - default: - break; - } - - if (propertyKey != null) { - addListener(listener, propertyKey, ChangeType.CONFIG, new EventType[] { - EventType.NodeChildrenChanged - }); - } else { - LOG.error("Can't add listener to config scope: " + scope); - } - } - - @Override - public void addMessageListener(MessageListener listener, String instanceName) { - addListener(listener, new Builder(_clusterName).messages(instanceName), ChangeType.MESSAGE, - new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }); - } - - @Override - public void addCurrentStateChangeListener(CurrentStateChangeListener listener, - String instanceName, String sessionId) throws Exception { - addListener(listener, new Builder(_clusterName).currentStates(instanceName, sessionId), - ChangeType.CURRENT_STATE, new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }); - } - - @Override - public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName) - throws Exception { - addListener(listener, new Builder(_clusterName).healthReports(instanceName), ChangeType.HEALTH, - new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }); - } - - @Override - public void addExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception { - addListener(listener, new Builder(_clusterName).externalViews(), ChangeType.EXTERNAL_VIEW, - new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }); - } - - @Override - public void addControllerListener(ControllerChangeListener listener) { - addListener(listener, new Builder(_clusterName).controller(), ChangeType.CONTROLLER, - new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }); - } - - void addControllerMessageListener(MessageListener listener) { - addListener(listener, new Builder(_clusterName).controllerMessages(), - ChangeType.MESSAGES_CONTROLLER, new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }); - } - - @Override - public boolean removeListener(PropertyKey key, Object listener) { - LOG.info("Removing listener: " + listener + " on path: " + key.getPath() + " from cluster: " - + _clusterName + " by instance: " + _instanceName); - - synchronized (this) { - List toRemove = new ArrayList(); - for (CallbackHandler handler : _handlers) { - // compare property-key path and listener reference - if (handler.getPath().equals(key.getPath()) && handler.getListener().equals(listener)) { - toRemove.add(handler); - } - } - - _handlers.removeAll(toRemove); - - // handler.reset() may modify the handlers list, so do it outside the iteration - for (CallbackHandler handler : toRemove) { - handler.reset(); - } - } - - return true; - } - - @Override - public HelixDataAccessor getHelixDataAccessor() { - checkConnected(); - return _dataAccessor; - } - - @Override - public ConfigAccessor getConfigAccessor() { - checkConnected(); - return _configAccessor; - } - - @Override - public String getClusterName() { - return _clusterName; - } - - @Override - public String getInstanceName() { - return _instanceName; - } - - @Override - public String getSessionId() { - checkConnected(); - return _sessionId; - } - - @Override - public long getLastNotificationTime() { - // TODO Auto-generated method stub - return 0; - } - - @Override - public HelixAdmin getClusterManagmentTool() { - checkConnected(); - if (_zkclient != null) { - return new ZKHelixAdmin(_zkclient); - } - - LOG.error("Couldn't get ZKClusterManagementTool because zkclient is null"); - return null; - } - - @Override - public synchronized ZkHelixPropertyStore getHelixPropertyStore() { - checkConnected(); - - if (_helixPropertyStore == null) { - String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, _clusterName); - - _helixPropertyStore = - new ZkHelixPropertyStore(new ZkBaseDataAccessor(_zkclient), path, - null); - } - - return _helixPropertyStore; - } - - @Override - public ClusterMessagingService getMessagingService() { - // The caller can register message handler factories on messaging service before the - // helix manager is connected. Thus we do not check connected here - return _messagingService; - } - - @Override - public ParticipantHealthReportCollector getHealthReportCollector() { - // helix-participant will override this - return null; - } - - @Override - public InstanceType getInstanceType() { - return _instanceType; - } - - @Override - public String getVersion() { - return _version; - } - - @Override - public HelixManagerProperties getProperties() { - return _properties; - } - - @Override - public StateMachineEngine getStateMachineEngine() { - // helix-participant will override this - return null; - } - - @Override - public abstract boolean isLeader(); - - @Override - public void startTimerTasks() { - for (HelixTimerTask task : _timerTasks) { - task.start(); - } - - } - - @Override - public void stopTimerTasks() { - for (HelixTimerTask task : _timerTasks) { - task.stop(); - } - - } - - @Override - public void addPreConnectCallback(PreConnectCallback callback) { - LOG.info("Adding preconnect callback: " + callback); - _preConnectCallbacks.add(callback); - } - - @Override - public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) { - _liveInstanceInfoProvider = liveInstanceInfoProvider; - } - - /** - * wait until we get a non-zero session-id. note that we might lose zkconnection - * right after we read session-id. but it's ok to get stale session-id and we will have - * another handle-new-session callback to correct this. - */ - protected void waitUntilConnected() { - boolean isConnected; - do { - isConnected = - _zkclient.waitUntilConnected(ZkClient.DEFAULT_CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS); - if (!isConnected) { - LOG.error("fail to connect zkserver: " + _zkAddress + " in " - + ZkClient.DEFAULT_CONNECTION_TIMEOUT + "ms. expiredSessionId: " + _sessionId - + ", clusterName: " + _clusterName); - continue; - } - - ZkConnection zkConnection = ((ZkConnection) _zkclient.getConnection()); - _sessionId = Long.toHexString(zkConnection.getZookeeper().getSessionId()); - - /** - * at the time we read session-id, zkconnection might be lost again - * wait until we get a non-zero session-id - */ - } while ("0".equals(_sessionId)); - - LOG.info("Handling new session, session id: " + _sessionId + ", instance: " + _instanceName - + ", instanceTye: " + _instanceType + ", cluster: " + _clusterName + ", zkconnection: " - + ((ZkConnection) _zkclient.getConnection()).getZookeeper()); - } - - protected void checkConnected() { - if (!isConnected()) { - throw new HelixException("ClusterManager not connected. Call clusterManager.connect()"); - } - } - - protected void addListener(Object listener, PropertyKey propertyKey, ChangeType changeType, - EventType[] eventType) { - checkConnected(); - - PropertyType type = propertyKey.getType(); - - synchronized (this) { - for (CallbackHandler handler : _handlers) { - // compare property-key path and listener reference - if (handler.getPath().equals(propertyKey.getPath()) - && handler.getListener().equals(listener)) { - LOG.info("Listener: " + listener + " on path: " + propertyKey.getPath() - + " already exists. skip add"); - - return; - } - } - - CallbackHandler newHandler = - new CallbackHandler(this, _zkclient, propertyKey, listener, eventType, changeType); - - _handlers.add(newHandler); - LOG.info("Added listener: " + listener + " for type: " + type + " to path: " - + newHandler.getPath()); - } - } - - protected void initHandlers(List handlers) { - synchronized (this) { - if (handlers != null) { - for (CallbackHandler handler : handlers) { - handler.init(); - LOG.info("init handler: " + handler.getPath() + ", " + handler.getListener()); - } - } - } - } - - protected void resetHandlers() { - synchronized (this) { - if (_handlers != null) { - // get a copy of the list and iterate over the copy list - // in case handler.reset() modify the original handler list - List tmpHandlers = new ArrayList(); - tmpHandlers.addAll(_handlers); - - for (CallbackHandler handler : tmpHandlers) { - handler.reset(); - LOG.info("reset handler: " + handler.getPath() + ", " + handler.getListener()); - } - } - } - } - - /** - * different helix-manager may override this to have a cache-enabled based-data-accessor - * @param baseDataAccessor - * @return - */ - BaseDataAccessor createBaseDataAccessor(ZkBaseDataAccessor baseDataAccessor) { - return baseDataAccessor; - } - - void createClient() throws Exception { - PathBasedZkSerializer zkSerializer = - ChainedPathZkSerializer.builder(new ZNRecordStreamingSerializer()).build(); - - _zkclient = - new ZkClient(_zkAddress, _sessionTimeout, ZkClient.DEFAULT_CONNECTION_TIMEOUT, zkSerializer); - - ZkBaseDataAccessor baseDataAccessor = new ZkBaseDataAccessor(_zkclient); - - _baseDataAccessor = createBaseDataAccessor(baseDataAccessor); - - _dataAccessor = new ZKHelixDataAccessor(_clusterName, _instanceType, _baseDataAccessor); - _configAccessor = new ConfigAccessor(_zkclient); - - int retryCount = 0; - - _zkclient.subscribeStateChanges(this); - while (retryCount < 3) { - try { - _zkclient.waitUntilConnected(_sessionTimeout, TimeUnit.MILLISECONDS); - handleStateChanged(KeeperState.SyncConnected); - handleNewSession(); - break; - } catch (HelixException e) { - LOG.error("fail to createClient.", e); - throw e; - } catch (Exception e) { - retryCount++; - - LOG.error("fail to createClient. retry " + retryCount, e); - if (retryCount == 3) { - throw e; - } - } - } - } - - // TODO separate out flapping detection code - @Override - public void handleStateChanged(KeeperState state) throws Exception { - switch (state) { - case SyncConnected: - ZkConnection zkConnection = (ZkConnection) _zkclient.getConnection(); - LOG.info("KeeperState: " + state + ", zookeeper:" + zkConnection.getZookeeper()); - break; - case Disconnected: - LOG.info("KeeperState:" + state + ", disconnectedSessionId: " + _sessionId + ", instance: " - + _instanceName + ", type: " + _instanceType); - - /** - * Track the time stamp that the disconnected happens, then check history and see if - * we should disconnect the helix-manager - */ - _disconnectTimeHistory.add(System.currentTimeMillis()); - if (isFlapping()) { - LOG.error("instanceName: " + _instanceName + " is flapping. diconnect it. " - + " maxDisconnectThreshold: " + _maxDisconnectThreshold + " disconnects in " - + _flappingTimeWindowMs + "ms."); - disconnect(); - } - break; - case Expired: - LOG.info("KeeperState:" + state + ", expiredSessionId: " + _sessionId + ", instance: " - + _instanceName + ", type: " + _instanceType); - break; - default: - break; - } - } - - /** - * If zk state has changed into Disconnected for _maxDisconnectThreshold times during previous - * _timeWindowLengthMs Ms - * time window, we think that there are something wrong going on and disconnect the zkHelixManager - * from zk. - */ - private boolean isFlapping() { - if (_disconnectTimeHistory.size() == 0) { - return false; - } - long mostRecentTimestamp = _disconnectTimeHistory.get(_disconnectTimeHistory.size() - 1); - - // Remove disconnect history timestamp that are older than _flappingTimeWindowMs ago - while ((_disconnectTimeHistory.get(0) + _flappingTimeWindowMs) < mostRecentTimestamp) { - _disconnectTimeHistory.remove(0); - } - return _disconnectTimeHistory.size() > _maxDisconnectThreshold; - } - - /** - * controller should override it to return a list of timers that need to start/stop when - * leadership changes - * @return - */ - protected List getControllerHelixTimerTasks() { - return null; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/Cache.java b/helix-core/src/main/java/org/apache/helix/manager/zk/Cache.java index 354a9f6f43..1935c5fe26 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/Cache.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/Cache.java @@ -19,7 +19,6 @@ * under the License. */ -import java.io.File; import java.util.List; import java.util.Map; import java.util.Set; diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java index b59976d2a2..24ccc02324 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java @@ -41,7 +41,6 @@ import org.apache.helix.ControllerChangeListener; import org.apache.helix.CurrentStateChangeListener; import org.apache.helix.ExternalViewChangeListener; -import org.apache.helix.HealthStateChangeListener; import org.apache.helix.HelixConstants.ChangeType; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixException; @@ -49,19 +48,16 @@ import org.apache.helix.HelixProperty; import org.apache.helix.IdealStateChangeListener; import org.apache.helix.InstanceConfigChangeListener; -import org.apache.helix.InstanceType; import org.apache.helix.LiveInstanceChangeListener; import org.apache.helix.MessageListener; import org.apache.helix.NotificationContext; import org.apache.helix.NotificationContext.Type; import org.apache.helix.PropertyKey; -import org.apache.helix.PropertyKey.Builder; import org.apache.helix.PropertyPathConfig; import org.apache.helix.ScopedConfigChangeListener; import org.apache.helix.ZNRecord; import org.apache.helix.model.CurrentState; import org.apache.helix.model.ExternalView; -import org.apache.helix.model.HealthStat; import org.apache.helix.model.IdealState; import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.LiveInstance; @@ -211,17 +207,7 @@ public void invoke(NotificationContext changeContext) throws Exception { ControllerChangeListener controllerChangelistener = (ControllerChangeListener) _listener; subscribeForChanges(changeContext, _path, true, false); controllerChangelistener.onControllerChange(changeContext); - } else if (_changeType == ChangeType.HEALTH) { - HealthStateChangeListener healthStateChangeListener = (HealthStateChangeListener) _listener; - subscribeForChanges(changeContext, _path, true, true); // TODO: figure out - // settings here - String instanceName = PropertyPathConfig.getInstanceNameFromPath(_path); - - List healthReportList = _accessor.getChildValues(_propertyKey); - - healthStateChangeListener.onHealthChange(instanceName, healthReportList, changeContext); } - long end = System.currentTimeMillis(); if (logger.isInfoEnabled()) { logger.info(Thread.currentThread().getId() + " END:INVOKE " + _path + " listener:" diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java deleted file mode 100644 index 1ed6dea35b..0000000000 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java +++ /dev/null @@ -1,175 +0,0 @@ -package org.apache.helix.manager.zk; - -/* - * 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.List; -import java.util.Timer; - -import org.apache.helix.BaseDataAccessor; -import org.apache.helix.HelixTimerTask; -import org.apache.helix.InstanceType; -import org.apache.helix.PropertyPathConfig; -import org.apache.helix.PropertyType; -import org.apache.helix.ZNRecord; -import org.apache.helix.HelixConstants.ChangeType; -import org.apache.helix.controller.GenericHelixController; -import org.apache.helix.healthcheck.HealthStatsAggregationTask; -import org.apache.helix.healthcheck.HealthStatsAggregator; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.monitoring.ZKPathDataDumpTask; -import org.apache.log4j.Logger; -import org.apache.zookeeper.Watcher.Event.EventType; - -public class ControllerManager extends AbstractManager { - private static Logger LOG = Logger.getLogger(ControllerManager.class); - - final GenericHelixController _controller = new GenericHelixController(); - - // TODO merge into GenericHelixController - private CallbackHandler _leaderElectionHandler = null; - - /** - * status dump timer-task - */ - static class StatusDumpTask extends HelixTimerTask { - Timer _timer = null; - final ZkClient zkclient; - final AbstractManager helixController; - - public StatusDumpTask(ZkClient zkclient, AbstractManager helixController) { - this.zkclient = zkclient; - this.helixController = helixController; - } - - @Override - public void start() { - long initialDelay = 30 * 60 * 1000; - long period = 120 * 60 * 1000; - int timeThresholdNoChange = 180 * 60 * 1000; - - if (_timer == null) { - LOG.info("Start StatusDumpTask"); - _timer = new Timer("StatusDumpTimerTask", true); - _timer.scheduleAtFixedRate(new ZKPathDataDumpTask(helixController, zkclient, - timeThresholdNoChange), initialDelay, period); - } - - } - - @Override - public void stop() { - if (_timer != null) { - LOG.info("Stop StatusDumpTask"); - _timer.cancel(); - _timer = null; - } - } - } - - public ControllerManager(String zkAddress, String clusterName, String instanceName) { - super(zkAddress, clusterName, instanceName, InstanceType.CONTROLLER); - - _timerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(this))); - _timerTasks.add(new StatusDumpTask(_zkclient, this)); - } - - @Override - protected List getControllerHelixTimerTasks() { - return _timerTasks; - } - - @Override - public void handleNewSession() throws Exception { - waitUntilConnected(); - - /** - * reset all handlers, make sure cleanup completed for previous session - * disconnect if fail to cleanup - */ - if (_leaderElectionHandler != null) { - _leaderElectionHandler.reset(); - } - // TODO reset user defined handlers only - resetHandlers(); - - /** - * from here on, we are dealing with new session - */ - - if (_leaderElectionHandler != null) { - _leaderElectionHandler.init(); - } else { - _leaderElectionHandler = - new CallbackHandler(this, _zkclient, _keyBuilder.controller(), - new DistributedLeaderElection(this, _controller), new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }, ChangeType.CONTROLLER); - } - - /** - * init handlers - * ok to init message handler and controller handlers twice - * the second init will be skipped (see CallbackHandler) - */ - initHandlers(_handlers); - } - - @Override - void doDisconnect() { - if (_leaderElectionHandler != null) { - _leaderElectionHandler.reset(); - } - } - - @Override - public boolean isLeader() { - if (!isConnected()) { - return false; - } - - try { - LiveInstance leader = _dataAccessor.getProperty(_keyBuilder.controllerLeader()); - if (leader != null) { - String leaderName = leader.getInstanceName(); - String sessionId = leader.getSessionId(); - if (leaderName != null && leaderName.equals(_instanceName) && sessionId != null - && sessionId.equals(_sessionId)) { - return true; - } - } - } catch (Exception e) { - // log - } - return false; - } - - /** - * helix-controller uses a write-through cache for external-view - */ - @Override - BaseDataAccessor createBaseDataAccessor(ZkBaseDataAccessor baseDataAccessor) { - String extViewPath = PropertyPathConfig.getPath(PropertyType.EXTERNALVIEW, _clusterName); - return new ZkCacheBaseDataAccessor(baseDataAccessor, Arrays.asList(extViewPath)); - - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java index ff3a264184..d2b520b03d 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java @@ -36,14 +36,14 @@ public class ControllerManagerHelper { private static Logger LOG = Logger.getLogger(ControllerManagerHelper.class); - final AbstractManager _manager; + final HelixManager _manager; final DefaultMessagingService _messagingService; final List _controllerTimerTasks; - public ControllerManagerHelper(AbstractManager manager) { + public ControllerManagerHelper(HelixManager manager, List controllerTimerTasks) { _manager = manager; _messagingService = (DefaultMessagingService) manager.getMessagingService(); - _controllerTimerTasks = manager.getControllerHelixTimerTasks(); + _controllerTimerTasks = controllerTimerTasks; } public void addListenersToController(GenericHelixController controller) { diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java b/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java index b96de1855f..11222e1e67 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java @@ -21,6 +21,9 @@ import org.I0Itec.zkclient.DataUpdater; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.model.CurrentState; /** @@ -52,15 +55,15 @@ public ZNRecord update(ZNRecord currentData) { curState = new CurrentState(_lastCurState.getId()); // copy all simple fields settings and overwrite session-id to current session curState.getRecord().setSimpleFields(_lastCurState.getRecord().getSimpleFields()); - curState.setSessionId(_curSessionId); + curState.setSessionId(SessionId.from(_curSessionId)); } else { curState = new CurrentState(currentData); } - for (String partitionName : _lastCurState.getPartitionStateMap().keySet()) { + for (PartitionId partitionId : _lastCurState.getTypedPartitionStateMap().keySet()) { // carry-over only when current-state not exist - if (curState.getState(partitionName) == null) { - curState.setState(partitionName, _initState); + if (curState.getState(partitionId) == null) { + curState.setState(partitionId, State.from(_initState)); } } return curState.getRecord(); diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultAlertMsgHandlerFactory.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultAlertMsgHandlerFactory.java new file mode 100644 index 0000000000..4766a3581c --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultAlertMsgHandlerFactory.java @@ -0,0 +1,132 @@ +package org.apache.helix.manager.zk; + +/* + * 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 org.apache.helix.HelixAdmin; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixException; +import org.apache.helix.HelixManager; +import org.apache.helix.NotificationContext; +import org.apache.helix.messaging.handling.HelixTaskResult; +import org.apache.helix.messaging.handling.MessageHandler; +import org.apache.helix.messaging.handling.MessageHandlerFactory; +import org.apache.helix.controller.alert.AlertAction; +import org.apache.helix.model.AlertConfig; +import org.apache.helix.controller.alert.AlertName; +import org.apache.helix.model.Message; +import org.apache.helix.model.Message.Attributes; +import org.apache.helix.tools.ClusterSetup; +import org.apache.log4j.Logger; + +public class DefaultAlertMsgHandlerFactory implements MessageHandlerFactory { + private static final Logger LOG = Logger.getLogger(DefaultAlertMsgHandlerFactory.class); + + public static class DefaultAlertMsgHandler extends MessageHandler { + public DefaultAlertMsgHandler(Message message, NotificationContext context) { + super(message, context); + } + + @Override + public HelixTaskResult handleMessage() throws InterruptedException { + LOG.info("Handling alert message: " + _message); + HelixManager manager = _notificationContext.getManager(); + HelixTaskResult result = new HelixTaskResult(); + + // Get alert-name from message + String alertNameStr = _message.getAttribute(Attributes.ALERT_NAME); + AlertName alertName = AlertName.from(alertNameStr); + + // Find action from alert config + HelixDataAccessor accessor = manager.getHelixDataAccessor(); + List alertConfigs = + accessor.getChildValues(accessor.keyBuilder().alertConfigs()); + + AlertAction action = null; + for (AlertConfig alertConfig : alertConfigs) { + action = alertConfig.findAlertAction(alertName); + if (action != null) { + LOG.info("Find alertAction: " + action + " for alertName " + alertName); + break; + } + } + + if (action != null) { + // perform action + HelixAdmin admin = manager.getClusterManagmentTool(); + try { + ClusterSetup setupTool = new ClusterSetup(admin); + ClusterSetup.processCommandLineArgs(setupTool, action.getCli()); + result.setSuccess(true); + + } catch (Exception e) { + String errMsg = "Exception execute action: " + action + " for alert: " + alertNameStr; + result.setSuccess(false); + result.setMessage(errMsg); + result.setException(e); + LOG.error(errMsg, e); + } + } else { + String errMsg = "Could NOT find action for alert: " + alertNameStr; + result.setSuccess(false); + result.setMessage(errMsg); + LOG.error(errMsg); + } + + return result; + } + + @Override + public void onError(Exception e, ErrorCode code, ErrorType type) { + LOG.error("Error processing message: " + _message + ", errCode: " + code + ", errType: " + + type, e); + } + + } + + public DefaultAlertMsgHandlerFactory() { + LOG.info("Construct default alert message handler factory"); + } + + @Override + public MessageHandler createHandler(Message message, NotificationContext context) { + String type = message.getMsgType(); + + if (!type.equals(getMessageType())) { + throw new HelixException("Unexpected msg type for message " + message.getMessageId() + + " type:" + message.getMsgType()); + } + + return new DefaultAlertMsgHandler(message, context); + + } + + @Override + public String getMessageType() { + return Message.MessageType.ALERT.name(); + } + + @Override + public void reset() { + LOG.info("Reset default alert message handler factory"); + } + +} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultControllerMessageHandlerFactory.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultControllerMessageHandlerFactory.java index 5f6d0835c2..33d271b916 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultControllerMessageHandlerFactory.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultControllerMessageHandlerFactory.java @@ -36,7 +36,7 @@ public MessageHandler createHandler(Message message, NotificationContext context String type = message.getMsgType(); if (!type.equals(getMessageType())) { - throw new HelixException("Unexpected msg type for message " + message.getMsgId() + " type:" + throw new HelixException("Unexpected msg type for message " + message.getMessageId() + " type:" + message.getMsgType()); } @@ -63,18 +63,18 @@ public HelixTaskResult handleMessage() throws InterruptedException { String type = _message.getMsgType(); HelixTaskResult result = new HelixTaskResult(); if (!type.equals(MessageType.CONTROLLER_MSG.toString())) { - throw new HelixException("Unexpected msg type for message " + _message.getMsgId() + throw new HelixException("Unexpected msg type for message " + _message.getMessageId() + " type:" + _message.getMsgType()); } result.getTaskResultMap().put("ControllerResult", - "msg " + _message.getMsgId() + " from " + _message.getMsgSrc() + " processed"); + "msg " + _message.getMessageId() + " from " + _message.getMsgSrc() + " processed"); result.setSuccess(true); return result; } @Override public void onError(Exception e, ErrorCode code, ErrorType type) { - _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMsgId(), e); + _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMessageId(), e); } } } diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultParticipantErrorMessageHandlerFactory.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultParticipantErrorMessageHandlerFactory.java index d2e56eb948..5e3a7ea455 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultParticipantErrorMessageHandlerFactory.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultParticipantErrorMessageHandlerFactory.java @@ -79,15 +79,15 @@ public HelixTaskResult handleMessage() throws InterruptedException { _logger.info("Instance " + _message.getMsgSrc() + " disabled"); } else if (actionOnError == ActionOnError.DISABLE_PARTITION) { _manager.getClusterManagmentTool().enablePartition(false, _manager.getClusterName(), - _message.getMsgSrc(), _message.getResourceName(), - Arrays.asList(_message.getPartitionName())); - _logger.info("partition " + _message.getPartitionName() + " disabled"); + _message.getMsgSrc(), _message.getResourceId().stringify(), + Arrays.asList(_message.getPartitionId().stringify())); + _logger.info("partition " + _message.getPartitionId() + " disabled"); } else if (actionOnError == ActionOnError.DISABLE_RESOURCE) { // NOT IMPLEMENTED, or we can disable all partitions // _manager.getClusterManagmentTool().en(_manager.getClusterName(), // _manager.getInstanceName(), // _message.getResourceName(), _message.getPartitionName(), false); - _logger.info("resource " + _message.getResourceName() + " disabled"); + _logger.info("resource " + _message.getResourceId() + " disabled"); } } catch (Exception e) { _logger.error("", e); @@ -99,7 +99,7 @@ public HelixTaskResult handleMessage() throws InterruptedException { @Override public void onError(Exception e, ErrorCode code, ErrorType type) { - _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMsgId(), e); + _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMessageId(), e); } } @@ -109,7 +109,7 @@ public MessageHandler createHandler(Message message, NotificationContext context String type = message.getMsgType(); if (!type.equals(getMessageType())) { - throw new HelixException("Unexpected msg type for message " + message.getMsgId() + " type:" + throw new HelixException("Unexpected msg type for message " + message.getMessageId() + " type:" + message.getMsgType()); } diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java index 5451a8128c..4fe916447e 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java @@ -20,9 +20,6 @@ */ import java.io.StringReader; -import java.text.DateFormat; -import java.text.SimpleDateFormat; -import java.util.Date; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -37,23 +34,28 @@ import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.messaging.AsyncCallback; import org.apache.helix.messaging.handling.HelixTaskResult; import org.apache.helix.messaging.handling.MessageHandler; import org.apache.helix.messaging.handling.MessageHandlerFactory; import org.apache.helix.model.IdealState; import org.apache.helix.model.Message; -import org.apache.helix.model.StatusUpdate; import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StatusUpdate; import org.apache.helix.util.StatusUpdateUtil; import org.apache.log4j.Logger; import org.codehaus.jackson.map.ObjectMapper; /* - * The current implementation supports throttling on STATE-TRANSITION type of message, transition SCHEDULED-COMPLETED. - * + * The current implementation supports throttling on STATE-TRANSITION type of message, transition SCHEDULED-COMPLETED. + * */ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFactory { public static final String WAIT_ALL = "WAIT_ALL"; @@ -76,7 +78,7 @@ public SchedulerAsyncCallback(Message originalMessage, HelixManager manager) { @Override public void onTimeOut() { - _logger.info("Scheduler msg timeout " + _originalMessage.getMsgId() + " timout with " + _logger.info("Scheduler msg timeout " + _originalMessage.getMessageId() + " timout with " + _timeout + " Ms"); _statusUpdateUtil.logError(_originalMessage, SchedulerAsyncCallback.class, "Task timeout", @@ -86,13 +88,13 @@ public void onTimeOut() { @Override public void onReplyMessage(Message message) { - _logger.info("Update for scheduler msg " + _originalMessage.getMsgId() + " Message " + _logger.info("Update for scheduler msg " + _originalMessage.getMessageId() + " Message " + message.getMsgSrc() + " id " + message.getCorrelationId() + " completed"); String key = "MessageResult " + message.getMsgSrc() + " " + UUID.randomUUID(); _resultSummaryMap.put(key, message.getResultMap()); if (this.isDone()) { - _logger.info("Scheduler msg " + _originalMessage.getMsgId() + " completed"); + _logger.info("Scheduler msg " + _originalMessage.getMessageId() + " completed"); _statusUpdateUtil.logInfo(_originalMessage, SchedulerAsyncCallback.class, "Scheduler task completed", _manager.getHelixDataAccessor()); addSummary(_resultSummaryMap, _originalMessage, _manager, false); @@ -110,13 +112,12 @@ private void addSummary(Map> _resultSummaryMap, Builder keyBuilder = accessor.keyBuilder(); ZNRecord statusUpdate = accessor.getProperty( - keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), - originalMessage.getMsgId())).getRecord(); + keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), originalMessage + .getMessageId().stringify())).getRecord(); statusUpdate.getMapFields().putAll(_resultSummaryMap); - accessor.setProperty( - keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), - originalMessage.getMsgId()), new StatusUpdate(statusUpdate)); + accessor.setProperty(keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), + originalMessage.getMessageId().stringify()), new StatusUpdate(statusUpdate)); } } @@ -133,7 +134,7 @@ public MessageHandler createHandler(Message message, NotificationContext context String type = message.getMsgType(); if (!type.equals(getMessageType())) { - throw new HelixException("Unexpected msg type for message " + message.getMsgId() + " type:" + throw new HelixException("Unexpected msg type for message " + message.getMessageId() + " type:" + message.getMsgType()); } @@ -159,7 +160,7 @@ public DefaultSchedulerMessageHandler(Message message, NotificationContext conte } void handleMessageUsingScheduledTaskQueue(Criteria recipientCriteria, Message messageTemplate, - String controllerMsgId) { + MessageId controllerMsgId) { HelixDataAccessor accessor = _manager.getHelixDataAccessor(); Builder keyBuilder = accessor.keyBuilder(); @@ -180,7 +181,7 @@ void handleMessageUsingScheduledTaskQueue(Criteria recipientCriteria, Message me } IdealState newAddedScheduledTasks = new IdealState(taskQueueName); newAddedScheduledTasks.setBucketSize(TASKQUEUE_BUCKET_NUM); - newAddedScheduledTasks.setStateModelDefRef(SCHEDULER_TASK_QUEUE); + newAddedScheduledTasks.setStateModelDefId(StateModelDefId.from(SCHEDULER_TASK_QUEUE)); synchronized (_manager) { int existingTopPartitionId = 0; @@ -196,9 +197,10 @@ void handleMessageUsingScheduledTaskQueue(Criteria recipientCriteria, Message me String partitionId = taskQueueName + "_" + existingTopPartitionId; existingTopPartitionId++; String instanceName = task.getTgtName(); - newAddedScheduledTasks.setPartitionState(partitionId, instanceName, "COMPLETED"); + newAddedScheduledTasks.setPartitionState(PartitionId.from(partitionId), + ParticipantId.from(instanceName), State.from("COMPLETED")); task.getRecord().setSimpleField(instanceName, "COMPLETED"); - task.getRecord().setSimpleField(CONTROLLER_MSG_ID, controllerMsgId); + task.getRecord().setSimpleField(CONTROLLER_MSG_ID, controllerMsgId.stringify()); List priorityList = new LinkedList(); priorityList.add(instanceName); @@ -206,7 +208,7 @@ void handleMessageUsingScheduledTaskQueue(Criteria recipientCriteria, Message me newAddedScheduledTasks.getRecord().setMapField(partitionId, task.getRecord().getSimpleFields()); _logger.info("Scheduling for controllerMsg " + controllerMsgId + " , sending task " - + partitionId + " " + task.getMsgId() + " to " + instanceName); + + partitionId + " " + task.getMessageId() + " to " + instanceName); if (_logger.isDebugEnabled()) { _logger.debug(task.getRecord().getSimpleFields()); @@ -222,18 +224,19 @@ void handleMessageUsingScheduledTaskQueue(Criteria recipientCriteria, Message me ZNRecord statusUpdate = accessor.getProperty( - keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), - _message.getMsgId())).getRecord(); + keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), _message + .getMessageId().stringify())).getRecord(); statusUpdate.getMapFields().put("SentMessageCount", sendSummary); accessor.updateProperty(keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), - _message.getMsgId()), new StatusUpdate(statusUpdate)); + _message.getMessageId().stringify()), new StatusUpdate(statusUpdate)); } private int findTopPartitionId(IdealState currentTaskQueue) { int topId = 0; - for (String partitionName : currentTaskQueue.getPartitionSet()) { + for (PartitionId partitionId : currentTaskQueue.getPartitionIdSet()) { try { + String partitionName = partitionId.stringify(); String partitionNumStr = partitionName.substring(partitionName.lastIndexOf('_') + 1); int num = Integer.parseInt(partitionNumStr); if (topId < num) { @@ -251,7 +254,7 @@ public HelixTaskResult handleMessage() throws InterruptedException { String type = _message.getMsgType(); HelixTaskResult result = new HelixTaskResult(); if (!type.equals(MessageType.SCHEDULER_MSG.toString())) { - throw new HelixException("Unexpected msg type for message " + _message.getMsgId() + throw new HelixException("Unexpected msg type for message " + _message.getMessageId() + " type:" + _message.getMsgType()); } // Parse timeout value @@ -298,11 +301,11 @@ public HelixTaskResult handleMessage() throws InterruptedException { if (InstanceType.PARTICIPANT == recipientCriteria.getRecipientInstanceType() && hasSchedulerTaskQueue) { handleMessageUsingScheduledTaskQueue(recipientCriteria, messageTemplate, - _message.getMsgId()); + _message.getMessageId()); result.setSuccess(true); - result.getTaskResultMap().put(SCHEDULER_MSG_ID, _message.getMsgId()); + result.getTaskResultMap().put(SCHEDULER_MSG_ID, _message.getMessageId().stringify()); result.getTaskResultMap().put("ControllerResult", - "msg " + _message.getMsgId() + " from " + _message.getMsgSrc() + " processed"); + "msg " + _message.getMessageId() + " from " + _message.getMsgSrc() + " processed"); return result; } @@ -318,6 +321,7 @@ public HelixTaskResult handleMessage() throws InterruptedException { _manager.getMessagingService().send(recipientCriteria, messageTemplate, callback, timeOut); } + HelixDataAccessor accessor = _manager.getHelixDataAccessor(); Builder keyBuilder = accessor.keyBuilder(); @@ -327,24 +331,24 @@ public HelixTaskResult handleMessage() throws InterruptedException { ZNRecord statusUpdate = accessor.getProperty( - keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), - _message.getMsgId())).getRecord(); + keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), _message + .getMessageId().stringify())).getRecord(); statusUpdate.getMapFields().put("SentMessageCount", sendSummary); accessor.setProperty(keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), - _message.getMsgId()), new StatusUpdate(statusUpdate)); + _message.getMessageId().stringify()), new StatusUpdate(statusUpdate)); result.getTaskResultMap().put("ControllerResult", - "msg " + _message.getMsgId() + " from " + _message.getMsgSrc() + " processed"); - result.getTaskResultMap().put(SCHEDULER_MSG_ID, _message.getMsgId()); + "msg " + _message.getMessageId() + " from " + _message.getMsgSrc() + " processed"); + result.getTaskResultMap().put(SCHEDULER_MSG_ID, _message.getMessageId().stringify()); result.setSuccess(true); return result; } @Override public void onError(Exception e, ErrorCode code, ErrorType type) { - _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMsgId(), e); + _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMessageId(), e); } } } diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java deleted file mode 100644 index c9ad0f3b72..0000000000 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java +++ /dev/null @@ -1,190 +0,0 @@ -package org.apache.helix.manager.zk; - -/* - * 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 org.apache.helix.HelixException; -import org.apache.helix.HelixTimerTask; -import org.apache.helix.InstanceType; -import org.apache.helix.PreConnectCallback; -import org.apache.helix.HelixConstants.ChangeType; -import org.apache.helix.controller.GenericHelixController; -import org.apache.helix.healthcheck.HealthStatsAggregationTask; -import org.apache.helix.healthcheck.HealthStatsAggregator; -import org.apache.helix.healthcheck.ParticipantHealthReportCollector; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.healthcheck.ParticipantHealthReportTask; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.participant.HelixStateMachineEngine; -import org.apache.helix.participant.StateMachineEngine; -import org.apache.log4j.Logger; -import org.apache.zookeeper.Watcher.Event.EventType; - -public class DistributedControllerManager extends AbstractManager { - private static Logger LOG = Logger.getLogger(DistributedControllerManager.class); - - final StateMachineEngine _stateMachineEngine; - final ParticipantHealthReportCollectorImpl _participantHealthInfoCollector; - - CallbackHandler _leaderElectionHandler = null; - final GenericHelixController _controller = new GenericHelixController(); - - /** - * hold timer tasks for controller only - * we need to add/remove controller timer tasks during handle new session - */ - final List _controllerTimerTasks = new ArrayList(); - - public DistributedControllerManager(String zkAddress, String clusterName, String instanceName) { - super(zkAddress, clusterName, instanceName, InstanceType.CONTROLLER_PARTICIPANT); - - _stateMachineEngine = new HelixStateMachineEngine(this); - _participantHealthInfoCollector = new ParticipantHealthReportCollectorImpl(this, _instanceName); - - _timerTasks.add(new ParticipantHealthReportTask(_participantHealthInfoCollector)); - - _controllerTimerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(this))); - _controllerTimerTasks.add(new ControllerManager.StatusDumpTask(_zkclient, this)); - - } - - @Override - public ParticipantHealthReportCollector getHealthReportCollector() { - checkConnected(); - return _participantHealthInfoCollector; - } - - @Override - public StateMachineEngine getStateMachineEngine() { - return _stateMachineEngine; - } - - @Override - protected List getControllerHelixTimerTasks() { - return _controllerTimerTasks; - } - - @Override - public void handleNewSession() throws Exception { - waitUntilConnected(); - - ParticipantManagerHelper participantHelper = - new ParticipantManagerHelper(this, _zkclient, _sessionTimeout); - - /** - * stop all timer tasks, reset all handlers, make sure cleanup completed for previous session - * disconnect if fail to cleanup - */ - stopTimerTasks(); - if (_leaderElectionHandler != null) { - _leaderElectionHandler.reset(); - } - resetHandlers(); - - /** - * clean up write-through cache - */ - _baseDataAccessor.reset(); - - /** - * from here on, we are dealing with new session - */ - if (!ZKUtil.isClusterSetup(_clusterName, _zkclient)) { - throw new HelixException("Cluster structure is not set up for cluster: " + _clusterName); - } - - /** - * auto-join - */ - participantHelper.joinCluster(); - - /** - * Invoke PreConnectCallbacks - */ - for (PreConnectCallback callback : _preConnectCallbacks) { - callback.onPreConnect(); - } - - participantHelper.createLiveInstance(); - - participantHelper.carryOverPreviousCurrentState(); - - participantHelper.setupMsgHandler(); - - /** - * leader election - */ - if (_leaderElectionHandler != null) { - _leaderElectionHandler.init(); - } else { - _leaderElectionHandler = - new CallbackHandler(this, _zkclient, _keyBuilder.controller(), - new DistributedLeaderElection(this, _controller), new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }, ChangeType.CONTROLLER); - } - - /** - * start health-check timer task - */ - participantHelper.createHealthCheckPath(); - startTimerTasks(); - - /** - * init handlers - * ok to init message handler, data-accessor, and controller handlers twice - * the second init will be skipped (see CallbackHandler) - */ - initHandlers(_handlers); - - } - - @Override - void doDisconnect() { - if (_leaderElectionHandler != null) { - _leaderElectionHandler.reset(); - } - } - - @Override - public boolean isLeader() { - if (!isConnected()) { - return false; - } - - try { - LiveInstance leader = _dataAccessor.getProperty(_keyBuilder.controllerLeader()); - if (leader != null) { - String leaderName = leader.getInstanceName(); - String sessionId = leader.getSessionId(); - if (leaderName != null && leaderName.equals(_instanceName) && sessionId != null - && sessionId.equals(_sessionId)) { - return true; - } - } - } catch (Exception e) { - // log - } - return false; - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java index 0ab83421d1..d4ffd80a45 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java @@ -19,19 +19,18 @@ * under the License. */ -import java.lang.management.ManagementFactory; +import java.util.List; import org.apache.helix.ControllerChangeListener; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; +import org.apache.helix.HelixTimerTask; import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; -import org.apache.helix.PropertyType; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.PropertyType; import org.apache.helix.controller.GenericHelixController; -import org.apache.helix.controller.restlet.ZKPropertyTransferServer; import org.apache.helix.model.LeaderHistory; -import org.apache.helix.model.LiveInstance; import org.apache.log4j.Logger; /** @@ -40,12 +39,15 @@ public class DistributedLeaderElection implements ControllerChangeListener { private static Logger LOG = Logger.getLogger(DistributedLeaderElection.class); - final AbstractManager _manager; + final HelixManager _manager; final GenericHelixController _controller; + final List _controllerTimerTasks; - public DistributedLeaderElection(AbstractManager manager, GenericHelixController controller) { + public DistributedLeaderElection(HelixManager manager, GenericHelixController controller, + List controllerTimerTasks) { _manager = manager; _controller = controller; + _controllerTimerTasks = controllerTimerTasks; } /** @@ -68,7 +70,8 @@ public synchronized void onControllerChange(NotificationContext changeContext) { return; } - ControllerManagerHelper controllerHelper = new ControllerManagerHelper(_manager); + ControllerManagerHelper controllerHelper = + new ControllerManagerHelper(_manager, _controllerTimerTasks); try { if (changeContext.getType().equals(NotificationContext.Type.INIT) || changeContext.getType().equals(NotificationContext.Type.CALLBACK)) { @@ -84,7 +87,7 @@ public synchronized void onControllerChange(NotificationContext changeContext) { + _manager.getClusterName()); updateHistory(manager); - _manager._baseDataAccessor.reset(); + _manager.getHelixDataAccessor().getBaseDataAccessor().reset(); controllerHelper.addListenersToController(_controller); controllerHelper.startControllerTimerTasks(); } @@ -98,7 +101,7 @@ public synchronized void onControllerChange(NotificationContext changeContext) { /** * clear write-through cache */ - _manager._baseDataAccessor.reset(); + _manager.getHelixDataAccessor().getBaseDataAccessor().reset(); } } catch (Exception e) { @@ -107,46 +110,7 @@ public synchronized void onControllerChange(NotificationContext changeContext) { } private boolean tryUpdateController(HelixManager manager) { - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - LiveInstance leader = new LiveInstance(manager.getInstanceName()); - try { - leader.setLiveInstance(ManagementFactory.getRuntimeMXBean().getName()); - leader.setSessionId(manager.getSessionId()); - leader.setHelixVersion(manager.getVersion()); - if (ZKPropertyTransferServer.getInstance() != null) { - String zkPropertyTransferServiceUrl = - ZKPropertyTransferServer.getInstance().getWebserviceUrl(); - if (zkPropertyTransferServiceUrl != null) { - leader.setWebserviceUrl(zkPropertyTransferServiceUrl); - } - } else { - LOG.warn("ZKPropertyTransferServer instnace is null"); - } - boolean success = accessor.createProperty(keyBuilder.controllerLeader(), leader); - if (success) { - return true; - } else { - LOG.info("Unable to become leader probably because some other controller becames the leader"); - } - } catch (Exception e) { - LOG.error( - "Exception when trying to updating leader record in cluster:" + manager.getClusterName() - + ". Need to check again whether leader node has been created or not", e); - } - - leader = accessor.getProperty(keyBuilder.controllerLeader()); - if (leader != null) { - String leaderSessionId = leader.getSessionId(); - LOG.info("Leader exists for cluster: " + manager.getClusterName() + ", currentLeader: " - + leader.getInstanceName() + ", leaderSessionId: " + leaderSessionId); - - if (leaderSessionId != null && leaderSessionId.equals(manager.getSessionId())) { - return true; - } - } - return false; + return ZkHelixLeaderElection.tryBecomingLeader(manager); } private void updateHistory(HelixManager manager) { diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java new file mode 100644 index 0000000000..10a55ffbdb --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java @@ -0,0 +1,303 @@ +package org.apache.helix.manager.zk; + +/* + * 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.ClusterMessagingService; +import org.apache.helix.ConfigAccessor; +import org.apache.helix.ConfigChangeListener; +import org.apache.helix.ControllerChangeListener; +import org.apache.helix.CurrentStateChangeListener; +import org.apache.helix.ExternalViewChangeListener; +import org.apache.helix.HelixAdmin; +import org.apache.helix.HelixAutoController; +import org.apache.helix.HelixConnection; +import org.apache.helix.HelixController; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixManager; +import org.apache.helix.HelixManagerProperties; +import org.apache.helix.HelixParticipant; +import org.apache.helix.HelixRole; +import org.apache.helix.IdealStateChangeListener; +import org.apache.helix.InstanceConfigChangeListener; +import org.apache.helix.InstanceType; +import org.apache.helix.LiveInstanceChangeListener; +import org.apache.helix.LiveInstanceInfoProvider; +import org.apache.helix.MessageListener; +import org.apache.helix.PreConnectCallback; +import org.apache.helix.PropertyKey; +import org.apache.helix.ScopedConfigChangeListener; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.Id; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.participant.StateMachineEngine; +import org.apache.helix.store.zk.ZkHelixPropertyStore; +import org.apache.log4j.Logger; + +/** + * Adapt helix-connection to helix-manager, so we can pass to callback-handler and + * notification-context + */ +public class HelixConnectionAdaptor implements HelixManager { + private static Logger LOG = Logger.getLogger(HelixConnectionAdaptor.class); + + final HelixRole _role; + final HelixConnection _connection; + final ClusterId _clusterId; + final Id _instanceId; + final InstanceType _instanceType; + final HelixDataAccessor _accessor; + final ClusterMessagingService _messagingService; + + public HelixConnectionAdaptor(HelixRole role) { + _role = role; + _connection = role.getConnection(); + _clusterId = role.getClusterId(); + _accessor = _connection.createDataAccessor(_clusterId); + + _instanceId = role.getId(); + _instanceType = role.getType(); + _messagingService = role.getMessagingService(); + } + + @Override + public void connect() throws Exception { + _connection.connect(); + } + + @Override + public boolean isConnected() { + return _connection.isConnected(); + } + + @Override + public void disconnect() { + _connection.disconnect(); + } + + @Override + public void addIdealStateChangeListener(IdealStateChangeListener listener) throws Exception { + _connection.addIdealStateChangeListener(_role, listener, _clusterId); + } + + @Override + public void addLiveInstanceChangeListener(LiveInstanceChangeListener listener) throws Exception { + _connection.addLiveInstanceChangeListener(_role, listener, _clusterId); + } + + @Override + public void addConfigChangeListener(ConfigChangeListener listener) throws Exception { + _connection.addConfigChangeListener(_role, listener, _clusterId); + } + + @Override + public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener) + throws Exception { + _connection.addInstanceConfigChangeListener(_role, listener, _clusterId); + } + + @Override + public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope) + throws Exception { + _connection.addConfigChangeListener(_role, listener, _clusterId, scope); + } + + @Override + public void addMessageListener(MessageListener listener, String instanceName) throws Exception { + _connection.addMessageListener(_role, listener, _clusterId, ParticipantId.from(instanceName)); + } + + @Override + public void addCurrentStateChangeListener(CurrentStateChangeListener listener, + String instanceName, String sessionId) throws Exception { + _connection.addCurrentStateChangeListener(_role, listener, _clusterId, + ParticipantId.from(instanceName), SessionId.from(sessionId)); + } + + @Override + public void addExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception { + _connection.addExternalViewChangeListener(_role, listener, _clusterId); + } + + @Override + public void addControllerListener(ControllerChangeListener listener) { + _connection.addControllerListener(_role, listener, _clusterId); + } + + @Override + public boolean removeListener(PropertyKey key, Object listener) { + return _connection.removeListener(_role, listener, key); + } + + @Override + public HelixDataAccessor getHelixDataAccessor() { + return _accessor; + } + + @Override + public ConfigAccessor getConfigAccessor() { + return _connection.getConfigAccessor(); + } + + @Override + public String getClusterName() { + return _clusterId.stringify(); + } + + @Override + public String getInstanceName() { + return _instanceId.stringify(); + } + + @Override + public String getSessionId() { + return _connection.getSessionId().stringify(); + } + + @Override + public long getLastNotificationTime() { + return 0; + } + + @Override + public HelixAdmin getClusterManagmentTool() { + return _connection.createClusterManagementTool(); + } + + @Override + public ZkHelixPropertyStore getHelixPropertyStore() { + return (ZkHelixPropertyStore) _connection.createPropertyStore(_clusterId); + } + + @Override + public ClusterMessagingService getMessagingService() { + return _messagingService; + } + + @Override + public InstanceType getInstanceType() { + return _instanceType; + } + + @Override + public String getVersion() { + return _connection.getHelixVersion(); + } + + @Override + public HelixManagerProperties getProperties() { + return _connection.getHelixProperties(); + } + + @Override + public StateMachineEngine getStateMachineEngine() { + StateMachineEngine engine = null; + switch (_role.getType()) { + case PARTICIPANT: + HelixParticipant participant = (HelixParticipant) _role; + engine = participant.getStateMachineEngine(); + break; + case CONTROLLER_PARTICIPANT: + HelixAutoController autoController = (HelixAutoController) _role; + engine = autoController.getStateMachineEngine(); + break; + default: + LOG.error("Helix role: " + _role.getType() + " does NOT have state-machine engine"); + break; + } + + return engine; + } + + @Override + public boolean isLeader() { + boolean isLeader = false; + switch (_role.getType()) { + case CONTROLLER: + HelixController controller = (HelixController) _role; + isLeader = controller.isLeader(); + break; + case CONTROLLER_PARTICIPANT: + HelixAutoController autoController = (HelixAutoController) _role; + isLeader = autoController.isLeader(); + break; + default: + LOG.error("Helix role: " + _role.getType() + " does NOT support leadership"); + break; + } + return isLeader; + } + + @Override + public void startTimerTasks() { + throw new UnsupportedOperationException( + "HelixConnectionAdaptor does NOT support start timer tasks"); + } + + @Override + public void stopTimerTasks() { + throw new UnsupportedOperationException( + "HelixConnectionAdaptor does NOT support stop timer tasks"); + } + + @Override + public void addPreConnectCallback(PreConnectCallback callback) { + switch (_role.getType()) { + case PARTICIPANT: + HelixParticipant participant = (HelixParticipant) _role; + participant.addPreConnectCallback(callback); + break; + case CONTROLLER_PARTICIPANT: + HelixAutoController autoController = (HelixAutoController) _role; + autoController.addPreConnectCallback(callback); + break; + default: + LOG.error("Helix role: " + _role.getType() + + " does NOT support adding a pre-connect callback"); + break; + } + } + + @Override + public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) { + switch (_role.getType()) { + case PARTICIPANT: + HelixParticipant participant = (HelixParticipant) _role; + participant.setLiveInstanceInfoProvider(liveInstanceInfoProvider); + break; + case CONTROLLER_PARTICIPANT: + HelixAutoController autoController = (HelixAutoController) _role; + autoController.setLiveInstanceInfoProvider(liveInstanceInfoProvider); + break; + default: + LOG.error("Helix role: " + _role.getType() + + " does NOT support setting additional live instance information"); + break; + } + } + + @Override + public void addControllerMessageListener(MessageListener listener) { + _connection.addControllerMessageListener(_role, listener, _clusterId); + } + +} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/HelixGroupCommit.java b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixGroupCommit.java index 8dd9d77189..48f0647c26 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/HelixGroupCommit.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixGroupCommit.java @@ -27,7 +27,6 @@ import org.I0Itec.zkclient.DataUpdater; import org.I0Itec.zkclient.exception.ZkBadVersionException; import org.I0Itec.zkclient.exception.ZkNoNodeException; -import org.apache.helix.AccessOption; import org.apache.log4j.Logger; import org.apache.zookeeper.data.Stat; diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/AccumulateAggregationType.java b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixManagerShutdownHook.java similarity index 58% rename from helix-core/src/main/java/org/apache/helix/healthcheck/AccumulateAggregationType.java rename to helix-core/src/main/java/org/apache/helix/manager/zk/HelixManagerShutdownHook.java index a3c443f497..7ada7c8cc6 100644 --- a/helix-core/src/main/java/org/apache/helix/healthcheck/AccumulateAggregationType.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixManagerShutdownHook.java @@ -1,4 +1,4 @@ -package org.apache.helix.healthcheck; +package org.apache.helix.manager.zk; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,23 +19,27 @@ * under the License. */ +import org.apache.helix.HelixManager; import org.apache.log4j.Logger; -public class AccumulateAggregationType implements AggregationType { - - private static final Logger logger = Logger.getLogger(AccumulateAggregationType.class); +/** + * Shutdown hook for helix manager + * Working for kill -2/-15 + * NOT working for kill -9 + */ +public class HelixManagerShutdownHook extends Thread { + private static Logger LOG = Logger.getLogger(HelixManagerShutdownHook.class); - public final static String TYPE_NAME = "accumulate"; + final HelixManager _manager; - @Override - public String getName() { - return TYPE_NAME; + public HelixManagerShutdownHook(HelixManager manager) { + _manager = manager; } @Override - public String merge(String iv, String ev, long prevTimestamp) { - double inVal = Double.parseDouble(iv); - double existingVal = Double.parseDouble(ev); - return String.valueOf(inVal + existingVal); + public void run() { + LOG.info("HelixControllerMainShutdownHook invoked on manager: " + _manager.getClusterName() + + ", " + _manager.getInstanceName()); + _manager.disconnect(); } } 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 deleted file mode 100644 index 0af7e77c69..0000000000 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java +++ /dev/null @@ -1,155 +0,0 @@ -package org.apache.helix.manager.zk; - -/* - * 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.List; - -import org.apache.helix.BaseDataAccessor; -import org.apache.helix.HelixException; -import org.apache.helix.InstanceType; -import org.apache.helix.PreConnectCallback; -import org.apache.helix.PropertyPathConfig; -import org.apache.helix.PropertyType; -import org.apache.helix.ZNRecord; -import org.apache.helix.healthcheck.ParticipantHealthReportCollector; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.healthcheck.ParticipantHealthReportTask; -import org.apache.helix.participant.HelixStateMachineEngine; -import org.apache.helix.participant.StateMachineEngine; -import org.apache.log4j.Logger; - -public class ParticipantManager extends AbstractManager { - - private static Logger LOG = Logger.getLogger(ParticipantManager.class); - - /** - * state-transition message handler factory for helix-participant - */ - final StateMachineEngine _stateMachineEngine; - - final ParticipantHealthReportCollectorImpl _participantHealthInfoCollector; - - public ParticipantManager(String zkAddress, String clusterName, String instanceName) { - super(zkAddress, clusterName, instanceName, InstanceType.PARTICIPANT); - - _stateMachineEngine = new HelixStateMachineEngine(this); - _participantHealthInfoCollector = new ParticipantHealthReportCollectorImpl(this, _instanceName); - - _timerTasks.add(new ParticipantHealthReportTask(_participantHealthInfoCollector)); - } - - @Override - public ParticipantHealthReportCollector getHealthReportCollector() { - checkConnected(); - return _participantHealthInfoCollector; - } - - @Override - public StateMachineEngine getStateMachineEngine() { - return _stateMachineEngine; - } - - @Override - public void handleNewSession() { - waitUntilConnected(); - - /** - * stop timer tasks, reset all handlers, make sure cleanup completed for previous session - * disconnect if cleanup fails - */ - stopTimerTasks(); - resetHandlers(); - - /** - * clear write-through cache - */ - _baseDataAccessor.reset(); - - /** - * from here on, we are dealing with new session - */ - if (!ZKUtil.isClusterSetup(_clusterName, _zkclient)) { - throw new HelixException("Cluster structure is not set up for cluster: " + _clusterName); - } - - /** - * auto-join - */ - ParticipantManagerHelper participantHelper = - new ParticipantManagerHelper(this, _zkclient, _sessionTimeout); - participantHelper.joinCluster(); - - /** - * Invoke PreConnectCallbacks - */ - for (PreConnectCallback callback : _preConnectCallbacks) { - callback.onPreConnect(); - } - - participantHelper.createLiveInstance(); - - participantHelper.carryOverPreviousCurrentState(); - - /** - * setup message listener - */ - participantHelper.setupMsgHandler(); - - /** - * start health check timer task - */ - participantHelper.createHealthCheckPath(); - startTimerTasks(); - - /** - * init handlers - * ok to init message handler and data-accessor twice - * the second init will be skipped (see CallbackHandler) - */ - initHandlers(_handlers); - - } - - /** - * helix-participant uses a write-through cache for current-state - */ - @Override - BaseDataAccessor createBaseDataAccessor(ZkBaseDataAccessor baseDataAccessor) { - String curStatePath = - PropertyPathConfig.getPath(PropertyType.CURRENTSTATES, _clusterName, _instanceName); - return new ZkCacheBaseDataAccessor(baseDataAccessor, Arrays.asList(curStatePath)); - - } - - @Override - public boolean isLeader() { - return false; - } - - /** - * disconnect logic for helix-participant - */ - @Override - void doDisconnect() { - // nothing for participant - } -} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java index 70dd592f44..b6d231f8ca 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java @@ -27,22 +27,21 @@ import org.apache.helix.AccessOption; import org.apache.helix.ConfigAccessor; import org.apache.helix.HelixAdmin; -import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixException; import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; +import org.apache.helix.LiveInstanceInfoProvider; import org.apache.helix.PropertyKey; import org.apache.helix.ZNRecord; import org.apache.helix.messaging.DefaultMessagingService; import org.apache.helix.model.CurrentState; import org.apache.helix.model.HelixConfigScope; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.StateModelDefinition; -import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StateModelDefinition; import org.apache.helix.model.builder.HelixConfigScopeBuilder; -import org.apache.helix.participant.HelixStateMachineEngine; import org.apache.helix.participant.StateMachineEngine; import org.apache.helix.participant.statemachine.ScheduledTaskStateModelFactory; import org.apache.log4j.Logger; @@ -55,7 +54,7 @@ public class ParticipantManagerHelper { private static Logger LOG = Logger.getLogger(ParticipantManagerHelper.class); final ZkClient _zkclient; - final AbstractManager _manager; + final HelixManager _manager; final PropertyKey.Builder _keyBuilder; final String _clusterName; final String _instanceName; @@ -67,8 +66,10 @@ public class ParticipantManagerHelper { final ZKHelixDataAccessor _dataAccessor; final DefaultMessagingService _messagingService; final StateMachineEngine _stateMachineEngine; + final LiveInstanceInfoProvider _liveInstanceInfoProvider; - public ParticipantManagerHelper(AbstractManager manager, ZkClient zkclient, int sessionTimeout) { + public ParticipantManagerHelper(HelixManager manager, ZkClient zkclient, int sessionTimeout, + LiveInstanceInfoProvider liveInstanceInfoProvider) { _zkclient = zkclient; _manager = manager; _clusterName = manager.getClusterName(); @@ -82,6 +83,7 @@ public ParticipantManagerHelper(AbstractManager manager, ZkClient zkclient, int _dataAccessor = (ZKHelixDataAccessor) manager.getHelixDataAccessor(); _messagingService = (DefaultMessagingService) manager.getMessagingService(); _stateMachineEngine = manager.getStateMachineEngine(); + _liveInstanceInfoProvider = liveInstanceInfoProvider; } public void joinCluster() { @@ -92,8 +94,8 @@ public void joinCluster() { new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster( _manager.getClusterName()).build(); autoJoin = - Boolean - .parseBoolean(_configAccessor.get(scope, HelixManager.ALLOW_PARTICIPANT_AUTO_JOIN)); + Boolean.parseBoolean(_configAccessor.get(scope, + ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN)); LOG.info("instance: " + _instanceName + " auto-joining " + _clusterName + " is " + autoJoin); } catch (Exception e) { // autoJoin is false @@ -128,6 +130,19 @@ public void createLiveInstance() { liveInstance.setHelixVersion(_manager.getVersion()); liveInstance.setLiveInstance(ManagementFactory.getRuntimeMXBean().getName()); + // LiveInstanceInfoProvider liveInstanceInfoProvider = _manager._liveInstanceInfoProvider; + if (_liveInstanceInfoProvider != null) { + LOG.info("invoke liveInstanceInfoProvider"); + ZNRecord additionalLiveInstanceInfo = + _liveInstanceInfoProvider.getAdditionalLiveInstanceInfo(); + if (additionalLiveInstanceInfo != null) { + additionalLiveInstanceInfo.merge(liveInstance.getRecord()); + ZNRecord mergedLiveInstance = new ZNRecord(additionalLiveInstanceInfo, _instanceName); + liveInstance = new LiveInstance(mergedLiveInstance); + LOG.info("instanceName: " + _instanceName + ", mergedLiveInstance: " + liveInstance); + } + } + boolean retry; do { retry = false; @@ -151,14 +166,14 @@ public void createLiveInstance() { * update sessionId field in live-instance if necessary */ LiveInstance curLiveInstance = new LiveInstance(record); - if (!curLiveInstance.getSessionId().equals(_sessionId)) { + if (!curLiveInstance.getTypedSessionId().stringify().equals(_sessionId)) { /** * in last handle-new-session, * live-instance is created by new zkconnection with stale session-id inside * just update session-id field */ LOG.info("overwriting session-id by ephemeralOwner: " + ephemeralOwner - + ", old-sessionId: " + curLiveInstance.getSessionId() + ", new-sessionId: " + + ", old-sessionId: " + curLiveInstance.getTypedSessionId() + ", new-sessionId: " + _sessionId); curLiveInstance.setSessionId(_sessionId); @@ -250,7 +265,7 @@ public void carryOverPreviousCurrentState() { } } - public void setupMsgHandler() { + public void setupMsgHandler() throws Exception { _messagingService.registerMessageHandlerFactory(MessageType.STATE_TRANSITION.toString(), _stateMachineEngine); _manager.addMessageListener(_messagingService.getExecutor(), _instanceName); @@ -263,15 +278,4 @@ public void setupMsgHandler() { } - /** - * create zk path for health check info - * TODO move it to cluster-setup - */ - public void createHealthCheckPath() { - String healthCheckInfoPath = _dataAccessor.keyBuilder().healthReports(_instanceName).getPath(); - if (!_zkclient.exists(healthCheckInfoPath)) { - _zkclient.createPersistent(healthCheckInfoPath, true); - LOG.info("Created healthcheck info path " + healthCheckInfoPath); - } - } } diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/WriteThroughCache.java b/helix-core/src/main/java/org/apache/helix/manager/zk/WriteThroughCache.java index 4ad0f0f18c..7c991aef07 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/WriteThroughCache.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/WriteThroughCache.java @@ -19,7 +19,6 @@ * under the License. */ -import java.io.File; import java.util.List; import org.I0Itec.zkclient.exception.ZkNoNodeException; diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java index 08a8208103..7223c0a34c 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java @@ -33,7 +33,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -52,12 +51,17 @@ import org.apache.helix.PropertyPathConfig; import org.apache.helix.PropertyType; import org.apache.helix.ZNRecord; -import org.apache.helix.alerts.AlertsHolder; -import org.apache.helix.alerts.StatsHolder; -import org.apache.helix.model.Alerts; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ConstraintId; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; import org.apache.helix.model.ClusterConstraints; import org.apache.helix.model.ClusterConstraints.ConstraintType; -import org.apache.helix.model.ConfigScope; import org.apache.helix.model.ConstraintItem; import org.apache.helix.model.CurrentState; import org.apache.helix.model.ExternalView; @@ -72,14 +76,14 @@ import org.apache.helix.model.Message.MessageState; import org.apache.helix.model.Message.MessageType; import org.apache.helix.model.PauseSignal; -import org.apache.helix.model.PersistentStats; import org.apache.helix.model.StateModelDefinition; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.apache.helix.util.HelixUtil; import org.apache.helix.util.RebalanceUtil; import org.apache.log4j.Logger; public class ZKHelixAdmin implements HelixAdmin { + static Logger LOG = Logger.getLogger(ZKHelixAdmin.class); + public static final String CONNECTION_TIMEOUT = "helixAdmin.timeOutInSec"; private final ZkClient _zkClient; private final ConfigAccessor _configAccessor; @@ -124,7 +128,10 @@ public void addInstance(String clusterName, InstanceConfig instanceConfig) { @Override public void dropInstance(String clusterName, InstanceConfig instanceConfig) { - // String instanceConfigsPath = HelixUtil.getConfigPath(clusterName); + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); + Builder keyBuilder = accessor.keyBuilder(); + String instanceConfigsPath = PropertyPathConfig.getPath(PropertyType.CONFIGS, clusterName, ConfigScopeProperty.PARTICIPANT.toString()); @@ -142,6 +149,29 @@ public void dropInstance(String clusterName, InstanceConfig instanceConfig) { + clusterName); } + String instanceId = instanceConfig.getInstanceName(); + + // ensure node is stopped + LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance(instanceId)); + if (liveInstance != null) { + throw new HelixException("Can't drop " + instanceId + ", please stop " + instanceId + + " before drop it"); + } + + InstanceConfig config = accessor.getProperty(keyBuilder.instanceConfig(instanceId)); + if (config == null) { + String error = "Node " + instanceId + " does not exist, cannot drop"; + LOG.warn(error); + throw new HelixException(error); + } + + // ensure node is disabled, otherwise fail + if (config.getInstanceEnabled()) { + String error = "Node " + instanceId + " is enabled, cannot drop"; + LOG.warn(error); + throw new HelixException(error); + } + // delete config path ZKUtil.dropChildren(_zkClient, instanceConfigsPath, instanceConfig.getRecord()); @@ -236,11 +266,9 @@ public void enablePartition(final boolean enabled, final String clusterName, IdealState idealState = new IdealState(idealStateRecord); for (String partitionName : partitionNames) { if ((idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO && idealState - .getPreferenceList(partitionName) == null) - || (idealState.getRebalanceMode() == RebalanceMode.USER_DEFINED && idealState - .getPreferenceList(partitionName) == null) + .getPreferenceList(PartitionId.from(partitionName)) == null) || (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED && idealState - .getInstanceStateMap(partitionName) == null)) { + .getParticipantStateMap(PartitionId.from(partitionName)) == null)) { logger.warn("Cluster: " + clusterName + ", resource: " + resourceName + ", partition: " + partitionName + ", partition does not exist in ideal state"); } @@ -330,9 +358,10 @@ public void resetPartition(String clusterName, String instanceName, String resou } // check partition is in ERROR state - String sessionId = liveInstance.getSessionId(); + SessionId sessionId = liveInstance.getTypedSessionId(); CurrentState curState = - accessor.getProperty(keyBuilder.currentState(instanceName, sessionId, resourceName)); + accessor.getProperty(keyBuilder.currentState(instanceName, sessionId.stringify(), + resourceName)); for (String partitionName : resetPartitionNames) { if (!curState.getState(partitionName).equals(HelixDefinedState.ERROR.toString())) { throw new HelixException("Can't reset state for " + resourceName + "/" + partitionNames @@ -341,8 +370,9 @@ public void resetPartition(String clusterName, String instanceName, String resou } // check stateModelDef exists and get initial state - String stateModelDef = idealState.getStateModelDefRef(); - StateModelDefinition stateModel = accessor.getProperty(keyBuilder.stateModelDef(stateModelDef)); + StateModelDefId stateModelDef = idealState.getStateModelDefId(); + StateModelDefinition stateModel = + accessor.getProperty(keyBuilder.stateModelDef(stateModelDef.stringify())); if (stateModel == null) { throw new HelixException("Can't reset state for " + resourceName + "/" + partitionNames + " on " + instanceName + ", because " + stateModelDef + " is NOT found"); @@ -352,9 +382,9 @@ public void resetPartition(String clusterName, String instanceName, String resou List messages = accessor.getChildValues(keyBuilder.messages(instanceName)); for (Message message : messages) { if (!MessageType.STATE_TRANSITION.toString().equalsIgnoreCase(message.getMsgType()) - || !sessionId.equals(message.getTgtSessionId()) - || !resourceName.equals(message.getResourceName()) - || !resetPartitionNames.contains(message.getPartitionName())) { + || !sessionId.equals(message.getTypedTgtSessionId()) + || !resourceName.equals(message.getResourceId().stringify()) + || !resetPartitionNames.contains(message.getPartitionId().stringify())) { continue; } @@ -375,18 +405,18 @@ public void resetPartition(String clusterName, String instanceName, String resou List messageKeys = new ArrayList(); for (String partitionName : resetPartitionNames) { // send ERROR to initialState message - String msgId = UUID.randomUUID().toString(); + MessageId msgId = MessageId.from(UUID.randomUUID().toString()); Message message = new Message(MessageType.STATE_TRANSITION, msgId); message.setSrcName(adminName); message.setTgtName(instanceName); message.setMsgState(MessageState.NEW); - message.setPartitionName(partitionName); - message.setResourceName(resourceName); + message.setPartitionId(PartitionId.from(partitionName)); + message.setResourceId(ResourceId.from(resourceName)); message.setTgtSessionId(sessionId); message.setStateModelDef(stateModelDef); - message.setFromState(HelixDefinedState.ERROR.toString()); - message.setToState(stateModel.getInitialState()); - message.setStateModelFactoryName(idealState.getStateModelFactoryName()); + message.setFromState(State.from(HelixDefinedState.ERROR.toString())); + message.setToState(stateModel.getTypedInitialState()); + message.setStateModelFactoryId(idealState.getStateModelFactoryId()); resetMessages.add(message); messageKeys.add(keyBuilder.message(instanceName, message.getId())); @@ -586,7 +616,7 @@ public void addResource(String clusterName, String resourceName, int partitions, @Override public void addResource(String clusterName, String resourceName, IdealState idealstate) { - String stateModelRef = idealstate.getStateModelDefRef(); + String stateModelRef = idealstate.getStateModelDefId().stringify(); String stateModelDefPath = PropertyPathConfig.getPath(PropertyType.STATEMODELDEFS, clusterName, stateModelRef); if (!_zkClient.exists(stateModelDefPath)) { @@ -619,18 +649,15 @@ public void addResource(String clusterName, String resourceName, int partitions, throw new HelixException("cluster " + clusterName + " is not setup yet"); } - RebalanceMode mode = RebalanceMode.SEMI_AUTO; - try { - mode = RebalanceMode.valueOf(rebalancerMode); - } catch (Exception e) { - logger.error("", e); - } IdealState idealState = new IdealState(resourceName); idealState.setNumPartitions(partitions); - idealState.setStateModelDefRef(stateModelRef); + idealState.setStateModelDefId(StateModelDefId.from(stateModelRef)); + RebalanceMode mode = + idealState.rebalanceModeFromString(rebalancerMode, RebalanceMode.SEMI_AUTO); idealState.setRebalanceMode(mode); idealState.setReplicas("" + 0); - idealState.setStateModelFactoryName(HelixConstants.DEFAULT_STATE_MODEL_FACTORY); + idealState.setStateModelFactoryId(StateModelFactoryId + .from(HelixConstants.DEFAULT_STATE_MODEL_FACTORY)); if (maxPartitionsPerInstance > 0 && maxPartitionsPerInstance < Integer.MAX_VALUE) { idealState.setMaxPartitionsPerInstance(maxPartitionsPerInstance); } @@ -726,79 +753,6 @@ public StateModelDefinition getStateModelDef(String clusterName, String stateMod return accessor.getProperty(keyBuilder.stateModelDef(stateModelName)); } - @Override - public void addStat(String clusterName, final String statName) { - if (!ZKUtil.isClusterSetup(clusterName, _zkClient)) { - throw new HelixException("cluster " + clusterName + " is not setup yet"); - } - - String persistentStatsPath = - PropertyPathConfig.getPath(PropertyType.PERSISTENTSTATS, clusterName); - ZkBaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_zkClient); - - baseAccessor.update(persistentStatsPath, new DataUpdater() { - - @Override - public ZNRecord update(ZNRecord statsRec) { - if (statsRec == null) { - // TODO: fix naming of this record, if it matters - statsRec = new ZNRecord(PersistentStats.nodeName); - } - - Map> currStatMap = statsRec.getMapFields(); - Map> newStatMap = StatsHolder.parseStat(statName); - for (String newStat : newStatMap.keySet()) { - if (!currStatMap.containsKey(newStat)) { - currStatMap.put(newStat, newStatMap.get(newStat)); - } - } - statsRec.setMapFields(currStatMap); - - return statsRec; - } - }, AccessOption.PERSISTENT); - } - - @Override - public void addAlert(final String clusterName, final String alertName) { - if (!ZKUtil.isClusterSetup(clusterName, _zkClient)) { - throw new HelixException("cluster " + clusterName + " is not setup yet"); - } - - ZkBaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_zkClient); - - String alertsPath = PropertyPathConfig.getPath(PropertyType.ALERTS, clusterName); - - baseAccessor.update(alertsPath, new DataUpdater() { - - @Override - public ZNRecord update(ZNRecord alertsRec) { - if (alertsRec == null) { - // TODO: fix naming of this record, if it matters - alertsRec = new ZNRecord(Alerts.nodeName); - - } - - Map> currAlertMap = alertsRec.getMapFields(); - StringBuilder newStatName = new StringBuilder(); - Map newAlertMap = new HashMap(); - - // use AlertsHolder to get map of new stats and map for this alert - AlertsHolder.parseAlert(alertName, newStatName, newAlertMap); - - // add stat - addStat(clusterName, newStatName.toString()); - - // add alert - currAlertMap.put(alertName, newAlertMap); - - alertsRec.setMapFields(currAlertMap); - - return alertsRec; - } - }, AccessOption.PERSISTENT); - } - @Override public void dropCluster(String clusterName) { logger.info("Deleting cluster " + clusterName); @@ -819,70 +773,6 @@ public void dropCluster(String clusterName) { _zkClient.deleteRecursive(root); } - @Override - public void dropStat(String clusterName, final String statName) { - if (!ZKUtil.isClusterSetup(clusterName, _zkClient)) { - throw new HelixException("cluster " + clusterName + " is not setup yet"); - } - - String persistentStatsPath = - PropertyPathConfig.getPath(PropertyType.PERSISTENTSTATS, clusterName); - ZkBaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_zkClient); - - baseAccessor.update(persistentStatsPath, new DataUpdater() { - - @Override - public ZNRecord update(ZNRecord statsRec) { - if (statsRec == null) { - throw new HelixException("No stats record in ZK, nothing to drop"); - } - - Map> currStatMap = statsRec.getMapFields(); - Map> newStatMap = StatsHolder.parseStat(statName); - - // delete each stat from stat map - for (String newStat : newStatMap.keySet()) { - if (currStatMap.containsKey(newStat)) { - currStatMap.remove(newStat); - } - } - statsRec.setMapFields(currStatMap); - - return statsRec; - } - }, AccessOption.PERSISTENT); - } - - @Override - public void dropAlert(String clusterName, final String alertName) { - if (!ZKUtil.isClusterSetup(clusterName, _zkClient)) { - throw new HelixException("cluster " + clusterName + " is not setup yet"); - } - - String alertsPath = PropertyPathConfig.getPath(PropertyType.ALERTS, clusterName); - - ZkBaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_zkClient); - - if (!baseAccessor.exists(alertsPath, 0)) { - throw new HelixException("No alerts node in ZK, nothing to drop"); - } - - baseAccessor.update(alertsPath, new DataUpdater() { - @Override - public ZNRecord update(ZNRecord alertsRec) { - if (alertsRec == null) { - throw new HelixException("No alerts record in ZK, nothing to drop"); - } - - Map> currAlertMap = alertsRec.getMapFields(); - currAlertMap.remove(alertName); - alertsRec.setMapFields(currAlertMap); - - return alertsRec; - } - }, AccessOption.PERSISTENT); - } - @Override public void addClusterToGrandCluster(String clusterName, String grandCluster) { if (!ZKUtil.isClusterSetup(grandCluster, _zkClient)) { @@ -896,7 +786,7 @@ public void addClusterToGrandCluster(String clusterName, String grandCluster) { IdealState idealState = new IdealState(clusterName); idealState.setNumPartitions(1); - idealState.setStateModelDefRef("LeaderStandby"); + idealState.setStateModelDefId(StateModelDefId.from("LeaderStandby")); List controllers = getInstancesInCluster(grandCluster); if (controllers.size() == 0) { @@ -914,7 +804,8 @@ public void addClusterToGrandCluster(String clusterName, String grandCluster) { new ZKHelixDataAccessor(grandCluster, new ZkBaseDataAccessor(_zkClient)); Builder keyBuilder = accessor.keyBuilder(); - accessor.setProperty(keyBuilder.idealStates(idealState.getResourceName()), idealState); + accessor + .setProperty(keyBuilder.idealStates(idealState.getResourceId().stringify()), idealState); } @Override @@ -982,7 +873,7 @@ void rebalance(String clusterName, String resourceName, int replica, String keyP } idealState.setReplicas(Integer.toString(replica)); int partitions = idealState.getNumPartitions(); - String stateModelName = idealState.getStateModelDefRef(); + String stateModelName = idealState.getStateModelDefId().stringify(); StateModelDefinition stateModDef = getStateModelDef(clusterName, stateModelName); if (stateModDef == null) { @@ -1023,14 +914,14 @@ void rebalance(String clusterName, String resourceName, int replica, String keyP if (masterStateValue == null) { masterStateValue = slaveStateValue; } - if (idealState.getRebalanceMode() != RebalanceMode.FULL_AUTO) { + if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO + || idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED) { ZNRecord newIdealState = - DefaultIdealStateCalculator.calculateIdealState(instanceNames, partitions, replica, + DefaultTwoStateStrategy.calculateIdealState(instanceNames, partitions, replica, keyPrefix, masterStateValue, slaveStateValue); // for now keep mapField in SEMI_AUTO mode and remove listField in CUSTOMIZED mode - if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO - || idealState.getRebalanceMode() == RebalanceMode.USER_DEFINED) { + if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO) { idealState.getRecord().setListFields(newIdealState.getListFields()); idealState.getRecord().setMapFields(newIdealState.getMapFields()); } @@ -1099,7 +990,7 @@ public ZNRecord update(ZNRecord currentData) { currentData == null ? new ClusterConstraints(constraintType) : new ClusterConstraints( currentData); - constraints.addConstraintItem(constraintId, constraintItem); + constraints.addConstraintItem(ConstraintId.from(constraintId), constraintItem); return constraints.getRecord(); } }, AccessOption.PERSISTENT); @@ -1119,7 +1010,7 @@ public ZNRecord update(ZNRecord currentData) { if (currentData != null) { ClusterConstraints constraints = new ClusterConstraints(currentData); - constraints.removeConstraintItem(constraintId); + constraints.removeConstraintItem(ConstraintId.from(constraintId)); return constraints.getRecord(); } return null; @@ -1148,27 +1039,27 @@ public ClusterConstraints getConstraints(String clusterName, ConstraintType cons @Override public void rebalance(String clusterName, IdealState currentIdealState, List instanceNames) { Set activeInstances = new HashSet(); - for (String partition : currentIdealState.getPartitionSet()) { - activeInstances.addAll(currentIdealState.getRecord().getListField(partition)); + for (PartitionId partition : currentIdealState.getPartitionIdSet()) { + activeInstances.addAll(IdealState.stringListFromPreferenceList(currentIdealState + .getPreferenceList(partition))); } instanceNames.removeAll(activeInstances); Map previousIdealState = RebalanceUtil.buildInternalIdealState(currentIdealState); Map balancedRecord = - DefaultIdealStateCalculator.calculateNextIdealState(instanceNames, previousIdealState); + DefaultTwoStateStrategy.calculateNextIdealState(instanceNames, previousIdealState); StateModelDefinition stateModDef = - this.getStateModelDef(clusterName, currentIdealState.getStateModelDefRef()); + this.getStateModelDef(clusterName, currentIdealState.getStateModelDefId().stringify()); if (stateModDef == null) { - throw new HelixException("cannot find state model: " - + currentIdealState.getStateModelDefRef()); + throw new HelixException("cannot find state model: " + currentIdealState.getStateModelDefId()); } String[] states = RebalanceUtil.parseStates(clusterName, stateModDef); ZNRecord newIdealStateRecord = - DefaultIdealStateCalculator.convertToZNRecord(balancedRecord, - currentIdealState.getResourceName(), states[0], states[1]); + DefaultTwoStateStrategy.convertToZNRecord(balancedRecord, currentIdealState.getResourceId() + .stringify(), states[0], states[1]); Set partitionSet = new HashSet(); partitionSet.addAll(newIdealStateRecord.getMapFields().keySet()); partitionSet.addAll(newIdealStateRecord.getListFields().keySet()); @@ -1234,10 +1125,79 @@ public void removeInstanceTag(String clusterName, String instanceName, String ta accessor.setProperty(keyBuilder.instanceConfig(instanceName), config); } + @Override public void close() { if (_zkClient != null) { _zkClient.close(); } } + @Override + public void swapInstance(String clusterName, String oldInstanceName, String newInstanceName) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); + Builder keyBuilder = accessor.keyBuilder(); + + InstanceConfig oldConfig = accessor.getProperty(keyBuilder.instanceConfig(oldInstanceName)); + if (oldConfig == null) { + String error = "Old instance " + oldInstanceName + " does not exist, cannot swap"; + LOG.warn(error); + throw new HelixException(error); + } + + InstanceConfig newConfig = accessor.getProperty(keyBuilder.instanceConfig(newInstanceName)); + if (newConfig == null) { + String error = "New instance " + newInstanceName + " does not exist, cannot swap"; + LOG.warn(error); + throw new HelixException(error); + } + + // ensure old instance is disabled, otherwise fail + if (oldConfig.getInstanceEnabled()) { + String error = + "Old instance " + oldInstanceName + " is enabled, it need to be disabled and turned off"; + LOG.warn(error); + throw new HelixException(error); + } + // ensure old instance is down, otherwise fail + List liveInstanceNames = accessor.getChildNames(accessor.keyBuilder().liveInstances()); + + if (liveInstanceNames.contains(oldInstanceName)) { + String error = + "Old instance " + oldInstanceName + " is still on, it need to be disabled and turned off"; + LOG.warn(error); + throw new HelixException(error); + } + + dropInstance(clusterName, oldConfig); + + List existingIdealStates = + accessor.getChildValues(accessor.keyBuilder().idealStates()); + for (IdealState idealState : existingIdealStates) { + swapInstanceInIdealState(idealState, oldInstanceName, newInstanceName); + accessor.setProperty(accessor.keyBuilder() + .idealStates(idealState.getResourceId().stringify()), idealState); + } + } + + private void swapInstanceInIdealState(IdealState idealState, String oldInstance, + String newInstance) { + for (String partition : idealState.getRecord().getMapFields().keySet()) { + Map valMap = idealState.getRecord().getMapField(partition); + if (valMap.containsKey(oldInstance)) { + valMap.put(newInstance, valMap.get(oldInstance)); + valMap.remove(oldInstance); + } + } + + for (String partition : idealState.getRecord().getListFields().keySet()) { + List valList = idealState.getRecord().getListField(partition); + for (int i = 0; i < valList.size(); i++) { + if (valList.get(i).equals(oldInstance)) { + valList.remove(i); + valList.add(i, newInstance); + } + } + } + } } diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java index 025402dcb8..f93d7c4aed 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java @@ -45,7 +45,7 @@ import org.apache.helix.controller.restlet.ZNRecordUpdate; import org.apache.helix.controller.restlet.ZNRecordUpdate.OpCode; import org.apache.helix.controller.restlet.ZkPropertyTransferClient; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Leader; import org.apache.log4j.Logger; import org.apache.zookeeper.data.Stat; @@ -76,7 +76,18 @@ public boolean createProperty(PropertyKey key, T value PropertyType type = key.getType(); String path = key.getPath(); int options = constructOptions(type); - return _baseDataAccessor.create(path, value.getRecord(), options); + boolean success = false; + switch (type) { + case STATEMODELDEFS: + if (value != null && value.isValid()) { + success = _baseDataAccessor.create(path, value.getRecord(), options); + } + break; + default: + success = _baseDataAccessor.create(path, value == null ? null : value.getRecord(), options); + break; + } + return success; } @Override @@ -431,6 +442,11 @@ public boolean[] setChildren(List keys, L bucketizedRecords.set(i, childBucketizedRecords); } break; + case STATEMODELDEFS: + if (value.isValid()) { + records.add(value.getRecord()); + } + break; default: records.add(value.getRecord()); break; @@ -453,7 +469,6 @@ public boolean[] setChildren(List keys, L // TODO: set success accordingly _baseDataAccessor.setChildren(allBucketizedPaths, allBucketizedRecords, options); - return success; } @@ -489,7 +504,7 @@ public void onControllerChange(NotificationContext changeContext) { void refreshZkPropertyTransferUrl() { try { - LiveInstance leader = getProperty(keyBuilder().controllerLeader()); + Leader leader = getProperty(keyBuilder().controllerLeader()); if (leader != null) { _zkPropertyTransferSvcUrl = leader.getWebserviceUrl(); LOG.info("_zkPropertyTransferSvcUrl : " + _zkPropertyTransferSvcUrl + " Controller " diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java index 621c18b47e..c89abd2972 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java @@ -19,31 +19,28 @@ * under the License. */ -import java.lang.management.ManagementFactory; import java.net.InetAddress; import java.net.UnknownHostException; import java.util.ArrayList; -import java.util.Arrays; -import java.util.LinkedList; import java.util.List; import java.util.Timer; import java.util.concurrent.TimeUnit; +import org.I0Itec.zkclient.IZkStateListener; import org.I0Itec.zkclient.ZkConnection; import org.apache.helix.BaseDataAccessor; import org.apache.helix.ClusterMessagingService; import org.apache.helix.ConfigAccessor; import org.apache.helix.ConfigChangeListener; -import org.apache.helix.AccessOption; import org.apache.helix.ControllerChangeListener; import org.apache.helix.CurrentStateChangeListener; import org.apache.helix.ExternalViewChangeListener; -import org.apache.helix.HealthStateChangeListener; import org.apache.helix.HelixAdmin; import org.apache.helix.HelixConstants.ChangeType; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixException; import org.apache.helix.HelixManager; +import org.apache.helix.HelixManagerProperties; import org.apache.helix.HelixTimerTask; import org.apache.helix.IdealStateChangeListener; import org.apache.helix.InstanceConfigChangeListener; @@ -52,150 +49,207 @@ import org.apache.helix.LiveInstanceInfoProvider; import org.apache.helix.MessageListener; import org.apache.helix.PreConnectCallback; -import org.apache.helix.HelixManagerProperties; import org.apache.helix.PropertyKey; import org.apache.helix.PropertyKey.Builder; import org.apache.helix.PropertyPathConfig; import org.apache.helix.PropertyType; import org.apache.helix.ScopedConfigChangeListener; import org.apache.helix.ZNRecord; -import org.apache.helix.controller.restlet.ZKPropertyTransferServer; -import org.apache.helix.healthcheck.HealthStatsAggregationTask; -import org.apache.helix.healthcheck.HealthStatsAggregator; -import org.apache.helix.healthcheck.ParticipantHealthReportCollector; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.healthcheck.ParticipantHealthReportTask; +import org.apache.helix.controller.GenericHelixController; import org.apache.helix.messaging.DefaultMessagingService; -import org.apache.helix.messaging.handling.MessageHandlerFactory; -import org.apache.helix.model.ConfigScope; -import org.apache.helix.model.CurrentState; -import org.apache.helix.model.HelixConfigScope; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; -import org.apache.helix.model.InstanceConfig; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.Message.MessageType; -import org.apache.helix.model.builder.ConfigScopeBuilder; -import org.apache.helix.model.builder.HelixConfigScopeBuilder; -import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.model.Leader; import org.apache.helix.monitoring.ZKPathDataDumpTask; -import org.apache.helix.participant.DistClusterControllerElection; import org.apache.helix.participant.HelixStateMachineEngine; import org.apache.helix.participant.StateMachineEngine; -import org.apache.helix.participant.statemachine.ScheduledTaskStateModelFactory; import org.apache.helix.store.zk.ZkHelixPropertyStore; import org.apache.log4j.Logger; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.apache.zookeeper.ZooKeeper.States; + +public class ZKHelixManager implements HelixManager, IZkStateListener { + private static Logger LOG = Logger.getLogger(ZKHelixManager.class); + + public static final int FLAPPING_TIME_WINDIOW = 300000; // Default to 300 sec + public static final int MAX_DISCONNECT_THRESHOLD = 5; + public static final String ALLOW_PARTICIPANT_AUTO_JOIN = "allowParticipantAutoJoin"; -public class ZKHelixManager implements HelixManager { - private static Logger logger = Logger.getLogger(ZKHelixManager.class); - private static final int RETRY_LIMIT = 3; - private static final int CONNECTIONTIMEOUT = 60 * 1000; + protected final String _zkAddress; private final String _clusterName; private final String _instanceName; - private final String _zkConnectString; - private static final int DEFAULT_SESSION_TIMEOUT = 30 * 1000; - private ZKHelixDataAccessor _helixAccessor; - private ConfigAccessor _configAccessor; - protected ZkClient _zkClient; - protected final List _handlers = new ArrayList(); - private final ZkStateChangeListener _zkStateChangeListener; private final InstanceType _instanceType; - volatile String _sessionId; - private Timer _timer; - private CallbackHandler _leaderElectionHandler; - private ParticipantHealthReportCollectorImpl _participantHealthCheckInfoCollector; - private ParticipantHealthReportTask _participantHealthReportTask; - private final DefaultMessagingService _messagingService; - private ZKHelixAdmin _managementTool; - private final String _version; + private final int _sessionTimeout; + private final List _preConnectCallbacks; + protected final List _handlers; private final HelixManagerProperties _properties; - private final StateMachineEngine _stateMachEngine; - private int _sessionTimeout; - private ZkHelixPropertyStore _helixPropertyStore; - private final List _controllerTimerTasks; + + /** + * helix version# + */ + private final String _version; + + protected ZkClient _zkclient = null; + private final DefaultMessagingService _messagingService; + private BaseDataAccessor _baseDataAccessor; - List _preConnectCallbacks = new LinkedList(); - ZKPropertyTransferServer _transferServer = null; - int _flappingTimeWindowMs; - int _maxDisconnectThreshold; - public static final int FLAPPING_TIME_WINDIOW = 300000; // Default to 300 sec - public static final int MAX_DISCONNECT_THRESHOLD = 5; - LiveInstanceInfoProvider _liveInstanceInfoProvider = null; - public static final String ALLOW_PARTICIPANT_AUTO_JOIN = "allowParticipantAutoJoin"; + private ZKHelixDataAccessor _dataAccessor; + private final Builder _keyBuilder; + private ConfigAccessor _configAccessor; + private ZkHelixPropertyStore _helixPropertyStore; + protected LiveInstanceInfoProvider _liveInstanceInfoProvider = null; - public ZKHelixManager(String clusterName, String instanceName, InstanceType instanceType, - String zkConnectString) { - logger.info("Create a zk-based cluster manager. clusterName:" + clusterName + ", instanceName:" - + instanceName + ", type:" + instanceType + ", zkSvr:" + zkConnectString); - _flappingTimeWindowMs = FLAPPING_TIME_WINDIOW; - try { - _flappingTimeWindowMs = - Integer.parseInt(System.getProperty("helixmanager.flappingTimeWindow", "" - + FLAPPING_TIME_WINDIOW)); - } catch (NumberFormatException e) { - logger.warn("Exception while parsing helixmanager.flappingTimeWindow: " - + System.getProperty("helixmanager.flappingTimeWindow", "" + FLAPPING_TIME_WINDIOW)); - } - _maxDisconnectThreshold = MAX_DISCONNECT_THRESHOLD; - try { - _maxDisconnectThreshold = - Integer.parseInt(System.getProperty("helixmanager.maxDisconnectThreshold", "" - + MAX_DISCONNECT_THRESHOLD)); - } catch (NumberFormatException e) { - logger.warn("Exception while parsing helixmanager.maxDisconnectThreshold: " - + System - .getProperty("helixmanager.maxDisconnectThreshold", "" + MAX_DISCONNECT_THRESHOLD)); - } - int sessionTimeoutInt = -1; - try { - sessionTimeoutInt = - Integer.parseInt(System.getProperty("zk.session.timeout", "" + DEFAULT_SESSION_TIMEOUT)); - } catch (NumberFormatException e) { - logger.warn("Exception while parsing session timeout: " - + System.getProperty("zk.session.timeout", "" + DEFAULT_SESSION_TIMEOUT)); + private volatile String _sessionId; + + /** + * Keep track of timestamps that zk State has become Disconnected + * If in a _timeWindowLengthMs window zk State has become Disconnected + * for more than_maxDisconnectThreshold times disconnect the zkHelixManager + */ + private final List _disconnectTimeHistory = new ArrayList(); + private final int _flappingTimeWindowMs; + private final int _maxDisconnectThreshold; + + /** + * participant fields + */ + private final StateMachineEngine _stateMachineEngine; + private final List _timerTasks = new ArrayList(); + + /** + * controller fields + */ + private final GenericHelixController _controller = new GenericHelixController(); + private CallbackHandler _leaderElectionHandler = null; + protected final List _controllerTimerTasks = new ArrayList(); + + /** + * status dump timer-task + */ + static class StatusDumpTask extends HelixTimerTask { + Timer _timer = null; + final ZkClient zkclient; + final HelixManager helixController; + + public StatusDumpTask(ZkClient zkclient, HelixManager helixController) { + this.zkclient = zkclient; + this.helixController = helixController; + } + + @Override + public void start() { + long initialDelay = 30 * 60 * 1000; + long period = 120 * 60 * 1000; + int timeThresholdNoChange = 180 * 60 * 1000; + + if (_timer == null) { + LOG.info("Start StatusDumpTask"); + _timer = new Timer("StatusDumpTimerTask", true); + _timer.scheduleAtFixedRate(new ZKPathDataDumpTask(helixController, zkclient, + timeThresholdNoChange), initialDelay, period); + } } - if (sessionTimeoutInt > 0) { - _sessionTimeout = sessionTimeoutInt; - } else { - _sessionTimeout = DEFAULT_SESSION_TIMEOUT; + + @Override + public void stop() { + if (_timer != null) { + LOG.info("Stop StatusDumpTask"); + _timer.cancel(); + _timer = null; + } } + } + + public ZKHelixManager(String clusterName, String instanceName, InstanceType instanceType, + String zkAddress) { + + LOG.info("Create a zk-based cluster manager. zkSvr: " + zkAddress + ", clusterName: " + + clusterName + ", instanceName: " + instanceName + ", type: " + instanceType); + + _zkAddress = zkAddress; + _clusterName = clusterName; + _instanceType = instanceType; + if (instanceName == null) { try { instanceName = InetAddress.getLocalHost().getCanonicalHostName() + "-" + instanceType.toString(); } catch (UnknownHostException e) { // can ignore it - logger.info("Unable to get host name. Will set it to UNKNOWN, mostly ignorable", e); + LOG.info("Unable to get host name. Will set it to UNKNOWN, mostly ignorable", e); instanceName = "UNKNOWN"; } } - _clusterName = clusterName; _instanceName = instanceName; - _instanceType = instanceType; - _zkConnectString = zkConnectString; - _zkStateChangeListener = - new ZkStateChangeListener(this, _flappingTimeWindowMs, _maxDisconnectThreshold); - _timer = null; + _preConnectCallbacks = new ArrayList(); + _handlers = new ArrayList(); + _properties = new HelixManagerProperties("cluster-manager-version.properties"); + _version = _properties.getVersion(); + _keyBuilder = new Builder(clusterName); _messagingService = new DefaultMessagingService(this); - _properties = new HelixManagerProperties("cluster-manager-version.properties"); - _version = _properties.getVersion(); + /** + * use system property if available + */ + _flappingTimeWindowMs = + getSystemPropertyAsInt("helixmanager.flappingTimeWindow", + ZKHelixManager.FLAPPING_TIME_WINDIOW); + + _maxDisconnectThreshold = + getSystemPropertyAsInt("helixmanager.maxDisconnectThreshold", + ZKHelixManager.MAX_DISCONNECT_THRESHOLD); - _stateMachEngine = new HelixStateMachineEngine(this); + _sessionTimeout = + getSystemPropertyAsInt("zk.session.timeout", ZkClient.DEFAULT_SESSION_TIMEOUT); + + /** + * instance type specific init + */ + switch (instanceType) { + case PARTICIPANT: + _stateMachineEngine = new HelixStateMachineEngine(this); + + break; + case CONTROLLER: + _stateMachineEngine = null; + _controllerTimerTasks.add(new StatusDumpTask(_zkclient, this)); - // add all timer tasks - _controllerTimerTasks = new ArrayList(); - if (_instanceType == InstanceType.CONTROLLER) { - _controllerTimerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(this))); + break; + case CONTROLLER_PARTICIPANT: + _stateMachineEngine = new HelixStateMachineEngine(this); + _controllerTimerTasks.add(new StatusDumpTask(_zkclient, this)); + + break; + case ADMINISTRATOR: + case SPECTATOR: + _stateMachineEngine = null; + break; + default: + throw new IllegalArgumentException("unrecognized type: " + instanceType); } } + private int getSystemPropertyAsInt(String propertyKey, int propertyDefaultValue) { + String valueString = System.getProperty(propertyKey, "" + propertyDefaultValue); + + try { + int value = Integer.parseInt(valueString); + if (value > 0) { + return value; + } + } catch (NumberFormatException e) { + LOG.warn("Exception while parsing property: " + propertyKey + ", string: " + valueString + + ", using default value: " + propertyDefaultValue); + } + + return propertyDefaultValue; + } + @Override public boolean removeListener(PropertyKey key, Object listener) { - logger.info("Removing listener: " + listener + " on path: " + key.getPath() + " from cluster: " + LOG.info("Removing listener: " + listener + " on path: " + key.getPath() + " from cluster: " + _clusterName + " by instance: " + _instanceName); synchronized (this) { @@ -218,7 +272,13 @@ public boolean removeListener(PropertyKey key, Object listener) { return true; } - private void addListener(Object listener, PropertyKey propertyKey, ChangeType changeType, + void checkConnected() { + if (!isConnected()) { + throw new HelixException("HelixManager is not connected. Call HelixManager#connect()"); + } + } + + void addListener(Object listener, PropertyKey propertyKey, ChangeType changeType, EventType[] eventType) { checkConnected(); @@ -229,16 +289,18 @@ private void addListener(Object listener, PropertyKey propertyKey, ChangeType ch // compare property-key path and listener reference if (handler.getPath().equals(propertyKey.getPath()) && handler.getListener().equals(listener)) { - logger.info("Listener: " + listener + " on path: " + propertyKey.getPath() - + " already exists. skip adding it"); + LOG.info("Listener: " + listener + " on path: " + propertyKey.getPath() + + " already exists. skip add"); + return; } } CallbackHandler newHandler = - createCallBackHandler(propertyKey, listener, eventType, changeType); + new CallbackHandler(this, _zkclient, propertyKey, listener, eventType, changeType); + _handlers.add(newHandler); - logger.info("Add listener: " + listener + " for type: " + type + " to path: " + LOG.info("Added listener: " + listener + " for type: " + type + " to path: " + newHandler.getPath()); } } @@ -261,7 +323,7 @@ public void addLiveInstanceChangeListener(LiveInstanceChangeListener listener) t } @Override - public void addConfigChangeListener(ConfigChangeListener listener) { + public void addConfigChangeListener(ConfigChangeListener listener) throws Exception { addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG, new EventType[] { EventType.NodeChildrenChanged @@ -269,7 +331,8 @@ public void addConfigChangeListener(ConfigChangeListener listener) { } @Override - public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener) { + public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener) + throws Exception { addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG, new EventType[] { EventType.NodeChildrenChanged @@ -277,7 +340,8 @@ public void addInstanceConfigChangeListener(InstanceConfigChangeListener listene } @Override - public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope) { + public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope) + throws Exception { Builder keyBuilder = new Builder(_clusterName); PropertyKey propertyKey = null; @@ -300,7 +364,7 @@ public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigS EventType.NodeChildrenChanged }); } else { - logger.error("Can't add listener to config scope: " + scope); + LOG.error("Can't add listener to config scope: " + scope); } } @@ -314,7 +378,8 @@ public void addMessageListener(MessageListener listener, String instanceName) { }); } - void addControllerMessageListener(MessageListener listener) { + @Override + public void addControllerMessageListener(MessageListener listener) { addListener(listener, new Builder(_clusterName).controllerMessages(), ChangeType.MESSAGES_CONTROLLER, new EventType[] { EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated @@ -323,7 +388,7 @@ void addControllerMessageListener(MessageListener listener) { @Override public void addCurrentStateChangeListener(CurrentStateChangeListener listener, - String instanceName, String sessionId) { + String instanceName, String sessionId) throws Exception { addListener(listener, new Builder(_clusterName).currentStates(instanceName, sessionId), ChangeType.CURRENT_STATE, new EventType[] { EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated @@ -331,15 +396,7 @@ public void addCurrentStateChangeListener(CurrentStateChangeListener listener, } @Override - public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName) { - addListener(listener, new Builder(_clusterName).healthReports(instanceName), ChangeType.HEALTH, - new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }); - } - - @Override - public void addExternalViewChangeListener(ExternalViewChangeListener listener) { + public void addExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception { addListener(listener, new Builder(_clusterName).externalViews(), ChangeType.EXTERNAL_VIEW, new EventType[] { EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated @@ -357,7 +414,7 @@ public void addControllerListener(ControllerChangeListener listener) { @Override public HelixDataAccessor getHelixDataAccessor() { checkConnected(); - return _helixAccessor; + return _dataAccessor; } @Override @@ -376,484 +433,158 @@ public String getInstanceName() { return _instanceName; } - @Override - public void connect() throws Exception { - logger.info("ClusterManager.connect()"); - if (_zkStateChangeListener.isConnected()) { - logger.warn("Cluster manager " + _clusterName + " " + _instanceName + " already connected"); - return; - } - - try { - createClient(_zkConnectString); - _messagingService.onConnected(); - } catch (Exception e) { - logger.error(e); - disconnect(); - throw e; - } - } - - @Override - public void disconnect() { - if (!isConnected()) { - logger.error("ClusterManager " + _instanceName + " already disconnected"); - return; - } - disconnectInternal(); - } - - void disconnectInternal() { - // This function can be called when the connection are in bad state(e.g. flapping), - // in which isConnected() could be false and we want to disconnect from cluster. - logger.info("disconnect " + _instanceName + "(" + _instanceType + ") from " + _clusterName); - - /** - * shutdown thread pool first to avoid reset() being invoked in the middle of state - * transition - */ - _messagingService.getExecutor().shutdown(); - resetHandlers(); - - _helixAccessor.shutdown(); - - if (_leaderElectionHandler != null) { - _leaderElectionHandler.reset(); - } - - if (_participantHealthCheckInfoCollector != null) { - _participantHealthReportTask.stop(); - } - - if (_timer != null) { - _timer.cancel(); - _timer = null; - } - - if (_instanceType == InstanceType.CONTROLLER) { - stopTimerTasks(); - } - - // unsubscribe accessor from controllerChange - _zkClient.unsubscribeAll(); - - _zkClient.close(); - - // HACK seems that zkClient is not sending DISCONNECT event - _zkStateChangeListener.disconnect(); - logger.info("Cluster manager: " + _instanceName + " disconnected"); - - } - - @Override - public String getSessionId() { - checkConnected(); - return _sessionId; - } - - @Override - public boolean isConnected() { - return _zkStateChangeListener.isConnected(); - } - - @Override - public long getLastNotificationTime() { - return -1; - } - - private void addLiveInstance() { - LiveInstance liveInstance = new LiveInstance(_instanceName); - liveInstance.setSessionId(_sessionId); - liveInstance.setHelixVersion(_version); - liveInstance.setLiveInstance(ManagementFactory.getRuntimeMXBean().getName()); - - if (_liveInstanceInfoProvider != null) { - logger.info("invoking _liveInstanceInfoProvider"); - ZNRecord additionalLiveInstanceInfo = - _liveInstanceInfoProvider.getAdditionalLiveInstanceInfo(); - if (additionalLiveInstanceInfo != null) { - additionalLiveInstanceInfo.merge(liveInstance.getRecord()); - ZNRecord mergedLiveInstance = new ZNRecord(additionalLiveInstanceInfo, _instanceName); - liveInstance = new LiveInstance(mergedLiveInstance); - logger.info("liveInstance content :" + _instanceName + " " + liveInstance.toString()); - } - } - - logger.info("Add live instance: InstanceName: " + _instanceName + " Session id:" + _sessionId); - Builder keyBuilder = _helixAccessor.keyBuilder(); - if (!_helixAccessor.createProperty(keyBuilder.liveInstance(_instanceName), liveInstance)) { - String errorMsg = - "Fail to create live instance node after waiting, so quit. instance:" + _instanceName; - logger.warn(errorMsg); - throw new HelixException(errorMsg); - - } - String currentStatePathParent = - PropertyPathConfig.getPath(PropertyType.CURRENTSTATES, _clusterName, _instanceName, - getSessionId()); - - if (!_zkClient.exists(currentStatePathParent)) { - _zkClient.createPersistent(currentStatePathParent); - logger.info("Creating current state path " + currentStatePathParent); - } - } - - private void startStatusUpdatedumpTask() { - long initialDelay = 30 * 60 * 1000; - long period = 120 * 60 * 1000; - int timeThresholdNoChange = 180 * 60 * 1000; + BaseDataAccessor createBaseDataAccessor() { + ZkBaseDataAccessor baseDataAccessor = new ZkBaseDataAccessor(_zkclient); - if (_timer == null) { - _timer = new Timer(true); - _timer.scheduleAtFixedRate(new ZKPathDataDumpTask(this, _zkClient, timeThresholdNoChange), - initialDelay, period); - } + return baseDataAccessor; } - private void createClient(String zkServers) throws Exception { - // by default use ZNRecordStreamingSerializer except for paths within the property - // store which expects raw byte[] serialization/deserialization + void createClient() throws Exception { PathBasedZkSerializer zkSerializer = ChainedPathZkSerializer.builder(new ZNRecordStreamingSerializer()).build(); - _zkClient = new ZkClient(zkServers, _sessionTimeout, CONNECTIONTIMEOUT, zkSerializer); - - ZkBaseDataAccessor baseDataAccessor = new ZkBaseDataAccessor(_zkClient); - if (_instanceType == InstanceType.PARTICIPANT) { - String curStatePath = - PropertyPathConfig.getPath(PropertyType.CURRENTSTATES, _clusterName, _instanceName); - _baseDataAccessor = - new ZkCacheBaseDataAccessor(baseDataAccessor, Arrays.asList(curStatePath)); - } else if (_instanceType == InstanceType.CONTROLLER) { - String extViewPath = PropertyPathConfig.getPath(PropertyType.EXTERNALVIEW, + _zkclient = + new ZkClient(_zkAddress, _sessionTimeout, ZkClient.DEFAULT_CONNECTION_TIMEOUT, zkSerializer); - _clusterName); - _baseDataAccessor = - new ZkCacheBaseDataAccessor(baseDataAccessor, Arrays.asList(extViewPath)); + _baseDataAccessor = createBaseDataAccessor(); - } else { - _baseDataAccessor = baseDataAccessor; - } + _dataAccessor = new ZKHelixDataAccessor(_clusterName, _instanceType, _baseDataAccessor); + _configAccessor = new ConfigAccessor(_zkclient); - _helixAccessor = new ZKHelixDataAccessor(_clusterName, _instanceType, _baseDataAccessor); - _configAccessor = new ConfigAccessor(_zkClient); int retryCount = 0; - _zkClient.subscribeStateChanges(_zkStateChangeListener); - while (retryCount < RETRY_LIMIT) { + _zkclient.subscribeStateChanges(this); + while (retryCount < 3) { try { - _zkClient.waitUntilConnected(_sessionTimeout, TimeUnit.MILLISECONDS); - _zkStateChangeListener.handleStateChanged(KeeperState.SyncConnected); - _zkStateChangeListener.handleNewSession(); + _zkclient.waitUntilConnected(_sessionTimeout, TimeUnit.MILLISECONDS); + handleStateChanged(KeeperState.SyncConnected); + handleNewSession(); break; } catch (HelixException e) { - logger.error("fail to createClient.", e); + LOG.error("fail to createClient.", e); throw e; } catch (Exception e) { retryCount++; - logger.error("fail to createClient. retry " + retryCount, e); - if (retryCount == RETRY_LIMIT) { + LOG.error("fail to createClient. retry " + retryCount, e); + if (retryCount == 3) { throw e; } } } } - private CallbackHandler createCallBackHandler(PropertyKey propertyKey, Object listener, - EventType[] eventTypes, ChangeType changeType) { - if (listener == null) { - throw new HelixException("Listener cannot be null"); + @Override + public void connect() throws Exception { + LOG.info("ClusterManager.connect()"); + if (isConnected()) { + LOG.warn("Cluster manager: " + _instanceName + " for cluster: " + _clusterName + + " already connected. skip connect"); + return; } - return new CallbackHandler(this, _zkClient, propertyKey, listener, eventTypes, changeType); - } - - /** - * This will be invoked when ever a new session is created
- * case 1: the cluster manager was a participant carry over current state, add live - * instance, and invoke message listener; case 2: the cluster manager was controller and - * was a leader before do leader election, and if it becomes leader again, invoke ideal - * state listener, current state listener, etc. if it fails to become leader in the new - * session, then becomes standby; case 3: the cluster manager was controller and was NOT - * a leader before do leader election, and if it becomes leader, instantiate and invoke - * ideal state listener, current state listener, etc. if if fails to become leader in - * the new session, stay as standby - */ - protected void handleNewSession() { - boolean isConnected = _zkClient.waitUntilConnected(CONNECTIONTIMEOUT, TimeUnit.MILLISECONDS); - while (!isConnected) { - logger.error("Could NOT connect to zk server in " + CONNECTIONTIMEOUT + "ms. zkServer: " - + _zkConnectString + ", expiredSessionId: " + _sessionId + ", clusterName: " - + _clusterName); - isConnected = _zkClient.waitUntilConnected(CONNECTIONTIMEOUT, TimeUnit.MILLISECONDS); + try { + createClient(); + _messagingService.onConnected(); + } catch (Exception e) { + LOG.error("fail to connect " + _instanceName, e); + disconnect(); + throw e; } + } - ZkConnection zkConnection = ((ZkConnection) _zkClient.getConnection()); - - synchronized (this) { - _sessionId = Long.toHexString(zkConnection.getZookeeper().getSessionId()); + @Override + public void disconnect() { + if (_zkclient == null) { + LOG.info("instanceName: " + _instanceName + " already disconnected"); + return; } - _baseDataAccessor.reset(); - - // reset all handlers so they have a chance to unsubscribe zk changes from zkclient - // abandon all callback-handlers added in expired session - resetHandlers(); - logger.info("Handling new session, session id:" + _sessionId + ", instance:" + _instanceName - + ", instanceTye: " + _instanceType + ", cluster: " + _clusterName); + LOG.info("disconnect " + _instanceName + "(" + _instanceType + ") from " + _clusterName); - logger.info(zkConnection.getZookeeper()); - - if (!ZKUtil.isClusterSetup(_clusterName, _zkClient)) { - throw new HelixException("Initial cluster structure is not set up for cluster:" - + _clusterName); - } - // Read cluster config and see if instance can auto join the cluster - boolean autoJoin = false; try { - HelixConfigScope scope = - new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(getClusterName()) - .build(); - autoJoin = Boolean.parseBoolean(getConfigAccessor().get(scope, ALLOW_PARTICIPANT_AUTO_JOIN)); - logger.info("Auto joining " + _clusterName + " is true"); - } catch (Exception e) { - } - if (!ZKUtil.isInstanceSetup(_zkClient, _clusterName, _instanceName, _instanceType)) { - if (!autoJoin) { - throw new HelixException("Initial cluster structure is not set up for instance:" - + _instanceName + " instanceType:" + _instanceType); - } else { - logger.info("Auto joining instance " + _instanceName); - InstanceConfig instanceConfig = new InstanceConfig(_instanceName); - String hostName = _instanceName; - String port = ""; - int lastPos = _instanceName.lastIndexOf("_"); - if (lastPos > 0) { - hostName = _instanceName.substring(0, lastPos); - port = _instanceName.substring(lastPos + 1); - } - instanceConfig.setHostName(hostName); - instanceConfig.setPort(port); - instanceConfig.setInstanceEnabled(true); - getClusterManagmentTool().addInstance(_clusterName, instanceConfig); - } - } + /** + * stop all timer tasks + */ + stopTimerTasks(); - if (_instanceType == InstanceType.PARTICIPANT - || _instanceType == InstanceType.CONTROLLER_PARTICIPANT) { - handleNewSessionAsParticipant(); - } + /** + * shutdown thread pool first to avoid reset() being invoked in the middle of state + * transition + */ + _messagingService.getExecutor().shutdown(); + + // TODO reset user defined handlers only + resetHandlers(); - if (_instanceType == InstanceType.CONTROLLER - || _instanceType == InstanceType.CONTROLLER_PARTICIPANT) { - addControllerMessageListener(_messagingService.getExecutor()); - MessageHandlerFactory defaultControllerMsgHandlerFactory = - new DefaultControllerMessageHandlerFactory(); - _messagingService.getExecutor().registerMessageHandlerFactory( - defaultControllerMsgHandlerFactory.getMessageType(), defaultControllerMsgHandlerFactory); - MessageHandlerFactory defaultSchedulerMsgHandlerFactory = - new DefaultSchedulerMessageHandlerFactory(this); - _messagingService.getExecutor().registerMessageHandlerFactory( - defaultSchedulerMsgHandlerFactory.getMessageType(), defaultSchedulerMsgHandlerFactory); - MessageHandlerFactory defaultParticipantErrorMessageHandlerFactory = - new DefaultParticipantErrorMessageHandlerFactory(this); - _messagingService.getExecutor().registerMessageHandlerFactory( - defaultParticipantErrorMessageHandlerFactory.getMessageType(), - defaultParticipantErrorMessageHandlerFactory); + _dataAccessor.shutdown(); if (_leaderElectionHandler != null) { _leaderElectionHandler.reset(); - _leaderElectionHandler.init(); - } else { - _leaderElectionHandler = - createCallBackHandler(new Builder(_clusterName).controller(), - new DistClusterControllerElection(_zkConnectString), new EventType[] { - EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated - }, ChangeType.CONTROLLER); } - } - if (_instanceType == InstanceType.PARTICIPANT - || _instanceType == InstanceType.CONTROLLER_PARTICIPANT - || (_instanceType == InstanceType.CONTROLLER && isLeader())) { - initHandlers(); + } finally { + _zkclient.close(); + _zkclient = null; + LOG.info("Cluster manager: " + _instanceName + " disconnected"); } } - private void handleNewSessionAsParticipant() { - // In case there is a live instance record on zookeeper - Builder keyBuilder = _helixAccessor.keyBuilder(); - - if (_helixAccessor.getProperty(keyBuilder.liveInstance(_instanceName)) != null) { - logger.warn("Found another instance with same instanceName: " + _instanceName - + " in cluster " + _clusterName); - // Wait for a while, in case previous storage node exits unexpectedly - // and its liveinstance - // still hangs around until session timeout happens - try { - Thread.sleep(_sessionTimeout + 5000); - } catch (InterruptedException e) { - logger.warn("Sleep interrupted while waiting for previous liveinstance to go away.", e); - } + @Override + public String getSessionId() { + checkConnected(); + return _sessionId; + } - if (_helixAccessor.getProperty(keyBuilder.liveInstance(_instanceName)) != null) { - String errorMessage = - "instance " + _instanceName + " already has a liveinstance in cluster " + _clusterName; - logger.error(errorMessage); - throw new HelixException(errorMessage); - } + @Override + public boolean isConnected() { + if (_zkclient == null) { + return false; } - // Invoke the PreConnectCallbacks - for (PreConnectCallback callback : _preConnectCallbacks) { - callback.onPreConnect(); + ZkConnection zkconnection = (ZkConnection) _zkclient.getConnection(); + if (zkconnection != null) { + States state = zkconnection.getZookeeperState(); + return state == States.CONNECTED; } - addLiveInstance(); - carryOverPreviousCurrentState(); - - // In case the cluster manager is running as a participant, setup message - // listener - _messagingService.registerMessageHandlerFactory(MessageType.STATE_TRANSITION.toString(), - _stateMachEngine); - addMessageListener(_messagingService.getExecutor(), _instanceName); - addControllerListener(_helixAccessor); - - ScheduledTaskStateModelFactory stStateModelFactory = - new ScheduledTaskStateModelFactory(_messagingService.getExecutor()); - _stateMachEngine.registerStateModelFactory( - DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, stStateModelFactory); + return false; + } - if (_participantHealthCheckInfoCollector == null) { - _participantHealthCheckInfoCollector = - new ParticipantHealthReportCollectorImpl(this, _instanceName); - _participantHealthReportTask = - new ParticipantHealthReportTask(_participantHealthCheckInfoCollector); - _participantHealthReportTask.start(); - } - // start the participant health check timer, also create zk path for health - // check info - String healthCheckInfoPath = _helixAccessor.keyBuilder().healthReports(_instanceName).getPath(); - if (!_zkClient.exists(healthCheckInfoPath)) { - _zkClient.createPersistent(healthCheckInfoPath, true); - logger.info("Creating healthcheck info path " + healthCheckInfoPath); - } + @Override + public long getLastNotificationTime() { + return 0; } @Override public void addPreConnectCallback(PreConnectCallback callback) { - logger.info("Adding preconnect callback"); + LOG.info("Adding preconnect callback: " + callback); _preConnectCallbacks.add(callback); } - private void resetHandlers() { - synchronized (this) { - if (_handlers != null) { - // get a copy of the list and iterate over the copy list - // in case handler.reset() will modify the original handler list - List tmpHandlers = new ArrayList(); - tmpHandlers.addAll(_handlers); - - for (CallbackHandler handler : tmpHandlers) { - handler.reset(); - logger.info("reset handler: " + handler.getPath() + ", " + handler.getListener()); - } - } - } - } - - private void initHandlers() { - synchronized (this) { - if (_handlers != null) { - // may add new currentState and message listeners during init() - // so make a copy and iterate over the copy - List tmpHandlers = new ArrayList(); - tmpHandlers.addAll(_handlers); - for (CallbackHandler handler : tmpHandlers) { - handler.init(); - logger.info("init handler: " + handler.getPath() + ", " + handler.getListener()); - } - } - } - } - @Override public boolean isLeader() { - if (!isConnected()) { - return false; - } - - if (_instanceType != InstanceType.CONTROLLER) { + if (_instanceType != InstanceType.CONTROLLER + && _instanceType != InstanceType.CONTROLLER_PARTICIPANT) { return false; } - Builder keyBuilder = _helixAccessor.keyBuilder(); - LiveInstance leader = _helixAccessor.getProperty(keyBuilder.controllerLeader()); - if (leader == null) { + if (!isConnected()) { return false; - } else { - String leaderName = leader.getInstanceName(); - // TODO need check sessionId also, but in distributed mode, leader's - // sessionId is - // not equal to - // the leader znode's sessionId field which is the sessionId of the - // controller_participant that - // successfully creates the leader node - if (leaderName == null || !leaderName.equals(_instanceName)) { - return false; - } } - return true; - } - - /** - * carry over current-states from last sessions - * set to initial state for current session only when the state doesn't exist in current session - */ - private void carryOverPreviousCurrentState() { - Builder keyBuilder = _helixAccessor.keyBuilder(); - List sessions = _helixAccessor.getChildNames(keyBuilder.sessions(_instanceName)); - - // carry-over - for (String session : sessions) { - if (session.equals(_sessionId)) { - continue; - } - List lastCurStates = - _helixAccessor.getChildValues(keyBuilder.currentStates(_instanceName, session)); - - for (CurrentState lastCurState : lastCurStates) { - logger.info("Carrying over old session: " + session + ", resource: " + lastCurState.getId() - + " to current session: " + _sessionId); - String stateModelDefRef = lastCurState.getStateModelDefRef(); - if (stateModelDefRef == null) { - logger - .error("skip carry-over because previous current state doesn't have a state model definition. previous current-state: " - + lastCurState); - continue; + try { + Leader leader = _dataAccessor.getProperty(_keyBuilder.controllerLeader()); + if (leader != null) { + String leaderName = leader.getInstanceName(); + String sessionId = leader.getSessionId(); + if (leaderName != null && leaderName.equals(_instanceName) && sessionId != null + && sessionId.equals(_sessionId)) { + return true; } - StateModelDefinition stateModel = - _helixAccessor.getProperty(keyBuilder.stateModelDef(stateModelDefRef)); - - String curStatePath = - keyBuilder.currentState(_instanceName, _sessionId, lastCurState.getResourceName()) - .getPath(); - _helixAccessor.getBaseDataAccessor().update(curStatePath, - new CurStateCarryOverUpdater(_sessionId, stateModel.getInitialState(), lastCurState), - AccessOption.PERSISTENT); } + } catch (Exception e) { + // log } - - // remove previous current states - for (String session : sessions) { - if (session.equals(_sessionId)) { - continue; - } - - String path = _helixAccessor.keyBuilder().currentStates(_instanceName, session).getPath(); - logger.info("Removing current states from previous sessions. path: " + path); - _zkClient.deleteRecursive(path); - } + return false; } @Override @@ -864,7 +595,7 @@ public synchronized ZkHelixPropertyStore getHelixPropertyStore() { String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, _clusterName); _helixPropertyStore = - new ZkHelixPropertyStore(new ZkBaseDataAccessor(_zkClient), path, + new ZkHelixPropertyStore(new ZkBaseDataAccessor(_zkclient), path, null); } @@ -874,13 +605,12 @@ public synchronized ZkHelixPropertyStore getHelixPropertyStore() { @Override public synchronized HelixAdmin getClusterManagmentTool() { checkConnected(); - if (_zkClient != null) { - _managementTool = new ZKHelixAdmin(_zkClient); - } else { - logger.error("Couldn't get ZKClusterManagementTool because zkClient is null"); + if (_zkclient != null) { + return new ZKHelixAdmin(_zkclient); } - return _managementTool; + LOG.error("Couldn't get ZKClusterManagementTool because zkclient is null"); + return null; } @Override @@ -890,23 +620,11 @@ public ClusterMessagingService getMessagingService() { return _messagingService; } - @Override - public ParticipantHealthReportCollector getHealthReportCollector() { - checkConnected(); - return _participantHealthCheckInfoCollector; - } - @Override public InstanceType getInstanceType() { return _instanceType; } - private void checkConnected() { - if (!isConnected()) { - throw new HelixException("ClusterManager not connected. Call clusterManager.connect()"); - } - } - @Override public String getVersion() { return _version; @@ -919,21 +637,20 @@ public HelixManagerProperties getProperties() { @Override public StateMachineEngine getStateMachineEngine() { - return _stateMachEngine; + return _stateMachineEngine; } // TODO: rename this and not expose this function as part of interface @Override public void startTimerTasks() { - for (HelixTimerTask task : _controllerTimerTasks) { + for (HelixTimerTask task : _timerTasks) { task.start(); } - startStatusUpdatedumpTask(); } @Override public void stopTimerTasks() { - for (HelixTimerTask task : _controllerTimerTasks) { + for (HelixTimerTask task : _timerTasks) { task.stop(); } } @@ -942,4 +659,206 @@ public void stopTimerTasks() { public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) { _liveInstanceInfoProvider = liveInstanceInfoProvider; } + + /** + * wait until we get a non-zero session-id. note that we might lose zkconnection + * right after we read session-id. but it's ok to get stale session-id and we will have + * another handle-new-session callback to correct this. + */ + void waitUntilConnected() { + boolean isConnected; + do { + isConnected = + _zkclient.waitUntilConnected(ZkClient.DEFAULT_CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS); + if (!isConnected) { + LOG.error("fail to connect zkserver: " + _zkAddress + " in " + + ZkClient.DEFAULT_CONNECTION_TIMEOUT + "ms. expiredSessionId: " + _sessionId + + ", clusterName: " + _clusterName); + continue; + } + + ZkConnection zkConnection = ((ZkConnection) _zkclient.getConnection()); + _sessionId = Long.toHexString(zkConnection.getZookeeper().getSessionId()); + + /** + * at the time we read session-id, zkconnection might be lost again + * wait until we get a non-zero session-id + */ + } while ("0".equals(_sessionId)); + + LOG.info("Handling new session, session id: " + _sessionId + ", instance: " + _instanceName + + ", instanceTye: " + _instanceType + ", cluster: " + _clusterName + ", zkconnection: " + + ((ZkConnection) _zkclient.getConnection()).getZookeeper()); + } + + void initHandlers(List handlers) { + synchronized (this) { + if (handlers != null) { + for (CallbackHandler handler : handlers) { + handler.init(); + LOG.info("init handler: " + handler.getPath() + ", " + handler.getListener()); + } + } + } + } + + void resetHandlers() { + synchronized (this) { + if (_handlers != null) { + // get a copy of the list and iterate over the copy list + // in case handler.reset() modify the original handler list + List tmpHandlers = new ArrayList(); + tmpHandlers.addAll(_handlers); + + for (CallbackHandler handler : tmpHandlers) { + handler.reset(); + LOG.info("reset handler: " + handler.getPath() + ", " + handler.getListener()); + } + } + } + } + + /** + * If zk state has changed into Disconnected for _maxDisconnectThreshold times during previous + * _timeWindowLengthMs Ms + * time window, we think that there are something wrong going on and disconnect the zkHelixManager + * from zk. + */ + boolean isFlapping() { + if (_disconnectTimeHistory.size() == 0) { + return false; + } + long mostRecentTimestamp = _disconnectTimeHistory.get(_disconnectTimeHistory.size() - 1); + + // Remove disconnect history timestamp that are older than _flappingTimeWindowMs ago + while ((_disconnectTimeHistory.get(0) + _flappingTimeWindowMs) < mostRecentTimestamp) { + _disconnectTimeHistory.remove(0); + } + return _disconnectTimeHistory.size() > _maxDisconnectThreshold; + } + + @Override + public void handleStateChanged(KeeperState state) throws Exception { + switch (state) { + case SyncConnected: + ZkConnection zkConnection = (ZkConnection) _zkclient.getConnection(); + LOG.info("KeeperState: " + state + ", zookeeper:" + zkConnection.getZookeeper()); + break; + case Disconnected: + LOG.info("KeeperState:" + state + ", disconnectedSessionId: " + _sessionId + ", instance: " + + _instanceName + ", type: " + _instanceType); + + /** + * Track the time stamp that the disconnected happens, then check history and see if + * we should disconnect the helix-manager + */ + _disconnectTimeHistory.add(System.currentTimeMillis()); + if (isFlapping()) { + LOG.error("instanceName: " + _instanceName + " is flapping. diconnect it. " + + " maxDisconnectThreshold: " + _maxDisconnectThreshold + " disconnects in " + + _flappingTimeWindowMs + "ms."); + disconnect(); + } + break; + case Expired: + LOG.info("KeeperState:" + state + ", expiredSessionId: " + _sessionId + ", instance: " + + _instanceName + ", type: " + _instanceType); + break; + default: + break; + } + } + + @Override + public void handleNewSession() throws Exception { + waitUntilConnected(); + + /** + * stop all timer tasks, reset all handlers, make sure cleanup completed for previous session + * disconnect if fail to cleanup + */ + stopTimerTasks(); + if (_leaderElectionHandler != null) { + _leaderElectionHandler.reset(); + } + resetHandlers(); + + /** + * clean up write-through cache + */ + _baseDataAccessor.reset(); + + /** + * from here on, we are dealing with new session + */ + if (!ZKUtil.isClusterSetup(_clusterName, _zkclient)) { + throw new HelixException("Cluster structure is not set up for cluster: " + _clusterName); + } + + switch (_instanceType) { + case PARTICIPANT: + handleNewSessionAsParticipant(); + break; + case CONTROLLER: + handleNewSessionAsController(); + break; + case CONTROLLER_PARTICIPANT: + handleNewSessionAsParticipant(); + handleNewSessionAsController(); + break; + case ADMINISTRATOR: + case SPECTATOR: + default: + break; + } + + startTimerTasks(); + + /** + * init handlers + * ok to init message handler and data-accessor twice + * the second init will be skipped (see CallbackHandler) + */ + initHandlers(_handlers); + } + + void handleNewSessionAsParticipant() throws Exception { + /** + * auto-join + */ + ParticipantManagerHelper participantHelper = + new ParticipantManagerHelper(this, _zkclient, _sessionTimeout, _liveInstanceInfoProvider); + participantHelper.joinCluster(); + + /** + * Invoke PreConnectCallbacks + */ + for (PreConnectCallback callback : _preConnectCallbacks) { + callback.onPreConnect(); + } + + participantHelper.createLiveInstance(); + participantHelper.carryOverPreviousCurrentState(); + + /** + * setup message listener + */ + participantHelper.setupMsgHandler(); + + } + + void handleNewSessionAsController() { + // get the leader election process going + if (_leaderElectionHandler != null) { + _leaderElectionHandler.init(); + } else { + _leaderElectionHandler = + new CallbackHandler(this, _zkclient, _keyBuilder.controller(), + new DistributedLeaderElection(this, _controller, _controllerTimerTasks), + new EventType[] { + EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated + }, ChangeType.CONTROLLER); + } + } + } diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordStreamingSerializer.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordStreamingSerializer.java index 23de9fd377..bd37d42340 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordStreamingSerializer.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordStreamingSerializer.java @@ -35,8 +35,9 @@ import org.codehaus.jackson.JsonGenerator; import org.codehaus.jackson.JsonParser; import org.codehaus.jackson.JsonToken; +import org.restlet.engine.util.Base64; -import com.noelios.restlet.util.Base64; +import com.google.common.collect.Maps; public class ZNRecordStreamingSerializer implements ZkSerializer { private static Logger LOG = Logger.getLogger(ZNRecordStreamingSerializer.class); @@ -173,7 +174,13 @@ public Object deserialize(byte[] bytes) throws ZkMarshallingError { } ByteArrayInputStream bais = new ByteArrayInputStream(bytes); + ZNRecord record = null; + String id = null; + Map simpleFields = Maps.newHashMap(); + Map> listFields = Maps.newHashMap(); + Map> mapFields = Maps.newHashMap(); + byte[] rawPayload = null; try { JsonFactory f = new JsonFactory(); @@ -185,24 +192,24 @@ public Object deserialize(byte[] bytes) throws ZkMarshallingError { jp.nextToken(); // move to value, or START_OBJECT/START_ARRAY if ("id".equals(fieldname)) { // contains an object - record = new ZNRecord(jp.getText()); + id = jp.getText(); } else if ("simpleFields".equals(fieldname)) { while (jp.nextToken() != JsonToken.END_OBJECT) { String key = jp.getCurrentName(); jp.nextToken(); // move to value - record.setSimpleField(key, jp.getText()); + simpleFields.put(key, jp.getText()); } } else if ("mapFields".equals(fieldname)) { // user.setVerified(jp.getCurrentToken() == JsonToken.VALUE_TRUE); while (jp.nextToken() != JsonToken.END_OBJECT) { String key = jp.getCurrentName(); - record.setMapField(key, new TreeMap()); + mapFields.put(key, new TreeMap()); jp.nextToken(); // move to value while (jp.nextToken() != JsonToken.END_OBJECT) { String mapKey = jp.getCurrentName(); jp.nextToken(); // move to value - record.getMapField(key).put(mapKey, jp.getText()); + mapFields.get(key).put(mapKey, jp.getText()); } } @@ -210,25 +217,33 @@ record = new ZNRecord(jp.getText()); // user.setUserImage(jp.getBinaryValue()); while (jp.nextToken() != JsonToken.END_OBJECT) { String key = jp.getCurrentName(); - record.setListField(key, new ArrayList()); + listFields.put(key, new ArrayList()); jp.nextToken(); // move to value while (jp.nextToken() != JsonToken.END_ARRAY) { - record.getListField(key).add(jp.getText()); + listFields.get(key).add(jp.getText()); } } } else if ("rawPayload".equals(fieldname)) { - record.setRawPayload(Base64.decode(jp.getText())); + rawPayload = Base64.decode(jp.getText()); } else { throw new IllegalStateException("Unrecognized field '" + fieldname + "'!"); } } jp.close(); // ensure resources get cleaned up timely and properly + + if (id == null) { + throw new IllegalStateException("ZNRecord id field is required!"); + } + record = new ZNRecord(id); + record.setSimpleFields(simpleFields); + record.setListFields(listFields); + record.setMapFields(mapFields); + record.setRawPayload(rawPayload); } catch (Exception e) { LOG.error("Exception during deserialization of bytes: " + new String(bytes), e); } - return record; } diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java index 586586341c..755ca52127 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java @@ -143,8 +143,7 @@ public boolean waitForSuccess() { } } } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Interrupted waiting for success", e); } return true; } diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java index 2ba76a306d..4c7798f2e9 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Iterator; import java.util.List; import org.I0Itec.zkclient.DataUpdater; @@ -34,7 +33,6 @@ import org.I0Itec.zkclient.exception.ZkNodeExistsException; import org.apache.helix.AccessOption; import org.apache.helix.BaseDataAccessor; -import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZkAsyncCallbacks.CreateCallbackHandler; import org.apache.helix.manager.zk.ZkAsyncCallbacks.DeleteCallbackHandler; import org.apache.helix.manager.zk.ZkAsyncCallbacks.ExistsCallbackHandler; @@ -46,34 +44,38 @@ import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.data.Stat; -import org.apache.zookeeper.server.DataTree; public class ZkBaseDataAccessor implements BaseDataAccessor { + /** + * return code for zk operations + */ enum RetCode { OK, NODE_EXISTS, + NONODE, ERROR } /** - * struct holding return information + * structure holding return information */ public class AccessResult { - RetCode _retCode; - List _pathCreated; + final RetCode _retCode; + final List _pathCreated; + + final Stat _stat; - Stat _stat; + final T _resultValue; - /** - * used by update only - */ - T _updatedValue; + public AccessResult(RetCode retCode) { + this(retCode, null, null, null); + } - public AccessResult() { - _retCode = RetCode.ERROR; - _pathCreated = new ArrayList(); - _stat = new Stat(); - _updatedValue = null; + public AccessResult(RetCode retCode, List pathCreated, Stat stat, T resultValue) { + _retCode = retCode; + _pathCreated = pathCreated; + _stat = stat; + _resultValue = resultValue; } } @@ -86,7 +88,7 @@ public ZkBaseDataAccessor(ZkClient zkClient) { } /** - * sync create + * sync create a znode */ @Override public boolean create(String path, T record, int options) { @@ -95,32 +97,40 @@ public boolean create(String path, T record, int options) { } /** - * sync create + * sync create a znode. create parent znodes if necessary + * @param path path to create + * @param record value to create, null for no value + * @param options + * @return */ public AccessResult doCreate(String path, T record, int options) { - AccessResult result = new AccessResult(); + if (path == null) { + throw new NullPointerException("path can't be null"); + } + CreateMode mode = AccessOption.getMode(options); if (mode == null) { - LOG.error("Invalid create mode. options: " + options); - result._retCode = RetCode.ERROR; - return result; + throw new IllegalArgumentException("Invalid create options: " + options); } boolean retry; + List pathCreated = null; do { retry = false; try { _zkClient.create(path, record, mode); - result._pathCreated.add(path); + if (pathCreated == null) { + pathCreated = new ArrayList(); + } + pathCreated.add(path); - result._retCode = RetCode.OK; - return result; + return new AccessResult(RetCode.OK, pathCreated, null, null); } catch (ZkNoNodeException e) { // this will happen if parent node does not exist String parentPath = HelixUtil.getZkParentPath(path); try { AccessResult res = doCreate(parentPath, null, AccessOption.PERSISTENT); - result._pathCreated.addAll(res._pathCreated); + pathCreated = res._pathCreated; RetCode rc = res._retCode; if (rc == RetCode.OK || rc == RetCode.NODE_EXISTS) { // if parent node created/exists, retry @@ -128,26 +138,22 @@ public AccessResult doCreate(String path, T record, int options) { } } catch (Exception e1) { LOG.error("Exception while creating path: " + parentPath, e1); - result._retCode = RetCode.ERROR; - return result; + return new AccessResult(RetCode.ERROR, pathCreated, null, null); } } catch (ZkNodeExistsException e) { LOG.warn("Node already exists. path: " + path); - result._retCode = RetCode.NODE_EXISTS; - return result; + return new AccessResult(RetCode.NODE_EXISTS); } catch (Exception e) { LOG.error("Exception while creating path: " + path, e); - result._retCode = RetCode.ERROR; - return result; + return new AccessResult(RetCode.ERROR); } } while (retry); - result._retCode = RetCode.OK; - return result; + return new AccessResult(RetCode.OK, pathCreated, null, null); } /** - * sync set + * sync set a znode */ @Override public boolean set(String path, T record, int options) { @@ -155,7 +161,7 @@ public boolean set(String path, T record, int options) { } /** - * sync set + * sync set a znode with expect version */ @Override public boolean set(String path, T record, int expectVersion, int options) { @@ -168,35 +174,39 @@ public boolean set(String path, T record, int expectVersion, int options) { } /** - * sync set + * sync set a znode, create parent paths if necessary + * @param path + * @param record + * @param expectVersion + * @param options */ public AccessResult doSet(String path, T record, int expectVersion, int options) { - AccessResult result = new AccessResult(); + if (path == null) { + throw new NullPointerException("path can't be null"); + } CreateMode mode = AccessOption.getMode(options); if (mode == null) { - LOG.error("Invalid set mode. options: " + options); - result._retCode = RetCode.ERROR; - return result; + throw new IllegalArgumentException("Invalid set options: " + options); } + Stat stat = null; + List pathCreated = null; boolean retry; do { retry = false; try { - Stat stat = _zkClient.writeDataGetStat(path, record, expectVersion); - DataTree.copyStat(stat, result._stat); + stat = _zkClient.writeDataGetStat(path, record, expectVersion); } catch (ZkNoNodeException e) { // node not exists, try create if expectedVersion == -1; in this case, stat will not be set if (expectVersion != -1) { LOG.error("Could not create node if expectVersion != -1, was " + expectVersion); - result._retCode = RetCode.ERROR; - return result; + return new AccessResult(RetCode.ERROR); } try { // may create recursively AccessResult res = doCreate(path, record, options); - result._pathCreated.addAll(res._pathCreated); + pathCreated = res._pathCreated; RetCode rc = res._retCode; switch (rc) { case OK: @@ -207,29 +217,25 @@ public AccessResult doSet(String path, T record, int expectVersion, int options) break; default: LOG.error("Fail to set path by creating: " + path); - result._retCode = RetCode.ERROR; - return result; + return new AccessResult(RetCode.ERROR, pathCreated, null, null); } } catch (Exception e1) { LOG.error("Exception while setting path by creating: " + path, e); - result._retCode = RetCode.ERROR; - return result; + return new AccessResult(RetCode.ERROR, pathCreated, null, null); } } catch (ZkBadVersionException e) { throw e; } catch (Exception e) { LOG.error("Exception while setting path: " + path, e); - result._retCode = RetCode.ERROR; - return result; + return new AccessResult(RetCode.ERROR, pathCreated, null, null); } } while (retry); - result._retCode = RetCode.OK; - return result; + return new AccessResult(RetCode.OK, pathCreated, stat, null); } /** - * sync update + * sync update a znode */ @Override public boolean update(String path, DataUpdater updater, int options) { @@ -238,27 +244,32 @@ public boolean update(String path, DataUpdater updater, int options) { } /** - * sync update + * sync update a znode, create parent paths if necessary + * @param path + * @param updater + * @param options */ public AccessResult doUpdate(String path, DataUpdater updater, int options) { - AccessResult result = new AccessResult(); + if (path == null || updater == null) { + throw new NullPointerException("path|updater can't be null"); + } + CreateMode mode = AccessOption.getMode(options); if (mode == null) { - LOG.error("Invalid update mode. options: " + options); - result._retCode = RetCode.ERROR; - return result; + throw new IllegalArgumentException("Invalid update options: " + options); } boolean retry; + Stat setStat = null; T updatedData = null; + List pathCreated = null; do { retry = false; try { Stat readStat = new Stat(); T oldData = (T) _zkClient.readData(path, readStat); T newData = updater.update(oldData); - Stat setStat = _zkClient.writeDataGetStat(path, newData, readStat.getVersion()); - DataTree.copyStat(setStat, result._stat); + setStat = _zkClient.writeDataGetStat(path, newData, readStat.getVersion()); updatedData = newData; } catch (ZkBadVersionException e) { @@ -268,7 +279,7 @@ public AccessResult doUpdate(String path, DataUpdater updater, int options) { try { T newData = updater.update(null); AccessResult res = doCreate(path, newData, options); - result._pathCreated.addAll(res._pathCreated); + pathCreated = res._pathCreated; RetCode rc = res._retCode; switch (rc) { case OK: @@ -279,31 +290,30 @@ public AccessResult doUpdate(String path, DataUpdater updater, int options) { break; default: LOG.error("Fail to update path by creating: " + path); - result._retCode = RetCode.ERROR; - return result; + return new AccessResult(RetCode.ERROR, pathCreated, null, null); } } catch (Exception e1) { LOG.error("Exception while updating path by creating: " + path, e1); - result._retCode = RetCode.ERROR; - return result; + return new AccessResult(RetCode.ERROR, pathCreated, null, null); } } catch (Exception e) { LOG.error("Exception while updating path: " + path, e); - result._retCode = RetCode.ERROR; - return result; + return new AccessResult(RetCode.ERROR, pathCreated, null, null); } } while (retry); - result._retCode = RetCode.OK; - result._updatedValue = updatedData; - return result; + return new AccessResult(RetCode.OK, pathCreated, setStat, updatedData); } /** - * sync get + * sync get a znode */ @Override public T get(String path, Stat stat, int options) { + if (path == null) { + throw new NullPointerException("path can't be null"); + } + T data = null; try { data = (T) _zkClient.readData(path, stat); @@ -316,38 +326,77 @@ public T get(String path, Stat stat, int options) { } /** - * async get + * async get a list of znodes */ @Override public List get(List paths, List stats, int options) { + if (paths == null) { + throw new NullPointerException("paths can't be null"); + } + + if (stats != null && stats.size() > 0) { + throw new IllegalArgumentException( + "stats list is an output parameter and should be empty, but was: " + stats); + } + boolean[] needRead = new boolean[paths.size()]; Arrays.fill(needRead, true); - return get(paths, stats, needRead); + List accessResults = doGet(paths, needRead); + List values = new ArrayList(); + + for (AccessResult accessResult : accessResults) { + values.add(accessResult._resultValue); + if (stats != null) { + stats.add(accessResult._stat); + } + } + + return values; } /** - * async get + * async get a list of znodes */ - List get(List paths, List stats, boolean[] needRead) { - if (paths == null || paths.size() == 0) { - return Collections.emptyList(); + List doGet(List paths, boolean[] needRead) { + if (paths == null || needRead == null) { + throw new NullPointerException("paths|needRead can't be null"); } - // init stats - if (stats != null) { - stats.clear(); - stats.addAll(Collections. nCopies(paths.size(), null)); + final int size = paths.size(); + if (size != needRead.length) { + throw new IllegalArgumentException( + "paths and needRead should of equal size, but paths size: " + size + ", needRead size: " + + needRead.length); } + for (int i = 0; i < size; i++) { + if (!needRead[i]) { + continue; + } + + if (paths.get(i) == null) { + throw new NullPointerException("path[" + i + "] can't be null, but was: " + paths); + } + } + + if (size == 0) { + return Collections.emptyList(); + } + + // init all results to null + List results = + new ArrayList(Collections. nCopies(size, null)); + long startT = System.nanoTime(); try { // issue asyn get requests - GetDataCallbackHandler[] cbList = new GetDataCallbackHandler[paths.size()]; - for (int i = 0; i < paths.size(); i++) { - if (!needRead[i]) + GetDataCallbackHandler[] cbList = new GetDataCallbackHandler[size]; + for (int i = 0; i < size; i++) { + if (!needRead[i]) { continue; + } String path = paths.get(i); cbList[i] = new GetDataCallbackHandler(); @@ -355,38 +404,46 @@ List get(List paths, List stats, boolean[] needRead) { } // wait for completion - for (int i = 0; i < cbList.length; i++) { - if (!needRead[i]) + for (int i = 0; i < size; i++) { + if (!needRead[i]) { continue; + } GetDataCallbackHandler cb = cbList[i]; cb.waitForSuccess(); } // construct return results - List records = new ArrayList(Collections. nCopies(paths.size(), null)); - for (int i = 0; i < paths.size(); i++) { - if (!needRead[i]) + if (!needRead[i]) { continue; + } GetDataCallbackHandler cb = cbList[i]; - if (Code.get(cb.getRc()) == Code.OK) { + switch (Code.get(cb.getRc())) { + case OK: { @SuppressWarnings("unchecked") - T record = (T) _zkClient.deserialize(cb._data, paths.get(i)); - records.set(i, record); - if (stats != null) { - stats.set(i, cb._stat); - } + T value = (T) _zkClient.deserialize(cb._data, paths.get(i)); + results.set(i, new AccessResult(RetCode.OK, null, cb._stat, value)); + break; + } + case NONODE: { + results.set(i, new AccessResult(RetCode.NONODE)); + break; + } + default: { + results.set(i, new AccessResult(RetCode.ERROR)); + break; + } } } - return records; + return results; } finally { long endT = System.nanoTime(); if (LOG.isTraceEnabled()) { - LOG.trace("getData_async, size: " + paths.size() + ", paths: " + paths.get(0) - + ",... time: " + (endT - startT) + " ns"); + LOG.trace("getData_async, size: " + size + ", paths: " + paths + ", time: " + + (endT - startT) + " ns"); } } } @@ -396,6 +453,11 @@ List get(List paths, List stats, boolean[] needRead) { */ @Override public List getChildren(String parentPath, List stats, int options) { + if (stats != null && stats.size() > 0) { + throw new IllegalArgumentException( + "stats list is an output parameter and should be empty, but was: " + stats); + } + try { // prepare child paths List childNames = getChildNames(parentPath, options); @@ -409,25 +471,21 @@ public List getChildren(String parentPath, List stats, int options) { paths.add(path); } - // remove null record - List curStats = new ArrayList(paths.size()); - List records = get(paths, curStats, options); - Iterator recordIter = records.iterator(); - Iterator statIter = curStats.iterator(); - while (statIter.hasNext()) { - recordIter.next(); - if (statIter.next() == null) { - statIter.remove(); - recordIter.remove(); - } - } + boolean[] needRead = new boolean[paths.size()]; + Arrays.fill(needRead, true); - if (stats != null) { - stats.clear(); - stats.addAll(curStats); + List results = doGet(paths, needRead); + List values = new ArrayList(); + for (AccessResult result : results) { + if (result._retCode == RetCode.OK) { + values.add(result._resultValue); + if (stats != null) { + stats.add(result._stat); + } + } } - return records; + return values; } catch (ZkNoNodeException e) { return Collections.emptyList(); } @@ -480,166 +538,219 @@ public boolean remove(String path, int options) { /** * async create. give up on error other than NONODE */ - CreateCallbackHandler[] create(List paths, List records, boolean[] needCreate, - List> pathsCreated, int options) { - if ((records != null && records.size() != paths.size()) || needCreate.length != paths.size() - || (pathsCreated != null && pathsCreated.size() != paths.size())) { + List doCreate(List paths, List records, boolean[] needCreate, int options) { + if (paths == null) { + throw new NullPointerException("paths can't be null"); + } + + for (int i = 0; i < paths.size(); i++) { + if (!needCreate[i]) { + continue; + } + + if (paths.get(i) == null) { + throw new NullPointerException("path[" + i + "] can't be null, but was: " + paths); + } + } + + if (records != null && records.size() != paths.size()) { throw new IllegalArgumentException( - "paths, records, needCreate, and pathsCreated should be of same size"); + "paths and records should be of same size, but paths size: " + paths.size() + + ", records size: " + records.size()); } - CreateCallbackHandler[] cbList = new CreateCallbackHandler[paths.size()]; + if (needCreate == null) { + throw new NullPointerException("needCreate can't be null"); + } + + if (needCreate.length != paths.size()) { + throw new IllegalArgumentException( + "paths and needCreate should be of same size, but paths size: " + paths.size() + + ", needCreate size: " + needCreate.length); + } CreateMode mode = AccessOption.getMode(options); if (mode == null) { - LOG.error("Invalid async set mode. options: " + options); - return cbList; + throw new IllegalArgumentException("Invalid async set options: " + options); } + CreateCallbackHandler[] cbList = new CreateCallbackHandler[paths.size()]; + List> pathsCreated = + new ArrayList>(Collections.> nCopies(paths.size(), null)); + RetCode retCodes[] = new RetCode[paths.size()]; + boolean retry; do { retry = false; for (int i = 0; i < paths.size(); i++) { - if (!needCreate[i]) + if (!needCreate[i]) { continue; + } String path = paths.get(i); - T record = records == null ? null : records.get(i); + T record = (records == null ? null : records.get(i)); cbList[i] = new CreateCallbackHandler(); + _zkClient.asyncCreate(path, record, mode, cbList[i]); } List parentPaths = new ArrayList(Collections. nCopies(paths.size(), null)); - boolean failOnNoNode = false; + boolean failOnNoParentNode = false; for (int i = 0; i < paths.size(); i++) { - if (!needCreate[i]) + if (!needCreate[i]) { continue; + } CreateCallbackHandler cb = cbList[i]; cb.waitForSuccess(); String path = paths.get(i); - if (Code.get(cb.getRc()) == Code.NONODE) { + Code code = Code.get(cb.getRc()); + switch (code) { + case NONODE: { + // we will try create parent nodes String parentPath = HelixUtil.getZkParentPath(path); parentPaths.set(i, parentPath); - failOnNoNode = true; - } else { - // if create succeed or fail on error other than NONODE, - // give up + failOnNoParentNode = true; + break; + } + case NODEEXISTS: { + retCodes[i] = RetCode.NODE_EXISTS; needCreate[i] = false; - - // if succeeds, record what paths we've created - if (Code.get(cb.getRc()) == Code.OK && pathsCreated != null) { - if (pathsCreated.get(i) == null) { - pathsCreated.set(i, new ArrayList()); - } - pathsCreated.get(i).add(path); + break; + } + case OK: { + retCodes[i] = RetCode.OK; + if (pathsCreated.get(i) == null) { + pathsCreated.set(i, new ArrayList()); } + pathsCreated.get(i).add(path); + needCreate[i] = false; + break; + } + default: { + retCodes[i] = RetCode.ERROR; + needCreate[i] = false; + break; + } } } - if (failOnNoNode) { - boolean[] needCreateParent = Arrays.copyOf(needCreate, needCreate.length); - - CreateCallbackHandler[] parentCbList = - create(parentPaths, null, needCreateParent, pathsCreated, AccessOption.PERSISTENT); - for (int i = 0; i < parentCbList.length; i++) { - CreateCallbackHandler parentCb = parentCbList[i]; - if (parentCb == null) + if (failOnNoParentNode) { + List createParentResults = + doCreate(parentPaths, null, Arrays.copyOf(needCreate, needCreate.length), AccessOption.PERSISTENT); + for (int i = 0; i < createParentResults.size(); i++) { + if (!needCreate[i]) { continue; - - Code rc = Code.get(parentCb.getRc()); + } // if parent is created, retry create child - if (rc == Code.OK || rc == Code.NODEEXISTS) { + AccessResult result = createParentResults.get(i); + pathsCreated.set(i, result._pathCreated); + + if (result._retCode == RetCode.OK || result._retCode == RetCode.NODE_EXISTS) { retry = true; - break; + } else { + retCodes[i] = RetCode.ERROR; + needCreate[i] = false; } } } } while (retry); - return cbList; + List results = new ArrayList(); + for (int i = 0; i < paths.size(); i++) { + results.add(new AccessResult(retCodes[i], pathsCreated.get(i), null, null)); + } + return results; } + // TODO: rename to create /** - * async create - * TODO: rename to create + * async create multiple znodes */ @Override public boolean[] createChildren(List paths, List records, int options) { boolean[] success = new boolean[paths.size()]; - CreateMode mode = AccessOption.getMode(options); - if (mode == null) { - LOG.error("Invalid async create mode. options: " + options); - return success; - } - boolean[] needCreate = new boolean[paths.size()]; Arrays.fill(needCreate, true); - List> pathsCreated = - new ArrayList>(Collections.> nCopies(paths.size(), null)); long startT = System.nanoTime(); try { + List results = doCreate(paths, records, needCreate, options); - CreateCallbackHandler[] cbList = create(paths, records, needCreate, pathsCreated, options); - - for (int i = 0; i < cbList.length; i++) { - CreateCallbackHandler cb = cbList[i]; - success[i] = (Code.get(cb.getRc()) == Code.OK); + for (int i = 0; i < paths.size(); i++) { + AccessResult result = results.get(i); + success[i] = (result._retCode == RetCode.OK); } return success; - } finally { long endT = System.nanoTime(); if (LOG.isTraceEnabled()) { - LOG.trace("create_async, size: " + paths.size() + ", paths: " + paths.get(0) - + ",... time: " + (endT - startT) + " ns"); + LOG.trace("create_async, size: " + paths.size() + ", paths: " + paths + ", time: " + + (endT - startT) + " ns"); } } } + // TODO: rename to set /** - * async set - * TODO: rename to set + * async set multiple znodes */ @Override public boolean[] setChildren(List paths, List records, int options) { - return set(paths, records, null, null, options); + List results = doSet(paths, records, options); + boolean[] success = new boolean[paths.size()]; + for (int i = 0; i < paths.size(); i++) { + success[i] = (results.get(i)._retCode == RetCode.OK); + } + + return success; } /** * async set, give up on error other than NoNode */ - boolean[] set(List paths, List records, List> pathsCreated, - List stats, int options) { - if (paths == null || paths.size() == 0) { - return new boolean[0]; + List doSet(List paths, List records, int options) { + if (paths == null) { + throw new NullPointerException("paths can't be null"); } - if ((records != null && records.size() != paths.size()) - || (pathsCreated != null && pathsCreated.size() != paths.size())) { - throw new IllegalArgumentException("paths, records, and pathsCreated should be of same size"); + for (String path : paths) { + if (path == null) { + throw new NullPointerException("path can't be null, but was: " + paths); + } } - boolean[] success = new boolean[paths.size()]; + final int size = paths.size(); + if (records != null && records.size() != size) { + throw new IllegalArgumentException( + "paths and records should be of same size, but paths size: " + size + + ", records size: " + records.size()); + } CreateMode mode = AccessOption.getMode(options); if (mode == null) { - LOG.error("Invalid async set mode. options: " + options); - return success; + throw new IllegalArgumentException("Invalid async set options: " + options); } - List setStats = new ArrayList(Collections. nCopies(paths.size(), null)); - SetDataCallbackHandler[] cbList = new SetDataCallbackHandler[paths.size()]; - CreateCallbackHandler[] createCbList = null; - boolean[] needSet = new boolean[paths.size()]; + if (size == 0) { + return Collections.emptyList(); + } + + Stat[] setStats = new Stat[size]; + RetCode[] retCodes = new RetCode[size]; + List> pathsCreated = + new ArrayList>(Collections.> nCopies(size, null)); + + SetDataCallbackHandler[] setCbList = new SetDataCallbackHandler[size]; + + boolean[] needSet = new boolean[size]; Arrays.fill(needSet, true); long startT = System.nanoTime(); @@ -649,94 +760,96 @@ boolean[] set(List paths, List records, List> pathsCreat do { retry = false; - for (int i = 0; i < paths.size(); i++) { - if (!needSet[i]) + for (int i = 0; i < size; i++) { + if (!needSet[i]) { continue; + } String path = paths.get(i); - T record = records.get(i); - cbList[i] = new SetDataCallbackHandler(); - _zkClient.asyncSetData(path, record, -1, cbList[i]); + T record = (records == null ? null : records.get(i)); + setCbList[i] = new SetDataCallbackHandler(); + _zkClient.asyncSetData(path, record, -1, setCbList[i]); } boolean failOnNoNode = false; - for (int i = 0; i < cbList.length; i++) { - SetDataCallbackHandler cb = cbList[i]; + for (int i = 0; i < size; i++) { + if (!needSet[i]) { + continue; + } + + SetDataCallbackHandler cb = setCbList[i]; cb.waitForSuccess(); Code rc = Code.get(cb.getRc()); switch (rc) { - case OK: - setStats.set(i, cb.getStat()); + case OK: { + setStats[i] = cb.getStat(); + retCodes[i] = RetCode.OK; needSet[i] = false; break; - case NONODE: + } + case NONODE: { // if fail on NoNode, try create the node failOnNoNode = true; break; - default: + } + default: { // if fail on error other than NoNode, give up + retCodes[i] = RetCode.ERROR; needSet[i] = false; break; } + } } // if failOnNoNode, try create if (failOnNoNode) { - boolean[] needCreate = Arrays.copyOf(needSet, needSet.length); - createCbList = create(paths, records, needCreate, pathsCreated, options); - for (int i = 0; i < createCbList.length; i++) { - CreateCallbackHandler createCb = createCbList[i]; - if (createCb == null) { + List createResults = + doCreate(paths, records, Arrays.copyOf(needSet, size), options); + for (int i = 0; i < size; i++) { + if (!needSet[i]) { continue; } - Code rc = Code.get(createCb.getRc()); - switch (rc) { - case OK: - setStats.set(i, ZNode.ZERO_STAT); + AccessResult createResult = createResults.get(i); + RetCode code = createResult._retCode; + pathsCreated.set(i, createResult._pathCreated); + + switch (code) { + case OK: { + setStats[i] = ZNode.ZERO_STAT; + retCodes[i] = RetCode.OK; needSet[i] = false; break; - case NODEEXISTS: + } + case NODE_EXISTS: { retry = true; break; - default: - // if creation fails on error other than NodeExists - // no need to retry set + } + default: { + // creation fails on error other than NodeExists + retCodes[i] = RetCode.ERROR; needSet[i] = false; break; } + } } } } while (retry); // construct return results - for (int i = 0; i < cbList.length; i++) { - SetDataCallbackHandler cb = cbList[i]; - - Code rc = Code.get(cb.getRc()); - if (rc == Code.OK) { - success[i] = true; - } else if (rc == Code.NONODE) { - CreateCallbackHandler createCb = createCbList[i]; - if (Code.get(createCb.getRc()) == Code.OK) { - success[i] = true; - } - } + List results = new ArrayList(); + for (int i = 0; i < size; i++) { + results.add(new AccessResult(retCodes[i], pathsCreated.get(i), setStats[i], null)); } - if (stats != null) { - stats.clear(); - stats.addAll(setStats); - } - - return success; + return results; } finally { long endT = System.nanoTime(); if (LOG.isTraceEnabled()) { - LOG.trace("setData_async, size: " + paths.size() + ", paths: " + paths.get(0) - + ",... time: " + (endT - startT) + " ns"); + LOG.trace("setData_async, size: " + size + ", paths: " + paths + ", time: " + + (endT - startT) + " ns"); } } } @@ -748,43 +861,59 @@ boolean[] set(List paths, List records, List> pathsCreat @Override public boolean[] updateChildren(List paths, List> updaters, int options) { - List updateData = update(paths, updaters, null, null, options); - boolean[] success = new boolean[paths.size()]; // init to false + List results = doUpdate(paths, updaters, options); + boolean[] success = new boolean[paths.size()]; for (int i = 0; i < paths.size(); i++) { - T data = updateData.get(i); - success[i] = (data != null); + success[i] = (results.get(i)._retCode == RetCode.OK); } + return success; } /** - * async update - * return: updatedData on success or null on fail + * async update multiple znodes */ - List update(List paths, List> updaters, - List> pathsCreated, List stats, int options) { - if (paths == null || paths.size() == 0) { - LOG.error("paths is null or empty"); - return Collections.emptyList(); + List doUpdate(List paths, List> updaters, int options) { + if (paths == null || updaters == null) { + throw new NullPointerException("paths|updaters can't be null"); + } + + for (String path : paths) { + if (path == null) { + throw new NullPointerException("path can't be null, but was: " + paths); + } } - if (updaters.size() != paths.size() - || (pathsCreated != null && pathsCreated.size() != paths.size())) { - throw new IllegalArgumentException("paths, updaters, and pathsCreated should be of same size"); + for (DataUpdater updater : updaters) { + if (updater == null) { + throw new NullPointerException("updater can't be null, but was: " + updaters + ", paths: " + + paths); + } } - List setStats = new ArrayList(Collections. nCopies(paths.size(), null)); - List updateData = new ArrayList(Collections. nCopies(paths.size(), null)); + final int size = paths.size(); + if (updaters.size() != size) { + throw new IllegalArgumentException( + "paths and updaters should be of same size, but paths size: " + size + + ", updaters size: " + updaters.size()); + } CreateMode mode = AccessOption.getMode(options); if (mode == null) { - LOG.error("Invalid update mode. options: " + options); - return updateData; + throw new IllegalArgumentException("Invalid update options: " + options); + } + + if (size == 0) { + return Collections.emptyList(); } - SetDataCallbackHandler[] cbList = new SetDataCallbackHandler[paths.size()]; - CreateCallbackHandler[] createCbList = null; - boolean[] needUpdate = new boolean[paths.size()]; + Stat[] updateStats = new Stat[size]; + RetCode[] retCodes = new RetCode[size]; + List> pathsCreated = + new ArrayList>(Collections.> nCopies(size, null)); + List updateData = new ArrayList(Collections. nCopies(size, null)); + + boolean[] needUpdate = new boolean[size]; Arrays.fill(needUpdate, true); long startT = System.nanoTime(); @@ -793,90 +922,102 @@ List update(List paths, List> updaters, boolean retry; do { retry = false; - boolean[] needCreate = new boolean[paths.size()]; // init'ed with false + SetDataCallbackHandler[] setCbList = new SetDataCallbackHandler[size]; + boolean[] needCreate = new boolean[size]; // init'ed with false boolean failOnNoNode = false; // asycn read all data - List curStats = new ArrayList(); - List curDataList = get(paths, curStats, Arrays.copyOf(needUpdate, needUpdate.length)); + List readResults = doGet(paths, Arrays.copyOf(needUpdate, size)); // async update - List newDataList = new ArrayList(); - for (int i = 0; i < paths.size(); i++) { + List newDataList = new ArrayList(Collections. nCopies(size, null)); + for (int i = 0; i < size; i++) { if (!needUpdate[i]) { - newDataList.add(null); continue; } String path = paths.get(i); DataUpdater updater = updaters.get(i); - T newData = updater.update(curDataList.get(i)); - newDataList.add(newData); - Stat curStat = curStats.get(i); - if (curStat == null) { + AccessResult readResult = readResults.get(i); + T newData = updater.update(readResult._resultValue); + newDataList.set(i, newData); + if (readResult._retCode == RetCode.NONODE) { // node not exists failOnNoNode = true; needCreate[i] = true; } else { - cbList[i] = new SetDataCallbackHandler(); - _zkClient.asyncSetData(path, newData, curStat.getVersion(), cbList[i]); + setCbList[i] = new SetDataCallbackHandler(); + _zkClient.asyncSetData(path, newData, readResult._stat.getVersion(), setCbList[i]); } } // wait for completion boolean failOnBadVersion = false; - for (int i = 0; i < paths.size(); i++) { - SetDataCallbackHandler cb = cbList[i]; - if (cb == null) + for (int i = 0; i < size; i++) { + SetDataCallbackHandler cb = setCbList[i]; + if (cb == null) { continue; + } cb.waitForSuccess(); switch (Code.get(cb.getRc())) { - case OK: + case OK: { updateData.set(i, newDataList.get(i)); - setStats.set(i, cb.getStat()); + updateStats[i] = cb.getStat(); + retCodes[i] = RetCode.OK; needUpdate[i] = false; break; - case NONODE: + } + case NONODE: { failOnNoNode = true; needCreate[i] = true; break; - case BADVERSION: + } + case BADVERSION: { failOnBadVersion = true; break; - default: - // if fail on error other than NoNode or BadVersion - // will not retry + } + default: { + // fail on error other than NoNode or BadVersion needUpdate[i] = false; + retCodes[i] = RetCode.ERROR; break; } + } } // if failOnNoNode, try create if (failOnNoNode) { - createCbList = create(paths, newDataList, needCreate, pathsCreated, options); - for (int i = 0; i < paths.size(); i++) { - CreateCallbackHandler createCb = createCbList[i]; - if (createCb == null) { + List createResults = + doCreate(paths, newDataList, Arrays.copyOf(needCreate, size), options); + for (int i = 0; i < size; i++) { + if (!needCreate[i]) { continue; } - switch (Code.get(createCb.getRc())) { - case OK: + AccessResult result = createResults.get(i); + pathsCreated.set(i, result._pathCreated); + + switch (result._retCode) { + case OK: { needUpdate[i] = false; updateData.set(i, newDataList.get(i)); - setStats.set(i, ZNode.ZERO_STAT); + updateStats[i] = ZNode.ZERO_STAT; + retCodes[i] = RetCode.OK; break; - case NODEEXISTS: + } + case NODE_EXISTS: { retry = true; break; - default: - // if fail on error other than NodeExists - // will not retry + } + default: { + // fail on error other than NodeExists + retCodes[i] = RetCode.ERROR; needUpdate[i] = false; break; } + } } } @@ -886,24 +1027,24 @@ List update(List paths, List> updaters, } } while (retry); - if (stats != null) { - stats.clear(); - stats.addAll(setStats); + List results = new ArrayList(); + for (int i = 0; i < size; i++) { + results.add(new AccessResult(retCodes[i], pathsCreated.get(i), updateStats[i], updateData + .get(i))); } - - return updateData; + return results; } finally { long endT = System.nanoTime(); if (LOG.isTraceEnabled()) { - LOG.trace("setData_async, size: " + paths.size() + ", paths: " + paths.get(0) - + ",... time: " + (endT - startT) + " ns"); + LOG.trace("updateData_async, size: " + size + ", paths: " + paths + ", time: " + + (endT - startT) + " ns"); } } } /** - * async exists + * async test existence on multiple znodes */ @Override public boolean[] exists(List paths, int options) { @@ -918,12 +1059,15 @@ public boolean[] exists(List paths, int options) { } /** - * async getStat + * async get stats of mulitple znodes */ @Override public Stat[] getStats(List paths, int options) { - if (paths == null || paths.size() == 0) { - LOG.error("paths is null or empty"); + if (paths == null) { + throw new NullPointerException("paths can't be null"); + } + + if (paths.size() == 0) { return new Stat[0]; } @@ -949,18 +1093,22 @@ public Stat[] getStats(List paths, int options) { } finally { long endT = System.nanoTime(); if (LOG.isTraceEnabled()) { - LOG.trace("exists_async, size: " + paths.size() + ", paths: " + paths.get(0) - + ",... time: " + (endT - startT) + " ns"); + LOG.trace("exists_async, size: " + paths.size() + ", paths: " + paths + ", time: " + + (endT - startT) + " ns"); } } } /** - * async remove + * async remove multiple znodes */ @Override public boolean[] remove(List paths, int options) { - if (paths == null || paths.size() == 0) { + if (paths == null) { + throw new NullPointerException("paths can't be null"); + } + + if (paths.size() == 0) { return new boolean[0]; } @@ -987,8 +1135,8 @@ public boolean[] remove(List paths, int options) { } finally { long endT = System.nanoTime(); if (LOG.isTraceEnabled()) { - LOG.trace("delete_async, size: " + paths.size() + ", paths: " + paths.get(0) - + ",... time: " + (endT - startT) + " ns"); + LOG.trace("delete_async, size: " + paths.size() + ", paths: " + paths + ", time: " + + (endT - startT) + " ns"); } } } @@ -1025,69 +1173,6 @@ public void unsubscribeChildChanges(String path, IZkChildListener childListener) _zkClient.unsubscribeChildChanges(path, childListener); } - // simple test - public static void main(String[] args) { - ZkClient zkclient = new ZkClient("localhost:2191"); - zkclient.setZkSerializer(new ZNRecordSerializer()); - ZkBaseDataAccessor accessor = new ZkBaseDataAccessor(zkclient); - - // test async create - List createPaths = Arrays.asList("/test/child1/child1", "/test/child2/child2"); - List createRecords = Arrays.asList(new ZNRecord("child1"), new ZNRecord("child2")); - - boolean[] needCreate = new boolean[createPaths.size()]; - Arrays.fill(needCreate, true); - List> pathsCreated = - new ArrayList>(Collections.> nCopies(createPaths.size(), null)); - accessor.create(createPaths, createRecords, needCreate, pathsCreated, AccessOption.PERSISTENT); - System.out.println("pathsCreated: " + pathsCreated); - - // test async set - List setPaths = Arrays.asList("/test/setChild1/setChild1", "/test/setChild2/setChild2"); - List setRecords = Arrays.asList(new ZNRecord("setChild1"), new ZNRecord("setChild2")); - - pathsCreated = - new ArrayList>(Collections.> nCopies(setPaths.size(), null)); - boolean[] success = - accessor.set(setPaths, setRecords, pathsCreated, null, AccessOption.PERSISTENT); - System.out.println("pathsCreated: " + pathsCreated); - System.out.println("setSuccess: " + Arrays.toString(success)); - - // test async update - List updatePaths = - Arrays.asList("/test/updateChild1/updateChild1", "/test/setChild2/setChild2"); - class TestUpdater implements DataUpdater { - final ZNRecord _newData; - - public TestUpdater(ZNRecord newData) { - _newData = newData; - } - - @Override - public ZNRecord update(ZNRecord currentData) { - return _newData; - - } - } - List> updaters = - Arrays.asList((DataUpdater) new TestUpdater(new ZNRecord("updateChild1")), - (DataUpdater) new TestUpdater(new ZNRecord("updateChild2"))); - - pathsCreated = - new ArrayList>(Collections.> nCopies(updatePaths.size(), null)); - - List updateRecords = - accessor.update(updatePaths, updaters, pathsCreated, null, AccessOption.PERSISTENT); - for (int i = 0; i < updatePaths.size(); i++) { - success[i] = updateRecords.get(i) != null; - } - System.out.println("pathsCreated: " + pathsCreated); - System.out.println("updateSuccess: " + Arrays.toString(success)); - - System.out.println("CLOSING"); - zkclient.close(); - } - /** * Reset */ diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java index 5e7355b26a..0d0bbe449a 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java @@ -35,14 +35,12 @@ import org.I0Itec.zkclient.exception.ZkNoNodeException; import org.I0Itec.zkclient.serialize.ZkSerializer; import org.apache.helix.AccessOption; -import org.apache.helix.manager.zk.ZkAsyncCallbacks.CreateCallbackHandler; import org.apache.helix.manager.zk.ZkBaseDataAccessor.RetCode; import org.apache.helix.store.HelixPropertyListener; import org.apache.helix.store.HelixPropertyStore; import org.apache.helix.store.zk.ZNode; import org.apache.helix.util.PathUtils; import org.apache.log4j.Logger; -import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.DataTree; @@ -250,7 +248,7 @@ public boolean set(String path, T data, int expectVersion, int options) { cache.lockWrite(); ZkBaseDataAccessor.AccessResult result = _baseAccessor.doSet(serverPath, data, expectVersion, options); - boolean success = result._retCode == RetCode.OK; + boolean success = (result._retCode == RetCode.OK); updateCache(cache, result._pathCreated, success, serverPath, data, result._stat); @@ -279,7 +277,7 @@ public boolean update(String path, DataUpdater updater, int options) { ZkBaseDataAccessor.AccessResult result = _baseAccessor.doUpdate(serverPath, updater, options); boolean success = (result._retCode == RetCode.OK); - updateCache(cache, result._pathCreated, success, serverPath, result._updatedValue, + updateCache(cache, result._pathCreated, success, serverPath, result._resultValue, result._stat); return success; @@ -421,17 +419,15 @@ public boolean[] createChildren(List paths, List records, int options cache.lockWrite(); boolean[] needCreate = new boolean[size]; Arrays.fill(needCreate, true); - List> pathsCreatedList = - new ArrayList>(Collections.> nCopies(size, null)); - CreateCallbackHandler[] createCbList = - _baseAccessor.create(serverPaths, records, needCreate, pathsCreatedList, options); + List.AccessResult> results = + _baseAccessor.doCreate(serverPaths, records, needCreate, options); boolean[] success = new boolean[size]; for (int i = 0; i < size; i++) { - CreateCallbackHandler cb = createCbList[i]; - success[i] = (Code.get(cb.getRc()) == Code.OK); + ZkBaseDataAccessor.AccessResult result = results.get(i); + success[i] = (result._retCode == RetCode.OK); - updateCache(cache, pathsCreatedList.get(i), success[i], serverPaths.get(i), + updateCache(cache, results.get(i)._pathCreated, success[i], serverPaths.get(i), records.get(i), ZNode.ZERO_STAT); } @@ -454,15 +450,14 @@ public boolean[] setChildren(List paths, List records, int options) { if (cache != null) { try { cache.lockWrite(); - List setStats = new ArrayList(); - List> pathsCreatedList = - new ArrayList>(Collections.> nCopies(size, null)); - boolean[] success = - _baseAccessor.set(serverPaths, records, pathsCreatedList, setStats, options); + List.AccessResult> results = + _baseAccessor.doSet(serverPaths, records, options); + boolean[] success = new boolean[size]; for (int i = 0; i < size; i++) { - updateCache(cache, pathsCreatedList.get(i), success[i], serverPaths.get(i), - records.get(i), setStats.get(i)); + success[i] = (results.get(i)._retCode == RetCode.OK); + updateCache(cache, results.get(i)._pathCreated, success[i], serverPaths.get(i), + records.get(i), results.get(i)._stat); } return success; @@ -484,23 +479,17 @@ public boolean[] updateChildren(List paths, List> updater try { cache.lockWrite(); - List setStats = new ArrayList(); boolean[] success = new boolean[size]; List> pathsCreatedList = new ArrayList>(Collections.> nCopies(size, null)); - List updateData = - _baseAccessor.update(serverPaths, updaters, pathsCreatedList, setStats, options); - // System.out.println("updateChild: "); - // for (T data : updateData) - // { - // System.out.println(data); - // } + List.AccessResult> results = _baseAccessor.doUpdate(serverPaths, updaters, options); for (int i = 0; i < size; i++) { - success[i] = (updateData.get(i) != null); - updateCache(cache, pathsCreatedList.get(i), success[i], serverPaths.get(i), - updateData.get(i), setStats.get(i)); + ZkBaseDataAccessor.AccessResult result = results.get(i); + success[i] = (result._retCode == RetCode.OK); + updateCache(cache, pathsCreatedList.get(i), success[i], serverPaths.get(i), + result._resultValue, results.get(i)._stat); } return success; } finally { @@ -516,11 +505,10 @@ public boolean[] updateChildren(List paths, List> updater @Override public boolean[] exists(List paths, int options) { final int size = paths.size(); - List serverPaths = prependChroot(paths); boolean exists[] = new boolean[size]; for (int i = 0; i < size; i++) { - exists[i] = exists(serverPaths.get(i), options); + exists[i] = exists(paths.get(i), options); } return exists; } @@ -590,11 +578,13 @@ public List get(List paths, List stats, int options) { if (needRead) { cache.lockWrite(); try { - List readRecords = _baseAccessor.get(serverPaths, readStats, needReads); + List.AccessResult> readResults = + _baseAccessor.doGet(serverPaths, needReads); for (int i = 0; i < size; i++) { if (needReads[i]) { - records.set(i, readRecords.get(i)); - cache.update(serverPaths.get(i), readRecords.get(i), readStats.get(i)); + records.set(i, readResults.get(i)._resultValue); + readStats.set(i, readResults.get(i)._stat); + cache.update(serverPaths.get(i), records.get(i), readStats.get(i)); } } } finally { @@ -664,7 +654,7 @@ public List getChildren(String parentPath, List stats, int options) { List paths = new ArrayList(); for (String childName : childNames) { - String path = parentPath + "/" + childName; + String path = parentPath.equals("/")? "/" + childName : parentPath + "/" + childName; paths.add(path); } diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackCache.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackCache.java index a02cedf978..d1e3af3453 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackCache.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackCache.java @@ -19,7 +19,6 @@ * under the License. */ -import java.io.File; import java.util.List; import java.util.Map; import java.util.Set; diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java new file mode 100644 index 0000000000..86d390a4f7 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java @@ -0,0 +1,449 @@ +package org.apache.helix.manager.zk; + +/* + * 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 static org.apache.helix.HelixConstants.ChangeType.CONFIG; +import static org.apache.helix.HelixConstants.ChangeType.CURRENT_STATE; +import static org.apache.helix.HelixConstants.ChangeType.EXTERNAL_VIEW; +import static org.apache.helix.HelixConstants.ChangeType.IDEAL_STATE; +import static org.apache.helix.HelixConstants.ChangeType.LIVE_INSTANCE; +import static org.apache.helix.HelixConstants.ChangeType.MESSAGE; +import static org.apache.helix.HelixConstants.ChangeType.MESSAGES_CONTROLLER; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.I0Itec.zkclient.IZkChildListener; +import org.I0Itec.zkclient.IZkDataListener; +import org.I0Itec.zkclient.exception.ZkNoNodeException; +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.ConfigChangeListener; +import org.apache.helix.ControllerChangeListener; +import org.apache.helix.CurrentStateChangeListener; +import org.apache.helix.ExternalViewChangeListener; +import org.apache.helix.HelixConnection; +import org.apache.helix.HelixConstants.ChangeType; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixException; +import org.apache.helix.HelixManager; +import org.apache.helix.HelixProperty; +import org.apache.helix.HelixRole; +import org.apache.helix.IdealStateChangeListener; +import org.apache.helix.InstanceConfigChangeListener; +import org.apache.helix.LiveInstanceChangeListener; +import org.apache.helix.MessageListener; +import org.apache.helix.NotificationContext; +import org.apache.helix.NotificationContext.Type; +import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyPathConfig; +import org.apache.helix.ScopedConfigChangeListener; +import org.apache.helix.ZNRecord; +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.log4j.Logger; +import org.apache.zookeeper.Watcher.Event.EventType; + +/** + * This is a copy of {@link CallbackHandler} We need to synchronize on ZkHelixConnection + * instead ofHelixManager to avoid dead-lock. + * Otherwise an example deadlock scenario would be: + * 1) main-thread calls ZkHelixConnection#disconnect(), results in: + * - ZkHelixController#reset(), holding ZkHelixConnection, waiting HelixConnectionAdaptor + * 2) zk-event-thread calls CallbackHandler#handleChildChange(), results in: + * - CallbackHandler#invoke(), holding HelixConnectionAdaptor, waiting ZkHelixConnection + * TODO remove code duplication + */ +public class ZkCallbackHandler implements IZkChildListener, IZkDataListener + +{ + private static Logger logger = Logger.getLogger(ZkCallbackHandler.class); + + /** + * define the next possible notification types + */ + private static Map> nextNotificationType = new HashMap>(); + static { + nextNotificationType.put(Type.INIT, Arrays.asList(Type.CALLBACK, Type.FINALIZE)); + nextNotificationType.put(Type.CALLBACK, Arrays.asList(Type.CALLBACK, Type.FINALIZE)); + nextNotificationType.put(Type.FINALIZE, Arrays.asList(Type.INIT)); + } + + private final String _path; + private final Object _listener; + private final EventType[] _eventTypes; + + private final ChangeType _changeType; + private final ZkClient _zkClient; + private final AtomicLong _lastNotificationTimeStamp; + + private final HelixRole _role; + private final HelixManager _manager; + private final String _instanceName; + private final HelixConnection _connection; + private final HelixDataAccessor _accessor; + + private final PropertyKey _propertyKey; + + /** + * maintain the expected notification types + * this is fix for HELIX-195: race condition between FINALIZE callbacks and Zk callbacks + */ + private List _expectTypes = nextNotificationType.get(Type.FINALIZE); + + public ZkCallbackHandler(HelixRole role, ZkClient client, + PropertyKey propertyKey, + Object listener, EventType[] eventTypes, ChangeType changeType) { + if (listener == null) { + throw new HelixException("listener could not be null"); + } + + _role = role; + _manager = new HelixConnectionAdaptor(role); + _instanceName = role.getId().stringify(); + _connection = role.getConnection(); + _accessor = _connection.createDataAccessor(role.getClusterId()); + _zkClient = client; + _propertyKey = propertyKey; + _path = propertyKey.getPath(); + _listener = listener; + _eventTypes = eventTypes; + _changeType = changeType; + _lastNotificationTimeStamp = new AtomicLong(System.nanoTime()); + init(); + } + + public Object getListener() { + return _listener; + } + + public String getPath() { + return _path; + } + + public void invoke(NotificationContext changeContext) throws Exception { + // This allows the listener to work with one change at a time + synchronized (_connection) { + Type type = changeContext.getType(); + if (!_expectTypes.contains(type)) { + logger.warn("Skip processing callbacks for listener: " + _listener + ", path: " + _path + + ", expected types: " + _expectTypes + " but was " + type); + return; + } + _expectTypes = nextNotificationType.get(type); + + // Builder keyBuilder = _accessor.keyBuilder(); + long start = System.currentTimeMillis(); + if (logger.isInfoEnabled()) { + logger.info(Thread.currentThread().getId() + " START:INVOKE " + _path + " listener:" + + _listener.getClass().getCanonicalName()); + } + + if (_changeType == IDEAL_STATE) { + + IdealStateChangeListener idealStateChangeListener = (IdealStateChangeListener) _listener; + subscribeForChanges(changeContext, _path, true, true); + List idealStates = _accessor.getChildValues(_propertyKey); + + idealStateChangeListener.onIdealStateChange(idealStates, changeContext); + + } else if (_changeType == ChangeType.INSTANCE_CONFIG) { + subscribeForChanges(changeContext, _path, true, true); + if (_listener instanceof ConfigChangeListener) { + ConfigChangeListener configChangeListener = (ConfigChangeListener) _listener; + List configs = _accessor.getChildValues(_propertyKey); + configChangeListener.onConfigChange(configs, changeContext); + } else if (_listener instanceof InstanceConfigChangeListener) { + InstanceConfigChangeListener listener = (InstanceConfigChangeListener) _listener; + List configs = _accessor.getChildValues(_propertyKey); + listener.onInstanceConfigChange(configs, changeContext); + } + } else if (_changeType == CONFIG) { + subscribeForChanges(changeContext, _path, true, true); + ScopedConfigChangeListener listener = (ScopedConfigChangeListener) _listener; + List configs = _accessor.getChildValues(_propertyKey); + listener.onConfigChange(configs, changeContext); + } else if (_changeType == LIVE_INSTANCE) { + LiveInstanceChangeListener liveInstanceChangeListener = + (LiveInstanceChangeListener) _listener; + subscribeForChanges(changeContext, _path, true, true); + List liveInstances = _accessor.getChildValues(_propertyKey); + + liveInstanceChangeListener.onLiveInstanceChange(liveInstances, changeContext); + + } else if (_changeType == CURRENT_STATE) { + CurrentStateChangeListener currentStateChangeListener = + (CurrentStateChangeListener) _listener; + subscribeForChanges(changeContext, _path, true, true); + String instanceName = PropertyPathConfig.getInstanceNameFromPath(_path); + + List currentStates = _accessor.getChildValues(_propertyKey); + + currentStateChangeListener.onStateChange(instanceName, currentStates, changeContext); + + } else if (_changeType == MESSAGE) { + MessageListener messageListener = (MessageListener) _listener; + subscribeForChanges(changeContext, _path, true, false); + String instanceName = PropertyPathConfig.getInstanceNameFromPath(_path); + List messages = _accessor.getChildValues(_propertyKey); + + messageListener.onMessage(instanceName, messages, changeContext); + + } else if (_changeType == MESSAGES_CONTROLLER) { + MessageListener messageListener = (MessageListener) _listener; + subscribeForChanges(changeContext, _path, true, false); + List messages = _accessor.getChildValues(_propertyKey); + + messageListener.onMessage(_instanceName, messages, changeContext); + + } else if (_changeType == EXTERNAL_VIEW) { + ExternalViewChangeListener externalViewListener = (ExternalViewChangeListener) _listener; + subscribeForChanges(changeContext, _path, true, true); + List externalViewList = _accessor.getChildValues(_propertyKey); + + externalViewListener.onExternalViewChange(externalViewList, changeContext); + } else if (_changeType == ChangeType.CONTROLLER) { + ControllerChangeListener controllerChangelistener = (ControllerChangeListener) _listener; + subscribeForChanges(changeContext, _path, true, false); + controllerChangelistener.onControllerChange(changeContext); + } + + long end = System.currentTimeMillis(); + if (logger.isInfoEnabled()) { + logger.info(Thread.currentThread().getId() + " END:INVOKE " + _path + " listener:" + + _listener.getClass().getCanonicalName() + " Took: " + (end - start) + "ms"); + } + } + } + + private void subscribeChildChange(String path, NotificationContext context) { + NotificationContext.Type type = context.getType(); + if (type == NotificationContext.Type.INIT || type == NotificationContext.Type.CALLBACK) { + logger.info(_instanceName + " subscribes child-change. path: " + path + + ", listener: " + _listener); + _zkClient.subscribeChildChanges(path, this); + } else if (type == NotificationContext.Type.FINALIZE) { + logger.info(_instanceName + " unsubscribe child-change. path: " + path + + ", listener: " + _listener); + + _zkClient.unsubscribeChildChanges(path, this); + } + } + + private void subscribeDataChange(String path, NotificationContext context) { + NotificationContext.Type type = context.getType(); + if (type == NotificationContext.Type.INIT || type == NotificationContext.Type.CALLBACK) { + if (logger.isDebugEnabled()) { + logger.debug(_instanceName + " subscribe data-change. path: " + path + + ", listener: " + _listener); + } + _zkClient.subscribeDataChanges(path, this); + + } else if (type == NotificationContext.Type.FINALIZE) { + logger.info(_instanceName + " unsubscribe data-change. path: " + path + + ", listener: " + _listener); + + _zkClient.unsubscribeDataChanges(path, this); + } + } + + // TODO watchParent is always true. consider remove it + private void subscribeForChanges(NotificationContext context, String path, boolean watchParent, + boolean watchChild) { + if (watchParent) { + subscribeChildChange(path, context); + } + + if (watchChild) { + try { + switch (_changeType) { + case CURRENT_STATE: + case IDEAL_STATE: + case EXTERNAL_VIEW: { + // check if bucketized + BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_zkClient); + List records = baseAccessor.getChildren(path, null, 0); + for (ZNRecord record : records) { + HelixProperty property = new HelixProperty(record); + String childPath = path + "/" + record.getId(); + + int bucketSize = property.getBucketSize(); + if (bucketSize > 0) { + // subscribe both data-change and child-change on bucketized parent node + // data-change gives a delete-callback which is used to remove watch + subscribeChildChange(childPath, context); + subscribeDataChange(childPath, context); + + // subscribe data-change on bucketized child + List bucketizedChildNames = _zkClient.getChildren(childPath); + if (bucketizedChildNames != null) { + for (String bucketizedChildName : bucketizedChildNames) { + String bucketizedChildPath = childPath + "/" + bucketizedChildName; + subscribeDataChange(bucketizedChildPath, context); + } + } + } else { + subscribeDataChange(childPath, context); + } + } + break; + } + default: { + List childNames = _zkClient.getChildren(path); + if (childNames != null) { + for (String childName : childNames) { + String childPath = path + "/" + childName; + subscribeDataChange(childPath, context); + } + } + break; + } + } + } catch (ZkNoNodeException e) { + logger.warn("fail to subscribe child/data change. path: " + path + ", listener: " + + _listener, e); + } + } + + } + + public EventType[] getEventTypes() { + return _eventTypes; + } + + /** + * Invoke the listener so that it sets up the initial values from the zookeeper if any + * exists + */ + public void init() { + updateNotificationTime(System.nanoTime()); + try { + NotificationContext changeContext = new NotificationContext(_manager); + changeContext.setType(NotificationContext.Type.INIT); + invoke(changeContext); + } catch (Exception e) { + String msg = "Exception while invoking init callback for listener:" + _listener; + ZKExceptionHandler.getInstance().handle(msg, e); + } + } + + @Override + public void handleDataChange(String dataPath, Object data) { + try { + updateNotificationTime(System.nanoTime()); + if (dataPath != null && dataPath.startsWith(_path)) { + NotificationContext changeContext = new NotificationContext(_manager); + changeContext.setType(NotificationContext.Type.CALLBACK); + invoke(changeContext); + } + } catch (Exception e) { + String msg = + "exception in handling data-change. path: " + dataPath + ", listener: " + _listener; + ZKExceptionHandler.getInstance().handle(msg, e); + } + } + + @Override + public void handleDataDeleted(String dataPath) { + try { + updateNotificationTime(System.nanoTime()); + if (dataPath != null && dataPath.startsWith(_path)) { + logger.info(_instanceName + " unsubscribe data-change. path: " + dataPath + + ", listener: " + _listener); + _zkClient.unsubscribeDataChanges(dataPath, this); + + // only needed for bucketized parent, but OK if we don't have child-change + // watch on the bucketized parent path + logger.info(_instanceName + " unsubscribe child-change. path: " + dataPath + + ", listener: " + _listener); + _zkClient.unsubscribeChildChanges(dataPath, this); + // No need to invoke() since this event will handled by child-change on parent-node + // NotificationContext changeContext = new NotificationContext(_manager); + // changeContext.setType(NotificationContext.Type.CALLBACK); + // invoke(changeContext); + } + } catch (Exception e) { + String msg = + "exception in handling data-delete-change. path: " + dataPath + ", listener: " + + _listener; + ZKExceptionHandler.getInstance().handle(msg, e); + } + } + + @Override + public void handleChildChange(String parentPath, List currentChilds) { + try { + updateNotificationTime(System.nanoTime()); + if (parentPath != null && parentPath.startsWith(_path)) { + NotificationContext changeContext = new NotificationContext(_manager); + + if (currentChilds == null) { + // parentPath has been removed + if (parentPath.equals(_path)) { + // _path has been removed, remove this listener + _manager.removeListener(_propertyKey, _listener); + } + changeContext.setType(NotificationContext.Type.FINALIZE); + } else { + changeContext.setType(NotificationContext.Type.CALLBACK); + } + invoke(changeContext); + } + } catch (Exception e) { + String msg = + "exception in handling child-change. instance: " + _instanceName + + ", parentPath: " + parentPath + ", listener: " + _listener; + ZKExceptionHandler.getInstance().handle(msg, e); + } + } + + /** + * Invoke the listener for the last time so that the listener could clean up resources + */ + public void reset() { + try { + NotificationContext changeContext = new NotificationContext(_manager); + changeContext.setType(NotificationContext.Type.FINALIZE); + invoke(changeContext); + } catch (Exception e) { + String msg = "Exception while resetting the listener:" + _listener; + ZKExceptionHandler.getInstance().handle(msg, e); + } + } + + private void updateNotificationTime(long nanoTime) { + long l = _lastNotificationTimeStamp.get(); + while (nanoTime > l) { + boolean b = _lastNotificationTimeStamp.compareAndSet(l, nanoTime); + if (b) { + break; + } else { + l = _lastNotificationTimeStamp.get(); + } + } + } + +} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java index 5b3af6d9b4..e43eddf5cc 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java @@ -37,7 +37,6 @@ import org.apache.helix.manager.zk.ZkAsyncCallbacks.SetDataCallbackHandler; import org.apache.log4j.Logger; import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; @@ -62,8 +61,7 @@ public ZkClient(IZkConnection connection, int connectionTimeout, _zkSerializer = zkSerializer; if (LOG.isTraceEnabled()) { StackTraceElement[] calls = Thread.currentThread().getStackTrace(); - int min = Math.min(calls.length, 5); - LOG.trace("creating a zkclient. callstack: " + Arrays.asList(calls).subList(0, min)); + LOG.trace("creating a zkclient. callstack: " + Arrays.asList(calls)); } } @@ -122,10 +120,45 @@ public IZkConnection getConnection() { public void close() throws ZkInterruptedException { if (LOG.isTraceEnabled()) { StackTraceElement[] calls = Thread.currentThread().getStackTrace(); - int min = Math.min(calls.length, 5); - LOG.trace("closing a zkclient. callStack: " + Arrays.asList(calls).subList(0, min)); + LOG.trace("closing a zkclient. callStack: " + Arrays.asList(calls)); + } + + getEventLock().lock(); + try { + if (_connection == null) { + return; + } + + LOG.info("Closing zkclient: " + ((ZkConnection) _connection).getZookeeper()); + super.close(); + } catch (ZkInterruptedException e) { + /** + * HELIX-264: calling ZkClient#close() in its own eventThread context will + * throw ZkInterruptedException and skip ZkConnection#close() + */ + if (_connection != null) { + try { + /** + * ZkInterruptedException#construct() honors InterruptedException by calling + * Thread.currentThread().interrupt(); clear it first, so we can safely close the + * zk-connection + */ + Thread.interrupted(); + _connection.close(); + _connection = null; + + /** + * restore interrupted status of current thread + */ + Thread.currentThread().interrupt(); + } catch (InterruptedException e1) { + throw new ZkInterruptedException(e1); + } + } + } finally { + getEventLock().unlock(); + LOG.info("Closed zkclient"); } - super.close(); } public Stat getStat(final String path) { @@ -312,6 +345,10 @@ public String call() throws Exception { @Override public boolean delete(final String path) { + return this.delete(path, -1); + } + + public boolean delete(final String path, final int version) { long startT = System.nanoTime(); try { try { @@ -319,7 +356,8 @@ public boolean delete(final String path) { @Override public Object call() throws Exception { - _connection.delete(path); + ZkConnection connection = (ZkConnection) _connection; + connection.getZookeeper().delete(path, version); return null; } }); diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixAutoController.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixAutoController.java new file mode 100644 index 0000000000..d17192fbc6 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixAutoController.java @@ -0,0 +1,134 @@ +package org.apache.helix.manager.zk; + +/* + * 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.ClusterMessagingService; +import org.apache.helix.HelixAutoController; +import org.apache.helix.HelixConnection; +import org.apache.helix.InstanceType; +import org.apache.helix.LiveInstanceInfoProvider; +import org.apache.helix.PreConnectCallback; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.api.id.Id; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.monitoring.MonitoringServer; +import org.apache.helix.participant.StateMachineEngine; +import org.apache.log4j.Logger; + +public class ZkHelixAutoController implements HelixAutoController { + private static Logger LOG = Logger.getLogger(ZkHelixAutoController.class); + + final ZkHelixConnection _connection; + final ClusterId _clusterId; + final ControllerId _controllerId; + final ZkHelixParticipant _participant; + final ZkHelixController _controller; + + public ZkHelixAutoController(ZkHelixConnection connection, ClusterId clusterId, + ControllerId controllerId) { + _connection = connection; + _clusterId = clusterId; + _controllerId = controllerId; + + _participant = + new ZkHelixParticipant(connection, clusterId, ParticipantId.from(controllerId.stringify())); + _controller = new ZkHelixController(connection, clusterId, controllerId); + } + + @Override + public HelixConnection getConnection() { + return _connection; + } + + @Override + public ClusterId getClusterId() { + return _clusterId; + } + + @Override + public Id getId() { + return getControllerId(); + } + + @Override + public InstanceType getType() { + return InstanceType.CONTROLLER_PARTICIPANT; + } + + @Override + public ClusterMessagingService getMessagingService() { + return _participant.getMessagingService(); + } + + @Override + public void startAsync() { + _connection.addConnectionStateListener(this); + onConnected(); + } + + @Override + public void stopAsync() { + _connection.removeConnectionStateListener(this); + onDisconnecting(); + } + + @Override + public void onConnected() { + _controller.reset(); + _participant.reset(); + + _participant.init(); + _controller.init(); + } + + @Override + public void onDisconnecting() { + LOG.info("disconnecting " + _controllerId + "(" + getType() + ") from " + _clusterId); + _controller.onDisconnecting(); + _participant.onDisconnecting(); + } + + @Override + public ControllerId getControllerId() { + return _controllerId; + } + + @Override + public StateMachineEngine getStateMachineEngine() { + return _participant.getStateMachineEngine(); + } + + @Override + public void addPreConnectCallback(PreConnectCallback callback) { + _participant.addPreConnectCallback(callback); + } + + @Override + public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) { + _participant.setLiveInstanceInfoProvider(liveInstanceInfoProvider); + } + + @Override + public boolean isLeader() { + return _controller.isLeader(); + } + +} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java new file mode 100644 index 0000000000..bbe8afe183 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java @@ -0,0 +1,618 @@ +package org.apache.helix.manager.zk; + +/* + * 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 java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; + +import org.I0Itec.zkclient.IZkStateListener; +import org.I0Itec.zkclient.ZkConnection; +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.ClusterMessagingService; +import org.apache.helix.ConfigAccessor; +import org.apache.helix.ConfigChangeListener; +import org.apache.helix.ControllerChangeListener; +import org.apache.helix.CurrentStateChangeListener; +import org.apache.helix.ExternalViewChangeListener; +import org.apache.helix.HelixAdmin; +import org.apache.helix.HelixAutoController; +import org.apache.helix.HelixConstants.ChangeType; +import org.apache.helix.HelixConnection; +import org.apache.helix.HelixConnectionStateListener; +import org.apache.helix.HelixController; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixManager; +import org.apache.helix.HelixManagerProperties; +import org.apache.helix.HelixParticipant; +import org.apache.helix.HelixRole; +import org.apache.helix.IdealStateChangeListener; +import org.apache.helix.InstanceConfigChangeListener; +import org.apache.helix.LiveInstanceChangeListener; +import org.apache.helix.MessageListener; +import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyType; +import org.apache.helix.ScopedConfigChangeListener; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.accessor.ParticipantAccessor; +import org.apache.helix.api.accessor.ResourceAccessor; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.messaging.DefaultMessagingService; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.monitoring.MonitoringClient; +import org.apache.helix.store.HelixPropertyStore; +import org.apache.helix.store.zk.ZkHelixPropertyStore; +import org.apache.log4j.Logger; +import org.apache.zookeeper.Watcher.Event.EventType; +import org.apache.zookeeper.Watcher.Event.KeeperState; + +public class ZkHelixConnection implements HelixConnection, IZkStateListener { + private static Logger LOG = Logger.getLogger(ZkHelixConnection.class); + + final String _zkAddr; + final int _sessionTimeout; + SessionId _sessionId; + ZkClient _zkclient; + BaseDataAccessor _baseAccessor; + ConfigAccessor _configAccessor; + final Set _connectionListener = + new CopyOnWriteArraySet(); + + final Map> _handlers; + final HelixManagerProperties _properties; + + /** + * Keep track of timestamps that zk State has become Disconnected + * If in a _timeWindowLengthMs window zk State has become Disconnected + * for more than_maxDisconnectThreshold times disconnect the zkHelixManager + */ + final List _disconnectTimeHistory = new ArrayList(); + final int _flappingTimeWindowMs; + final int _maxDisconnectThreshold; + + final ReentrantLock _lock = new ReentrantLock(); + + /** + * helix version# + */ + final String _version; + + public ZkHelixConnection(String zkAddr) { + _zkAddr = zkAddr; + _handlers = new HashMap>(); + + /** + * use system property if available + */ + _flappingTimeWindowMs = + getSystemPropertyAsInt("helixmanager.flappingTimeWindow", + ZKHelixManager.FLAPPING_TIME_WINDIOW); + + _maxDisconnectThreshold = + getSystemPropertyAsInt("helixmanager.maxDisconnectThreshold", + ZKHelixManager.MAX_DISCONNECT_THRESHOLD); + + _sessionTimeout = + getSystemPropertyAsInt("zk.session.timeout", ZkClient.DEFAULT_SESSION_TIMEOUT); + + _properties = new HelixManagerProperties("cluster-manager-version.properties"); + _version = _properties.getVersion(); + + } + + private int getSystemPropertyAsInt(String propertyKey, int propertyDefaultValue) { + String valueString = System.getProperty(propertyKey, "" + propertyDefaultValue); + + try { + int value = Integer.parseInt(valueString); + if (value > 0) { + return value; + } + } catch (NumberFormatException e) { + LOG.warn("Exception while parsing property: " + propertyKey + ", string: " + valueString + + ", using default value: " + propertyDefaultValue); + } + + return propertyDefaultValue; + } + + @Override + public void connect() { + boolean isStarted = false; + try { + _lock.lock(); + _zkclient = + new ZkClient(_zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT, + ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer()); + // waitUntilConnected(); + + _baseAccessor = new ZkBaseDataAccessor(_zkclient); + _configAccessor = new ConfigAccessor(_zkclient); + + _zkclient.subscribeStateChanges(this); + handleNewSession(); + + isStarted = true; + } catch (Exception e) { + LOG.error("Exception connect", e); + } finally { + _lock.unlock(); + if (!isStarted) { + disconnect(); + } + } + } + + @Override + public void disconnect() { + if (_zkclient == null) { + return; + } + + LOG.info("Disconnecting connection: " + this); + + try { + _lock.lock(); + for (final HelixConnectionStateListener listener : _connectionListener) { + try { + + listener.onDisconnecting(); + } catch (Exception e) { + LOG.error("Exception in calling disconnect on listener: " + listener, e); + } + } + _zkclient.close(); + _zkclient = null; + LOG.info("Disconnected connection: " + this); + } catch (Exception e) { + LOG.error("Exception disconnect", e); + } finally { + _lock.unlock(); + } + } + + @Override + public boolean isConnected() { + try { + _lock.lock(); + return _zkclient != null; + } finally { + _lock.unlock(); + } + } + + @Override + public HelixParticipant createParticipant(ClusterId clusterId, ParticipantId participantId) { + return new ZkHelixParticipant(this, clusterId, participantId); + } + + @Override + public HelixController createController(ClusterId clusterId, ControllerId controllerId) { + return new ZkHelixController(this, clusterId, controllerId); + } + + @Override + public HelixAutoController createAutoController(ClusterId clusterId, ControllerId controllerId) { + return new ZkHelixAutoController(this, clusterId, controllerId); + } + + @Override + public ClusterAccessor createClusterAccessor(ClusterId clusterId) { + return new ClusterAccessor(clusterId, createDataAccessor(clusterId)); + } + + @Override + public ResourceAccessor createResourceAccessor(ClusterId clusterId) { + return new ResourceAccessor(createDataAccessor(clusterId)); + } + + @Override + public ParticipantAccessor createParticipantAccessor(ClusterId clusterId) { + return new ParticipantAccessor(createDataAccessor(clusterId)); + } + + @Override + public HelixAdmin createClusterManagementTool() { + return new ZKHelixAdmin(_zkclient); + } + + @Override + public HelixPropertyStore createPropertyStore(ClusterId clusterId) { + PropertyKey key = new PropertyKey.Builder(clusterId.stringify()).propertyStore(); + return new ZkHelixPropertyStore(new ZkBaseDataAccessor(_zkclient), + key.getPath(), null); + } + + @Override + public HelixDataAccessor createDataAccessor(ClusterId clusterId) { + return new ZKHelixDataAccessor(clusterId.stringify(), _baseAccessor); + } + + @Override + public ConfigAccessor getConfigAccessor() { + return _configAccessor; + } + + @Override + public void addControllerListener(HelixRole role, ControllerChangeListener listener, + ClusterId clusterId) { + + addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).controller(), + ChangeType.CONTROLLER, new EventType[] { + EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated + }); + } + + @Override + public void addMessageListener(HelixRole role, MessageListener listener, ClusterId clusterId, + ParticipantId participantId) { + + addListener(role, listener, + new PropertyKey.Builder(clusterId.stringify()).messages(participantId.stringify()), + ChangeType.MESSAGE, new EventType[] { + EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated + }); + } + + @Override + public void addControllerMessageListener(HelixRole role, MessageListener listener, + ClusterId clusterId) { + + addListener(role, listener, + new PropertyKey.Builder(clusterId.stringify()).controllerMessages(), + ChangeType.MESSAGES_CONTROLLER, new EventType[] { + EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated + }); + } + + @Override + public void addIdealStateChangeListener(HelixRole role, IdealStateChangeListener listener, + ClusterId clusterId) { + + addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).idealStates(), + ChangeType.IDEAL_STATE, new EventType[] { + EventType.NodeDataChanged, EventType.NodeDeleted, EventType.NodeCreated + }); + } + + @Override + public void addLiveInstanceChangeListener(HelixRole role, LiveInstanceChangeListener listener, + ClusterId clusterId) { + + addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).liveInstances(), + ChangeType.LIVE_INSTANCE, new EventType[] { + EventType.NodeDataChanged, EventType.NodeChildrenChanged, EventType.NodeDeleted, + EventType.NodeCreated + }); + } + + @Override + public void addConfigChangeListener(HelixRole role, ConfigChangeListener listener, + ClusterId clusterId) { + + addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).instanceConfigs(), + ChangeType.INSTANCE_CONFIG, new EventType[] { + EventType.NodeChildrenChanged + }); + } + + @Override + public void addInstanceConfigChangeListener(HelixRole role, + InstanceConfigChangeListener listener, ClusterId clusterId) { + addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).instanceConfigs(), + ChangeType.INSTANCE_CONFIG, new EventType[] { + EventType.NodeChildrenChanged + }); + } + + @Override + public void addConfigChangeListener(HelixRole role, ScopedConfigChangeListener listener, + ClusterId clusterId, ConfigScopeProperty scope) { + PropertyKey.Builder keyBuilder = new PropertyKey.Builder(clusterId.stringify()); + + PropertyKey propertyKey = null; + switch (scope) { + case CLUSTER: + propertyKey = keyBuilder.clusterConfigs(); + break; + case PARTICIPANT: + propertyKey = keyBuilder.instanceConfigs(); + break; + case RESOURCE: + propertyKey = keyBuilder.resourceConfigs(); + break; + default: + break; + } + + if (propertyKey == null) { + LOG.error("Failed to add listener: " + listener + ", unrecognized config scope: " + scope); + return; + } + + addListener(role, listener, propertyKey, ChangeType.CONFIG, new EventType[] { + EventType.NodeChildrenChanged + }); + } + + @Override + public void addCurrentStateChangeListener(HelixRole role, CurrentStateChangeListener listener, + ClusterId clusterId, ParticipantId participantId, SessionId sessionId) { + + addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).currentStates( + participantId.stringify(), sessionId.stringify()), ChangeType.CURRENT_STATE, + new EventType[] { + EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated + }); + } + + @Override + public void addExternalViewChangeListener(HelixRole role, ExternalViewChangeListener listener, + ClusterId clusterId) { + addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).externalViews(), + ChangeType.EXTERNAL_VIEW, new EventType[] { + EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated + }); + } + + @Override + public boolean removeListener(HelixRole role, Object listener, PropertyKey key) { + LOG.info("role: " + role + " removing listener: " + listener + " on path: " + key.getPath() + + " from connection: " + this); + List toRemove = new ArrayList(); + List handlerList = _handlers.get(role); + if (handlerList == null) { + return true; + } + + synchronized (this) { + for (ZkCallbackHandler handler : handlerList) { + // compare property-key path and listener reference + if (handler.getPath().equals(key.getPath()) && handler.getListener().equals(listener)) { + toRemove.add(handler); + } + } + + handlerList.removeAll(toRemove); + if (handlerList.isEmpty()) { + _handlers.remove(role); + } + } + + // handler.reset() may modify the handlers list, so do it outside the iteration + for (ZkCallbackHandler handler : toRemove) { + handler.reset(); + } + + return true; + } + + @Override + public void addConnectionStateListener(HelixConnectionStateListener listener) { + synchronized (_connectionListener) { + _connectionListener.add(listener); + } + } + + @Override + public void removeConnectionStateListener(HelixConnectionStateListener listener) { + synchronized (_connectionListener) { + _connectionListener.remove(listener); + } + } + + @Override + public void handleStateChanged(KeeperState state) throws Exception { + try { + _lock.lock(); + + switch (state) { + case SyncConnected: + ZkConnection zkConnection = (ZkConnection) _zkclient.getConnection(); + LOG.info("KeeperState: " + state + ", zookeeper:" + zkConnection.getZookeeper()); + break; + case Disconnected: + LOG.info("KeeperState:" + state + ", disconnectedSessionId: " + _sessionId); + + /** + * Track the time stamp that the disconnected happens, then check history and see if + * we should disconnect the helix-manager + */ + _disconnectTimeHistory.add(System.currentTimeMillis()); + if (isFlapping()) { + LOG.error("helix-connection: " + this + ", sessionId: " + _sessionId + + " is flapping. diconnect it. " + " maxDisconnectThreshold: " + + _maxDisconnectThreshold + " disconnects in " + _flappingTimeWindowMs + "ms"); + disconnect(); + } + break; + case Expired: + LOG.info("KeeperState:" + state + ", expiredSessionId: " + _sessionId); + break; + default: + break; + } + } finally { + _lock.unlock(); + } + } + + @Override + public void handleNewSession() throws Exception { + waitUntilConnected(); + + try { + _lock.lock(); + + for (final HelixConnectionStateListener listener : _connectionListener) { + try { + listener.onConnected(); + } catch (Exception e) { + LOG.error("Exception invoking connect on listener: " + listener, e); + } + } + } finally { + _lock.unlock(); + } + } + + @Override + public SessionId getSessionId() { + return _sessionId; + } + + @Override + public String getHelixVersion() { + return _version; + } + + @Override + public HelixManagerProperties getHelixProperties() { + return _properties; + } + + /** + * wait until we get a non-zero session-id. note that we might lose zkconnection + * right after we read session-id. but it's ok to get stale session-id and we will have + * another handle-new-session callback to correct this. + */ + private void waitUntilConnected() { + boolean isConnected; + do { + isConnected = + _zkclient.waitUntilConnected(ZkClient.DEFAULT_CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS); + if (!isConnected) { + LOG.error("fail to connect zkserver: " + _zkAddr + " in " + + ZkClient.DEFAULT_CONNECTION_TIMEOUT + "ms. expiredSessionId: " + _sessionId); + continue; + } + + ZkConnection zkConnection = ((ZkConnection) _zkclient.getConnection()); + _sessionId = SessionId.from(Long.toHexString(zkConnection.getZookeeper().getSessionId())); + + /** + * at the time we read session-id, zkconnection might be lost again + * wait until we get a non-zero session-id + */ + } while ("0".equals(_sessionId)); + + LOG.info("Handling new session, session id: " + _sessionId + ", zkconnection: " + + ((ZkConnection) _zkclient.getConnection()).getZookeeper()); + } + + @Override + public int getSessionTimeout() { + return _sessionTimeout; + } + + @Override + public ClusterMessagingService createMessagingService(HelixRole role) { + HelixManager manager = new HelixConnectionAdaptor(role); + return new DefaultMessagingService(manager); + } + + void addListener(HelixRole role, Object listener, PropertyKey propertyKey, ChangeType changeType, + EventType[] eventType) { + // checkConnected(); + PropertyType type = propertyKey.getType(); + + synchronized (this) { + if (!_handlers.containsKey(role)) { + _handlers.put(role, new CopyOnWriteArrayList()); + } + List handlerList = _handlers.get(role); + + for (ZkCallbackHandler handler : handlerList) { + // compare property-key path and listener reference + if (handler.getPath().equals(propertyKey.getPath()) + && handler.getListener().equals(listener)) { + LOG.info("role: " + role + ", listener: " + listener + " on path: " + + propertyKey.getPath() + " already exists. skip add"); + + return; + } + } + + ZkCallbackHandler newHandler = + new ZkCallbackHandler(role, _zkclient, propertyKey, listener, eventType, changeType); + + handlerList.add(newHandler); + LOG.info("role: " + role + " added listener: " + listener + " for type: " + type + + " to path: " + newHandler.getPath()); + } + } + + void initHandlers(HelixRole role) { + synchronized (this) { + List handlerList = _handlers.get(role); + + if (handlerList != null) { + for (ZkCallbackHandler handler : handlerList) { + handler.init(); + LOG.info("role: " + role + ", init handler: " + handler.getPath() + ", " + + handler.getListener()); + } + } + } + } + + void resetHandlers(HelixRole role) { + synchronized (this) { + List handlerList = _handlers.get(role); + + if (handlerList != null) { + for (ZkCallbackHandler handler : handlerList) { + handler.reset(); + LOG.info("role: " + role + ", reset handler: " + handler.getPath() + ", " + + handler.getListener()); + } + } + } + } + + /** + * If zk state has changed into DISCONNECTED for _maxDisconnectThreshold times during + * _timeWindowLengthMs time window, it's flapping and we tear down the zk-connection + */ + private boolean isFlapping() { + if (_disconnectTimeHistory.size() == 0) { + return false; + } + long mostRecentTimestamp = _disconnectTimeHistory.get(_disconnectTimeHistory.size() - 1); + + // Remove disconnect history timestamp that are older than _flappingTimeWindowMs ago + while ((_disconnectTimeHistory.get(0) + _flappingTimeWindowMs) < mostRecentTimestamp) { + _disconnectTimeHistory.remove(0); + } + return _disconnectTimeHistory.size() > _maxDisconnectThreshold; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java new file mode 100644 index 0000000000..7bbd490826 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java @@ -0,0 +1,254 @@ +package org.apache.helix.manager.zk; + +/* + * 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 org.I0Itec.zkclient.exception.ZkInterruptedException; +import org.apache.helix.ClusterMessagingService; +import org.apache.helix.HelixConnection; +import org.apache.helix.HelixController; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixException; +import org.apache.helix.HelixManager; +import org.apache.helix.HelixTimerTask; +import org.apache.helix.InstanceType; +import org.apache.helix.PropertyKey; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.api.id.Id; +import org.apache.helix.controller.GenericHelixController; +import org.apache.helix.messaging.DefaultMessagingService; +import org.apache.helix.messaging.handling.MessageHandlerFactory; +import org.apache.helix.model.Leader; +import org.apache.helix.model.MonitoringConfig; +import org.apache.helix.monitoring.MonitoringServer; +import org.apache.helix.monitoring.StatusDumpTask; +import org.apache.log4j.Logger; + +public class ZkHelixController implements HelixController { + private static Logger LOG = Logger.getLogger(ZkHelixController.class); + + final ZkHelixConnection _connection; + final ClusterId _clusterId; + final ControllerId _controllerId; + final GenericHelixController _pipeline; + final DefaultMessagingService _messagingService; + final List _timerTasks; + final ClusterAccessor _clusterAccessor; + final HelixDataAccessor _accessor; + final HelixManager _manager; + final ZkHelixLeaderElection _leaderElection; + + public ZkHelixController(ZkHelixConnection connection, ClusterId clusterId, + ControllerId controllerId) { + _connection = connection; + _clusterId = clusterId; + _controllerId = controllerId; + _pipeline = new GenericHelixController(); + _clusterAccessor = connection.createClusterAccessor(clusterId); + _accessor = connection.createDataAccessor(clusterId); + + _messagingService = (DefaultMessagingService) connection.createMessagingService(this); + _timerTasks = new ArrayList(); + + _manager = new HelixConnectionAdaptor(this); + _leaderElection = new ZkHelixLeaderElection(this, _pipeline); + + _timerTasks.add(new StatusDumpTask(clusterId, _manager.getHelixDataAccessor())); + + } + + void startTimerTasks() { + for (HelixTimerTask task : _timerTasks) { + task.start(); + } + } + + void stopTimerTasks() { + for (HelixTimerTask task : _timerTasks) { + task.stop(); + } + } + + @Override + public HelixConnection getConnection() { + return _connection; + } + + @Override + public void startAsync() { + _connection.addConnectionStateListener(this); + onConnected(); + } + + @Override + public void stopAsync() { + _connection.removeConnectionStateListener(this); + onDisconnecting(); + } + + void reset() { + /** + * reset all handlers, make sure cleanup completed for previous session + * disconnect if fail to cleanup + */ + _connection.resetHandlers(this); + + } + + void init() { + /** + * from here on, we are dealing with new session + * init handlers + */ + if (!_clusterAccessor.isClusterStructureValid()) { + throw new HelixException("Cluster structure is not set up for cluster: " + _clusterId); + } + + /** + * leader-election listener should be reset/init before all other controller listeners; + * it's ok to add a listener multiple times, since we check existence in + * ZkHelixConnection#addXXXListner() + */ + _connection.addControllerListener(this, _leaderElection, _clusterId); + + /** + * ok to init message handler and controller handlers twice + * the second init will be skipped (see CallbackHandler) + */ + _connection.initHandlers(this); + } + + @Override + public void onConnected() { + reset(); + init(); + } + + @Override + public void onDisconnecting() { + LOG.info("disconnecting " + _controllerId + "(" + getType() + ") from " + _clusterId); + + reset(); + } + + @Override + public ClusterMessagingService getMessagingService() { + return _messagingService; + } + + @Override + public ClusterId getClusterId() { + return _clusterId; + } + + @Override + public ControllerId getControllerId() { + return _controllerId; + } + + @Override + public Id getId() { + return getControllerId(); + } + + @Override + public InstanceType getType() { + return InstanceType.CONTROLLER; + } + + @Override + public boolean isLeader() { + PropertyKey.Builder keyBuilder = _accessor.keyBuilder(); + try { + Leader leader = _accessor.getProperty(keyBuilder.controllerLeader()); + if (leader != null) { + String leaderName = leader.getInstanceName(); + String sessionId = leader.getSessionId(); + if (leaderName != null && leaderName.equals(_controllerId.stringify()) && sessionId != null + && sessionId.equals(_connection.getSessionId().stringify())) { + return true; + } + } + } catch (Exception e) { + // log + } + return false; + } + + void addListenersToController(GenericHelixController pipeline) { + try { + /** + * setup controller message listener and register message handlers + */ + _connection.addControllerMessageListener(this, _messagingService.getExecutor(), _clusterId); + MessageHandlerFactory defaultControllerMsgHandlerFactory = + new DefaultControllerMessageHandlerFactory(); + _messagingService.getExecutor().registerMessageHandlerFactory( + defaultControllerMsgHandlerFactory.getMessageType(), defaultControllerMsgHandlerFactory); + MessageHandlerFactory defaultSchedulerMsgHandlerFactory = + new DefaultSchedulerMessageHandlerFactory(_manager); + _messagingService.getExecutor().registerMessageHandlerFactory( + defaultSchedulerMsgHandlerFactory.getMessageType(), defaultSchedulerMsgHandlerFactory); + MessageHandlerFactory defaultParticipantErrorMessageHandlerFactory = + new DefaultParticipantErrorMessageHandlerFactory(_manager); + _messagingService.getExecutor().registerMessageHandlerFactory( + defaultParticipantErrorMessageHandlerFactory.getMessageType(), + defaultParticipantErrorMessageHandlerFactory); + + /** + * setup generic-controller + */ + _connection.addConfigChangeListener(this, pipeline, _clusterId); + _connection.addLiveInstanceChangeListener(this, pipeline, _clusterId); + _connection.addIdealStateChangeListener(this, pipeline, _clusterId); + _connection.addControllerListener(this, pipeline, _clusterId); + } catch (ZkInterruptedException e) { + LOG.warn("zk connection is interrupted during addListenersToController()", e); + } catch (Exception e) { + LOG.error("Error addListenersToController", e); + } + } + + void removeListenersFromController(GenericHelixController pipeline) { + PropertyKey.Builder keyBuilder = new PropertyKey.Builder(_manager.getClusterName()); + /** + * reset generic-controller + */ + _connection.removeListener(this, pipeline, keyBuilder.instanceConfigs()); + _connection.removeListener(this, pipeline, keyBuilder.liveInstances()); + _connection.removeListener(this, pipeline, keyBuilder.idealStates()); + _connection.removeListener(this, pipeline, keyBuilder.controller()); + + /** + * reset controller message listener and unregister all message handlers + */ + _connection.removeListener(this, _messagingService.getExecutor(), + keyBuilder.controllerMessages()); + } + + HelixManager getManager() { + return _manager; + } + +} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixLeaderElection.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixLeaderElection.java new file mode 100644 index 0000000000..c69d1a3532 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixLeaderElection.java @@ -0,0 +1,187 @@ +package org.apache.helix.manager.zk; + +/* + * 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.lang.management.ManagementFactory; + +import org.apache.helix.ControllerChangeListener; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixManager; +import org.apache.helix.InstanceType; +import org.apache.helix.NotificationContext; +import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyType; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.controller.GenericHelixController; +import org.apache.helix.controller.restlet.ZKPropertyTransferServer; +import org.apache.helix.model.Leader; +import org.apache.helix.model.LeaderHistory; +import org.apache.helix.monitoring.MonitoringServer; +import org.apache.log4j.Logger; + +// TODO GenericHelixController has a controller-listener, we can invoke leader-election from there +public class ZkHelixLeaderElection implements ControllerChangeListener { + private static Logger LOG = Logger.getLogger(ZkHelixLeaderElection.class); + + final ZkHelixController _controller; + final ClusterId _clusterId; + final ControllerId _controllerId; + final HelixManager _manager; + final GenericHelixController _pipeline; + + public ZkHelixLeaderElection(ZkHelixController controller, GenericHelixController pipeline) { + _controller = controller; + _clusterId = controller.getClusterId(); + _controllerId = controller.getControllerId(); + _pipeline = pipeline; + _manager = controller.getManager(); + } + + /** + * may be accessed by multiple threads: zk-client thread and + * ZkHelixManager.disconnect()->reset() TODO: Refactor accessing + * HelixMangerMain class statically + */ + @Override + public synchronized void onControllerChange(NotificationContext changeContext) { + HelixManager manager = changeContext.getManager(); + if (manager == null) { + LOG.error("missing attributes in changeContext. requires HelixManager"); + return; + } + + InstanceType type = _manager.getInstanceType(); + if (type != InstanceType.CONTROLLER && type != InstanceType.CONTROLLER_PARTICIPANT) { + LOG.error("fail to become controller because incorrect instanceType (was " + type.toString() + + ", requires CONTROLLER | CONTROLLER_PARTICIPANT)"); + return; + } + + try { + HelixDataAccessor accessor = _manager.getHelixDataAccessor(); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + if (changeContext.getType().equals(NotificationContext.Type.INIT) + || changeContext.getType().equals(NotificationContext.Type.CALLBACK)) { + LOG.info(_controllerId + " is trying to acquire leadership for cluster: " + _clusterId); + + while (accessor.getProperty(keyBuilder.controllerLeader()) == null) { + boolean success = tryUpdateController(_manager); + if (success) { + LOG.info(_controllerId + " acquires leadership of cluster: " + _clusterId); + + updateHistory(_manager); + _manager.getHelixDataAccessor().getBaseDataAccessor().reset(); + _controller.addListenersToController(_pipeline); + _controller.startTimerTasks(); + } + } + } else if (changeContext.getType().equals(NotificationContext.Type.FINALIZE)) { + LOG.info(_controllerId + " reqlinquishes leadership of cluster: " + _clusterId); + _controller.stopTimerTasks(); + _controller.removeListenersFromController(_pipeline); + + /** + * clear write-through cache + */ + accessor.getBaseDataAccessor().reset(); + + /** + * remove leader ephemeral znode if this is the leader + * note that session expiry may happen during checking leader and remove leader + * in this race condition, we may remove a leader node created by another controller + * this is fine since it will just invoke another round of leader-election + */ + if (_controller.isLeader()) { + accessor.removeProperty(keyBuilder.controllerLeader()); + } + } + + } catch (Exception e) { + LOG.error("Exception when trying to become leader", e); + } + } + + private boolean tryUpdateController(HelixManager manager) { + return tryBecomingLeader(manager); + } + + private void updateHistory(HelixManager manager) { + HelixDataAccessor accessor = manager.getHelixDataAccessor(); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + LeaderHistory history = accessor.getProperty(keyBuilder.controllerLeaderHistory()); + if (history == null) { + history = new LeaderHistory(PropertyType.HISTORY.toString()); + } + history.updateHistory(manager.getClusterName(), manager.getInstanceName()); + accessor.setProperty(keyBuilder.controllerLeaderHistory(), history); + } + + /** + * Try to make this controller the leader + * @param manager HelixManager connection for the controller + * @return true if the controller is the leader, false otherwise + */ + public static boolean tryBecomingLeader(HelixManager manager) { + HelixDataAccessor accessor = manager.getHelixDataAccessor(); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + Leader leader = new Leader(ControllerId.from(manager.getInstanceName())); + try { + leader.setLiveInstance(ManagementFactory.getRuntimeMXBean().getName()); + leader.setSessionId(manager.getSessionId()); + leader.setHelixVersion(manager.getVersion()); + if (ZKPropertyTransferServer.getInstance() != null) { + String zkPropertyTransferServiceUrl = + ZKPropertyTransferServer.getInstance().getWebserviceUrl(); + if (zkPropertyTransferServiceUrl != null) { + leader.setWebserviceUrl(zkPropertyTransferServiceUrl); + } + } else { + LOG.warn("ZKPropertyTransferServer instance is null"); + } + + boolean success = accessor.createProperty(keyBuilder.controllerLeader(), leader); + if (success) { + return true; + } else { + LOG.info("Unable to become leader probably because some other controller became the leader"); + } + } catch (Exception e) { + LOG.error( + "Exception when trying to updating leader record in cluster:" + manager.getClusterName() + + ". Need to check again whether leader node has been created or not", e); + } + + leader = accessor.getProperty(keyBuilder.controllerLeader()); + if (leader != null) { + String leaderSessionId = leader.getSessionId(); + LOG.info("Leader exists for cluster: " + manager.getClusterName() + ", currentLeader: " + + leader.getInstanceName() + ", leaderSessionId: " + leaderSessionId); + + if (leaderSessionId != null && leaderSessionId.equals(manager.getSessionId())) { + return true; + } + } + return false; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java new file mode 100644 index 0000000000..6bb33c0530 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java @@ -0,0 +1,478 @@ +package org.apache.helix.manager.zk; + +/* + * 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.lang.management.ManagementFactory; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.apache.helix.AccessOption; +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.ClusterMessagingService; +import org.apache.helix.ConfigAccessor; +import org.apache.helix.HelixConnection; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixException; +import org.apache.helix.HelixManager; +import org.apache.helix.HelixParticipant; +import org.apache.helix.HelixTimerTask; +import org.apache.helix.InstanceType; +import org.apache.helix.LiveInstanceInfoProvider; +import org.apache.helix.PreConnectCallback; +import org.apache.helix.PropertyKey; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.accessor.ParticipantAccessor; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.Id; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.messaging.DefaultMessagingService; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.HelixConfigScope; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.model.builder.HelixConfigScopeBuilder; +import org.apache.helix.participant.HelixStateMachineEngine; +import org.apache.helix.participant.StateMachineEngine; +import org.apache.helix.participant.statemachine.ScheduledTaskStateModelFactory; +import org.apache.log4j.Logger; +import org.apache.zookeeper.data.Stat; + +public class ZkHelixParticipant implements HelixParticipant { + private static Logger LOG = Logger.getLogger(ZkHelixParticipant.class); + + final ZkHelixConnection _connection; + final ClusterId _clusterId; + final ParticipantId _participantId; + final ZKHelixDataAccessor _accessor; + final BaseDataAccessor _baseAccessor; + final PropertyKey.Builder _keyBuilder; + final ConfigAccessor _configAccessor; + final ClusterAccessor _clusterAccessor; + final ParticipantAccessor _participantAccessor; + final DefaultMessagingService _messagingService; + final List _preConnectCallbacks; + final List _timerTasks; + + /** + * state-transition message handler factory for helix-participant + */ + final StateMachineEngine _stateMachineEngine; + + LiveInstanceInfoProvider _liveInstanceInfoProvider; + + public ZkHelixParticipant(ZkHelixConnection connection, ClusterId clusterId, + ParticipantId participantId) { + _connection = connection; + _accessor = (ZKHelixDataAccessor) connection.createDataAccessor(clusterId); + _baseAccessor = _accessor.getBaseDataAccessor(); + _keyBuilder = _accessor.keyBuilder(); + _clusterAccessor = connection.createClusterAccessor(clusterId); + _participantAccessor = connection.createParticipantAccessor(clusterId); + _configAccessor = connection.getConfigAccessor(); + + _clusterId = clusterId; + _participantId = participantId; + + _messagingService = (DefaultMessagingService) connection.createMessagingService(this); + HelixManager manager = new HelixConnectionAdaptor(this); + _stateMachineEngine = new HelixStateMachineEngine(manager); + _preConnectCallbacks = new ArrayList(); + _timerTasks = new ArrayList(); + } + + @Override + public ClusterId getClusterId() { + return _clusterId; + } + + @Override + public ParticipantId getParticipantId() { + return _participantId; + } + + @Override + public HelixConnection getConnection() { + return _connection; + } + + void startTimerTasks() { + for (HelixTimerTask task : _timerTasks) { + task.start(); + } + } + + void stopTimerTasks() { + for (HelixTimerTask task : _timerTasks) { + task.stop(); + } + } + + void reset() { + /** + * stop timer tasks, reset all handlers, make sure cleanup completed for previous session, + * disconnect if cleanup fails + */ + stopTimerTasks(); + _connection.resetHandlers(this); + + /** + * clear write-through cache + */ + _accessor.getBaseDataAccessor().reset(); + } + + private void createLiveInstance() { + String liveInstancePath = _keyBuilder.liveInstance(_participantId.stringify()).getPath(); + String sessionId = _connection.getSessionId().stringify(); + LiveInstance liveInstance = new LiveInstance(_participantId.stringify()); + liveInstance.setSessionId(sessionId); + liveInstance.setHelixVersion(_connection.getHelixVersion()); + liveInstance.setLiveInstance(ManagementFactory.getRuntimeMXBean().getName()); + + if (_liveInstanceInfoProvider != null) { + LOG.info("Additional live instance information is provided: " + _liveInstanceInfoProvider); + ZNRecord additionalLiveInstanceInfo = + _liveInstanceInfoProvider.getAdditionalLiveInstanceInfo(); + if (additionalLiveInstanceInfo != null) { + additionalLiveInstanceInfo.merge(liveInstance.getRecord()); + ZNRecord mergedLiveInstance = + new ZNRecord(additionalLiveInstanceInfo, _participantId.stringify()); + liveInstance = new LiveInstance(mergedLiveInstance); + LOG.info("Participant: " + _participantId + ", mergedLiveInstance: " + liveInstance); + } + } + + boolean retry; + do { + retry = false; + boolean success = + _baseAccessor.create(liveInstancePath, liveInstance.getRecord(), AccessOption.EPHEMERAL); + if (!success) { + LOG.warn("found another participant with same name: " + _participantId + " in cluster " + + _clusterId); + + Stat stat = new Stat(); + ZNRecord record = _baseAccessor.get(liveInstancePath, stat, 0); + if (record == null) { + /** + * live-instance is gone as we check it, retry create live-instance + */ + retry = true; + } else { + String ephemeralOwner = Long.toHexString(stat.getEphemeralOwner()); + if (ephemeralOwner.equals(sessionId)) { + /** + * update sessionId field in live-instance if necessary + */ + LiveInstance curLiveInstance = new LiveInstance(record); + if (!curLiveInstance.getSessionId().equals(sessionId)) { + /** + * in last handle-new-session, + * live-instance is created by new zkconnection with stale session-id inside + * just update session-id field + */ + LOG.info("overwriting session-id by ephemeralOwner: " + ephemeralOwner + + ", old-sessionId: " + curLiveInstance.getSessionId() + ", new-sessionId: " + + sessionId); + + curLiveInstance.setSessionId(sessionId); + success = + _baseAccessor.set(liveInstancePath, curLiveInstance.getRecord(), + stat.getVersion(), AccessOption.EPHEMERAL); + if (!success) { + LOG.error("Someone changes sessionId as we update, should not happen"); + throw new HelixException("fail to create live-instance for " + _participantId); + } + } + } else { + /** + * wait for a while, in case previous helix-participant exits unexpectedly + * and its live-instance still hangs around until session timeout + */ + try { + TimeUnit.MILLISECONDS.sleep(_connection.getSessionTimeout() + 5000); + } catch (InterruptedException ex) { + LOG.warn("Sleep interrupted while waiting for previous live-instance to go away", ex); + } + /** + * give a last try after exit while loop + */ + retry = true; + break; + } + } + } + } while (retry); + + /** + * give a last shot + */ + if (retry) { + boolean success = + _baseAccessor.create(liveInstancePath, liveInstance.getRecord(), AccessOption.EPHEMERAL); + if (!success) { + LOG.error("instance: " + _participantId + " already has a live-instance in cluster " + + _clusterId); + throw new HelixException("fail to create live-instance for " + _participantId); + } + } + } + + /** + * carry over current-states from last sessions + * set to initial state for current session only when state doesn't exist in current session + */ + private void carryOverPreviousCurrentState() { + String sessionId = _connection.getSessionId().stringify(); + String participantName = _participantId.stringify(); + List sessions = _accessor.getChildNames(_keyBuilder.sessions(participantName)); + + for (String session : sessions) { + if (session.equals(sessionId)) { + continue; + } + + List lastCurStates = + _accessor.getChildValues(_keyBuilder.currentStates(participantName, session)); + + for (CurrentState lastCurState : lastCurStates) { + LOG.info("Carrying over old session: " + session + ", resource: " + lastCurState.getId() + + " to current session: " + sessionId); + String stateModelDefRef = lastCurState.getStateModelDefRef(); + if (stateModelDefRef == null) { + LOG.error("skip carry-over because previous current state doesn't have a state model definition. previous current-state: " + + lastCurState); + continue; + } + StateModelDefinition stateModel = + _accessor.getProperty(_keyBuilder.stateModelDef(stateModelDefRef)); + + String curStatePath = + _keyBuilder.currentState(participantName, sessionId, lastCurState.getResourceName()) + .getPath(); + _accessor.getBaseDataAccessor().update( + curStatePath, + new CurStateCarryOverUpdater(sessionId, stateModel.getInitialState(), lastCurState), + AccessOption.PERSISTENT); + } + } + + /** + * remove previous current state parent nodes + */ + for (String session : sessions) { + if (session.equals(sessionId)) { + continue; + } + + PropertyKey key = _keyBuilder.currentStates(participantName, session); + LOG.info("Removing current states from previous sessions. path: " + key.getPath()); + _accessor.removeProperty(key); + } + } + + /** + * Read cluster config and see if instance can auto join the cluster + */ + private void joinCluster() { + boolean autoJoin = false; + try { + HelixConfigScope scope = + new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster( + _clusterId.stringify()).build(); + autoJoin = + Boolean + .parseBoolean(_configAccessor.get(scope, HelixManager.ALLOW_PARTICIPANT_AUTO_JOIN)); + LOG.info("instance: " + _participantId + " auto-joining " + _clusterId + " is " + autoJoin); + } catch (Exception e) { + // autoJoin is false + } + + if (!_participantAccessor.isParticipantStructureValid(_participantId)) { + if (!autoJoin) { + throw new HelixException("Initial cluster structure is not set up for instance: " + + _participantId + ", instanceType: " + getType()); + } else { + LOG.info(_participantId + " is auto-joining cluster: " + _clusterId); + String participantName = _participantId.stringify(); + String hostName = participantName; + int port = -1; + int lastPos = participantName.lastIndexOf("_"); + if (lastPos > 0) { + hostName = participantName.substring(0, lastPos); + try { + port = Integer.parseInt(participantName.substring(lastPos + 1)); + } catch (Exception e) { + // use port = -1 + } + } + ParticipantConfig.Builder builder = + new ParticipantConfig.Builder(_participantId).hostName(hostName).port(port) + .enabled(true); + _clusterAccessor.addParticipantToCluster(builder.build()); + } + } + } + + private void setupMsgHandler() { + _messagingService.registerMessageHandlerFactory(MessageType.STATE_TRANSITION.toString(), + _stateMachineEngine); + + /** + * it's ok to add a listener multiple times, since we check existence in + * ZkHelixConnection#addXXXListner() + */ + _connection.addMessageListener(this, _messagingService.getExecutor(), _clusterId, + _participantId); + _connection.addControllerListener(this, _accessor, _clusterId); + + ScheduledTaskStateModelFactory stStateModelFactory = + new ScheduledTaskStateModelFactory(_messagingService.getExecutor()); + _stateMachineEngine.registerStateModelFactory( + DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, stStateModelFactory); + } + + void init() { + /** + * from here on, we are dealing with new session + */ + if (!_clusterAccessor.isClusterStructureValid()) { + throw new HelixException("Cluster structure is not set up for cluster: " + _clusterId); + } + + /** + * auto-join + */ + joinCluster(); + + /** + * Invoke PreConnectCallbacks + */ + for (PreConnectCallback callback : _preConnectCallbacks) { + callback.onPreConnect(); + } + + createLiveInstance(); + carryOverPreviousCurrentState(); + + /** + * setup message listener + */ + setupMsgHandler(); + + /** + * start health check timer task + */ + startTimerTasks(); + + /** + * init handlers + * ok to init message handler and data-accessor twice + * the second init will be skipped (see CallbackHandler) + */ + _connection.initHandlers(this); + } + + @Override + public void onConnected() { + reset(); + init(); + } + + @Override + public void onDisconnecting() { + LOG.info("disconnecting " + _participantId + "(" + getType() + ") from " + _clusterId); + + reset(); + + /** + * shall we shutdown thread pool first to avoid reset() being invoked in the middle of state + * transition? + */ + _messagingService.getExecutor().shutdown(); + _accessor.shutdown(); + + /** + * remove live instance ephemeral znode + */ + _accessor.removeProperty(_keyBuilder.liveInstance(_participantId.stringify())); + } + + @Override + public void startAsync() { + _connection.addConnectionStateListener(this); + onConnected(); + } + + @Override + public void stopAsync() { + _connection.removeConnectionStateListener(this); + onDisconnecting(); + } + + @Override + public ClusterMessagingService getMessagingService() { + return _messagingService; + } + + @Override + public StateMachineEngine getStateMachineEngine() { + return _stateMachineEngine; + } + + @Override + public Id getId() { + return getParticipantId(); + } + + @Override + public InstanceType getType() { + return InstanceType.PARTICIPANT; + } + + @Override + public void addPreConnectCallback(PreConnectCallback callback) { + LOG.info("Adding preconnect callback: " + callback); + _preConnectCallbacks.add(callback); + } + + @Override + public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) { + _liveInstanceInfoProvider = liveInstanceInfoProvider; + } + + public HelixDataAccessor getAccessor() { + return _accessor; + } + + public ClusterAccessor getClusterAccessor() { + return _clusterAccessor; + } + + public ParticipantAccessor getParticipantAccessor() { + return _participantAccessor; + } + +} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkStateChangeListener.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkStateChangeListener.java deleted file mode 100644 index e11444b458..0000000000 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkStateChangeListener.java +++ /dev/null @@ -1,127 +0,0 @@ -package org.apache.helix.manager.zk; - -/* - * 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.LinkedList; -import java.util.List; - -import org.I0Itec.zkclient.IZkStateListener; -import org.I0Itec.zkclient.ZkConnection; -import org.apache.log4j.Logger; -import org.apache.zookeeper.Watcher.Event.KeeperState; - -public class ZkStateChangeListener implements IZkStateListener { - private volatile boolean _isConnected; - private volatile boolean _hasSessionExpired; - private final ZKHelixManager _zkHelixManager; - - // Keep track of timestamps that zk State has become Disconnected - // If in a _timeWindowLengthMs window zk State has become Disconnected - // for more than_maxDisconnectThreshold times disconnect the zkHelixManager - List _disconnectTimeHistory = new LinkedList(); - int _timeWindowLengthMs; - int _maxDisconnectThreshold; - - private static Logger logger = Logger.getLogger(ZkStateChangeListener.class); - - public ZkStateChangeListener(ZKHelixManager zkHelixManager, int timeWindowLengthMs, - int maxDisconnectThreshold) { - this._zkHelixManager = zkHelixManager; - _timeWindowLengthMs = timeWindowLengthMs; - // _maxDisconnectThreshold min value is 1. - // We don't want to disconnect from zk for the first time zkState become Disconnected - _maxDisconnectThreshold = maxDisconnectThreshold > 0 ? maxDisconnectThreshold : 1; - } - - @Override - public void handleNewSession() { - // TODO:bug in zkclient . - // zkclient does not invoke handleStateChanged when a session expires but - // directly invokes handleNewSession - _isConnected = true; - _hasSessionExpired = false; - _zkHelixManager.handleNewSession(); - } - - @Override - public void handleStateChanged(KeeperState keeperState) throws Exception { - switch (keeperState) { - case SyncConnected: - ZkConnection zkConnection = ((ZkConnection) _zkHelixManager._zkClient.getConnection()); - logger.info("KeeperState: " + keeperState + ", zookeeper:" + zkConnection.getZookeeper()); - _isConnected = true; - break; - case Disconnected: - logger.info("KeeperState:" + keeperState + ", disconnectedSessionId: " - + _zkHelixManager._sessionId + ", instance: " + _zkHelixManager.getInstanceName() - + ", type: " + _zkHelixManager.getInstanceType()); - - _isConnected = false; - // Track the time stamp that the disconnected happens, then check history and see if - // we should disconnect the _zkHelixManager - _disconnectTimeHistory.add(System.currentTimeMillis()); - if (isFlapping()) { - logger.error("isFlapping() returns true, so disconnect the helix manager. " - + _zkHelixManager.getInstanceName() + " " + _maxDisconnectThreshold - + " disconnects in " + _timeWindowLengthMs + " Ms."); - _zkHelixManager.disconnectInternal(); - } - break; - case Expired: - logger.info("KeeperState:" + keeperState + ", expiredSessionId: " - + _zkHelixManager._sessionId + ", instance: " + _zkHelixManager.getInstanceName() - + ", type: " + _zkHelixManager.getInstanceType()); - - _isConnected = false; - _hasSessionExpired = true; - break; - } - } - - boolean isConnected() { - return _isConnected; - } - - void disconnect() { - _isConnected = false; - } - - boolean hasSessionExpired() { - return _hasSessionExpired; - } - - /** - * If zk state has changed into Disconnected for _maxDisconnectThreshold times during previous - * _timeWindowLengthMs Ms - * time window, we think that there are something wrong going on and disconnect the zkHelixManager - * from zk. - */ - boolean isFlapping() { - if (_disconnectTimeHistory.size() == 0) { - return false; - } - long mostRecentTimestamp = _disconnectTimeHistory.get(_disconnectTimeHistory.size() - 1); - // Remove disconnect history timestamp that are older than _timeWindowLengthMs ago - while ((_disconnectTimeHistory.get(0) + _timeWindowLengthMs) < mostRecentTimestamp) { - _disconnectTimeHistory.remove(0); - } - return _disconnectTimeHistory.size() > _maxDisconnectThreshold; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/serializer/JacksonPayloadSerializer.java b/helix-core/src/main/java/org/apache/helix/manager/zk/serializer/JacksonPayloadSerializer.java index 13609f1911..41470127ff 100644 --- a/helix-core/src/main/java/org/apache/helix/manager/zk/serializer/JacksonPayloadSerializer.java +++ b/helix-core/src/main/java/org/apache/helix/manager/zk/serializer/JacksonPayloadSerializer.java @@ -23,7 +23,6 @@ import java.io.StringWriter; import org.apache.helix.HelixException; -import org.apache.helix.manager.zk.serializer.PayloadSerializer; import org.apache.log4j.Logger; import org.codehaus.jackson.map.DeserializationConfig; import org.codehaus.jackson.map.ObjectMapper; diff --git a/helix-core/src/main/java/org/apache/helix/messaging/AsyncCallback.java b/helix-core/src/main/java/org/apache/helix/messaging/AsyncCallback.java index f9743a42b6..9fea0c8547 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/AsyncCallback.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/AsyncCallback.java @@ -71,7 +71,7 @@ public void setInterrupted(boolean b) { } public synchronized final void onReply(Message message) { - _logger.info("OnReply msg " + message.getMsgId()); + _logger.info("OnReply msg " + message.getMessageId()); if (!isDone()) { _messageReplied.add(message); try { diff --git a/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java b/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java index 2eec354244..422d35e6ca 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java @@ -28,19 +28,23 @@ import org.apache.helix.ClusterMessagingService; import org.apache.helix.ConfigAccessor; -import org.apache.helix.model.ConfigScope; -import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.helix.Criteria; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.messaging.handling.AsyncCallbackService; import org.apache.helix.messaging.handling.HelixTaskExecutor; import org.apache.helix.messaging.handling.MessageHandlerFactory; +import org.apache.helix.model.ConfigScope; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.log4j.Logger; public class DefaultMessagingService implements ClusterMessagingService { @@ -113,9 +117,6 @@ public int send(final Criteria recipientCriteria, final Message message, Builder keyBuilder = accessor.keyBuilder(); if (receiverType == InstanceType.CONTROLLER) { - // _manager.getDataAccessor().setProperty(PropertyType.MESSAGES_CONTROLLER, - // tempMessage, - // tempMessage.getId()); accessor.setProperty(keyBuilder.controllerMessage(tempMessage.getId()), tempMessage); } @@ -133,6 +134,7 @@ public int send(final Criteria recipientCriteria, final Message message, return totalMessageCount; } + @Override public Map> generateMessage(final Criteria recipientCriteria, final Message message) { Map> messagesToSendMap = new HashMap>(); @@ -157,11 +159,12 @@ public Map> generateMessage(final Criteria recipient List liveInstances = accessor.getChildValues(keyBuilder.liveInstances()); for (LiveInstance liveInstance : liveInstances) { - sessionIdMap.put(liveInstance.getInstanceName(), liveInstance.getSessionId()); + sessionIdMap.put(liveInstance.getInstanceName(), liveInstance.getTypedSessionId() + .stringify()); } } for (Map map : matchedList) { - String id = UUID.randomUUID().toString(); + MessageId id = MessageId.from(UUID.randomUUID().toString()); Message newMessage = new Message(message.getRecord(), id); String srcInstanceName = _manager.getInstanceName(); String tgtInstanceName = map.get("instanceName"); @@ -172,10 +175,10 @@ public Map> generateMessage(final Criteria recipient } newMessage.setSrcName(srcInstanceName); newMessage.setTgtName(tgtInstanceName); - newMessage.setResourceName(map.get("resourceName")); - newMessage.setPartitionName(map.get("partitionName")); + newMessage.setResourceId(ResourceId.from(map.get("resourceName"))); + newMessage.setPartitionId(PartitionId.from(map.get("partitionName"))); if (recipientCriteria.isSessionSpecific()) { - newMessage.setTgtSessionId(sessionIdMap.get(tgtInstanceName)); + newMessage.setTgtSessionId(SessionId.from(sessionIdMap.get(tgtInstanceName))); } messages.add(newMessage); } @@ -187,9 +190,9 @@ public Map> generateMessage(final Criteria recipient private List generateMessagesForController(Message message) { List messages = new ArrayList(); - String id = UUID.randomUUID().toString(); + MessageId id = MessageId.from(UUID.randomUUID().toString()); Message newMessage = new Message(message.getRecord(), id); - newMessage.setMsgId(id); + newMessage.setMessageId(id); newMessage.setSrcName(_manager.getInstanceName()); newMessage.setTgtName("Controller"); messages.add(newMessage); @@ -262,7 +265,8 @@ void registerMessageHandlerFactoryInternal(String type, MessageHandlerFactory fa public void sendNopMessage() { if (_manager.isConnected()) { try { - Message nopMsg = new Message(MessageType.NO_OP, UUID.randomUUID().toString()); + Message nopMsg = + new Message(MessageType.NO_OP, MessageId.from(UUID.randomUUID().toString())); nopMsg.setSrcName(_manager.getInstanceName()); HelixDataAccessor accessor = _manager.getHelixDataAccessor(); diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java index c218a159e6..17fc67da7d 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java @@ -19,22 +19,13 @@ * under the License. */ -import java.util.Collection; -import java.util.Date; -import java.util.Map; -import java.util.Random; -import java.util.Timer; -import java.util.TimerTask; import java.util.concurrent.ConcurrentHashMap; import org.apache.helix.HelixException; import org.apache.helix.NotificationContext; import org.apache.helix.messaging.AsyncCallback; -import org.apache.helix.messaging.handling.MessageHandler.ErrorCode; -import org.apache.helix.messaging.handling.MessageHandler.ErrorType; import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageType; -import org.apache.helix.participant.HelixStateMachineEngine; import org.apache.log4j.Logger; public class AsyncCallbackService implements MessageHandlerFactory { @@ -56,14 +47,14 @@ public void registerAsyncCallback(String correlationId, AsyncCallback callback) void verifyMessage(Message message) { if (!message.getMsgType().toString().equalsIgnoreCase(MessageType.TASK_REPLY.toString())) { String errorMsg = - "Unexpected msg type for message " + message.getMsgId() + " type:" + message.getMsgType() + "Unexpected msg type for message " + message.getMessageId() + " type:" + message.getMsgType() + " Expected : " + MessageType.TASK_REPLY; _logger.error(errorMsg); throw new HelixException(errorMsg); } String correlationId = message.getCorrelationId(); if (correlationId == null) { - String errorMsg = "Message " + message.getMsgId() + " does not have correlation id"; + String errorMsg = "Message " + message.getMessageId() + " does not have correlation id"; _logger.error(errorMsg); throw new HelixException(errorMsg); } @@ -71,13 +62,13 @@ void verifyMessage(Message message) { if (!_callbackMap.containsKey(correlationId)) { String errorMsg = "Message " - + message.getMsgId() + + message.getMessageId() + " does not have correponding callback. Probably timed out already. Correlation id: " + correlationId; _logger.error(errorMsg); throw new HelixException(errorMsg); } - _logger.info("Verified reply message " + message.getMsgId() + " correlation:" + correlationId); + _logger.info("Verified reply message " + message.getMessageId() + " correlation:" + correlationId); } @Override @@ -110,7 +101,7 @@ public HelixTaskResult handleMessage() throws InterruptedException { verifyMessage(_message); HelixTaskResult result = new HelixTaskResult(); assert (_correlationId.equalsIgnoreCase(_message.getCorrelationId())); - _logger.info("invoking reply message " + _message.getMsgId() + ", correlationid:" + _logger.info("invoking reply message " + _message.getMessageId() + ", correlationid:" + _correlationId); AsyncCallback callback = _callbackMap.get(_correlationId); @@ -127,7 +118,7 @@ public HelixTaskResult handleMessage() throws InterruptedException { @Override public void onError(Exception e, ErrorCode code, ErrorType type) { - _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMsgId(), e); + _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMessageId(), e); } } } diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java index b50614811d..5bb67fd74a 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java @@ -35,6 +35,8 @@ import org.apache.helix.NotificationContext; import org.apache.helix.NotificationContext.MapKey; import org.apache.helix.PropertyKey; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.model.CurrentState; import org.apache.helix.model.Message; import org.apache.helix.model.Message.Attributes; @@ -63,11 +65,12 @@ public BatchMessageHandler(Message msg, NotificationContext context, MessageHand // create sub-messages _subMessages = new ArrayList(); - List partitionKeys = _message.getPartitionNames(); - for (String partitionKey : partitionKeys) { + List partitionKeys = _message.getPartitionIds(); + for (PartitionId partitionKey : partitionKeys) { // assign a new message id, put batch-msg-id to parent-id field - Message subMsg = new Message(_message.getRecord(), UUID.randomUUID().toString()); - subMsg.setPartitionName(partitionKey); + Message subMsg = + new Message(_message.getRecord(), MessageId.from(UUID.randomUUID().toString())); + subMsg.setPartitionId(partitionKey); subMsg.setAttribute(Attributes.PARENT_MSG_ID, _message.getId()); subMsg.setBatchMessageMode(false); @@ -131,7 +134,7 @@ public HelixTaskResult handleMessage() { preHandleMessage(); int exeBatchSize = 1; // TODO: getExeBatchSize from msg - List partitionKeys = _message.getPartitionNames(); + List partitionKeys = _message.getPartitionIds(); for (int i = 0; i < partitionKeys.size(); i += exeBatchSize) { if (i + exeBatchSize <= partitionKeys.size()) { List msgs = _subMessages.subList(i, i + exeBatchSize); diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/GroupMessageHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/GroupMessageHandler.java index 68729073f9..81b5155ea9 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/handling/GroupMessageHandler.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/GroupMessageHandler.java @@ -26,6 +26,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.helix.PropertyKey; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.model.CurrentState; import org.apache.helix.model.Message; import org.apache.helix.model.Message.Attributes; @@ -52,7 +53,7 @@ static class GroupMessageInfo { public GroupMessageInfo(Message message) { _message = message; - List partitionNames = message.getPartitionNames(); + List partitionNames = message.getPartitionIds(); _countDown = new AtomicInteger(partitionNames.size()); _curStateUpdateList = new ConcurrentLinkedQueue(); } diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java index 627babcb11..8381f4af0c 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java @@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentHashMap; import org.apache.helix.HelixAdmin; -import org.apache.helix.HelixConstants; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixDefinedState; import org.apache.helix.HelixException; @@ -36,10 +35,14 @@ import org.apache.helix.NotificationContext; import org.apache.helix.NotificationContext.MapKey; import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.ZNRecordBucketizer; import org.apache.helix.ZNRecordDelta; -import org.apache.helix.PropertyKey.Builder; import org.apache.helix.ZNRecordDelta.MergeOperation; +import org.apache.helix.api.State; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.model.CurrentState; import org.apache.helix.model.Message; import org.apache.helix.model.Message.Attributes; @@ -92,18 +95,18 @@ void preHandleMessage() throws Exception { HelixDataAccessor accessor = _manager.getHelixDataAccessor(); - String partitionName = _message.getPartitionName(); - String fromState = _message.getFromState(); + PartitionId partitionId = _message.getPartitionId(); + State fromState = _message.getTypedFromState(); // Verify the fromState and current state of the stateModel - String state = _currentStateDelta.getState(partitionName); + String state = _currentStateDelta.getState(partitionId.stringify()); - if (fromState != null && !fromState.equals("*") && !fromState.equalsIgnoreCase(state)) { + if (fromState != null && !fromState.equals("*") + && !fromState.toString().equalsIgnoreCase(state)) { String errorMessage = "Current state of stateModel does not match the fromState in Message" + ", Current State:" + state + ", message expected:" + fromState + ", partition: " - + partitionName + ", from: " + _message.getMsgSrc() + ", to: " - + _message.getTgtName(); + + partitionId + ", from: " + _message.getMsgSrc() + ", to: " + _message.getTgtName(); _statusUpdateUtil.logError(_message, HelixStateTransitionHandler.class, errorMessage, accessor); @@ -117,9 +120,9 @@ void postHandleMessage() { (HelixTaskResult) _notificationContext.get(MapKey.HELIX_TASK_RESULT.toString()); Exception exception = taskResult.getException(); - String partitionKey = _message.getPartitionName(); - String resource = _message.getResourceName(); - String sessionId = _message.getTgtSessionId(); + PartitionId partitionId = _message.getPartitionId(); + ResourceId resource = _message.getResourceId(); + SessionId sessionId = _message.getTypedTgtSessionId(); String instanceName = _manager.getInstanceName(); HelixDataAccessor accessor = _manager.getHelixDataAccessor(); @@ -132,18 +135,18 @@ void postHandleMessage() { // new session // sessionId might change when we update the state model state. // for zk current state it is OK as we have the per-session current state node - if (!_message.getTgtSessionId().equals(_manager.getSessionId())) { + if (!_message.getTypedTgtSessionId().stringify().equals(_manager.getSessionId())) { logger.warn("Session id has changed. Skip postExecutionMessage. Old session " - + _message.getExecutionSessionId() + " , new session : " + _manager.getSessionId()); + + _message.getTypedExecutionSessionId() + " , new session : " + _manager.getSessionId()); return; } if (taskResult.isSuccess()) { // String fromState = message.getFromState(); - String toState = _message.getToState(); - _currentStateDelta.setState(partitionKey, toState); + State toState = _message.getTypedToState(); + _currentStateDelta.setState(partitionId, toState); - if (toState.equalsIgnoreCase(HelixDefinedState.DROPPED.toString())) { + if (toState.toString().equalsIgnoreCase(HelixDefinedState.DROPPED.toString())) { // for "OnOfflineToDROPPED" message, we need to remove the resource key record // from the current state of the instance because the resource key is dropped. // In the state model it will be stayed as "OFFLINE", which is OK. @@ -155,18 +158,18 @@ void postHandleMessage() { List deltaList = new ArrayList(); deltaList.add(delta); _currentStateDelta.setDeltaList(deltaList); - _stateModelFactory.removeStateModel(partitionKey); + _stateModelFactory.removeStateModel(partitionId.stringify()); } else { // if the partition is not to be dropped, update _stateModel to the TO_STATE - _stateModel.updateState(toState); + _stateModel.updateState(toState.toString()); } } else { if (exception instanceof HelixStateMismatchException) { // if fromState mismatch, set current state on zk to stateModel's current state logger.warn("Force CurrentState on Zk to be stateModel's CurrentState. partitionKey: " - + partitionKey + ", currentState: " + _stateModel.getCurrentState() + ", message: " + + partitionId + ", currentState: " + _stateModel.getCurrentState() + ", message: " + _message); - _currentStateDelta.setState(partitionKey, _stateModel.getCurrentState()); + _currentStateDelta.setState(partitionId, State.from(_stateModel.getCurrentState())); } else { StateTransitionError error = new StateTransitionError(ErrorType.INTERNAL, ErrorCode.ERROR, exception); @@ -178,16 +181,16 @@ void postHandleMessage() { // state in this case logger .error("State transition interrupted but not timeout. Not updating state. Partition : " - + _message.getPartitionName() + " MsgId : " + _message.getMsgId()); + + _message.getPartitionId() + " MsgId : " + _message.getMessageId()); return; } } _stateModel.rollbackOnError(_message, _notificationContext, error); - _currentStateDelta.setState(partitionKey, HelixDefinedState.ERROR.toString()); + _currentStateDelta.setState(partitionId, State.from(HelixDefinedState.ERROR.toString())); _stateModel.updateState(HelixDefinedState.ERROR.toString()); // if we have errors transit from ERROR state, disable the partition - if (_message.getFromState().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) { + if (_message.getTypedFromState().toString().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) { disablePartition(); } } @@ -196,8 +199,8 @@ void postHandleMessage() { try { // Update the ZK current state of the node PropertyKey key = - keyBuilder.currentState(instanceName, sessionId, resource, - bucketizer.getBucketName(partitionKey)); + keyBuilder.currentState(instanceName, sessionId.stringify(), resource.stringify(), + bucketizer.getBucketName(partitionId.stringify())); if (_message.getAttribute(Attributes.PARENT_MSG_ID) == null) { // normal message accessor.updateProperty(key, _currentStateDelta); @@ -206,7 +209,7 @@ void postHandleMessage() { ConcurrentHashMap csUpdateMap = (ConcurrentHashMap) _notificationContext .get(MapKey.CURRENT_STATE_UPDATE.toString()); - csUpdateMap.put(partitionKey, new CurrentStateUpdate(key, _currentStateDelta)); + csUpdateMap.put(partitionId.stringify(), new CurrentStateUpdate(key, _currentStateDelta)); } } catch (Exception e) { logger.error("Error when updating current-state ", e); @@ -220,14 +223,14 @@ void postHandleMessage() { void disablePartition() { String instanceName = _manager.getInstanceName(); - String resourceName = _message.getResourceName(); - String partitionName = _message.getPartitionName(); + ResourceId resourceId = _message.getResourceId(); + PartitionId partitionId = _message.getPartitionId(); String clusterName = _manager.getClusterName(); HelixAdmin admin = _manager.getClusterManagmentTool(); - admin.enablePartition(false, clusterName, instanceName, resourceName, - Arrays.asList(partitionName)); - logger.info("error in transit from ERROR to " + _message.getToState() + " for partition: " - + partitionName + ". disable it on " + instanceName); + admin.enablePartition(false, clusterName, instanceName, resourceId.stringify(), + Arrays.asList(partitionId.stringify())); + logger.info("error in transit from ERROR to " + _message.getTypedToState() + " for partition: " + + partitionId + ". disable it on " + instanceName); } @@ -256,7 +259,7 @@ public HelixTaskResult handleMessage() { taskResult.setException(e); } catch (Exception e) { String errorMessage = - "Exception while executing a state transition task " + message.getPartitionName(); + "Exception while executing a state transition task " + message.getPartitionId(); logger.error(errorMessage, e); if (e.getCause() != null && e.getCause() instanceof InterruptedException) { e = (InterruptedException) e.getCause(); @@ -285,11 +288,11 @@ private void invoke(HelixDataAccessor accessor, NotificationContext context, // by default, we invoke state transition function in state model Method methodToInvoke = null; - String fromState = message.getFromState(); - String toState = message.getToState(); + State fromState = message.getTypedFromState(); + State toState = message.getTypedToState(); methodToInvoke = - _transitionMethodFinder.getMethodForTransition(_stateModel.getClass(), fromState, toState, - new Class[] { + _transitionMethodFinder.getMethodForTransition(_stateModel.getClass(), + fromState.toString(), toState.toString(), new Class[] { Message.class, NotificationContext.class }); if (methodToInvoke != null) { @@ -314,8 +317,8 @@ public void onError(Exception e, ErrorCode code, ErrorType type) { HelixDataAccessor accessor = _manager.getHelixDataAccessor(); Builder keyBuilder = accessor.keyBuilder(); String instanceName = _manager.getInstanceName(); - String resourceName = _message.getResourceName(); - String partition = _message.getPartitionName(); + ResourceId resourceId = _message.getResourceId(); + PartitionId partition = _message.getPartitionId(); // All internal error has been processed already, so we can skip them if (type == ErrorType.INTERNAL) { @@ -327,17 +330,16 @@ public void onError(Exception e, ErrorCode code, ErrorType type) { // set current state to ERROR for the partition // if the transition is not canceled, it should go into error state if (code == ErrorCode.ERROR) { - CurrentState currentStateDelta = new CurrentState(resourceName); - currentStateDelta.setState(partition, HelixDefinedState.ERROR.toString()); + CurrentState currentStateDelta = new CurrentState(resourceId.stringify()); + currentStateDelta.setState(partition, State.from(HelixDefinedState.ERROR.toString())); _stateModel.updateState(HelixDefinedState.ERROR.toString()); // if transit from ERROR state, disable the partition - if (_message.getFromState().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) { + if (_message.getTypedFromState().toString().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) { disablePartition(); } - accessor.updateProperty( - keyBuilder.currentState(instanceName, _message.getTgtSessionId(), resourceName), - currentStateDelta); + accessor.updateProperty(keyBuilder.currentState(instanceName, _message.getTypedTgtSessionId() + .stringify(), resourceId.stringify()), currentStateDelta); } } finally { StateTransitionError error = new StateTransitionError(type, code, e); diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java index d9f7ae23f1..3bcc260920 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java @@ -19,13 +19,7 @@ * under the License. */ -import java.util.ArrayList; import java.util.Date; -import java.util.List; -import java.util.Map; -import java.util.Timer; -import java.util.TimerTask; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import org.apache.helix.HelixDataAccessor; @@ -33,12 +27,10 @@ import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; import org.apache.helix.NotificationContext.MapKey; -import org.apache.helix.PropertyKey; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.messaging.handling.GroupMessageHandler.GroupMessageInfo; +import org.apache.helix.api.State; import org.apache.helix.messaging.handling.MessageHandler.ErrorCode; import org.apache.helix.messaging.handling.MessageHandler.ErrorType; -import org.apache.helix.model.CurrentState; import org.apache.helix.model.Message; import org.apache.helix.model.Message.Attributes; import org.apache.helix.model.Message.MessageType; @@ -98,14 +90,14 @@ public HelixTaskResult call() { _statusUpdateUtil.logError(_message, HelixTask.class, e, "State transition interrupted, timeout:" + _isTimeout, accessor); - logger.info("Message " + _message.getMsgId() + " is interrupted"); + logger.info("Message " + _message.getMessageId() + " is interrupted"); } catch (Exception e) { taskResult = new HelixTaskResult(); taskResult.setException(e); taskResult.setMessage(e.getMessage()); String errorMessage = - "Exception while executing a message. " + e + " msgId: " + _message.getMsgId() + "Exception while executing a message. " + e + " msgId: " + _message.getMessageId() + " type: " + _message.getMsgType(); logger.error(errorMessage, e); _statusUpdateUtil.logError(_message, HelixTask.class, e, errorMessage, accessor); @@ -119,17 +111,17 @@ public HelixTaskResult call() { if (taskResult.isSuccess()) { _statusUpdateUtil.logInfo(_message, _handler.getClass(), "Message handling task completed successfully", accessor); - logger.info("Message " + _message.getMsgId() + " completed."); + logger.info("Message " + _message.getMessageId() + " completed."); } else { type = ErrorType.INTERNAL; if (taskResult.isInterrupted()) { - logger.info("Message " + _message.getMsgId() + " is interrupted"); + logger.info("Message " + _message.getMessageId() + " is interrupted"); code = _isTimeout ? ErrorCode.TIMEOUT : ErrorCode.CANCEL; if (_isTimeout) { int retryCount = _message.getRetryCount(); logger.info("Message timeout, retry count: " + retryCount + " msgId:" - + _message.getMsgId()); + + _message.getMessageId()); _statusUpdateUtil.logInfo(_message, _handler.getClass(), "Message handling task timeout, retryCount:" + retryCount, accessor); // Notify the handler that timeout happens, and the number of retries left @@ -166,12 +158,12 @@ public HelixTaskResult call() { code = ErrorCode.ERROR; String errorMessage = - "Exception after executing a message, msgId: " + _message.getMsgId() + e; + "Exception after executing a message, msgId: " + _message.getMessageId() + e; logger.error(errorMessage, e); _statusUpdateUtil.logError(_message, HelixTask.class, errorMessage, accessor); } finally { long end = System.currentTimeMillis(); - logger.info("msg: " + _message.getMsgId() + " handling task completed, results:" + logger.info("msg: " + _message.getMessageId() + " handling task completed, results:" + taskResult.isSuccess() + ", at: " + end + ", took:" + (end - start)); // Notify the handler about any error happened in the handling procedure, so that @@ -190,9 +182,10 @@ private void removeMessageFromZk(HelixDataAccessor accessor, Message message) { Builder keyBuilder = accessor.keyBuilder(); if (message.getTgtName().equalsIgnoreCase("controller")) { // TODO: removeProperty returns boolean - accessor.removeProperty(keyBuilder.controllerMessage(message.getMsgId())); + accessor.removeProperty(keyBuilder.controllerMessage(message.getMessageId().stringify())); } else { - accessor.removeProperty(keyBuilder.message(_manager.getInstanceName(), message.getMsgId())); + accessor.removeProperty(keyBuilder.message(_manager.getInstanceName(), message.getMessageId() + .stringify())); } } @@ -214,11 +207,13 @@ private void sendReply(HelixDataAccessor accessor, Message message, HelixTaskRes if (message.getSrcInstanceType() == InstanceType.PARTICIPANT) { Builder keyBuilder = accessor.keyBuilder(); - accessor.setProperty(keyBuilder.message(message.getMsgSrc(), replyMessage.getMsgId()), + accessor.setProperty( + keyBuilder.message(message.getMsgSrc(), replyMessage.getMessageId().stringify()), replyMessage); } else if (message.getSrcInstanceType() == InstanceType.CONTROLLER) { Builder keyBuilder = accessor.keyBuilder(); - accessor.setProperty(keyBuilder.controllerMessage(replyMessage.getMsgId()), replyMessage); + accessor.setProperty(keyBuilder.controllerMessage(replyMessage.getMessageId().stringify()), + replyMessage); } _statusUpdateUtil.logInfo(message, HelixTask.class, "1 msg replied to " + replyMessage.getTgtName(), accessor); @@ -237,13 +232,13 @@ private void reportMessageStat(HelixManager manager, Message message, HelixTaskR long totalDelay = now - msgReadTime; long executionDelay = now - msgExecutionStartTime; if (totalDelay > 0 && executionDelay > 0) { - String fromState = message.getFromState(); - String toState = message.getToState(); + State fromState = message.getTypedFromState(); + State toState = message.getTypedToState(); String transition = fromState + "--" + toState; StateTransitionContext cxt = - new StateTransitionContext(manager.getClusterName(), manager.getInstanceName(), - message.getResourceName(), transition); + new StateTransitionContext(manager.getClusterName(), manager.getInstanceName(), message + .getResourceId().stringify(), transition); StateTransitionDataPoint data = new StateTransitionDataPoint(totalDelay, executionDelay, taskResult.isSuccess()); diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java index 600a3ab9ca..8da53ea519 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java @@ -35,8 +35,6 @@ import java.util.concurrent.TimeUnit; import org.apache.helix.ConfigAccessor; -import org.apache.helix.model.ConfigScope; -import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.helix.HelixConstants; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixException; @@ -44,13 +42,17 @@ import org.apache.helix.MessageListener; import org.apache.helix.NotificationContext; import org.apache.helix.NotificationContext.MapKey; -import org.apache.helix.PropertyKey; import org.apache.helix.NotificationContext.Type; +import org.apache.helix.PropertyKey; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.model.ConfigScope; import org.apache.helix.model.CurrentState; import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageState; import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.helix.monitoring.ParticipantMonitor; import org.apache.helix.participant.HelixStateMachineEngine; import org.apache.helix.util.StatusUpdateUtil; @@ -161,9 +163,9 @@ void checkResourceConfig(String resourceName, HelixManager manager) { ExecutorService findExecutorServiceForMsg(Message message) { ExecutorService executorService = _executorMap.get(message.getMsgType()); if (message.getMsgType().equals(MessageType.STATE_TRANSITION.toString())) { - String resourceName = message.getResourceName(); - if (resourceName != null) { - String key = message.getMsgType() + "." + resourceName; + ResourceId resourceId = message.getResourceId(); + if (resourceId != null) { + String key = message.getMsgType() + "." + resourceId; if (_executorMap.containsKey(key)) { LOG.info("Find per-resource thread pool with key: " + key); executorService = _executorMap.get(key); @@ -223,7 +225,7 @@ public boolean scheduleTask(MessageTask task) { try { if (message.getMsgType().equals(MessageType.STATE_TRANSITION.toString())) { - checkResourceConfig(message.getResourceName(), notificationContext.getManager()); + checkResourceConfig(message.getResourceId().toString(), notificationContext.getManager()); } LOG.info("Scheduling message: " + taskId); @@ -430,14 +432,14 @@ public void onMessage(String instanceName, List messages, continue; } - String tgtSessionId = message.getTgtSessionId(); + SessionId tgtSessionId = message.getTypedTgtSessionId(); // sessionId mismatch normally means message comes from expired session, just remove it - if (!sessionId.equals(tgtSessionId) && !tgtSessionId.equals("*")) { + if (!sessionId.equals(tgtSessionId.toString()) && !tgtSessionId.toString().equals("*")) { String warningMessage = "SessionId does NOT match. expected sessionId: " + sessionId + ", tgtSessionId in message: " + tgtSessionId + ", messageId: " - + message.getMsgId(); + + message.getMessageId(); LOG.warn(warningMessage); accessor.removeProperty(message.getKey(keyBuilder, instanceName)); _statusUpdateUtil.logWarning(message, HelixStateMachineEngine.class, warningMessage, @@ -452,7 +454,7 @@ public void onMessage(String instanceName, List messages, // We will read the message again if there is a new message but we // check for the status and ignore if its already read if (LOG.isTraceEnabled()) { - LOG.trace("Message already read. msgId: " + message.getMsgId()); + LOG.trace("Message already read. msgId: " + message.getMessageId()); } continue; } @@ -465,9 +467,9 @@ public void onMessage(String instanceName, List messages, } handlers.add(createHandler); } catch (Exception e) { - LOG.error("Failed to create message handler for " + message.getMsgId(), e); + LOG.error("Failed to create message handler for " + message.getMessageId(), e); String error = - "Failed to create message handler for " + message.getMsgId() + ", exception: " + e; + "Failed to create message handler for " + message.getMessageId() + ", exception: " + e; _statusUpdateUtil.logError(message, HelixStateMachineEngine.class, e, error, accessor); @@ -481,7 +483,7 @@ public void onMessage(String instanceName, List messages, // update msgState to read message.setMsgState(MessageState.READ); message.setReadTimeStamp(new Date().getTime()); - message.setExecuteSessionId(changeContext.getManager().getSessionId()); + message.setExecuteSessionId(SessionId.from(changeContext.getManager().getSessionId())); _statusUpdateUtil.logInfo(message, HelixStateMachineEngine.class, "New Message", accessor); @@ -491,15 +493,17 @@ public void onMessage(String instanceName, List messages, // do it for non-controller and state transition messages only if (!message.isControlerMsg() && message.getMsgType().equals(Message.MessageType.STATE_TRANSITION.toString())) { - String resourceName = message.getResourceName(); - if (!curResourceNames.contains(resourceName) && !createCurStateNames.contains(resourceName)) { - createCurStateNames.add(resourceName); - createCurStateKeys.add(keyBuilder.currentState(instanceName, sessionId, resourceName)); - - CurrentState metaCurState = new CurrentState(resourceName); + ResourceId resourceId = message.getResourceId(); + if (!curResourceNames.contains(resourceId.stringify()) + && !createCurStateNames.contains(resourceId.stringify())) { + createCurStateNames.add(resourceId.stringify()); + createCurStateKeys.add(keyBuilder.currentState(instanceName, sessionId, + resourceId.stringify())); + + CurrentState metaCurState = new CurrentState(resourceId.stringify()); metaCurState.setBucketSize(message.getBucketSize()); metaCurState.setStateModelDefRef(message.getStateModelDef()); - metaCurState.setSessionId(sessionId); + metaCurState.setSessionId(SessionId.from(sessionId)); metaCurState.setBatchMessageMode(message.getBatchMessageMode()); String ftyName = message.getStateModelFactoryName(); if (ftyName != null) { @@ -543,7 +547,7 @@ public MessageHandler createMessageHandler(Message message, NotificationContext // the corresponding MessageHandlerFactory is registered if (handlerFactory == null) { LOG.warn("Fail to find message handler factory for type: " + msgType + " msgId: " - + message.getMsgId()); + + message.getMessageId()); return null; } diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageHandler.java index 96784c2b2d..5715571faa 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageHandler.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageHandler.java @@ -20,8 +20,6 @@ */ import org.apache.helix.NotificationContext; -import org.apache.helix.messaging.handling.MessageHandler.ErrorCode; -import org.apache.helix.messaging.handling.MessageHandler.ErrorType; import org.apache.helix.model.Message; /** diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageTimeoutTask.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageTimeoutTask.java index e1b4f0f964..17fc041c0c 100644 --- a/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageTimeoutTask.java +++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageTimeoutTask.java @@ -40,7 +40,7 @@ public void run() { Message message = _task.getMessage(); // NotificationContext context = _task.getNotificationContext(); // System.out.println("msg: " + message.getMsgId() + " timeouot."); - LOG.warn("Message time out, canceling. id:" + message.getMsgId() + " timeout : " + LOG.warn("Message time out, canceling. id:" + message.getMessageId() + " timeout : " + message.getExecutionTimeout()); _task.onTimeout(); _executor.cancelTask(_task); diff --git a/helix-core/src/main/java/org/apache/helix/model/AlertConfig.java b/helix-core/src/main/java/org/apache/helix/model/AlertConfig.java new file mode 100644 index 0000000000..54da31d3b4 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/AlertConfig.java @@ -0,0 +1,89 @@ +package org.apache.helix.model; + +/* + * 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.HelixProperty; +import org.apache.helix.ZNRecord; +import org.apache.helix.controller.alert.AlertAction; +import org.apache.helix.controller.alert.AlertName; +import org.apache.helix.controller.alert.AlertScope; +import org.apache.log4j.Logger; + +public class AlertConfig extends HelixProperty { + private static final Logger LOG = Logger.getLogger(AlertConfig.class); + + public AlertConfig(ZNRecord record) { + super(record); + } + + public AlertConfig(String id) { + super(id); + } + + public void putConfig(AlertName alertName, AlertAction action) { + _record.setSimpleField(alertName.toString(), action.toString()); + } + + /** + * Given an alert name, find the matching action + * @param matchingName + * @return + */ + public AlertAction findAlertAction(AlertName matchingName) { + // search each alert config entry to find a match + for (String alertNameStr : _record.getSimpleFields().keySet()) { + String actionStr = _record.getSimpleField(alertNameStr); + + try { + AlertName name = AlertName.from(alertNameStr); + if (!name.match(matchingName)) { + continue; + } + + // find a match + // replace "{scope}" in action.args with actual values + // e.g. "{node}" -> "localhost_12918" + int start = actionStr.indexOf("{"); + while (start != -1) { + int end = actionStr.indexOf("}", start + 1); + String fieldStr = actionStr.substring(start + 1, end); + AlertScope.AlertScopeField field = AlertScope.AlertScopeField.valueOf(fieldStr); + + String fieldValue = matchingName.getScope().get(field); + if (fieldValue == null) { + throw new NullPointerException("Null value for alert scope field: " + field + + ", in alert: " + matchingName); + } + + actionStr = actionStr.replace("{" + fieldStr + "}", fieldValue); + start = actionStr.indexOf("{"); + } + + return AlertAction.from(actionStr); + } catch (Exception e) { + LOG.error("Exception in find action for alert: " + matchingName + + ", matching alertEntry: \"" + alertNameStr + ":" + actionStr + "\"", e); + } + } + + return null; + } + +} diff --git a/helix-core/src/main/java/org/apache/helix/model/AlertStatus.java b/helix-core/src/main/java/org/apache/helix/model/AlertStatus.java deleted file mode 100644 index d90ec1a63f..0000000000 --- a/helix-core/src/main/java/org/apache/helix/model/AlertStatus.java +++ /dev/null @@ -1,112 +0,0 @@ -package org.apache.helix.model; - -/* - * 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 org.apache.helix.HelixProperty; -import org.apache.helix.ZNRecord; -import org.apache.helix.model.Alerts.AlertsProperty; - -/** - * Get characteristics of alerts, whether they were fired, and their context. An alert is triggered - * when cluster health violates pre-defined constraints to ensure that the cluster meets SLAs. - */ -public class AlertStatus extends HelixProperty { - - /** - * The name of the ZNode containing alert status - */ - public final static String nodeName = "AlertStatus"; - - /** - * Instantiate with an identifier - * @param id identifier representing this group of alert statuses - */ - public AlertStatus(String id) { - super(id); - } - - /** - * Instantiate with a pre-populated record corresponding to alert status - * @param record ZNRecord representing alert statuses - */ - public AlertStatus(ZNRecord record) { - // _record = record; - super(record); - - } - - /* - * public Alerts(ZNRecord record, Stat stat) { super(record, stat); } - */ - - /** - * Set the session that these alerts correspond to - * @param sessionId session for which to look up alerts - */ - public void setSessionId(String sessionId) { - _record.setSimpleField(AlertsProperty.SESSION_ID.toString(), sessionId); - } - - /** - * Get the session that these alerts correspond to - * @return session identifier - */ - public String getSessionId() { - return _record.getSimpleField(AlertsProperty.SESSION_ID.toString()); - } - - /** - * Get the instance that these alerts correspond to - * @return name of the instance - */ - public String getInstanceName() { - return _record.getId(); - } - - /* - * public String getVersion() { return - * _record.getSimpleField(AlertsProperty.CLUSTER_MANAGER_VERSION.toString()); } - */ - - /** - * Get the properties of all alerts, such as if they were fired - * @return all alert statuses as a Map of alert to the status properties - */ - public Map> getMapFields() { - return _record.getMapFields(); - } - - /** - * Get the statistics of a single alert - * @param statName Name of the alert - * @return alert statistics as a map of name, value pairs - */ - public Map getStatFields(String statName) { - return _record.getMapField(statName); - } - - @Override - public boolean isValid() { - // TODO Auto-generated method stub - return true; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/model/Alerts.java b/helix-core/src/main/java/org/apache/helix/model/Alerts.java deleted file mode 100644 index 506e3d5e6a..0000000000 --- a/helix-core/src/main/java/org/apache/helix/model/Alerts.java +++ /dev/null @@ -1,124 +0,0 @@ -package org.apache.helix.model; - -/* - * 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 org.apache.helix.HelixProperty; -import org.apache.helix.ZNRecord; - -/** - * Describe alerts and corresponding metrics. An alert is triggered when cluster health - * violates pre-defined constraints to ensure that the cluster meets SLAs. - */ -public class Alerts extends HelixProperty { - - // private final ZNRecord _record; - - /** - * The name of the ZNode corresponding to this property - */ - public final static String nodeName = "Alerts"; - - /** - * Supported fields corresponding to a set of alerts - */ - public enum AlertsProperty { - SESSION_ID, - FIELDS - } - - // private final ZNRecord _record; - - /** - * Instantiate with an identifier - * @param id A string that identifies the alerts - */ - public Alerts(String id) { - super(id); - } - - /** - * Instantiate with a pre-populated Alerts record - * @param record ZNRecord representing Alerts - */ - public Alerts(ZNRecord record) { - // _record = record; - super(record); - - } - - /* - * public Alerts(ZNRecord record, Stat stat) { super(record, stat); } - */ - - /** - * Set the session that the alerts correspond to - * @param sessionId the session for which alerts occurred - */ - public void setSessionId(String sessionId) { - _record.setSimpleField(AlertsProperty.SESSION_ID.toString(), sessionId); - } - - /** - * Get the session that the alerts correspond to - * @return session identifier - */ - public String getSessionId() { - return _record.getSimpleField(AlertsProperty.SESSION_ID.toString()); - } - - /** - * Get the instance that the alerts correspond to - * @return the name of the instance - */ - public String getInstanceName() { - return _record.getId(); - } - - /* - * public String getVersion() { return - * _record.getSimpleField(AlertsProperty.CLUSTER_MANAGER_VERSION.toString()); - * } - */ - - /** - * Get the alerts - * @return a mapping of alert stat name to alert properties - */ - public Map> getMapFields() { - return _record.getMapFields(); - } - - /** - * Get specific alert statistics - * @param statName the name of the statistic group - * @return a mapping of property and value for the statistic - */ - public Map getStatFields(String statName) { - return _record.getMapField(statName); - } - - @Override - public boolean isValid() { - // TODO Auto-generated method stub - return true; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java new file mode 100644 index 0000000000..8386d6c793 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java @@ -0,0 +1,109 @@ +package org.apache.helix.model; + +import org.apache.helix.HelixProperty; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.config.NamespacedConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.manager.zk.ZKHelixManager; + +/* + * 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. + */ + +/** + * Persisted configuration properties for a cluster + */ +public class ClusterConfiguration extends HelixProperty { + /** + * Instantiate for an id + * @param id cluster id + */ + public ClusterConfiguration(ClusterId id) { + super(id.stringify()); + } + + /** + * Instantiate from a record + * @param record configuration properties + */ + public ClusterConfiguration(ZNRecord record) { + super(record); + } + + /** + * Get a typed cluster id + * @return ClusterId + */ + public ClusterId getClusterId() { + return ClusterId.from(getId()); + } + + /** + * Determine if participants can automatically join the cluster + * @return true if allowed, false if disallowed + */ + public boolean autoJoinAllowed() { + return _record.getBooleanField(ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, false); + } + + /** + * Set if participants can automatically join the cluster + * @param autoJoinAllowed true if allowed, false if disallowed + */ + public void setAutoJoinAllowed(boolean autoJoinAllowed) { + _record.setBooleanField(ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, autoJoinAllowed); + } + + /** + * Get a backward-compatible cluster user config + * @return UserConfig + */ + public UserConfig getUserConfig() { + UserConfig userConfig = UserConfig.from(this); + for (String simpleField : _record.getSimpleFields().keySet()) { + if (!simpleField.contains(NamespacedConfig.PREFIX_CHAR + "") + && !simpleField.equals(ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN)) { + userConfig.setSimpleField(simpleField, _record.getSimpleField(simpleField)); + } + } + for (String listField : _record.getListFields().keySet()) { + if (!listField.contains(NamespacedConfig.PREFIX_CHAR + "")) { + userConfig.setListField(listField, _record.getListField(listField)); + } + } + for (String mapField : _record.getMapFields().keySet()) { + if (!mapField.contains(NamespacedConfig.PREFIX_CHAR + "")) { + userConfig.setMapField(mapField, _record.getMapField(mapField)); + } + } + return userConfig; + } + + /** + * Create a new ClusterConfiguration from a UserConfig + * @param userConfig user-defined configuration properties + * @return ClusterConfiguration + */ + public static ClusterConfiguration from(UserConfig userConfig) { + ClusterConfiguration clusterConfiguration = + new ClusterConfiguration(ClusterId.from(userConfig.getId())); + clusterConfiguration.addNamespacedConfig(userConfig); + return clusterConfiguration; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConstraints.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConstraints.java index f69a7ce1ae..daefe6ee2a 100644 --- a/helix-core/src/main/java/org/apache/helix/model/ClusterConstraints.java +++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConstraints.java @@ -27,6 +27,7 @@ import org.apache.helix.HelixProperty; import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.ConstraintId; import org.apache.helix.model.Message.MessageType; import org.apache.helix.model.builder.ConstraintItemBuilder; import org.apache.log4j.Logger; @@ -42,9 +43,11 @@ public class ClusterConstraints extends HelixProperty { */ public enum ConstraintAttribute { STATE, + STATE_MODEL, MESSAGE_TYPE, TRANSITION, RESOURCE, + PARTITION, INSTANCE, CONSTRAINT_VALUE } @@ -53,7 +56,9 @@ public enum ConstraintAttribute { * Possible special values that constraint attributes can take */ public enum ConstraintValue { - ANY + ANY, + N, + R } /** @@ -65,7 +70,8 @@ public enum ConstraintType { } // constraint-id -> constraint-item - private final Map _constraints = new HashMap(); + private final Map _constraints = + new HashMap(); /** * Instantiate constraints as a given type @@ -75,6 +81,14 @@ public ClusterConstraints(ConstraintType type) { super(type.toString()); } + /** + * Get the type of constraint this object represents + * @return constraint type + */ + public ConstraintType getType() { + return ConstraintType.valueOf(getId()); + } + /** * Instantiate constraints from a pre-populated ZNRecord * @param record ZNRecord containing all constraints @@ -88,7 +102,7 @@ public ClusterConstraints(ZNRecord record) { builder.addConstraintAttributes(_record.getMapField(constraintId)).build(); // ignore item with empty attributes or no constraint-value if (item.getAttributes().size() > 0 && item.getConstraintValue() != null) { - addConstraintItem(constraintId, item); + addConstraintItem(ConstraintId.from(constraintId), item); } else { LOG.error("Skip invalid constraint. key: " + constraintId + ", value: " + _record.getMapField(constraintId)); @@ -101,16 +115,25 @@ public ClusterConstraints(ZNRecord record) { * @param constraintId unique constraint identifier * @param item the constraint as a {@link ConstraintItem} */ - public void addConstraintItem(String constraintId, ConstraintItem item) { + public void addConstraintItem(ConstraintId constraintId, ConstraintItem item) { Map map = new TreeMap(); for (ConstraintAttribute attr : item.getAttributes().keySet()) { map.put(attr.toString(), item.getAttributeValue(attr)); } map.put(ConstraintAttribute.CONSTRAINT_VALUE.toString(), item.getConstraintValue()); - _record.setMapField(constraintId, map); + _record.setMapField(constraintId.stringify(), map); _constraints.put(constraintId, item); } + /** + * add the constraint, overwrite existing one if constraint with same constraint-id already exists + * @param constraintId unique constraint identifier + * @param item the constraint as a {@link ConstraintItem} + */ + public void addConstraintItem(String constraintId, ConstraintItem item) { + addConstraintItem(ConstraintId.from(constraintId), item); + } + /** * Add multiple constraint items. * @param items (constraint identifier, {@link ConstrantItem}) pairs @@ -125,9 +148,17 @@ public void addConstraintItems(Map items) { * remove a constraint-item * @param constraintId unique constraint identifier */ - public void removeConstraintItem(String constraintId) { + public void removeConstraintItem(ConstraintId constraintId) { _constraints.remove(constraintId); - _record.getMapFields().remove(constraintId); + _record.getMapFields().remove(constraintId.stringify()); + } + + /** + * remove a constraint-item + * @param constraintId unique constraint identifier + */ + public void removeConstraintItem(String constraintId) { + removeConstraintItem(ConstraintId.from(constraintId)); } /** @@ -135,10 +166,19 @@ public void removeConstraintItem(String constraintId) { * @param constraintId unique constraint identifier * @return {@link ConstraintItem} or null if not present */ - public ConstraintItem getConstraintItem(String constraintId) { + public ConstraintItem getConstraintItem(ConstraintId constraintId) { return _constraints.get(constraintId); } + /** + * get a constraint-item + * @param constraintId unique constraint identifier + * @return {@link ConstraintItem} or null if not present + */ + public ConstraintItem getConstraintItem(String constraintId) { + return getConstraintItem(ConstraintId.from(constraintId)); + } + /** * return a set of constraints that match the attribute pairs * @param attributes (constraint scope, constraint string) pairs @@ -154,6 +194,14 @@ public Set match(Map attributes) { return matches; } + /** + * Get all constraint items in this collection of constraints + * @return map of constraint id to constraint item + */ + public Map getConstraintItems() { + return _constraints; + } + /** * convert a message to constraint attribute pairs * @param msg a {@link Message} containing constraint attributes @@ -164,22 +212,25 @@ public static Map toConstraintAttributes(Message ms String msgType = msg.getMsgType(); attributes.put(ConstraintAttribute.MESSAGE_TYPE, msgType); if (MessageType.STATE_TRANSITION.toString().equals(msgType)) { - if (msg.getFromState() != null && msg.getToState() != null) { - attributes.put(ConstraintAttribute.TRANSITION, msg.getFromState() + "-" + msg.getToState()); + if (msg.getTypedFromState() != null && msg.getTypedToState() != null) { + attributes.put(ConstraintAttribute.TRANSITION, + Transition.from(msg.getTypedFromState(), msg.getTypedToState()).toString()); } - if (msg.getResourceName() != null) { - attributes.put(ConstraintAttribute.RESOURCE, msg.getResourceName()); + if (msg.getResourceId() != null) { + attributes.put(ConstraintAttribute.RESOURCE, msg.getResourceId().stringify()); } if (msg.getTgtName() != null) { attributes.put(ConstraintAttribute.INSTANCE, msg.getTgtName()); } + if (msg.getStateModelDefId() != null) { + attributes.put(ConstraintAttribute.STATE_MODEL, msg.getStateModelDefId().stringify()); + } } return attributes; } @Override public boolean isValid() { - // TODO Auto-generated method stub return true; } diff --git a/helix-core/src/main/java/org/apache/helix/model/CurrentState.java b/helix-core/src/main/java/org/apache/helix/model/CurrentState.java index 32854ab717..5c9bcbc7a1 100644 --- a/helix-core/src/main/java/org/apache/helix/model/CurrentState.java +++ b/helix-core/src/main/java/org/apache/helix/model/CurrentState.java @@ -19,12 +19,18 @@ * under the License. */ +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.TreeMap; import org.apache.helix.HelixProperty; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.log4j.Logger; /** @@ -53,6 +59,14 @@ public CurrentState(String resourceName) { super(resourceName); } + /** + * Instantiate a current state with a resource + * @param resourceId identifier for the resource + */ + public CurrentState(ResourceId resourceId) { + super(resourceId.stringify()); + } + /** * Instantiate a current state with a pre-populated ZNRecord * @param record a ZNRecord corresponding to the current state @@ -69,6 +83,14 @@ public String getResourceName() { return _record.getId(); } + /** + * Get the resource id + * @return ResourceId + */ + public ResourceId getResourceId() { + return ResourceId.from(getResourceName()); + } + /** * Get the partitions on this instance and the state that each partition is currently in. * @return (partition, state) pairs @@ -85,9 +107,33 @@ public Map getPartitionStateMap() { return map; } + /** + * Get the partitions on this instance and the state that each partition is currently in + * @return (partition id, state) pairs + */ + public Map getTypedPartitionStateMap() { + Map map = new HashMap(); + for (String partitionName : _record.getMapFields().keySet()) { + Map stateMap = _record.getMapField(partitionName); + if (stateMap != null) { + map.put(PartitionId.from(partitionName), + State.from(stateMap.get(CurrentStateProperty.CURRENT_STATE.toString()))); + } + } + return map; + } + /** * Get the session that this current state corresponds to - * @return String session identifier + * @return session identifier + */ + public SessionId getTypedSessionId() { + return SessionId.from(getSessionId()); + } + + /** + * Get the session that this current state corresponds to + * @return session identifier */ public String getSessionId() { return _record.getSimpleField(CurrentStateProperty.SESSION_ID.toString()); @@ -95,7 +141,15 @@ public String getSessionId() { /** * Set the session that this current state corresponds to - * @param sessionId String session identifier + * @param sessionId session identifier + */ + public void setSessionId(SessionId sessionId) { + setSessionId(sessionId.stringify()); + } + + /** + * Set the session that this current state corresponds to + * @param sessionId session identifier */ public void setSessionId(String sessionId) { _record.setSimpleField(CurrentStateProperty.SESSION_ID.toString(), sessionId); @@ -115,6 +169,15 @@ public String getState(String partitionName) { return null; } + /** + * Get the state of a partition on this instance + * @param partitionId partition id + * @return State + */ + public State getState(PartitionId partitionId) { + return State.from(getState(partitionId.stringify())); + } + /** * Set the state model that the resource follows * @param stateModelName an identifier of the state model @@ -131,6 +194,37 @@ public String getStateModelDefRef() { return _record.getSimpleField(CurrentStateProperty.STATE_MODEL_DEF.toString()); } + /** + * Set the state model that the resource follows + * @param stateModelName an identifier of the state model + */ + public void setStateModelDefId(StateModelDefId stateModelId) { + _record.setSimpleField(CurrentStateProperty.STATE_MODEL_DEF.toString(), + stateModelId.stringify()); + } + + /** + * Get the state model that the resource follows + * @return an identifier of the state model + */ + public StateModelDefId getStateModelDefId() { + return StateModelDefId.from(getStateModelDefRef()); + } + + /** + * Set the state that a partition is currently in on this instance + * @param partitionId the id of the partition + * @param state the state of the partition + */ + public void setState(PartitionId partitionId, State state) { + Map> mapFields = _record.getMapFields(); + if (mapFields.get(partitionId.stringify()) == null) { + mapFields.put(partitionId.stringify(), new TreeMap()); + } + mapFields.get(partitionId.stringify()).put(CurrentStateProperty.CURRENT_STATE.toString(), + state.toString()); + } + /** * Set the state that a partition is currently in on this instance * @param partitionName the name of the partition @@ -202,4 +296,36 @@ public boolean isValid() { return true; } + /** + * Convert a string map to a concrete partition map + * @param rawMap map of partition name to state name + * @return map of partition id to state + */ + public static Map partitionStateMapFromStringMap(Map rawMap) { + if (rawMap == null) { + return Collections.emptyMap(); + } + Map partitionStateMap = new HashMap(); + for (String partitionId : rawMap.keySet()) { + partitionStateMap.put(PartitionId.from(partitionId), State.from(rawMap.get(partitionId))); + } + return partitionStateMap; + } + + /** + * Convert a partition map to a string map + * @param partitionStateMap map of partition id to state + * @return map of partition name to state name + */ + public static Map stringMapFromPartitionStateMap( + Map partitionStateMap) { + if (partitionStateMap == null) { + return Collections.emptyMap(); + } + Map rawMap = new HashMap(); + for (PartitionId partitionId : partitionStateMap.keySet()) { + rawMap.put(partitionId.stringify(), partitionStateMap.get(partitionId).toString()); + } + return rawMap; + } } diff --git a/helix-core/src/main/java/org/apache/helix/model/ExternalView.java b/helix-core/src/main/java/org/apache/helix/model/ExternalView.java index d5f1afc7ae..0f7b6db47a 100644 --- a/helix-core/src/main/java/org/apache/helix/model/ExternalView.java +++ b/helix-core/src/main/java/org/apache/helix/model/ExternalView.java @@ -25,6 +25,13 @@ import org.apache.helix.HelixProperty; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; /** * External view is an aggregation (across all instances) @@ -39,6 +46,14 @@ public ExternalView(String resource) { super(new ZNRecord(resource)); } + /** + * Instantiate an external view with the resource it corresponds to + * @param resource the id of the resource + */ + public ExternalView(ResourceId resource) { + super(new ZNRecord(resource.stringify())); + } + /** * Instantiate an external view with a pre-populated record * @param record ZNRecord corresponding to an external view @@ -61,6 +76,20 @@ public void setState(String partition, String instance, String state) { _record.getMapField(partition).put(instance, state); } + /** + * For a given replica, specify which partition it corresponds to, where it is served, and its + * current state + * @param partitionId the partition of the replica being served + * @param participantId the instance serving the replica + * @param state the state the replica is in + */ + public void setState(PartitionId partitionId, ParticipantId participantId, State state) { + if (_record.getMapField(partitionId.stringify()) == null) { + _record.setMapField(partitionId.stringify(), new TreeMap()); + } + _record.getMapField(partitionId.stringify()).put(participantId.stringify(), state.toString()); + } + /** * For a given partition, indicate where and in what state each of its replicas is in * @param partitionName the partition to set @@ -70,6 +99,17 @@ public void setStateMap(String partitionName, Map currentStateMa _record.setMapField(partitionName, currentStateMap); } + /** + * For a given partition, indicate where and in what state each of its replicas is in + * @param partitionId the partition to set + * @param currentStateMap (participant, state) pairs + */ + public void setStateMap(PartitionId partitionId, Map currentStateMap) { + for (ParticipantId participantId : currentStateMap.keySet()) { + setState(partitionId, participantId, currentStateMap.get(participantId)); + } + } + /** * Get all the partitions of the resource * @return a set of partition names @@ -78,6 +118,18 @@ public Set getPartitionSet() { return _record.getMapFields().keySet(); } + /** + * Get all the partitions of the resource + * @return a set of partition ids + */ + public Set getPartitionIdSet() { + Set partitionSet = Sets.newHashSet(); + for (String partitionName : getPartitionSet()) { + partitionSet.add(PartitionId.from(partitionName)); + } + return partitionSet; + } + /** * Get the instance and the state for each partition replica * @param partitionName the partition to look up @@ -87,6 +139,24 @@ public Map getStateMap(String partitionName) { return _record.getMapField(partitionName); } + /** + * Get the participant and the state for each partition replica + * @param partitionId the partition to look up + * @return (participant, state) pairs + */ + public Map getStateMap(PartitionId partitionId) { + Map rawStateMap = getStateMap(partitionId.stringify()); + if (rawStateMap == null) { + return null; + } + Map stateMap = Maps.newHashMap(); + for (String participantName : rawStateMap.keySet()) { + stateMap.put(ParticipantId.from(participantName), + State.from(rawStateMap.get(participantName))); + } + return stateMap; + } + /** * Get the resource represented by this view * @return the name of the resource @@ -95,8 +165,25 @@ public String getResourceName() { return _record.getId(); } + /** + * Get the resource represented by this view + * @return resource id + */ + public ResourceId getResourceId() { + return ResourceId.from(getResourceName()); + } + @Override public boolean isValid() { return true; } + + /** + * Convert a partition mapping as strings into a participant state map + * @param rawMap the map of participant name to state + * @return converted map + */ + public static Map stateMapFromStringMap(Map rawMap) { + return ResourceAssignment.replicaMapFromStringMap(rawMap); + } } diff --git a/helix-core/src/main/java/org/apache/helix/model/HealthStat.java b/helix-core/src/main/java/org/apache/helix/model/HealthStat.java deleted file mode 100644 index b8845340e9..0000000000 --- a/helix-core/src/main/java/org/apache/helix/model/HealthStat.java +++ /dev/null @@ -1,147 +0,0 @@ -package org.apache.helix.model; - -/* - * 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; - -import org.apache.helix.HelixProperty; -import org.apache.helix.ZNRecord; -import org.apache.helix.alerts.ExpressionParser; -import org.apache.helix.alerts.StatsHolder; -import org.apache.helix.model.Message.Attributes; -import org.apache.log4j.Logger; - -/** - * Represents a set of properties that can be queried to determine the health of instances on a - * Helix-managed cluster - */ -public class HealthStat extends HelixProperty { - /** - * Queryable health statistic properties - */ - public enum HealthStatProperty { - FIELDS - } - - private static final Logger _logger = Logger.getLogger(HealthStat.class.getName()); - - /** - * Instantiate with an identifier - * @param id the name of these statistics - */ - public HealthStat(String id) { - super(id); - } - - /** - * Instantiate with a pre-populated record - * @param record a ZNRecord corresponding to health statistics - */ - public HealthStat(ZNRecord record) { - super(record); - if (getCreateTimeStamp() == 0) { - _record.setLongField(Attributes.CREATE_TIMESTAMP.toString(), new Date().getTime()); - } - } - - /** - * Get when these statistics were last modified - * @return a UNIX timestamp - */ - public long getLastModifiedTimeStamp() { - return _record.getModifiedTime(); - } - - /** - * Get when these statistics were created - * @return a UNIX timestamp - */ - public long getCreateTimeStamp() { - return _record.getLongField(Attributes.CREATE_TIMESTAMP.toString(), 0L); - } - - /** - * Get the value of a test field corresponding to a request count - * @return the number of requests - */ - public String getTestField() { - return _record.getSimpleField("requestCountStat"); - } - - /** - * Set a group of heath statistics, grouped by the statistic - * @param healthFields a map of statistic name, the corresponding entity, and the value - */ - public void setHealthFields(Map> healthFields) { - _record.setMapFields(healthFields); - } - - /** - * Create a key based on a parent key, instance, and statistic - * @param instance the instance for which these statistics exist - * @param parentKey the originating key - * @param statName the statistic - * @return a unified key - */ - public String buildCompositeKey(String instance, String parentKey, String statName) { - String delim = ExpressionParser.statFieldDelim; - return instance + delim + parentKey + delim + statName; - } - - /** - * Get all the health statistics for a given instance - * @param instanceName the instance for which to get health statistics - * @return a map of (instance and statistic, value or timestamp, value) triples - */ - public Map> getHealthFields(String instanceName) // , - // String - // timestamp) - { - // XXX: need to do some conversion of input format to the format that stats - // computation wants - Map> currMapFields = _record.getMapFields(); - Map> convertedMapFields = - new HashMap>(); - for (String key : currMapFields.keySet()) { - Map currMap = currMapFields.get(key); - String timestamp = _record.getStringField(StatsHolder.TIMESTAMP_NAME, "-1"); - for (String subKey : currMap.keySet()) { - if (subKey.equals("StatsHolder.TIMESTAMP_NAME")) { // don't want to get timestamp again - continue; - } - String compositeKey = buildCompositeKey(instanceName, key, subKey); - String value = currMap.get(subKey); - Map convertedMap = new HashMap(); - convertedMap.put(StatsHolder.VALUE_NAME, value); - convertedMap.put(StatsHolder.TIMESTAMP_NAME, timestamp); - convertedMapFields.put(compositeKey, convertedMap); - } - } - return convertedMapFields; - } - - @Override - public boolean isValid() { - // TODO Auto-generated method stub - return true; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/model/HelixConfigScope.java b/helix-core/src/main/java/org/apache/helix/model/HelixConfigScope.java index 9d3b41aae4..3492187501 100644 --- a/helix-core/src/main/java/org/apache/helix/model/HelixConfigScope.java +++ b/helix-core/src/main/java/org/apache/helix/model/HelixConfigScope.java @@ -35,7 +35,9 @@ public enum ConfigScopeProperty { PARTICIPANT(2, 0), RESOURCE(2, 0), PARTITION(2, 1), - CONSTRAINT(2, 0); + CONSTRAINT(2, 0), + MONITORING(2, 0), + ALERT(2, 0); final int _zkPathArgNum; final int _mapKeyArgNum; @@ -77,11 +79,16 @@ public int getMapKeyArgNum() { "/{clusterName}/CONFIGS/RESOURCE/{resourceName}"); template.addEntry(ConfigScopeProperty.PARTITION, 2, "/{clusterName}/CONFIGS/RESOURCE/{resourceName}"); + template.addEntry(ConfigScopeProperty.MONITORING, 2, + "/{clusterName}/CONFIGS/MONITORING/{configName}"); + template.addEntry(ConfigScopeProperty.ALERT, 2, "/{clusterName}/CONFIGS/ALERT/{configName}"); // get children template.addEntry(ConfigScopeProperty.CLUSTER, 1, "/{clusterName}/CONFIGS/CLUSTER"); template.addEntry(ConfigScopeProperty.PARTICIPANT, 1, "/{clusterName}/CONFIGS/PARTICIPANT"); template.addEntry(ConfigScopeProperty.RESOURCE, 1, "/{clusterName}/CONFIGS/RESOURCE"); + template.addEntry(ConfigScopeProperty.MONITORING, 1, "/{clusterName}/CONFIGS/MONITORING"); + template.addEntry(ConfigScopeProperty.ALERT, 1, "/{clusterName}/CONFIGS/ALERT"); } final ConfigScopeProperty _type; @@ -92,6 +99,14 @@ public int getMapKeyArgNum() { */ final String _participantName; + /** + * this is the monitoring config name if type is MONITORING, or null otherwise + */ + final String _monitoringConfigName; + final String _alertConfigName; + + final String _resourceName; + final String _zkPath; final String _mapKey; @@ -131,6 +146,27 @@ public HelixConfigScope(ConfigScopeProperty type, List zkPathKeys, Strin _participantName = null; } + // init resourceName + if (type == ConfigScopeProperty.RESOURCE && _isFullKey) { + _resourceName = zkPathKeys.get(1); + } else { + _resourceName = null; + } + + // init monitoring config name + if (type == ConfigScopeProperty.MONITORING && _isFullKey) { + _monitoringConfigName = zkPathKeys.get(1); + } else { + _monitoringConfigName = null; + } + + // init alert config name + if (type == ConfigScopeProperty.ALERT && _isFullKey) { + _alertConfigName = zkPathKeys.get(1); + } else { + _alertConfigName = null; + } + _zkPath = template.instantiate(type, zkPathKeys.toArray(new String[0])); _mapKey = mapKey; } @@ -159,6 +195,30 @@ public String getParticipantName() { return _participantName; } + /** + * Get the resource name if it exists + * @return the resource name if the type is RESOURCE, or null + */ + public String getResourceName() { + return _resourceName; + } + + /** + * Get the monitoring config name if it exists + * @return the monitoring config name if the type is MONITORING, or null + */ + public String getMonitoringConfigName() { + return _monitoringConfigName; + } + + /** + * Get the alert config name if exists + * @return the alert config name if the type is ALERT, or null otherwise + */ + public String getAlertConfigName() { + return _alertConfigName; + } + /** * Get the path to the corresponding ZNode * @return a Zookeeper path diff --git a/helix-core/src/main/java/org/apache/helix/model/IdealState.java b/helix-core/src/main/java/org/apache/helix/model/IdealState.java index 463369a4ee..173e251e9a 100644 --- a/helix-core/src/main/java/org/apache/helix/model/IdealState.java +++ b/helix-core/src/main/java/org/apache/helix/model/IdealState.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -28,10 +29,31 @@ import java.util.TreeSet; import org.apache.helix.HelixConstants; +import org.apache.helix.HelixDefinedState; import org.apache.helix.HelixProperty; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.config.NamespacedConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; +import org.apache.helix.controller.rebalancer.HelixRebalancer; +import org.apache.helix.controller.rebalancer.RebalancerRef; import org.apache.log4j.Logger; +import com.google.common.base.Enums; +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimaps; +import com.google.common.collect.Sets; + /** * The ideal states of all partitions in a resource */ @@ -89,6 +111,14 @@ public IdealState(String resourceName) { super(resourceName); } + /** + * Instantiate an ideal state for a resource + * @param resourceId the id of the resource + */ + public IdealState(ResourceId resourceId) { + super(resourceId.stringify()); + } + /** * Instantiate an ideal state from a record * @param record ZNRecord corresponding to an ideal state @@ -105,6 +135,14 @@ public String getResourceName() { return _record.getId(); } + /** + * Get the associated resource + * @return the id of the resource + */ + public ResourceId getResourceId() { + return ResourceId.from(getResourceName()); + } + /** * Get the rebalance mode of the ideal state * @param mode {@link IdealStateModeProperty} @@ -136,7 +174,7 @@ public int getMaxPartitionsPerInstance() { } /** - * Define a custom rebalancer that implements {@link Rebalancer} + * Define a custom rebalancer that implements {@link HelixRebalancer} * @param rebalancerClassName the name of the custom rebalancing class */ public void setRebalancerClassName(String rebalancerClassName) { @@ -152,6 +190,30 @@ public String getRebalancerClassName() { return _record.getSimpleField(IdealStateProperty.REBALANCER_CLASS_NAME.toString()); } + /** + * Set a reference to the user-defined rebalancer associated with this resource(if any) + * @param rebalancerRef a reference to a user-defined rebalancer + */ + public void setRebalancerRef(RebalancerRef rebalancerRef) { + if (rebalancerRef != null) { + setRebalancerClassName(rebalancerRef.toString()); + } else { + setRebalancerClassName(null); + } + } + + /** + * Get a reference to the user-defined rebalancer associated with this resource(if any) + * @return RebalancerRef + */ + public RebalancerRef getRebalancerRef() { + String className = getRebalancerClassName(); + if (className != null) { + return RebalancerRef.from(getRebalancerClassName()); + } + return null; + } + /** * Set the maximum number of partitions of this resource that an instance can serve * @param max the maximum number of partitions supported @@ -199,16 +261,30 @@ public void setPartitionState(String partitionName, String instanceName, String _record.getMapField(partitionName).put(instanceName, state); } + /** + * Set the preferred participant placement and state for a partition replica + * @param partitionId the replica to set + * @param participantId the assigned participant + * @param state the replica state in this instance + */ + public void setPartitionState(PartitionId partitionId, ParticipantId participantId, State state) { + Map mapField = _record.getMapField(partitionId.stringify()); + if (mapField == null) { + _record.setMapField(partitionId.stringify(), new TreeMap()); + } + _record.getMapField(partitionId.stringify()).put(participantId.stringify(), state.toString()); + } + /** * Get all of the partitions * @return a set of partition names */ public Set getPartitionSet() { if (getRebalanceMode() == RebalanceMode.SEMI_AUTO - || getRebalanceMode() == RebalanceMode.FULL_AUTO - || getRebalanceMode() == RebalanceMode.USER_DEFINED) { + || getRebalanceMode() == RebalanceMode.FULL_AUTO) { return _record.getListFields().keySet(); - } else if (getRebalanceMode() == RebalanceMode.CUSTOMIZED) { + } else if (getRebalanceMode() == RebalanceMode.CUSTOMIZED + || getRebalanceMode() == RebalanceMode.USER_DEFINED) { return _record.getMapFields().keySet(); } else { logger.error("Invalid ideal state mode:" + getResourceName()); @@ -216,6 +292,18 @@ public Set getPartitionSet() { } } + /** + * Get all of the partitions + * @return a set of partitions + */ + public Set getPartitionIdSet() { + Set partitionSet = Sets.newHashSet(); + for (String partitionName : getPartitionSet()) { + partitionSet.add(PartitionId.from(partitionName)); + } + return partitionSet; + } + /** * Get the current mapping of a partition * @param partitionName the name of the partition @@ -225,6 +313,47 @@ public Map getInstanceStateMap(String partitionName) { return _record.getMapField(partitionName); } + /** + * Set the current mapping of a partition + * @param partitionName the partition to set + * @param instanceStateMap (participant name, state) pairs + */ + public void setInstanceStateMap(String partitionName, Map instanceStateMap) { + _record.setMapField(partitionName, instanceStateMap); + } + + /** + * Set the current mapping of a partition + * @param partitionId the partition to set + * @param participantStateMap (participant id, state) pairs + */ + public void setParticipantStateMap(PartitionId partitionId, + Map participantStateMap) { + Map rawMap = new HashMap(); + for (ParticipantId participantId : participantStateMap.keySet()) { + rawMap.put(participantId.stringify(), participantStateMap.get(participantId).toString()); + } + _record.setMapField(partitionId.stringify(), rawMap); + } + + /** + * Get the current mapping of a partition + * @param partitionId the name of the partition + * @return the instances where the replicas live and the state of each + */ + public Map getParticipantStateMap(PartitionId partitionId) { + Map instanceStateMap = getInstanceStateMap(partitionId.stringify()); + Map participantStateMap = Maps.newHashMap(); + if (instanceStateMap != null) { + for (String participantId : instanceStateMap.keySet()) { + participantStateMap.put(ParticipantId.from(participantId), + State.from(instanceStateMap.get(participantId))); + } + return participantStateMap; + } + return null; + } + /** * Get the instances who host replicas of a partition * @param partitionName the partition to look up @@ -232,8 +361,7 @@ public Map getInstanceStateMap(String partitionName) { */ public Set getInstanceSet(String partitionName) { if (getRebalanceMode() == RebalanceMode.SEMI_AUTO - || getRebalanceMode() == RebalanceMode.FULL_AUTO - || getRebalanceMode() == RebalanceMode.USER_DEFINED) { + || getRebalanceMode() == RebalanceMode.FULL_AUTO) { List prefList = _record.getListField(partitionName); if (prefList != null) { return new TreeSet(prefList); @@ -241,7 +369,8 @@ public Set getInstanceSet(String partitionName) { logger.warn(partitionName + " does NOT exist"); return Collections.emptySet(); } - } else if (getRebalanceMode() == RebalanceMode.CUSTOMIZED) { + } else if (getRebalanceMode() == RebalanceMode.CUSTOMIZED + || getRebalanceMode() == RebalanceMode.USER_DEFINED) { Map stateMap = _record.getMapField(partitionName); if (stateMap != null) { return new TreeSet(stateMap.keySet()); @@ -253,7 +382,41 @@ public Set getInstanceSet(String partitionName) { logger.error("Invalid ideal state mode: " + getResourceName()); return Collections.emptySet(); } + } + /** + * Get the participants who host replicas of a partition + * @param partitionId the partition to look up + * @return set of participant ids + */ + public Set getParticipantSet(PartitionId partitionId) { + Set participantSet = Sets.newHashSet(); + for (String participantName : getInstanceSet(partitionId.stringify())) { + participantSet.add(ParticipantId.from(participantName)); + } + return participantSet; + } + + /** + * Set the preference list of a partition + * @param partitionName the name of the partition to set + * @param preferenceList a list of participants that can serve replicas of the partition + */ + public void setPreferenceList(String partitionName, List preferenceList) { + _record.setListField(partitionName, preferenceList); + } + + /** + * Set the preference list of a partition + * @param partitionId the id of the partition to set + * @param preferenceList a list of participants that can serve replicas of the partition + */ + public void setPreferenceList(PartitionId partitionId, List preferenceList) { + List rawPreferenceList = new ArrayList(); + for (ParticipantId participantId : preferenceList) { + rawPreferenceList.add(participantId.stringify()); + } + _record.setListField(partitionId.stringify(), rawPreferenceList); } /** @@ -271,6 +434,23 @@ public List getPreferenceList(String partitionName) { return null; } + /** + * Get the preference list of a partition + * @param partitionId the partition id + * @return an ordered list of participants that can serve replicas of the partition + */ + public List getPreferenceList(PartitionId partitionId) { + List preferenceList = Lists.newArrayList(); + List preferenceStringList = getPreferenceList(partitionId.stringify()); + if (preferenceStringList != null) { + for (String participantName : preferenceStringList) { + preferenceList.add(ParticipantId.from(participantName)); + } + return preferenceList; + } + return null; + } + /** * Get the state model associated with this resource * @return an identifier of the state model @@ -279,6 +459,14 @@ public String getStateModelDefRef() { return _record.getSimpleField(IdealStateProperty.STATE_MODEL_DEF_REF.toString()); } + /** + * Get the state model associated with this resource + * @return an identifier of the state model + */ + public StateModelDefId getStateModelDefId() { + return StateModelDefId.from(getStateModelDefRef()); + } + /** * Set the state model associated with this resource * @param stateModel state model identifier @@ -287,6 +475,14 @@ public void setStateModelDefRef(String stateModel) { _record.setSimpleField(IdealStateProperty.STATE_MODEL_DEF_REF.toString(), stateModel); } + /** + * Set the state model associated with this resource + * @param stateModel state model identifier + */ + public void setStateModelDefId(StateModelDefId stateModelDefId) { + setStateModelDefRef(stateModelDefId.stringify()); + } + /** * Set the number of partitions of this resource * @param numPartitions the number of partitions @@ -352,7 +548,7 @@ public String getReplicas() { break; default: replica = "0"; - logger.error("could NOT determine replicas. set to 0"); + logger.warn("could NOT determine replicas. set to 0"); break; } } @@ -368,6 +564,16 @@ public void setStateModelFactoryName(String name) { _record.setSimpleField(IdealStateProperty.STATE_MODEL_FACTORY_NAME.toString(), name); } + /** + * Set the state model factory associated with this resource + * @param name state model factory id + */ + public void setStateModelFactoryId(StateModelFactoryId stateModelFactoryId) { + if (stateModelFactoryId != null) { + setStateModelFactoryName(stateModelFactoryId.stringify()); + } + } + /** * Get the state model factory associated with this resource * @return state model factory name @@ -377,6 +583,14 @@ public String getStateModelFactoryName() { HelixConstants.DEFAULT_STATE_MODEL_FACTORY); } + /** + * Get the state model factory associated with this resource + * @return state model factory id + */ + public StateModelFactoryId getStateModelFactoryId() { + return StateModelFactoryId.from(getStateModelFactoryName()); + } + /** * Set the frequency with which to rebalance * @return the rebalancing timer period @@ -398,8 +612,7 @@ public boolean isValid() { return false; } - if (getRebalanceMode() == RebalanceMode.SEMI_AUTO - || getRebalanceMode() == RebalanceMode.USER_DEFINED) { + if (getRebalanceMode() == RebalanceMode.SEMI_AUTO) { String replicaStr = getReplicas(); if (replicaStr == null) { logger.error("invalid ideal-state. missing replicas in auto mode. record was: " + _record); @@ -443,6 +656,42 @@ public String getInstanceGroupTag() { return _record.getSimpleField(IdealStateProperty.INSTANCE_GROUP_TAG.toString()); } + /** + * Update the ideal state from a ResourceAssignment computed during a rebalance + * @param assignment the new resource assignment + * @param stateModelDef state model of the resource + */ + public void updateFromAssignment(ResourceAssignment assignment, StateModelDefinition stateModelDef) { + // clear all preference lists and maps + _record.getMapFields().clear(); + _record.getListFields().clear(); + + // assign a partition at a time + for (PartitionId partition : assignment.getMappedPartitionIds()) { + List preferenceList = new ArrayList(); + Map participantStateMap = new HashMap(); + + // invert the map to get in state order + Map replicaMap = assignment.getReplicaMap(partition); + ListMultimap inverseMap = ArrayListMultimap.create(); + Multimaps.invertFrom(Multimaps.forMap(replicaMap), inverseMap); + + // update the ideal state in order of state priorities + for (State state : stateModelDef.getTypedStatesPriorityList()) { + if (!state.equals(State.from(HelixDefinedState.DROPPED)) + && !state.equals(State.from(HelixDefinedState.ERROR))) { + List stateParticipants = inverseMap.get(state); + for (ParticipantId participant : stateParticipants) { + preferenceList.add(participant); + participantStateMap.put(participant, state); + } + } + } + setPreferenceList(partition, preferenceList); + setParticipantStateMap(partition, participantStateMap); + } + } + private RebalanceMode normalizeRebalanceMode(IdealStateModeProperty mode) { RebalanceMode property; switch (mode) { @@ -484,4 +733,152 @@ private IdealStateModeProperty denormalizeRebalanceMode(RebalanceMode rebalancer } return property; } + + /** + * Parse a rebalance mode from a string. It can also understand IdealStateModeProperty values + * @param mode string containing a RebalanceMode value + * @param defaultMode the mode to use if the string is not valid + * @return converted RebalanceMode value + */ + public RebalanceMode rebalanceModeFromString(String mode, RebalanceMode defaultMode) { + RebalanceMode rebalanceMode = defaultMode; + try { + rebalanceMode = RebalanceMode.valueOf(mode); + } catch (Exception rebalanceModeException) { + try { + IdealStateModeProperty oldMode = IdealStateModeProperty.valueOf(mode); + rebalanceMode = normalizeRebalanceMode(oldMode); + } catch (Exception e) { + logger.error(e); + } + } + return rebalanceMode; + } + + /** + * Get the non-Helix simple fields from this property and add them to a UserConfig + * @param userConfig the user config to update + */ + public void updateUserConfig(UserConfig userConfig) { + for (String simpleField : _record.getSimpleFields().keySet()) { + Optional enumField = + Enums.getIfPresent(IdealStateProperty.class, simpleField); + if (!simpleField.contains(NamespacedConfig.PREFIX_CHAR + "") && !enumField.isPresent()) { + userConfig.setSimpleField(simpleField, _record.getSimpleField(simpleField)); + } + } + } + + /** + * Convert a preference list of strings into a preference list of participants + * @param rawPreferenceList the list of strings representing participant names + * @return converted list + */ + public static List preferenceListFromStringList(List rawPreferenceList) { + if (rawPreferenceList == null) { + return Collections.emptyList(); + } + return Lists.transform(new ArrayList(rawPreferenceList), + new Function() { + @Override + public ParticipantId apply(String participantName) { + return ParticipantId.from(participantName); + } + }); + } + + /** + * Convert preference lists of strings into preference lists of participants + * @param rawPreferenceLists a map of partition name to a list of participant names + * @return converted lists as a map + */ + public static Map> preferenceListsFromStringLists( + Map> rawPreferenceLists) { + if (rawPreferenceLists == null) { + return Collections.emptyMap(); + } + Map> preferenceLists = + new HashMap>(); + for (String partitionId : rawPreferenceLists.keySet()) { + preferenceLists.put(PartitionId.from(partitionId), + preferenceListFromStringList(rawPreferenceLists.get(partitionId))); + } + return preferenceLists; + } + + /** + * Convert a preference list of participants into a preference list of strings + * @param preferenceList the list of strings representing participant ids + * @return converted list + */ + public static List stringListFromPreferenceList(List preferenceList) { + if (preferenceList == null) { + return Collections.emptyList(); + } + return Lists.transform(new ArrayList(preferenceList), + new Function() { + @Override + public String apply(ParticipantId participantId) { + return participantId.stringify(); + } + }); + } + + /** + * Convert preference lists of participants into preference lists of strings + * @param preferenceLists a map of partition id to a list of participant ids + * @return converted lists as a map + */ + public static Map> stringListsFromPreferenceLists( + Map> preferenceLists) { + if (preferenceLists == null) { + return Collections.emptyMap(); + } + Map> rawPreferenceLists = new HashMap>(); + for (PartitionId partitionId : preferenceLists.keySet()) { + rawPreferenceLists.put(partitionId.stringify(), + stringListFromPreferenceList(preferenceLists.get(partitionId))); + } + return rawPreferenceLists; + } + + /** + * Convert a partition mapping as strings into a participant state map + * @param rawMap the map of participant name to state + * @return converted map + */ + public static Map participantStateMapFromStringMap( + Map rawMap) { + return ResourceAssignment.replicaMapFromStringMap(rawMap); + } + + /** + * Convert a full state mapping as strings into participant state maps + * @param rawMaps the map of partition name to participant name and state + * @return converted maps + */ + public static Map> participantStateMapsFromStringMaps( + Map> rawMaps) { + return ResourceAssignment.replicaMapsFromStringMaps(rawMaps); + } + + /** + * Convert a partition mapping into a mapping of string names + * @param participantStateMap the map of participant id to state + * @return converted map + */ + public static Map stringMapFromParticipantStateMap( + Map participantStateMap) { + return ResourceAssignment.stringMapFromReplicaMap(participantStateMap); + } + + /** + * Convert a full state mapping into a mapping of string names + * @param participantStateMaps the map of partition id to participant id and state + * @return converted maps + */ + public static Map> stringMapsFromParticipantStateMaps( + Map> participantStateMaps) { + return ResourceAssignment.stringMapsFromReplicaMaps(participantStateMaps); + } } diff --git a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java index eb1c652f94..35b4bd45e3 100644 --- a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java +++ b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java @@ -27,7 +27,13 @@ import org.apache.helix.HelixProperty; import org.apache.helix.ZNRecord; -import org.apache.log4j.Logger; +import org.apache.helix.api.config.NamespacedConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; + +import com.google.common.base.Enums; +import com.google.common.base.Optional; /** * Instance configurations @@ -44,8 +50,6 @@ public enum InstanceConfigProperty { TAG_LIST } - private static final Logger _logger = Logger.getLogger(InstanceConfig.class.getName()); - /** * Instantiate for a specific instance * @param instanceId the instance identifier @@ -54,6 +58,14 @@ public InstanceConfig(String instanceId) { super(instanceId); } + /** + * Instantiate for a specific instance + * @param participantId the instance identifier + */ + public InstanceConfig(ParticipantId participantId) { + super(participantId.stringify()); + } + /** * Instantiate with a pre-populated record * @param record a ZNRecord corresponding to an instance configuration @@ -213,6 +225,15 @@ public void setInstanceEnabledForPartition(String partitionName, boolean enabled _record.setListField(InstanceConfigProperty.HELIX_DISABLED_PARTITION.toString(), list); } + /** + * Set the enabled state for a partition on this instance + * @param partitionId the partition to set + * @param enabled true to enable, false to disable + */ + public void setParticipantEnabledForPartition(PartitionId partitionId, boolean enabled) { + setInstanceEnabledForPartition(partitionId.stringify(), enabled); + } + @Override public boolean equals(Object obj) { if (obj instanceof InstanceConfig) { @@ -238,6 +259,44 @@ public String getInstanceName() { return _record.getId(); } + /** + * Get the identifier of this participant + * @return the participant id + */ + public ParticipantId getParticipantId() { + return ParticipantId.from(getInstanceName()); + } + + /** + * Get a backward-compatible participant user config + * @return UserConfig + */ + public UserConfig getUserConfig() { + UserConfig userConfig = UserConfig.from(this); + for (String simpleField : _record.getSimpleFields().keySet()) { + Optional enumField = + Enums.getIfPresent(InstanceConfigProperty.class, simpleField); + if (!simpleField.contains(NamespacedConfig.PREFIX_CHAR + "") && !enumField.isPresent()) { + userConfig.setSimpleField(simpleField, _record.getSimpleField(simpleField)); + } + } + for (String listField : _record.getListFields().keySet()) { + Optional enumField = + Enums.getIfPresent(InstanceConfigProperty.class, listField); + if (!listField.contains(NamespacedConfig.PREFIX_CHAR + "") && !enumField.isPresent()) { + userConfig.setListField(listField, _record.getListField(listField)); + } + } + for (String mapField : _record.getMapFields().keySet()) { + Optional enumField = + Enums.getIfPresent(InstanceConfigProperty.class, mapField); + if (!mapField.contains(NamespacedConfig.PREFIX_CHAR + "") && !enumField.isPresent()) { + userConfig.setMapField(mapField, _record.getMapField(mapField)); + } + } + return userConfig; + } + @Override public boolean isValid() { // HELIX-65: remove check for hostname/port existence diff --git a/helix-core/src/main/java/org/apache/helix/model/Leader.java b/helix-core/src/main/java/org/apache/helix/model/Leader.java new file mode 100644 index 0000000000..1aaf4658e9 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/Leader.java @@ -0,0 +1,98 @@ +package org.apache.helix.model; + +/* + * 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.ZNRecord; +import org.apache.helix.api.id.ControllerId; + +/** + * A special live instance that represents the leader controller + */ +public class Leader extends LiveInstance { + public enum LeaderProperty { + ZKPROPERTYTRANSFERURL, + MONITORING_HOST, + MONITORING_PORT + } + + /** + * Instantiate with a controller identifier + * @param controllerId typed controller identifier + */ + public Leader(ControllerId controllerId) { + super(controllerId.toString()); + } + + /** + * Instantiate with a pre-populated record + * @param record ZNRecord corresponding to a live instance + */ + public Leader(ZNRecord record) { + super(record); + } + + /** + * Get a web service URL where ZK properties can be transferred to + * @return a fully-qualified URL + */ + public String getWebserviceUrl() { + return _record.getSimpleField(LeaderProperty.ZKPROPERTYTRANSFERURL.toString()); + } + + /** + * Set a web service URL where ZK properties can be transferred to + * @param url a fully-qualified URL + */ + public void setWebserviceUrl(String url) { + _record.setSimpleField(LeaderProperty.ZKPROPERTYTRANSFERURL.toString(), url); + } + + /** + * Get the monitoring port attached to this instance + * @return port, or -1 + */ + public int getMonitoringPort() { + return _record.getIntField(LeaderProperty.MONITORING_PORT.toString(), -1); + } + + /** + * Set the monitoring port attached to this instance + * @param port the port to contact for monitoring + */ + public void setMonitoringPort(int port) { + _record.setIntField(LeaderProperty.MONITORING_PORT.toString(), port); + } + + /** + * Get the monitoring host attached to this instance + * @return host, or null + */ + public String getMonitoringHost() { + return _record.getSimpleField(LeaderProperty.MONITORING_HOST.toString()); + } + + /** + * Set the monitoring host attached to this instance + * @param host the host to contact for monitoring + */ + public void setMonitoringHost(String host) { + _record.setSimpleField(LeaderProperty.MONITORING_HOST.toString(), host); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java b/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java index 75e0cf398f..8f919fb3f8 100644 --- a/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java +++ b/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java @@ -21,6 +21,10 @@ import org.apache.helix.HelixProperty; import org.apache.helix.ZNRecord; +import org.apache.helix.api.HelixVersion; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ProcId; +import org.apache.helix.api.id.SessionId; import org.apache.log4j.Logger; /** @@ -33,8 +37,7 @@ public class LiveInstance extends HelixProperty { public enum LiveInstanceProperty { SESSION_ID, HELIX_VERSION, - LIVE_INSTANCE, - ZKPROPERTYTRANSFERURL + LIVE_INSTANCE } private static final Logger _logger = Logger.getLogger(LiveInstance.class.getName()); @@ -47,6 +50,14 @@ public LiveInstance(String id) { super(id); } + /** + * Instantiate with an participant identifier + * @param id participant identifier + */ + public LiveInstance(ParticipantId id) { + super(id.stringify()); + } + /** * Instantiate with a pre-populated record * @param record ZNRecord corresponding to a live instance @@ -64,7 +75,15 @@ public void setSessionId(String sessionId) { } /** - * Get the session that this instance corresponds to + * Get the session that this participant corresponds to + * @return session identifier + */ + public SessionId getTypedSessionId() { + return SessionId.from(getSessionId()); + } + + /** + * Get the session that this participant corresponds to * @return session identifier */ public String getSessionId() { @@ -80,7 +99,23 @@ public String getInstanceName() { } /** - * Get the version of Helix that this instance is running + * Get the id of this participant + * @return participant id + */ + public ParticipantId getParticipantId() { + return ParticipantId.from(getInstanceName()); + } + + /** + * Get the version of Helix that this participant is running + * @return the version + */ + public HelixVersion getTypedHelixVersion() { + return HelixVersion.from(getHelixVersion()); + } + + /** + * Get the version of Helix that this participant is running * @return the version */ public String getHelixVersion() { @@ -103,6 +138,14 @@ public String getLiveInstance() { return _record.getSimpleField(LiveInstanceProperty.LIVE_INSTANCE.toString()); } + /** + * Get an identifier that represents the instance and where it is located + * @return process identifier + */ + public ProcId getProcessId() { + return ProcId.from(getLiveInstance()); + } + /** * Set an identifier that represents the process * @param liveInstance process identifier, e.g. process_id@host @@ -119,29 +162,13 @@ public long getModifiedTime() { return _record.getModifiedTime(); } - /** - * Get a web service URL where ZK properties can be transferred to - * @return a fully-qualified URL - */ - public String getWebserviceUrl() { - return _record.getSimpleField(LiveInstanceProperty.ZKPROPERTYTRANSFERURL.toString()); - } - - /** - * Set a web service URL where ZK properties can be transferred to - * @param url a fully-qualified URL - */ - public void setWebserviceUrl(String url) { - _record.setSimpleField(LiveInstanceProperty.ZKPROPERTYTRANSFERURL.toString(), url); - } - @Override public boolean isValid() { - if (getSessionId() == null) { + if (getTypedSessionId() == null) { _logger.error("liveInstance does not have session id. id:" + _record.getId()); return false; } - if (getHelixVersion() == null) { + if (getTypedHelixVersion() == null) { _logger.error("liveInstance does not have CLM verion. id:" + _record.getId()); return false; } diff --git a/helix-core/src/main/java/org/apache/helix/model/Message.java b/helix-core/src/main/java/org/apache/helix/model/Message.java index d599b8bf09..5dbb57ceaa 100644 --- a/helix-core/src/main/java/org/apache/helix/model/Message.java +++ b/helix-core/src/main/java/org/apache/helix/model/Message.java @@ -27,12 +27,24 @@ import java.util.Map; import java.util.UUID; +import org.apache.helix.HelixConstants; import org.apache.helix.HelixException; import org.apache.helix.HelixProperty; import org.apache.helix.InstanceType; import org.apache.helix.PropertyKey; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; +import org.apache.helix.controller.stages.ClusterEvent; +import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory; + +import com.google.common.collect.Lists; /** * Messages sent internally among nodes in the system to respond to changes in state. @@ -48,7 +60,8 @@ public enum MessageType { CONTROLLER_MSG, TASK_REPLY, NO_OP, - PARTICIPANT_ERROR_REPORT + PARTICIPANT_ERROR_REPORT, + ALERT; }; /** @@ -80,7 +93,8 @@ public enum Attributes { STATE_MODEL_FACTORY_NAME, BUCKET_SIZE, PARENT_MSG_ID, // used for group message mode - INNER_MESSAGE + INNER_MESSAGE, + ALERT_NAME; } /** @@ -111,23 +125,60 @@ public int compare(Message m1, Message m2) { * @param type the message category * @param msgId unique message identifier */ - public Message(MessageType type, String msgId) { + public Message(MessageType type, MessageId msgId) { this(type.toString(), msgId); } + /** + * Instantiate a message + * @param type the message category + * @param msgId unique message identifier + */ + public Message(MessageType type, String msgId) { + this(type, MessageId.from(msgId)); + } + /** * Instantiate a message * @param type {@link MessageType} as a string or a custom message type * @param msgId unique message identifier */ - public Message(String type, String msgId) { - super(new ZNRecord(msgId)); + public Message(String type, MessageId msgId) { + super(new ZNRecord(msgId.stringify())); _record.setSimpleField(Attributes.MSG_TYPE.toString(), type); - setMsgId(msgId); + setMessageId(msgId); setMsgState(MessageState.NEW); _record.setLongField(Attributes.CREATE_TIMESTAMP.toString(), new Date().getTime()); } + /** + * Instantiate a message + * @param type {@link MessageType} as a string or a custom message type + * @param msgId unique message identifier + */ + public Message(String type, String msgId) { + this(type, MessageId.from(msgId)); + } + + /** + * Instantiate a message with a new id + * @param record a ZNRecord corresponding to a message + * @param id unique message identifier + */ + public Message(ZNRecord record, MessageId id) { + super(new ZNRecord(record, id.stringify())); + setMessageId(id); + } + + /** + * Instantiate a message with a new id + * @param record a ZNRecord corresponding to a message + * @param id unique message identifier + */ + public Message(ZNRecord record, String id) { + this(record, MessageId.from(id)); + } + /** * Instantiate a message * @param record a ZNRecord corresponding to a message @@ -143,21 +194,27 @@ public Message(ZNRecord record) { } /** - * Set the time that the message was created - * @param timestamp a UNIX timestamp + * Convert a string map to a message + * @param msgStrMap + * @return message */ - public void setCreateTimeStamp(long timestamp) { - _record.setLongField(Attributes.CREATE_TIMESTAMP.toString(), timestamp); + public static Message toMessage(Map msgStrMap) { + String msgId = msgStrMap.get(Attributes.MSG_ID.name()); + if (msgId == null) { + throw new IllegalArgumentException("Missing msgId in message string map: " + msgStrMap); + } + + ZNRecord record = new ZNRecord(msgId); + record.getSimpleFields().putAll(msgStrMap); + return new Message(record); } /** - * Instantiate a message with a new id - * @param record a ZNRecord corresponding to a message - * @param id unique message identifier + * Set the time that the message was created + * @param timestamp a UNIX timestamp */ - public Message(ZNRecord record, String id) { - super(new ZNRecord(record, id)); - setMsgId(id); + public void setCreateTimeStamp(long timestamp) { + _record.setLongField(Attributes.CREATE_TIMESTAMP.toString(), timestamp); } /** @@ -192,6 +249,14 @@ public String getMsgType() { return _record.getSimpleField(Attributes.MSG_TYPE.toString()); } + /** + * Get the session identifier of the destination node + * @return session identifier + */ + public SessionId getTypedTgtSessionId() { + return SessionId.from(getTgtSessionId()); + } + /** * Get the session identifier of the destination node * @return session identifier @@ -200,6 +265,16 @@ public String getTgtSessionId() { return _record.getSimpleField(Attributes.TGT_SESSION_ID.toString()); } + /** + * Set the session identifier of the destination node + * @param tgtSessionId session identifier + */ + public void setTgtSessionId(SessionId tgtSessionId) { + if (tgtSessionId != null) { + setTgtSessionId(tgtSessionId.stringify()); + } + } + /** * Set the session identifier of the destination node * @param tgtSessionId session identifier @@ -208,6 +283,14 @@ public void setTgtSessionId(String tgtSessionId) { _record.setSimpleField(Attributes.TGT_SESSION_ID.toString(), tgtSessionId); } + /** + * Get the session identifier of the source node + * @return session identifier + */ + public SessionId getTypedSrcSessionId() { + return SessionId.from(getSrcSessionId()); + } + /** * Get the session identifier of the source node * @return session identifier @@ -216,6 +299,16 @@ public String getSrcSessionId() { return _record.getSimpleField(Attributes.SRC_SESSION_ID.toString()); } + /** + * Set the session identifier of the source node + * @param srcSessionId session identifier + */ + public void setSrcSessionId(SessionId srcSessionId) { + if (srcSessionId != null) { + setSrcSessionId(srcSessionId.stringify()); + } + } + /** * Set the session identifier of the source node * @param srcSessionId session identifier @@ -224,6 +317,14 @@ public void setSrcSessionId(String srcSessionId) { _record.setSimpleField(Attributes.SRC_SESSION_ID.toString(), srcSessionId); } + /** + * Get the session identifier of the node that executes the message + * @return session identifier + */ + public SessionId getTypedExecutionSessionId() { + return SessionId.from(getExecutionSessionId()); + } + /** * Get the session identifier of the node that executes the message * @return session identifier @@ -232,6 +333,16 @@ public String getExecutionSessionId() { return _record.getSimpleField(Attributes.EXE_SESSION_ID.toString()); } + /** + * Set the session identifier of the node that executes the message + * @param exeSessionId session identifier + */ + public void setExecuteSessionId(SessionId exeSessionId) { + if (exeSessionId != null) { + setExecuteSessionId(exeSessionId.stringify()); + } + } + /** * Set the session identifier of the node that executes the message * @param exeSessionId session identifier @@ -301,13 +412,31 @@ public MessageState getMsgState() { } /** - * Set the name of the partition this message concerns - * @param partitionName + * Set the id of the partition this message concerns + * @param partitionId + */ + public void setPartitionId(PartitionId partitionId) { + if (partitionId != null) { + setPartitionName(partitionId.stringify()); + } + } + + /** + * Set the id of the partition this message concerns + * @param partitionId */ public void setPartitionName(String partitionName) { _record.setSimpleField(Attributes.PARTITION_NAME.toString(), partitionName); } + /** + * Get the unique identifier of this message + * @return message identifier + */ + public MessageId getMessageId() { + return MessageId.from(getMsgId()); + } + /** * Get the unique identifier of this message * @return message identifier @@ -316,6 +445,16 @@ public String getMsgId() { return _record.getSimpleField(Attributes.MSG_ID.toString()); } + /** + * Set the unique identifier of this message + * @param msgId message identifier + */ + public void setMessageId(MessageId msgId) { + if (msgId != null) { + setMsgId(msgId.stringify()); + } + } + /** * Set the unique identifier of this message * @param msgId message identifier @@ -326,15 +465,33 @@ public void setMsgId(String msgId) { /** * Set the "from state" for transition-related messages - * @param state the state name + * @param state the state + */ + public void setFromState(State state) { + if (state != null) { + setFromState(state.toString()); + } + } + + /** + * Set the "from state" for transition-related messages + * @param state the state */ public void setFromState(String state) { - _record.setSimpleField(Attributes.FROM_STATE.toString(), state); + _record.setSimpleField(Attributes.FROM_STATE.toString(), state.toString()); + } + + /** + * Get the "from-state" for transition-related messages + * @return state, or null for other message types + */ + public State getTypedFromState() { + return State.from(getFromState()); } /** * Get the "from-state" for transition-related messages - * @return state name, or null for other message types + * @return state, or null for other message types */ public String getFromState() { return _record.getSimpleField(Attributes.FROM_STATE.toString()); @@ -342,15 +499,33 @@ public String getFromState() { /** * Set the "to state" for transition-related messages - * @param state the state name + * @param state the state + */ + public void setToState(State state) { + if (state != null) { + setToState(state.toString()); + } + } + + /** + * Set the "to state" for transition-related messages + * @param state the state */ public void setToState(String state) { - _record.setSimpleField(Attributes.TO_STATE.toString(), state); + _record.setSimpleField(Attributes.TO_STATE.toString(), state.toString()); + } + + /** + * Get the "to state" for transition-related messages + * @return state, or null for other message types + */ + public State getTypedToState() { + return State.from(getToState()); } /** * Get the "to state" for transition-related messages - * @return state name, or null for other message types + * @return state, or null for other message types */ public String getToState() { return _record.getSimpleField(Attributes.TO_STATE.toString()); @@ -380,6 +555,16 @@ public Integer getGeneration() { return 1; } + /** + * Set the resource associated with this message + * @param resourceId resource name to set + */ + public void setResourceId(ResourceId resourceId) { + if (resourceId != null) { + setResourceName(resourceId.stringify()); + } + } + /** * Set the resource associated with this message * @param resourceName resource name to set @@ -388,6 +573,14 @@ public void setResourceName(String resourceName) { _record.setSimpleField(Attributes.RESOURCE_NAME.toString(), resourceName); } + /** + * Get the resource associated with this message + * @return resource name + */ + public ResourceId getResourceId() { + return ResourceId.from(getResourceName()); + } + /** * Get the resource associated with this message * @return resource name @@ -398,7 +591,15 @@ public String getResourceName() { /** * Get the resource partition associated with this message - * @return partition name + * @return partition id + */ + public PartitionId getPartitionId() { + return PartitionId.from(getPartitionName()); + } + + /** + * Get the resource partition associated with this message + * @return partition id */ public String getPartitionName() { return _record.getSimpleField(Attributes.PARTITION_NAME.toString()); @@ -413,11 +614,29 @@ public String getStateModelDef() { } /** - * Set the state model definition name + * Get the state model definition id + * @return a reference to the state model definition + */ + public StateModelDefId getStateModelDefId() { + return StateModelDefId.from(getStateModelDef()); + } + + /** + * Set the state model definition + * @param stateModelDefName a reference to the state model definition, e.g. "MasterSlave" + */ + public void setStateModelDef(StateModelDefId stateModelDefId) { + if (stateModelDefId != null) { + setStateModelDef(stateModelDefId.stringify()); + } + } + + /** + * Set the state model definition * @param stateModelDefName a reference to the state model definition, e.g. "MasterSlave" */ - public void setStateModelDef(String stateModelDefName) { - _record.setSimpleField(Attributes.STATE_MODEL_DEF.toString(), stateModelDefName); + public void setStateModelDef(String stateModelDefId) { + _record.setSimpleField(Attributes.STATE_MODEL_DEF.toString(), stateModelDefId); } /** @@ -540,6 +759,18 @@ public void setStateModelFactoryName(String factoryName) { _record.setSimpleField(Attributes.STATE_MODEL_FACTORY_NAME.toString(), factoryName); } + /** + * Set the state model factory associated with this message + * @param factoryName the name of the factory + */ + public void setStateModelFactoryId(StateModelFactoryId factoryId) { + if (factoryId != null) { + setStateModelFactoryName(factoryId.stringify()); + } else { + setStateModelFactoryName(HelixConstants.DEFAULT_STATE_MODEL_FACTORY); + } + } + // TODO: remove this. impl in HelixProperty @Override public int getBucketSize() { @@ -581,13 +812,14 @@ public String getAttribute(Attributes attr) { public static Message createReplyMessage(Message srcMessage, String instanceName, Map taskResultMap) { if (srcMessage.getCorrelationId() == null) { - throw new HelixException("Message " + srcMessage.getMsgId() + throw new HelixException("Message " + srcMessage.getMessageId() + " does not contain correlation id"); } - Message replyMessage = new Message(MessageType.TASK_REPLY, UUID.randomUUID().toString()); + Message replyMessage = + new Message(MessageType.TASK_REPLY, MessageId.from(UUID.randomUUID().toString())); replyMessage.setCorrelationId(srcMessage.getCorrelationId()); replyMessage.setResultMap(taskResultMap); - replyMessage.setTgtSessionId("*"); + replyMessage.setTgtSessionId(SessionId.from("*")); replyMessage.setMsgState(MessageState.NEW); replyMessage.setSrcName(instanceName); if (srcMessage.getSrcInstanceType() == InstanceType.CONTROLLER) { @@ -615,26 +847,27 @@ public void addPartitionName(String partitionName) { /** * Get a list of partitions associated with this message - * @return list of partition names + * @return list of partition ids */ - public List getPartitionNames() { + public List getPartitionIds() { List partitionNames = _record.getListField(Attributes.PARTITION_NAME.toString()); if (partitionNames == null) { return Collections.emptyList(); } - - return partitionNames; + List partitionIds = Lists.newArrayList(); + for (String partitionName : partitionNames) { + partitionIds.add(PartitionId.from(partitionName)); + } + return partitionIds; } - // public AtomicInteger getGroupMsgCountDown() - // { - // return _groupMsgCountDown; - // } - // - // public void setGroupMsgCountDown(AtomicInteger countDown) - // { - // _groupMsgCountDown = countDown; - // } + /** + * Get a list of partitions associated with this message + * @return list of partition ids + */ + public List getPartitionNames() { + return _record.getListField(Attributes.PARTITION_NAME.toString()); + } /** * Check if this message is targetted for a controller @@ -658,6 +891,55 @@ public PropertyKey getKey(Builder keyBuilder, String instanceName) { } } + /** + * Get timeout + * @return timeout or -1 if not available + */ + public int getTimeout() { + String timeoutStr = _record.getSimpleField(Attributes.TIMEOUT.name()); + int timeout = -1; + if (timeoutStr != null) { + try { + timeout = Integer.parseInt(timeoutStr); + } catch (Exception e) { + // ignore + } + } + return timeout; + } + + /** + * Get controller message id, used for scheduler-task-queue state model only + * @return controller message id + */ + public String getControllerMessageId() { + return _record.getSimpleField(DefaultSchedulerMessageHandlerFactory.CONTROLLER_MSG_ID); + } + + /** + * Set an inner message + * @param inner message + */ + public void setInnerMessage(Message message) { + _record.setMapField(Attributes.INNER_MESSAGE.name(), message.getRecord().getSimpleFields()); + } + + /** + * Set the cluster event generating this message + * @param event cluster event + */ + public void setClusterEvent(ClusterEvent event) { + _record.setSimpleField("ClusterEventName", event.getName()); + } + + /** + * Get the cluster event name generating this message + * @param the cluster event event name + */ + public String getClusterEventName() { + return _record.getSimpleField("ClusterEventName"); + } + private boolean isNullOrEmpty(String data) { return data == null || data.length() == 0 || data.trim().length() == 0; } @@ -669,10 +951,11 @@ public boolean isValid() { if (getMsgType().equals(MessageType.STATE_TRANSITION.toString())) { boolean isNotValid = - isNullOrEmpty(getTgtName()) || isNullOrEmpty(getPartitionName()) - || isNullOrEmpty(getResourceName()) || isNullOrEmpty(getStateModelDef()) - || isNullOrEmpty(getToState()) || isNullOrEmpty(getStateModelFactoryName()) - || isNullOrEmpty(getFromState()); + isNullOrEmpty(getTgtName()) || isNullOrEmpty(getPartitionId().stringify()) + || isNullOrEmpty(getResourceId().stringify()) || isNullOrEmpty(getStateModelDef()) + || isNullOrEmpty(getTypedToState().toString()) + || isNullOrEmpty(getStateModelFactoryName()) + || isNullOrEmpty(getTypedFromState().toString()); return !isNotValid; } diff --git a/helix-core/src/main/java/org/apache/helix/model/MonitoringConfig.java b/helix-core/src/main/java/org/apache/helix/model/MonitoringConfig.java new file mode 100644 index 0000000000..e95b545cdb --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/MonitoringConfig.java @@ -0,0 +1,87 @@ +package org.apache.helix.model; + +/* + * 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.io.InputStream; + +import org.apache.commons.io.IOUtils; +import org.apache.helix.HelixProperty; +import org.apache.helix.ZNRecord; +import org.apache.log4j.Logger; + +/** + * Wrapper for a monitoring config file + */ +public class MonitoringConfig extends HelixProperty { + private static final Logger LOG = Logger.getLogger(MonitoringConfig.class); + + /** + * Properties describing the config + */ + public enum MonitoringConfigProperty { + CONFIG_FILE + } + + /** + * Instantiate from a record + * @param record populated ZNRecord + */ + public MonitoringConfig(ZNRecord record) { + super(record); + } + + /** + * Create an empty config with a name + * @param configName the name, will map to a file name + */ + public MonitoringConfig(String configName) { + super(configName); + } + + /** + * Set the config from an input stream + * @param is an open InputStream + */ + public void setConfig(InputStream is) { + try { + String config = IOUtils.toString(is); + setConfig(config); + } catch (IOException e) { + LOG.error("Could not persist the monitoring config!"); + } + } + + /** + * Set the config from a String + * @param config a String containing the entire config + */ + public void setConfig(String config) { + _record.setSimpleField(MonitoringConfigProperty.CONFIG_FILE.toString(), config); + } + + /** + * Get the config + * @return String containing the config + */ + public String getConfig() { + return _record.getSimpleField(MonitoringConfigProperty.CONFIG_FILE.toString()); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/Partition.java b/helix-core/src/main/java/org/apache/helix/model/Partition.java index 1d694ab3f6..6a3e054f14 100644 --- a/helix-core/src/main/java/org/apache/helix/model/Partition.java +++ b/helix-core/src/main/java/org/apache/helix/model/Partition.java @@ -21,7 +21,9 @@ /** * A distinct partition of a resource + * Deprecated. Use {@link org.apache.helix.api.Partition} */ +@Deprecated public class Partition { private final String _partitionName; diff --git a/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java b/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java new file mode 100644 index 0000000000..c2741c8604 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java @@ -0,0 +1,58 @@ +package org.apache.helix.model; + +import org.apache.helix.HelixProperty; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.PartitionId; + +/* + * 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. + */ + +/** + * Persisted configuration properties for a partition + */ +public class PartitionConfiguration extends HelixProperty { + /** + * Instantiate for an id + * @param id partition id + */ + public PartitionConfiguration(PartitionId id) { + super(id.stringify()); + } + + /** + * Instantiate from a record + * @param record configuration properties + */ + public PartitionConfiguration(ZNRecord record) { + super(record); + } + + /** + * Create a new PartitionConfiguration from a UserConfig + * @param userConfig user-defined configuration properties + * @return PartitionConfiguration + */ + public static PartitionConfiguration from(UserConfig userConfig) { + PartitionConfiguration partitionConfiguration = + new PartitionConfiguration(PartitionId.from(userConfig.getId())); + partitionConfiguration.addNamespacedConfig(userConfig); + return partitionConfiguration; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/PersistentStats.java b/helix-core/src/main/java/org/apache/helix/model/PersistentStats.java deleted file mode 100644 index 9cd1934be9..0000000000 --- a/helix-core/src/main/java/org/apache/helix/model/PersistentStats.java +++ /dev/null @@ -1,128 +0,0 @@ -package org.apache.helix.model; - -/* - * 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 org.apache.helix.HelixProperty; -import org.apache.helix.ZNRecord; -import org.apache.log4j.Logger; - -/** - * Statistics for an instance - */ -public class PersistentStats extends HelixProperty { - private static final Logger _logger = Logger.getLogger(PersistentStats.class.getName()); - - /** - * High-level properties to provide context for these statistics - */ - public enum PersistentStatsProperty { - SESSION_ID, - FIELDS - } - - // private final ZNRecord _record; - - /** - * The name of the statistics ZNode - */ - public final static String nodeName = "PersistentStats"; - - /** - * Instantiate with an identifier - * @param id record identifier - */ - public PersistentStats(String id) { - super(id); - } - - /** - * Instantiate with a pre-populated record - * @param record ZNRecord with fields corresponding to persistent stats - */ - public PersistentStats(ZNRecord record) { - // _record = record; - super(record); - - } - - /* - * public PersistentStats(ZNRecord record, Stat stat) - * { - * super(record, stat); - * } - */ - - /** - * Set the session corresponding to these statistics - * @param sessionId session id - */ - public void setSessionId(String sessionId) { - _record.setSimpleField(PersistentStatsProperty.SESSION_ID.toString(), sessionId); - } - - /** - * Get the session corresponding to these statistics - * @return session id - */ - public String getSessionId() { - return _record.getSimpleField(PersistentStatsProperty.SESSION_ID.toString()); - } - - /** - * Get the instance for which these stats have been collected - * @return instance name - */ - public String getInstanceName() { - return _record.getId(); - } - - /* - * public String getVersion() - * { - * return _record.getSimpleField(CLUSTER_MANAGER_VERSION.toString()); - * } - */ - - /** - * Get all the statistics currently stored - * @return map of (stat name, stat attribute, value) - */ - public Map> getMapFields() { - return _record.getMapFields(); - } - - /** - * Get a specific statistic - * @param statName the statistic to look up - * @return map of (stat attribute, value) - */ - public Map getStatFields(String statName) { - return _record.getMapField(statName); - } - - @Override - public boolean isValid() { - // TODO Auto-generated method stub - return true; - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/model/Resource.java b/helix-core/src/main/java/org/apache/helix/model/Resource.java index 1544514b2c..437aca6b3e 100644 --- a/helix-core/src/main/java/org/apache/helix/model/Resource.java +++ b/helix-core/src/main/java/org/apache/helix/model/Resource.java @@ -24,14 +24,12 @@ import java.util.Map; import org.apache.helix.HelixConstants; -import org.apache.log4j.Logger; /** * A resource contains a set of partitions and its replicas are managed by a state model */ +@Deprecated public class Resource { - private static Logger LOG = Logger.getLogger(Resource.class); - private final String _resourceName; private final Map _partitionMap; private String _stateModelDefRef; diff --git a/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java b/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java index b0d7f1fdc8..6da3dc287f 100644 --- a/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java +++ b/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java @@ -19,6 +19,171 @@ * under the License. */ -public class ResourceAssignment { +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.helix.HelixProperty; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; + +/** + * Represents the assignments of replicas for an entire resource, keyed on partitions of the + * resource. Each partition has its replicas assigned to a node, and each replica is in a state. + * For example, if there is a partition p with 2 replicas, a valid assignment is:
+ *
+ * p: {(n1, s1), (n2, s2)}
+ *
+ * This means one replica of p is located at node n1 and is in state s1, and another is in node n2 + * and is in state s2. n1 cannot be equal to n2, but s1 can be equal to s2 if at least two replicas + * can be in s1. + */ +public class ResourceAssignment extends HelixProperty { + /** + * Initialize an empty mapping + * @param resourceId the resource being mapped + */ + public ResourceAssignment(ResourceId resourceId) { + super(resourceId.stringify()); + } + + /** + * Instantiate from a record. This supports reading the assignment directly from the backing store + * @param record backing record + */ + public ResourceAssignment(ZNRecord record) { + super(record); + } + + /** + * Get the resource for which this assignment was created + * @return resource id + */ + public ResourceId getResourceId() { + return ResourceId.from(getId()); + } + + /** + * Get the currently mapped partitions + * @return list of Partition objects (immutable) + */ + public List getMappedPartitionIds() { + ImmutableList.Builder builder = new ImmutableList.Builder(); + for (String partitionName : _record.getMapFields().keySet()) { + builder.add(PartitionId.from(partitionName)); + } + return builder.build(); + } + + /** + * Get the entire map of a resource + * @return map of partition to participant to state + */ + public Map> getResourceMap() { + return replicaMapsFromStringMaps(_record.getMapFields()); + } + + /** + * Get the participant, state pairs for a partition + * @param partition the Partition to look up + * @return map of (participant id, state) + */ + public Map getReplicaMap(PartitionId partitionId) { + Map rawReplicaMap = _record.getMapField(partitionId.stringify()); + Map replicaMap = Maps.newHashMap(); + if (rawReplicaMap != null) { + for (String participantName : rawReplicaMap.keySet()) { + replicaMap.put(ParticipantId.from(participantName), + State.from(rawReplicaMap.get(participantName))); + } + } + return replicaMap; + } + + /** + * Add participant, state pairs for a partition + * @param partitionId the partition to set + * @param replicaMap map of (participant name, state) + */ + public void addReplicaMap(PartitionId partitionId, Map replicaMap) { + Map convertedMap = Maps.newHashMap(); + for (ParticipantId participantId : replicaMap.keySet()) { + convertedMap.put(participantId.stringify(), replicaMap.get(participantId).toString()); + } + _record.setMapField(partitionId.stringify(), convertedMap); + } + + /** + * Helper for converting a map of strings to a concrete replica map + * @param rawMap map of participant name to state name + * @return map of participant id to state + */ + public static Map replicaMapFromStringMap(Map rawMap) { + if (rawMap == null) { + return Collections.emptyMap(); + } + Map replicaMap = Maps.newHashMap(); + for (String participantName : rawMap.keySet()) { + replicaMap.put(ParticipantId.from(participantName), State.from(rawMap.get(participantName))); + } + return replicaMap; + } + + /** + * Convert a full replica mapping as strings into participant state maps + * @param rawMaps the map of partition name to participant name and state + * @return converted maps + */ + public static Map> replicaMapsFromStringMaps( + Map> rawMaps) { + if (rawMaps == null) { + return Collections.emptyMap(); + } + Map> participantStateMaps = Maps.newHashMap(); + for (String partitionId : rawMaps.keySet()) { + participantStateMaps.put(PartitionId.from(partitionId), + replicaMapFromStringMap(rawMaps.get(partitionId))); + } + return participantStateMaps; + } + + /** + * Helper for converting a replica map to a map of strings + * @param replicaMap map of participant id to state + * @return map of participant name to state name + */ + public static Map stringMapFromReplicaMap(Map replicaMap) { + if (replicaMap == null) { + return Collections.emptyMap(); + } + Map rawMap = new HashMap(); + for (ParticipantId participantId : replicaMap.keySet()) { + rawMap.put(participantId.stringify(), replicaMap.get(participantId).toString()); + } + return rawMap; + } + + /** + * Convert a full state mapping into a mapping of string names + * @param replicaMaps the map of partition id to participant id and state + * @return converted maps + */ + public static Map> stringMapsFromReplicaMaps( + Map> replicaMaps) { + if (replicaMaps == null) { + return Collections.emptyMap(); + } + Map> rawMaps = Maps.newHashMap(); + for (PartitionId partitionId : replicaMaps.keySet()) { + rawMaps.put(partitionId.stringify(), stringMapFromReplicaMap(replicaMaps.get(partitionId))); + } + return rawMaps; + } } diff --git a/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java b/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java new file mode 100644 index 0000000000..f32649ff7f --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java @@ -0,0 +1,115 @@ +package org.apache.helix.model; + +import org.apache.helix.HelixProperty; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.config.NamespacedConfig; +import org.apache.helix.api.config.ResourceConfig.ResourceType; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; + +import com.google.common.base.Enums; +import com.google.common.base.Optional; + +/* + * 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. + */ + +/** + * Persisted configuration properties for a resource + */ +public class ResourceConfiguration extends HelixProperty { + public enum Fields { + TYPE + } + + /** + * Instantiate for an id + * @param id resource id + */ + public ResourceConfiguration(ResourceId id) { + super(id.stringify()); + } + + /** + * Get the resource that is rebalanced + * @return resource id + */ + public ResourceId getResourceId() { + return ResourceId.from(getId()); + } + + /** + * Instantiate from a record + * @param record configuration properties + */ + public ResourceConfiguration(ZNRecord record) { + super(record); + } + + /** + * Set the resource type + * @param type ResourceType type + */ + public void setType(ResourceType type) { + _record.setEnumField(Fields.TYPE.toString(), type); + } + + /** + * Get the resource type + * @return ResourceType type + */ + public ResourceType getType() { + return _record.getEnumField(Fields.TYPE.toString(), ResourceType.class, ResourceType.DATA); + } + + /** + * Get a backward-compatible resource user config + * @return UserConfig + */ + public UserConfig getUserConfig() { + UserConfig userConfig = UserConfig.from(this); + for (String simpleField : _record.getSimpleFields().keySet()) { + Optional enumField = Enums.getIfPresent(Fields.class, simpleField); + if (!simpleField.contains(NamespacedConfig.PREFIX_CHAR + "") && !enumField.isPresent()) { + userConfig.setSimpleField(simpleField, _record.getSimpleField(simpleField)); + } + } + for (String listField : _record.getListFields().keySet()) { + if (!listField.contains(NamespacedConfig.PREFIX_CHAR + "")) { + userConfig.setListField(listField, _record.getListField(listField)); + } + } + for (String mapField : _record.getMapFields().keySet()) { + if (!mapField.contains(NamespacedConfig.PREFIX_CHAR + "")) { + userConfig.setMapField(mapField, _record.getMapField(mapField)); + } + } + return userConfig; + } + + /** + * Get a RebalancerContext if available + * @return RebalancerContext, or null + */ + public RebalancerContext getRebalancerContext(Class clazz) { + RebalancerConfig config = new RebalancerConfig(this); + return config.getRebalancerContext(clazz); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java index 7f08b6fc7c..a74d2174ab 100644 --- a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java +++ b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java @@ -30,8 +30,12 @@ import org.apache.helix.HelixDefinedState; import org.apache.helix.HelixProperty; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.model.builder.StateTransitionTableBuilder; -import org.apache.log4j.Logger; +import org.apache.helix.model.util.StateModelDefinitionValidator; + +import com.google.common.collect.ImmutableList; /** * Describe the state model @@ -43,7 +47,6 @@ public enum StateModelDefinitionProperty { STATE_PRIORITY_LIST } - private static final Logger _logger = Logger.getLogger(StateModelDefinition.class.getName()); /** * state model's initial state */ @@ -137,6 +140,14 @@ void addDefaultTransition(String from, String to, String next) { } } + /** + * Get a concrete state model definition id + * @return StateModelDefId + */ + public StateModelDefId getStateModelDefId() { + return StateModelDefId.from(getId()); + } + /** * Get an ordered priority list of transitions * @return transitions in the form SRC-DEST, the first of which is highest priority @@ -145,6 +156,20 @@ public List getStateTransitionPriorityList() { return _stateTransitionPriorityList; } + /** + * Get an ordered priority list of transitions + * @return Transition objects, the first of which is highest priority (immutable) + */ + public List getTypedStateTransitionPriorityList() { + ImmutableList.Builder builder = new ImmutableList.Builder(); + for (String transition : getStateTransitionPriorityList()) { + String fromState = transition.substring(0, transition.indexOf('-')); + String toState = transition.substring(transition.indexOf('-') + 1); + builder.add(Transition.from(State.from(fromState), State.from(toState))); + } + return builder.build(); + } + /** * Get an ordered priority list of states * @return state names, the first of which is highest priority @@ -153,6 +178,18 @@ public List getStatesPriorityList() { return _statesPriorityList; } + /** + * Get an ordered priority list of states + * @return immutable list of states, the first of which is highest priority (immutable) + */ + public List getTypedStatesPriorityList() { + ImmutableList.Builder builder = new ImmutableList.Builder(); + for (String state : getStatesPriorityList()) { + builder.add(State.from(state)); + } + return builder.build(); + } + /** * Get the intermediate state required to transition from one state to the other * @param fromState the source @@ -167,6 +204,20 @@ public String getNextStateForTransition(String fromState, String toState) { return null; } + /** + * Get the intermediate state required to transition from one state to the other + * @param fromState the source + * @param toState the destination + * @return the intermediate state, or null if not present + */ + public State getNextStateForTransition(State fromState, State toState) { + String next = getNextStateForTransition(fromState.toString(), toState.toString()); + if (next != null) { + return State.from(getNextStateForTransition(fromState.toString(), toState.toString())); + } + return null; + } + /** * Get the starting state in the model * @return name of the initial state @@ -177,6 +228,16 @@ public String getInitialState() { return _initialState; } + /** + * Get the starting state in the model + * @return name of the initial state + */ + public State getTypedInitialState() { + // return _record.getSimpleField(StateModelDefinitionProperty.INITIAL_STATE + // .toString()); + return State.from(_initialState); + } + /** * Number of instances that can be in each state * @param state the state name @@ -186,18 +247,18 @@ public String getNumInstancesPerState(String state) { return _statesCountMap.get(state); } + /** + * Number of participants that can be in each state + * @param state the state + * @return maximum instance count per state, can be "N" or "R" + */ + public String getNumParticipantsPerState(State state) { + return _statesCountMap.get(state.toString()); + } + @Override public boolean isValid() { - if (getInitialState() == null) { - _logger.error("State model does not contain init state, statemodel:" + _record.getId()); - return false; - } - if (_record.getListField(StateModelDefinitionProperty.STATE_PRIORITY_LIST.toString()) == null) { - _logger.error("CurrentState does not contain StatesPriorityList, state model : " - + _record.getId()); - return false; - } - return true; + return StateModelDefinitionValidator.isStateModelDefinitionValid(this); } // TODO move this to model.builder package, refactor StateModelConfigGenerator to use this @@ -212,20 +273,37 @@ public static class Builder { Map stateConstraintMap; /** - * Start building a state model with a name - * @param name state model name + * Start building a state model with a id + * @param stateModelDefId state model id */ - public Builder(String name) { - this._statemodelName = name; + public Builder(StateModelDefId stateModelDefId) { + this._statemodelName = stateModelDefId.stringify(); statesMap = new HashMap(); transitionMap = new HashMap(); stateConstraintMap = new HashMap(); } + /** + * Start building a state model with a name + * @param stateModelDefId state model name + */ + public Builder(String stateModelName) { + this(StateModelDefId.from(stateModelName)); + } + /** * initial state of a replica when it starts, most commonly used initial * state is OFFLINE - * @param state + * @param initialState + */ + public Builder initialState(State initialState) { + return initialState(initialState.toString()); + } + + /** + * initial state of a replica when it starts, most commonly used initial + * state is OFFLINE + * @param initialState */ public Builder initialState(String initialState) { this.initialState = initialState; @@ -238,13 +316,36 @@ public Builder initialState(String initialState) { * STATE2 has a constraint of 3 but only one node is up then Helix will uses * the priority to see STATE constraint has to be given higher preference
* Use -1 to indicates states with no constraints, like OFFLINE - * @param states + * @param state the state to add + * @param priority the state priority, lower number is higher priority + */ + public Builder addState(State state, int priority) { + return addState(state.toString(), priority); + } + + /** + * Define all valid states using this method. Set the priority in which the + * constraints must be satisfied. Lets say STATE1 has a constraint of 1 and + * STATE2 has a constraint of 3 but only one node is up then Helix will uses + * the priority to see STATE constraint has to be given higher preference
+ * Use -1 to indicates states with no constraints, like OFFLINE + * @param state the state to add + * @param priority the state priority, lower number is higher priority */ public Builder addState(String state, int priority) { statesMap.put(state, priority); return this; } + /** + * Sets the priority to Integer.MAX_VALUE + * @param state + */ + public Builder addState(State state) { + addState(state, Integer.MAX_VALUE); + return this; + } + /** * Sets the priority to Integer.MAX_VALUE * @param state @@ -254,6 +355,23 @@ public Builder addState(String state) { return this; } + /** + * Define all legal transitions between states using this method. Priority + * is used to order the transitions. Helix tries to maximize the number of + * transitions that can be fired in parallel without violating the + * constraint. The transitions are first sorted based on priority and + * transitions are selected in a greedy way until the constriants are not + * violated. + * @param fromState source + * @param toState destination + * @param priority priority, higher value is higher priority + * @return Builder + */ + public Builder addTransition(State fromState, State toState, int priority) { + transitionMap.put(new Transition(fromState, toState), priority); + return this; + } + /** * Define all legal transitions between states using this method. Priority * is used to order the transitions. Helix tries to maximize the number of @@ -271,6 +389,18 @@ public Builder addTransition(String fromState, String toState, int priority) { return this; } + /** + * Add a state transition with maximal priority value + * @see #addTransition(String, String, int) + * @param fromState + * @param toState + * @return Builder + */ + public Builder addTransition(State fromState, State toState) { + addTransition(fromState, toState, Integer.MAX_VALUE); + return this; + } + /** * Add a state transition with maximal priority value * @see #addTransition(String, String, int) @@ -283,6 +413,16 @@ public Builder addTransition(String fromState, String toState) { return this; } + /** + * Set a maximum for replicas in this state + * @param state state name + * @param upperBound maximum + * @return Builder + */ + public Builder upperBound(State state, int upperBound) { + return upperBound(state.toString(), upperBound); + } + /** * Set a maximum for replicas in this state * @param state state name @@ -294,6 +434,24 @@ public Builder upperBound(String state, int upperBound) { return this; } + /** + * You can use this to have the bounds dynamically change based on other + * parameters.
+ * Currently support 2 values
+ * R --> Refers to the number of replicas specified during resource + * creation. This allows having different replication factor for each + * resource without having to create a different state machine.
+ * N --> Refers to all nodes in the cluster. Useful for resources that need + * to exist on all nodes. This way one can add/remove nodes without having + * the change the bounds. + * @param state + * @param bound + * @return Builder + */ + public Builder dynamicUpperBound(State state, String bound) { + return dynamicUpperBound(state.toString(), bound); + } + /** * You can use this to have the bounds dynamically change based on other * parameters.
@@ -319,6 +477,8 @@ public Builder dynamicUpperBound(String state, String bound) { */ public StateModelDefinition build() { ZNRecord record = new ZNRecord(_statemodelName); + + // get sorted state priorities by specified values ArrayList statePriorityList = new ArrayList(statesMap.keySet()); Comparator c1 = new Comparator() { @@ -328,8 +488,9 @@ public int compare(String o1, String o2) { } }; Collections.sort(statePriorityList, c1); - ArrayList transitionList = new ArrayList(transitionMap.keySet()); + // get sorted transition priorities by specified values + ArrayList transitionList = new ArrayList(transitionMap.keySet()); Comparator c2 = new Comparator() { @Override public int compare(Transition o1, Transition o2) { @@ -347,6 +508,8 @@ public int compare(Transition o1, Transition o2) { statePriorityList); record.setListField(StateModelDefinitionProperty.STATE_TRANSITION_PRIORITYLIST.toString(), transitionPriorityList); + + // compute full paths for next states StateTransitionTableBuilder stateTransitionTableBuilder = new StateTransitionTableBuilder(); Map> transitionTable = stateTransitionTableBuilder.buildTransitionTable(statePriorityList, @@ -354,6 +517,8 @@ public int compare(Transition o1, Transition o2) { for (String state : transitionTable.keySet()) { record.setMapField(state + ".next", transitionTable.get(state)); } + + // state counts for (String state : statePriorityList) { HashMap metadata = new HashMap(); if (stateConstraintMap.get(state) != null) { diff --git a/helix-core/src/main/java/org/apache/helix/model/Transition.java b/helix-core/src/main/java/org/apache/helix/model/Transition.java index 2151c44c7c..70f86353d2 100644 --- a/helix-core/src/main/java/org/apache/helix/model/Transition.java +++ b/helix-core/src/main/java/org/apache/helix/model/Transition.java @@ -1,5 +1,7 @@ package org.apache.helix.model; +import org.apache.helix.api.State; + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -23,19 +25,28 @@ * Defines a transition from one state to another */ public class Transition { - final private String _fromState; - final private String _toState; + final private State _fromState; + final private State _toState; /** * Instantiate with a source and destination state * @param fromState source name * @param toState destination name */ - public Transition(String fromState, String toState) { + public Transition(State fromState, State toState) { _fromState = fromState; _toState = toState; } + /** + * Instantiate with a source and destination state + * @param fromState source name + * @param toState destination name + */ + public Transition(String fromState, String toState) { + this(State.from(fromState), State.from(toState)); + } + @Override public String toString() { return _fromState + "-" + _toState; @@ -58,7 +69,7 @@ public boolean equals(Object that) { * Get the source state * @return source state name */ - public String getFromState() { + public State getTypedFromState() { return _fromState; } @@ -66,8 +77,33 @@ public String getFromState() { * Get the destination state * @return destination state name */ - public String getToState() { + public State getTypedToState() { return _toState; } + /** + * Get the source state + * @return source state name + */ + public String getFromState() { + return _fromState.toString(); + } + + /** + * Get the destination state + * @return destination state name + */ + public String getToState() { + return _toState.toString(); + } + + /** + * Create a new transition + * @param fromState string source state + * @param toState string destination state + * @return Transition + */ + public static Transition from(State fromState, State toState) { + return new Transition(fromState, toState); + } } diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/AutoModeISBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/AutoModeISBuilder.java index 535f25b857..cda2f9e3c0 100644 --- a/helix-core/src/main/java/org/apache/helix/model/builder/AutoModeISBuilder.java +++ b/helix-core/src/main/java/org/apache/helix/model/builder/AutoModeISBuilder.java @@ -19,27 +19,87 @@ * under the License. */ -import org.apache.helix.model.IdealState.RebalanceMode; - import java.util.ArrayList; import java.util.Arrays; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.IdealState.RebalanceMode; + +/** + * IdealState builder for SEMI_AUTO mode + */ public class AutoModeISBuilder extends IdealStateBuilder { + /** + * Start building a SEMI_AUTO IdealState + * @param resourceName the resource + */ public AutoModeISBuilder(String resourceName) { super(resourceName); setRebalancerMode(RebalanceMode.SEMI_AUTO); } - public void add(String partitionName) { + /** + * Start building a SEMI_AUTO IdealState + * @param resourceId the resource + */ + public AutoModeISBuilder(ResourceId resourceId) { + this(resourceId.stringify()); + } + + /** + * Add a partition; Helix will assign replicas of the partition according to preference lists + * @param partitionName the name of the new partition + * @return AutoModeISBuilder + */ + public AutoModeISBuilder add(String partitionName) { if (_record.getListField(partitionName) == null) { _record.setListField(partitionName, new ArrayList()); } + return this; + } + + /** + * Add a partition; Helix will assign replicas of the partition according to preference lists + * @param partitionId the id of the new partition + * @return AutoModeISBuilder + */ + public AutoModeISBuilder add(PartitionId partitionId) { + if (partitionId != null) { + add(partitionId.stringify()); + } + return this; } + /** + * Define where replicas of a partition should live + * @param partitionName the partition + * @param instanceNames ordered list of participant names + * @return AutoModeISBuilder + */ public AutoModeISBuilder assignPreferenceList(String partitionName, String... instanceNames) { add(partitionName); _record.getListField(partitionName).addAll(Arrays.asList(instanceNames)); return this; } + /** + * Define where replicas of a partition should live + * @param partitionId the partition + * @param participantIds ordered list of participant ids + * @return AutoModeISBuilder + */ + public AutoModeISBuilder assignPreferenceList(PartitionId partitionId, + ParticipantId... participantIds) { + if (partitionId != null) { + String[] participantNames = new String[participantIds.length]; + for (int i = 0; i < participantIds.length; i++) { + participantNames[i] = participantIds[i].stringify(); + } + assignPreferenceList(partitionId.stringify(), participantNames); + } + return this; + } + } diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/AutoRebalanceModeISBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/AutoRebalanceModeISBuilder.java index 3368c87468..8ac3b82713 100644 --- a/helix-core/src/main/java/org/apache/helix/model/builder/AutoRebalanceModeISBuilder.java +++ b/helix-core/src/main/java/org/apache/helix/model/builder/AutoRebalanceModeISBuilder.java @@ -19,26 +19,56 @@ * under the License. */ -import org.apache.helix.model.IdealState.RebalanceMode; - import java.util.ArrayList; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.IdealState.RebalanceMode; + +/** + * IdealState builder for FULL_AUTO mode + */ public class AutoRebalanceModeISBuilder extends IdealStateBuilder { + /** + * Start building a SEMI_AUTO IdealState + * @param resourceName the resource + */ public AutoRebalanceModeISBuilder(String resourceName) { super(resourceName); setRebalancerMode(RebalanceMode.FULL_AUTO); } + /** + * Start building a SEMI_AUTO IdealState + * @param resourceId the resource + */ + public AutoRebalanceModeISBuilder(ResourceId resourceId) { + this(resourceId.stringify()); + } + /** * Add a partition, Helix will automatically assign the placement and state * for this partition at runtime. - * @param partitionName + * @param partitionName the partition to add + * @return AutoRebalanceModeISBuilder */ public AutoRebalanceModeISBuilder add(String partitionName) { if (_record.getListField(partitionName) == null) { _record.setListField(partitionName, new ArrayList()); } + return this; + } + /** + * Add a partition, Helix will automatically assign the placement and state + * for this partition at runtime. + * @param partitionId the partition to add + * @return AutoRebalanceModeISBuilder + */ + public AutoRebalanceModeISBuilder add(PartitionId partitionId) { + if (partitionId != null) { + add(partitionId.stringify()); + } return this; } diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/ClusterConstraintsBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/ClusterConstraintsBuilder.java index 59cd825d3f..13b2a7e42f 100644 --- a/helix-core/src/main/java/org/apache/helix/model/builder/ClusterConstraintsBuilder.java +++ b/helix-core/src/main/java/org/apache/helix/model/builder/ClusterConstraintsBuilder.java @@ -22,13 +22,11 @@ import java.util.HashMap; import java.util.Map; +import org.apache.helix.api.id.ConstraintId; import org.apache.helix.model.ClusterConstraints; import org.apache.helix.model.ClusterConstraints.ConstraintType; -import org.apache.log4j.Logger; public class ClusterConstraintsBuilder { - private static Logger LOG = Logger.getLogger(ClusterConstraintsBuilder.class); - final private ConstraintType _constraintType; /** @@ -40,8 +38,8 @@ public class ClusterConstraintsBuilder { * TRANSITION : OFFLINE->SLAVE * CONSTRAINT_VALUE : 1 */ - private final Map _constraintBuilderMap = - new HashMap(); + private final Map _constraintBuilderMap = + new HashMap(); public ClusterConstraintsBuilder(ConstraintType type) { if (type == null) { @@ -50,8 +48,8 @@ public ClusterConstraintsBuilder(ConstraintType type) { _constraintType = type; } - public ClusterConstraintsBuilder addConstraintAttribute(String constraintId, String attribute, - String value) { + public ClusterConstraintsBuilder addConstraintAttribute(ConstraintId constraintId, + String attribute, String value) { if (!_constraintBuilderMap.containsKey(constraintId)) { _constraintBuilderMap.put(constraintId, new ConstraintItemBuilder()); } @@ -60,10 +58,15 @@ public ClusterConstraintsBuilder addConstraintAttribute(String constraintId, Str return this; } + public ClusterConstraintsBuilder addConstraintAttribute(String constraintId, String attribute, + String value) { + return addConstraintAttribute(ConstraintId.from(constraintId), attribute, value); + } + public ClusterConstraints build() { ClusterConstraints constraints = new ClusterConstraints(_constraintType); - for (String constraintId : _constraintBuilderMap.keySet()) { + for (ConstraintId constraintId : _constraintBuilderMap.keySet()) { ConstraintItemBuilder builder = _constraintBuilderMap.get(constraintId); constraints.addConstraintItem(constraintId, builder.build()); } diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/ConstraintItemBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/ConstraintItemBuilder.java index 6296b04882..fd3be82e5c 100644 --- a/helix-core/src/main/java/org/apache/helix/model/builder/ConstraintItemBuilder.java +++ b/helix-core/src/main/java/org/apache/helix/model/builder/ConstraintItemBuilder.java @@ -22,9 +22,9 @@ import java.util.Map; import java.util.TreeMap; -import org.apache.helix.model.ConstraintItem; import org.apache.helix.model.ClusterConstraints.ConstraintAttribute; import org.apache.helix.model.ClusterConstraints.ConstraintValue; +import org.apache.helix.model.ConstraintItem; import org.apache.log4j.Logger; public class ConstraintItemBuilder { diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/CurrentStateBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/CurrentStateBuilder.java new file mode 100644 index 0000000000..fb6235f71a --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/builder/CurrentStateBuilder.java @@ -0,0 +1,124 @@ +package org.apache.helix.model.builder; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.CurrentState.CurrentStateProperty; + +/* + * 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. + */ + +/** + * Assemble a CurrentState + */ +public class CurrentStateBuilder { + private final ResourceId _resourceId; + private final Map _partitionStateMap; + private SessionId _sessionId; + private StateModelDefId _stateModelDefId; + private StateModelFactoryId _stateModelFactoryId; + + /** + * Build a current state for a given resource + * @param resourceId resource identifier + */ + public CurrentStateBuilder(ResourceId resourceId) { + _resourceId = resourceId; + _partitionStateMap = new HashMap(); + } + + /** + * Add partition-state mappings for this instance and resource + * @param mappings map of partition to state + * @return CurrentStateBuilder + */ + public CurrentStateBuilder addMappings(Map mappings) { + _partitionStateMap.putAll(mappings); + return this; + } + + /** + * Add a single partition-state mapping for this instance and resource + * @param partitionId the partition to map + * @param state the replica state + * @return CurrentStateBuilder + */ + public CurrentStateBuilder addMapping(PartitionId partitionId, State state) { + _partitionStateMap.put(partitionId, state); + return this; + } + + /** + * Set the session id for this current state + * @param sessionId session identifier + * @return CurrentStateBuilder + */ + public CurrentStateBuilder sessionId(SessionId sessionId) { + _sessionId = sessionId; + return this; + } + + /** + * Set the state model for this current state + * @param stateModelDefId state model definition identifier + * @return CurrentStateBuilder + */ + public CurrentStateBuilder stateModelDef(StateModelDefId stateModelDefId) { + _stateModelDefId = stateModelDefId; + return this; + } + + /** + * Set the name of the state model factory + * @param stateModelFactoryIde state model factory identifier + * @return CurrentStateBuilder + */ + public CurrentStateBuilder stateModelFactory(StateModelFactoryId stateModelFactoryId) { + _stateModelFactoryId = stateModelFactoryId; + return this; + } + + /** + * Create a CurrentState + * @return instantiated CurrentState + */ + public CurrentState build() { + ZNRecord record = new ZNRecord(_resourceId.stringify()); + for (PartitionId partitionId : _partitionStateMap.keySet()) { + Map stateMap = new HashMap(); + stateMap.put(CurrentStateProperty.CURRENT_STATE.toString(), + _partitionStateMap.get(partitionId).toString()); + record.setMapField(partitionId.toString(), stateMap); + } + record.setSimpleField(CurrentStateProperty.SESSION_ID.toString(), _sessionId.toString()); + record.setSimpleField(CurrentStateProperty.STATE_MODEL_DEF.toString(), + _stateModelDefId.toString()); + record.setSimpleField(CurrentStateProperty.STATE_MODEL_FACTORY_NAME.toString(), + _stateModelFactoryId.toString()); + return new CurrentState(record); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/CustomModeISBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/CustomModeISBuilder.java index 78d8ca3aee..566452a03b 100644 --- a/helix-core/src/main/java/org/apache/helix/model/builder/CustomModeISBuilder.java +++ b/helix-core/src/main/java/org/apache/helix/model/builder/CustomModeISBuilder.java @@ -19,34 +19,66 @@ * under the License. */ -import org.apache.helix.model.IdealState.RebalanceMode; - import java.util.Map; import java.util.TreeMap; -public class CustomModeISBuilder extends IdealStateBuilder { +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.IdealState.RebalanceMode; +/** + * IdealState builder for CUSTOMIZED mode + */ +public class CustomModeISBuilder extends IdealStateBuilder { + /** + * Start building a CUSTOMIZED IdealState + * @param resourceName the resource + */ public CustomModeISBuilder(String resourceName) { super(resourceName); setRebalancerMode(RebalanceMode.CUSTOMIZED); } + /** + * Start building a SEMI_AUTO IdealState + * @param resourceId the resource + */ + public CustomModeISBuilder(ResourceId resourceId) { + this(resourceId.stringify()); + } + /** * Add a sub-resource - * @param partitionName + * @param partitionName partition to add + * @return CustomModeISBuilder */ - public void add(String partitionName) { + public CustomModeISBuilder add(String partitionName) { if (_record.getMapField(partitionName) == null) { _record.setMapField(partitionName, new TreeMap()); } + return this; + } + + /** + * Add a sub-resource + * @param partitionId partition to add + * @return CustomModeISBuilder + */ + public CustomModeISBuilder add(PartitionId partitionId) { + if (partitionId != null) { + add(partitionId.stringify()); + } + return this; } /** * add an instance->state assignment - * @param partitionName - * @param instanceName - * @param state - * @return + * @param partitionName partition to update + * @param instanceName participant name + * @param state state the replica should be in + * @return CustomModeISBuilder */ public CustomModeISBuilder assignInstanceAndState(String partitionName, String instanceName, String state) { @@ -56,4 +88,19 @@ public CustomModeISBuilder assignInstanceAndState(String partitionName, String i return this; } + /** + * add an instance->state assignment + * @param partitionId partition to update + * @param participantId participant to assign to + * @param state state the replica should be in + * @return CustomModeISBuilder + */ + public CustomModeISBuilder assignParticipantAndState(PartitionId partitionId, + ParticipantId participantId, State state) { + if (partitionId != null && participantId != null && state != null) { + assignInstanceAndState(partitionId.stringify(), participantId.stringify(), state.toString()); + } + return this; + } + } diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/HelixConfigScopeBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/HelixConfigScopeBuilder.java index d65ffd3d6f..87cabcf562 100644 --- a/helix-core/src/main/java/org/apache/helix/model/builder/HelixConfigScopeBuilder.java +++ b/helix-core/src/main/java/org/apache/helix/model/builder/HelixConfigScopeBuilder.java @@ -34,6 +34,7 @@ public class HelixConfigScopeBuilder { private String _participantName; private String _resourceName; private String _partitionName; + private String _monitoringConfigName; public HelixConfigScopeBuilder(ConfigScopeProperty type, String... keys) { int argNum = type.getZkPathArgNum() + type.getMapKeyArgNum(); @@ -64,6 +65,11 @@ public HelixConfigScopeBuilder(ConfigScopeProperty type, String... keys) { _partitionName = keys[2]; } break; + case MONITORING: + if (keys.length > 1) { + _monitoringConfigName = keys[1]; + } + break; default: break; } @@ -93,6 +99,11 @@ public HelixConfigScopeBuilder forPartition(String partitionName) { return this; } + public HelixConfigScopeBuilder forMonitoringConfig(String monitoringConfigName) { + _participantName = monitoringConfigName; + return this; + } + public HelixConfigScope build() { HelixConfigScope scope = null; switch (_type) { @@ -121,6 +132,13 @@ public HelixConfigScope build() { new HelixConfigScope(_type, Arrays.asList(_clusterName, _resourceName), _partitionName); } break; + case MONITORING: + if (_monitoringConfigName == null) { + scope = new HelixConfigScope(_type, Arrays.asList(_clusterName), null); + } else { + scope = new HelixConfigScope(_type, Arrays.asList(_clusterName, _monitoringConfigName), null); + } + break; default: break; } diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/IdealStateBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/IdealStateBuilder.java index a7c0335f59..a6707e91a4 100644 --- a/helix-core/src/main/java/org/apache/helix/model/builder/IdealStateBuilder.java +++ b/helix-core/src/main/java/org/apache/helix/model/builder/IdealStateBuilder.java @@ -22,6 +22,9 @@ import org.apache.helix.HelixConstants; import org.apache.helix.HelixException; import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; import org.apache.helix.model.IdealState; public abstract class IdealStateBuilder { @@ -73,6 +76,14 @@ public IdealStateBuilder(String resourceName) { _record = new ZNRecord(resourceName); } + /** + * Instantiate with a resource id + * @param resourceId the resource for which to build an ideal state + */ + public IdealStateBuilder(ResourceId resourceId) { + this(resourceId.stringify()); + } + /** * @param numReplica */ @@ -97,6 +108,15 @@ public IdealStateBuilder setStateModel(String stateModel) { return this; } + /** + * Set the state model definition to use with this ideal state + * @param stateModelDefId state model identifier + */ + public IdealStateBuilder setStateModelDefId(StateModelDefId stateModelDefId) { + this.stateModel = stateModelDefId.stringify(); + return this; + } + /** * @param stateModelFactoryName */ @@ -137,8 +157,8 @@ public IdealState build() { IdealState idealstate = new IdealState(_record); idealstate.setNumPartitions(numPartitions); idealstate.setMaxPartitionsPerInstance(maxPartitionsPerNode); - idealstate.setStateModelDefRef(stateModel); - idealstate.setStateModelFactoryName(stateModelFactoryName); + idealstate.setStateModelDefId(StateModelDefId.from(stateModel)); + idealstate.setStateModelFactoryId(StateModelFactoryId.from(stateModelFactoryName)); idealstate.setRebalanceMode(rebalancerMode); idealstate.setReplicas("" + numReplica); @@ -147,5 +167,4 @@ public IdealState build() { } return idealstate; } - } diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/MessageConstraintItemBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/MessageConstraintItemBuilder.java new file mode 100644 index 0000000000..da463e16f2 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/builder/MessageConstraintItemBuilder.java @@ -0,0 +1,107 @@ +package org.apache.helix.model.builder; + +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.ClusterConstraints.ConstraintAttribute; +import org.apache.helix.model.ConstraintItem; +import org.apache.helix.model.Message; +import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.Transition; + +/* + * 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. + */ + +/** + * Specify a ConstraintItem based on a message (commonly used for transition constraints) + */ +public class MessageConstraintItemBuilder { + private ConstraintItemBuilder _builder; + + /** + * Instantiate the builder + */ + public MessageConstraintItemBuilder() { + _builder = new ConstraintItemBuilder(); + } + + /** + * Set the message type of the constraint + * @param messageType message type + * @return MessageConstraintItemBuilder + */ + public MessageConstraintItemBuilder messageType(MessageType messageType) { + _builder.addConstraintAttribute(ConstraintAttribute.MESSAGE_TYPE.toString(), + messageType.toString()); + return this; + } + + /** + * Set a participant as part of the constraint scope + * @param participantId the participant to constrain + * @return MessageConstraintItemBuilder + */ + public MessageConstraintItemBuilder participant(ParticipantId participantId) { + _builder.addConstraintAttribute(ConstraintAttribute.INSTANCE.toString(), + participantId.stringify()); + return this; + } + + /** + * Set a resource as part of the constraint scope + * @param resourceId the resource to constrain + * @return MessageConstraintItemBuilder + */ + public MessageConstraintItemBuilder resource(ResourceId resourceId) { + _builder + .addConstraintAttribute(ConstraintAttribute.RESOURCE.toString(), resourceId.stringify()); + return this; + } + + /** + * Set the transition to constrain for transition message types + * @param transition the transition to constrain + * @return MessageConstraintItemBuilder + */ + public MessageConstraintItemBuilder transition(Transition transition) { + // if this is a transition constraint, the message type must be STATE_TRANSITION + _builder.addConstraintAttribute(ConstraintAttribute.MESSAGE_TYPE.toString(), + Message.MessageType.STATE_TRANSITION.toString()); + _builder.addConstraintAttribute(ConstraintAttribute.TRANSITION.toString(), + transition.toString()); + return this; + } + + /** + * Set the value of the constraint + * @param value constraint value + * @return MessageConstraintItemBuilder + */ + public MessageConstraintItemBuilder constraintValue(String value) { + _builder.addConstraintAttribute(ConstraintAttribute.CONSTRAINT_VALUE.toString(), value); + return this; + } + + /** + * Get the ConstraintItem instance that is built + * @return ConstraintItem + */ + public ConstraintItem build() { + return _builder.build(); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/ResourceAssignmentBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/ResourceAssignmentBuilder.java new file mode 100644 index 0000000000..8dfb0e59c2 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/builder/ResourceAssignmentBuilder.java @@ -0,0 +1,93 @@ +package org.apache.helix.model.builder; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.ResourceAssignment; + +/* + * 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. + */ + +/** + * Build an ideal assignment of resources + */ +public class ResourceAssignmentBuilder { + private final ResourceId _resourceId; + private final Map> _mapping; + + /** + * Create an assignment for a given resource + * @param resourceId resource id + */ + public ResourceAssignmentBuilder(ResourceId resourceId) { + _resourceId = resourceId; + _mapping = new HashMap>(); + } + + /** + * Add multiple assignments of partition replicas + * @param partitionId the partition to assign + * @param replicaMap participant-state map of assignments + * @return ResourceAssignmentBuilder + */ + public ResourceAssignmentBuilder addAssignments(PartitionId partitionId, + Map replicaMap) { + if (!_mapping.containsKey(partitionId)) { + _mapping.put(partitionId, new HashMap()); + } + _mapping.get(partitionId).putAll(replicaMap); + return this; + } + + /** + * Add a single replica assignment + * @param partitonId the partition to assign + * @param participantId participant of assignment + * @param state replica state + * @return ResourceAssignmentBuilder + */ + public ResourceAssignmentBuilder addAssignment(PartitionId partitonId, + ParticipantId participantId, State state) { + Map replicaMap; + if (!_mapping.containsKey(partitonId)) { + replicaMap = new HashMap(); + _mapping.put(partitonId, replicaMap); + } else { + replicaMap = _mapping.get(partitonId); + } + replicaMap.put(participantId, state); + return this; + } + + /** + * Get a complete resource assignment + * @return ResourceAssignment + */ + public ResourceAssignment build() { + ResourceAssignment assignment = new ResourceAssignment(_resourceId); + for (PartitionId partitionId : _mapping.keySet()) { + assignment.addReplicaMap(partitionId, _mapping.get(partitionId)); + } + return assignment; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/StateConstraintItemBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/StateConstraintItemBuilder.java new file mode 100644 index 0000000000..9a2678acda --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/builder/StateConstraintItemBuilder.java @@ -0,0 +1,92 @@ +package org.apache.helix.model.builder; + +import org.apache.helix.api.State; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.model.ClusterConstraints.ConstraintAttribute; +import org.apache.helix.model.ConstraintItem; + +/* + * 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. + */ + +/** + * Build a ConstraintItem corresponding to a state bound + */ +public class StateConstraintItemBuilder { + private ConstraintItemBuilder _builder; + + /** + * Instantiate the builder + */ + public StateConstraintItemBuilder() { + _builder = new ConstraintItemBuilder(); + } + + /** + * Get the state model definition to constrain + * @param stateModelDefId state model definition identifier + * @return StateConstraintItemBuilder + */ + public StateConstraintItemBuilder stateModel(StateModelDefId stateModelDefId) { + _builder.addConstraintAttribute(ConstraintAttribute.STATE_MODEL.toString(), + stateModelDefId.stringify()); + return this; + } + + /** + * Get the state to constrain + * @param state state object + * @return StateConstraintItemBuilder + */ + public StateConstraintItemBuilder state(State state) { + _builder.addConstraintAttribute(ConstraintAttribute.STATE.toString(), state.toString()); + return this; + } + + /** + * Set a numerical upper bound for the replicas that can be in a state + * @param upperBound maximum replica count for a state, per partition + * @return StateConstraintItemBuilder + */ + public StateConstraintItemBuilder upperBound(int upperBound) { + _builder.addConstraintAttribute(ConstraintAttribute.CONSTRAINT_VALUE.toString(), + Integer.toString(upperBound)); + return this; + } + + /** + * Set an upper bound for the replicas that can be in a state. This can be numerical, or "N" for + * number of nodes, or "R" for total number of replicas per partition + * @param dynamicUpperBound maximum replica count for a state, per partition, can also be "N" or + * "R" + * @return StateConstraintItemBuilder + */ + public StateConstraintItemBuilder dynamicUpperBound(String dynamicUpperBound) { + _builder.addConstraintAttribute(ConstraintAttribute.CONSTRAINT_VALUE.toString(), + dynamicUpperBound); + return this; + } + + /** + * Get the ConstraintItem instance that is built + * @return ConstraintItem + */ + public ConstraintItem build() { + return _builder.build(); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/StateTransitionTableBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/StateTransitionTableBuilder.java index bbd7b3352a..779f220ac7 100644 --- a/helix-core/src/main/java/org/apache/helix/model/builder/StateTransitionTableBuilder.java +++ b/helix-core/src/main/java/org/apache/helix/model/builder/StateTransitionTableBuilder.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; +import org.apache.helix.api.State; import org.apache.helix.model.Transition; public class StateTransitionTableBuilder { @@ -125,10 +126,10 @@ public Map> buildTransitionTable(List states } for (Transition transition : transitions) { - String fromState = transition.getFromState(); - String toState = transition.getToState(); - setPathVal(path, fromState, toState, 1); - setNext(next, fromState, toState, toState); + State fromState = transition.getTypedFromState(); + State toState = transition.getTypedToState(); + setPathVal(path, fromState.toString(), toState.toString(), 1); + setNext(next, fromState.toString(), toState.toString(), toState.toString()); } // iterate @@ -162,11 +163,11 @@ public static void main(String[] args) { states.add("OFFLINE"); List transitions = new ArrayList(); - transitions.add(new Transition("SLAVE", "OFFLINE")); - transitions.add(new Transition("OFFLINE", "SLAVE")); - transitions.add(new Transition("SLAVE", "MASTER")); - transitions.add(new Transition("OFFLINE", "DROPPED")); - transitions.add(new Transition("MASTER", "SLAVE")); + transitions.add(Transition.from(State.from("SLAVE"), State.from("OFFLINE"))); + transitions.add(Transition.from(State.from("OFFLINE"), State.from("SLAVE"))); + transitions.add(Transition.from(State.from("SLAVE"), State.from("MASTER"))); + transitions.add(Transition.from(State.from("OFFLINE"), State.from("DROPPED"))); + transitions.add(Transition.from(State.from("MASTER"), State.from("SLAVE"))); StateTransitionTableBuilder builder = new StateTransitionTableBuilder(); Map> next = builder.buildTransitionTable(states, transitions); diff --git a/helix-core/src/main/java/org/apache/helix/model/util/StateModelDefinitionValidator.java b/helix-core/src/main/java/org/apache/helix/model/util/StateModelDefinitionValidator.java new file mode 100644 index 0000000000..6ecc80026d --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/model/util/StateModelDefinitionValidator.java @@ -0,0 +1,290 @@ +package org.apache.helix.model.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.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.HelixDefinedState; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +/** + * Validator logic for a StateModelDefinition.
+ *
+ * Usage:
+ * StateModelDefinition stateModelDef = ...;
+ * StateModelDefinitionValidator.isStateModelDefinitionValid(stateModelDef); + */ +public class StateModelDefinitionValidator { + private static final Logger _logger = Logger.getLogger(StateModelDefinitionValidator.class); + private final StateModelDefinition _stateModelDef; + private final List _statePriorityList; + private final List _transitionPriorityList; + private final Set _stateSet; + + /** + * Instantiate a validator instance + * @param stateModelDef the state model definition to validate + */ + private StateModelDefinitionValidator(StateModelDefinition stateModelDef) { + _stateModelDef = stateModelDef; + _statePriorityList = stateModelDef.getStatesPriorityList(); + _transitionPriorityList = stateModelDef.getStateTransitionPriorityList(); + _stateSet = Sets.newHashSet(_statePriorityList); + } + + /** + * Check if the StateModelDefinition passes all validation checks + * @return true if state model definition is valid, false otherwise + */ + public boolean isStateModelDefinitionValid() { + // has a name + if (_stateModelDef.getId() == null || _stateModelDef.getId().isEmpty()) { + _logger.error("State model does not have a name"); + return false; + } + + // has an initial state + if (_stateModelDef.getInitialState() == null || _stateModelDef.getInitialState().isEmpty()) { + _logger + .error("State model does not contain init state, statemodel:" + _stateModelDef.getId()); + return false; + } + + // has states + if (_statePriorityList == null || _statePriorityList.isEmpty()) { + _logger.error("CurrentState does not contain StatesPriorityList, state model : " + + _stateModelDef.getId()); + return false; + } + + // initial state is a state + if (!_stateSet.contains(_stateModelDef.getInitialState())) { + _logger.error("Defined states does not include the initial state, state model: " + + _stateModelDef.getId()); + return false; + } + + // has a dropped state + if (!_stateSet.contains(HelixDefinedState.DROPPED.toString())) { + _logger.error("Defined states does not include the DROPPED state, state model: " + + _stateModelDef.getId()); + return false; + } + + // make sure individual checks all pass + if (!areStateCountsValid() || !areNextStatesValid() || !isTransitionPriorityListValid() + || !arePathsValid()) { + return false; + } + + return true; + } + + /** + * Check if state counts are properly defined for each state + * @return true if state counts valid, false otherwise + */ + private boolean areStateCountsValid() { + for (String state : _statePriorityList) { + // all states should have a count + String count = _stateModelDef.getNumInstancesPerState(state); + if (count == null) { + _logger.error("State " + state + " needs an upper bound constraint, state model: " + + _stateModelDef.getId()); + return false; + } + + // count should be a number, N, or R + try { + Integer.parseInt(count); + } catch (NumberFormatException e) { + if (!count.equals("N") && !count.equals("R")) { + _logger.error("State " + state + " has invalid count " + count + ", state model: " + + _stateModelDef.getId()); + return false; + } + } + } + return true; + } + + /** + * Check if the state transition priority list is properly formed + * @return true if the transition priority list is valid, false otherwise + */ + private boolean isTransitionPriorityListValid() { + if (_transitionPriorityList != null) { + for (String transition : _transitionPriorityList) { + // ensure that transition is of form FROM-TO + int index = transition.indexOf('-'); + int lastIndex = transition.indexOf('-'); + if (index <= 0 || index >= transition.length() - 1 || index != lastIndex) { + _logger.error("Transition " + transition + " is not of the form SRC-DEST, state model: " + + _stateModelDef.getId()); + return false; + } + + // from and to states should be valid states + String from = transition.substring(0, index); + String to = transition.substring(index + 1); + if (!_stateSet.contains(from)) { + _logger.error("State " + from + " in " + transition + + " is not a defined state, state model" + _stateModelDef.getId()); + return false; + } + if (!_stateSet.contains(to)) { + _logger.error("State " + to + " in " + transition + + " is not a defined state, state model: " + _stateModelDef.getId()); + return false; + } + + // the next state for the transition should be the to state + if (!to.equals(_stateModelDef.getNextStateForTransition(from, to))) { + _logger.error("Transition " + transition + " must have " + to + " as the next state"); + return false; + } + } + } + return true; + } + + /** + * Check if the "next" states in the state model definition are valid. These check the next values + * at a single level. To check full paths, use {@link #arePathsValid()}. + * @return true if next states are properly defined, false otherwise + */ + private boolean areNextStatesValid() { + for (String state : _statePriorityList) { + // all states can reach DROPPED + if (!state.equals(HelixDefinedState.DROPPED.toString()) + && _stateModelDef.getNextStateForTransition(state, HelixDefinedState.DROPPED.toString()) == null) { + _logger.error("State " + state + " cannot reach the DROPPED state, state model: " + + _stateModelDef.getId()); + return false; + } + + // initial state should reach all states (other than error) + if (!state.equals(_stateModelDef.getInitialState()) + && !state.equals(HelixDefinedState.ERROR.toString()) + && _stateModelDef.getNextStateForTransition(_stateModelDef.getInitialState(), state) == null) { + _logger.error("Initial state " + _stateModelDef.getInitialState() + + " should be able to reach all states, state model: " + _stateModelDef.getId()); + return false; + } + + // validate "next" states + for (String destState : _statePriorityList) { + if (state.equals(destState)) { + continue; + } + // the next state should exist + String intermediate = _stateModelDef.getNextStateForTransition(state, destState); + if (intermediate != null && !_stateSet.contains(intermediate)) { + _logger.error("Intermediate state " + intermediate + " for transition " + state + "-" + + destState + " is not a valid state, state model: " + _stateModelDef.getId()); + return false; + } + + // the next state should not allow a self loop + if (intermediate != null && intermediate.equals(state)) { + _logger.error("Intermediate state " + intermediate + " for transition " + state + "-" + + destState + " should never be the from state, state model: " + + _stateModelDef.getId()); + return false; + } + } + } + return true; + } + + /** + * Check that the state model does not have loops or unreachable states and that next states + * actually help make progress + * @return true if the transitions are valid, false otherwise + */ + private boolean arePathsValid() { + // create a map for memoized path checking + Map> alreadyChecked = Maps.newHashMap(); + for (String state : _statePriorityList) { + alreadyChecked.put(state, new HashSet()); + } + + // check all pairs for paths + for (String from : _statePriorityList) { + for (String to : _statePriorityList) { + // ignore self transitions + if (from.equals(to)) { + continue; + } + + // see if a path is claimed to exist + Set used = Sets.newHashSet(from); + String next = _stateModelDef.getNextStateForTransition(from, to); + if (next == null) { + if (from.equals(_stateModelDef.getInitialState()) + && !to.equals(HelixDefinedState.ERROR.toString())) { + _logger.error("Initial state " + from + " cannot reach " + to + ", state model: " + + _stateModelDef.getId()); + return false; + } + continue; + } + // if a path exists, follow it all the way + while (!to.equals(next)) { + // no need to proceed if this path has already been traversed + if (alreadyChecked.get(next).contains(to)) { + break; + } + if (used.contains(next)) { + _logger.error("Path from " + from + " to " + to + + " contains an infinite loop, state model: " + _stateModelDef.getId()); + return false; + } + alreadyChecked.get(next).add(to); + used.add(next); + next = _stateModelDef.getNextStateForTransition(next, to); + if (next == null) { + _logger.error("Path from " + from + " to " + to + " is incomplete, state model: " + + _stateModelDef.getId()); + return false; + } + } + alreadyChecked.get(from).add(to); + } + } + return true; + } + + /** + * Validate a StateModelDefinition instance + * @param stateModelDef the state model definition to validate + * @return true if the state model definition is valid, false otherwise + */ + public static boolean isStateModelDefinitionValid(StateModelDefinition stateModelDef) { + return new StateModelDefinitionValidator(stateModelDef).isStateModelDefinitionValid(); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringClient.java b/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringClient.java new file mode 100644 index 0000000000..a055354769 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringClient.java @@ -0,0 +1,85 @@ +package org.apache.helix.monitoring; + +/* + * 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.TimeUnit; + +/** + * Interface for a client that can register with a monitoring server and send events for monitoring + */ +public interface MonitoringClient { + /** + * Connect. May be asynchronous. + * @throws Exception + */ + void connect() throws Exception; + + /** + * Disconnect synchronously. + */ + void disconnect(); + + /** + * Send an event + * @param e the event + * @return true if the event was sent (or queued for batching), false otherwise + */ + boolean send(MonitoringEvent e); + + /** + * Send an event and flush any outstanding messages + * @param e the event + * @return true if events were successfully sent, false otherwise + */ + boolean sendAndFlush(MonitoringEvent e); + + /** + * Schedule an operation to run + * @param interval the frequency + * @param delay the amount of time to wait before the first execution + * @param unit the unit of time to use + * @param r the code to run + */ + void every(long interval, long delay, TimeUnit unit, Runnable r); + + /** + * Check if there is a valid connection to a monitoring server + * @return true if connected, false otherwise + */ + boolean isConnected(); + + /** + * Check if batching is being used + * @return true if enabled, false otherwise + */ + boolean isBatchingEnabled(); + + /** + * Check the number of events sent as a batch + * @return the batch size, or 1 if batching is not used + */ + int getBatchSize(); + + /** + * Flush all outstanding events + * @return true if all events were flushed, false otherwise + */ + boolean flush(); +} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertComparator.java b/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringClientOwner.java similarity index 59% rename from helix-core/src/main/java/org/apache/helix/alerts/AlertComparator.java rename to helix-core/src/main/java/org/apache/helix/monitoring/MonitoringClientOwner.java index c0330c5025..c623ab1f58 100644 --- a/helix-core/src/main/java/org/apache/helix/alerts/AlertComparator.java +++ b/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringClientOwner.java @@ -1,4 +1,4 @@ -package org.apache.helix.alerts; +package org.apache.helix.monitoring; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,12 +19,19 @@ * under the License. */ -public abstract class AlertComparator { +public interface MonitoringClientOwner { - public AlertComparator() { - - } - - public abstract boolean evaluate(Tuple leftTup, Tuple rightTup); + /** + * Register a monitoring client that can be used to report statistics. + * This will connect a monitoring client. If an existing client is connected, it will be + * disconnected. + * @param monitoringClient a reference to an instantiated client + */ + void registerMonitoringClient(MonitoringClient monitoringClient); + /** + * Get the registered monitoring client + * @return a MonitoringClient object, or null + */ + MonitoringClient getMonitoringClient(); } diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringEvent.java b/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringEvent.java new file mode 100644 index 0000000000..2044a3a6b3 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringEvent.java @@ -0,0 +1,378 @@ +package org.apache.helix.monitoring; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.api.Scope; +import org.apache.helix.api.Scope.ScopeType; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SpectatorId; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +/** + * A generic monitoring event based on Helix constructs. This is based on Riemann's EventDSL. + */ +public class MonitoringEvent { + private ClusterId _clusterId; + private ResourceId _resourceId; + private PartitionId _partitionId; + private String _name; + private String _host; + private String _eventState; + private String _description; + private Long _time; + private Long _longMetric; + private Float _floatMetric; + private Double _doubleMetric; + private Float _ttl; + private final List _tags; + private final Map _attributes; + private String _shardingStr; + private final Set _shardingScopes; + + /** + * Create an empty MonitoringEvent + */ + public MonitoringEvent() { + _clusterId = null; + _resourceId = null; + _partitionId = null; + _name = null; + _host = null; + _eventState = null; + _description = null; + _time = null; + _longMetric = null; + _floatMetric = null; + _doubleMetric = null; + _ttl = null; + _tags = Lists.newLinkedList(); + _attributes = Maps.newHashMap(); + _shardingStr = null; + _shardingScopes = Sets.newHashSet(); + } + + /** + * Give this event a name + * @param name the name + * @return MonitoringEvent + */ + public MonitoringEvent name(String name) { + _name = name; + return this; + } + + /** + * Set the cluster this event corresponds to + * @param clusterId the cluster id + * @return MonitoringEvent + */ + public MonitoringEvent cluster(ClusterId clusterId) { + _clusterId = clusterId; + return this; + } + + /** + * Set the participant this event corresponds to + * @param participantId the participant id + * @return MonitoringEvent + */ + public MonitoringEvent participant(ParticipantId participantId) { + _host = participantId.stringify(); + return this; + } + + /** + * Set the spectator this event corresponds to + * @param spectatorId the spectator id + * @return MonitoringEvent + */ + public MonitoringEvent spectator(SpectatorId spectatorId) { + _host = spectatorId.stringify(); + return this; + } + + /** + * Set the controller this event corresponds to + * @param controllerId the controller id + * @return MonitoringEvent + */ + public MonitoringEvent controller(ControllerId controllerId) { + _host = controllerId.stringify(); + return this; + } + + /** + * Set the resource this event corresponds to + * @param resourceId the resource id + * @return MonitoringEvent + */ + public MonitoringEvent resource(ResourceId resourceId) { + _resourceId = resourceId; + return this; + } + + /** + * Set the partition this event corresponds to + * @param partitionId the partition id + * @return MonitoringEvent + */ + public MonitoringEvent partition(PartitionId partitionId) { + _partitionId = partitionId; + return this; + } + + /** + * Set the state of the metric + * @param eventState the event state (e.g. "OK", "Failing", etc) + * @return MonitoringEvent + */ + public MonitoringEvent eventState(String eventState) { + _eventState = eventState; + return this; + } + + /** + * Give the event a description + * @param description descriptive text + * @return MonitoringEvent + */ + public MonitoringEvent description(String description) { + _description = description; + return this; + } + + /** + * Set the time that the event occurred + * @param time long UNIX timestamp + * @return MonitoringEvent + */ + public MonitoringEvent time(long time) { + _time = time; + return this; + } + + /** + * Give the event a long metric + * @param metric the metric (the measured quantity) + * @return MonitoringEvent + */ + public MonitoringEvent metric(long metric) { + _longMetric = metric; + return this; + } + + /** + * Give the event a float metric + * @param metric the metric (the measured quantity) + * @return MonitoringEvent + */ + public MonitoringEvent metric(float metric) { + _floatMetric = metric; + return this; + } + + /** + * Give the event a double metric + * @param metric the metric (the measured quantity) + * @return MonitoringEvent + */ + public MonitoringEvent metric(double metric) { + _doubleMetric = metric; + return this; + } + + /** + * Give the time before the event will expire + * @param ttl time to live + * @return MonitoringEvent + */ + public MonitoringEvent ttl(float ttl) { + _ttl = ttl; + return this; + } + + /** + * Add a tag to the event + * @param tag arbitrary string + * @return MonitoringEvent + */ + public MonitoringEvent tag(String tag) { + _tags.add(tag); + return this; + } + + /** + * Add multiple tags to an event + * @param tags a collection of tags + * @return MonitoringEvent + */ + public MonitoringEvent tags(Collection tags) { + _tags.addAll(tags); + return this; + } + + /** + * Add an attribute (a key-value pair) + * @param name the attribute name + * @param value the attribute value + * @return MonitoringEvent + */ + public MonitoringEvent attribute(String name, String value) { + _attributes.put(name, value); + return this; + } + + /** + * Add multiple attributes + * @param attributes map of attribute name to value + * @return MonitoringEvent + */ + public MonitoringEvent attributes(Map attributes) { + _attributes.putAll(attributes); + return this; + } + + /** + * Set sharding key using string + * @param shardingStr + * @return MonitoringEvent + */ + public MonitoringEvent shardingString(String shardingStr) { + _shardingStr = shardingStr; + return this; + } + + /** + * Set sharding key using scopes + * @param scopes + * @return MonitoringEvent + */ + public MonitoringEvent shardingScopes(ScopeType... scopes) { + _shardingScopes.clear(); + _shardingScopes.addAll(Arrays.asList(scopes)); + return this; + } + + /** + * Return sharding key which is used by MonitoringClient to choose MonitoringServer + * @return sharding key + */ + public String shardingKey() { + // if shardingStr exists, use shardingStr + if (_shardingStr != null) { + return _shardingStr; + } + + // if shardingStr doesn't exist, use shardingScopes + if (_shardingScopes.isEmpty()) { + _shardingScopes.addAll(Arrays.asList(ScopeType.CLUSTER, ScopeType.RESOURCE)); + } + + StringBuilder sb = new StringBuilder(); + if (_shardingScopes.contains(ScopeType.CLUSTER)) { + sb.append(_clusterId == null ? "%" : _clusterId.stringify()); + } + if (_shardingScopes.contains(ScopeType.RESOURCE)) { + sb.append("|"); + sb.append(_resourceId == null ? "%" : _resourceId.stringify()); + } + if (_shardingScopes.contains(ScopeType.PARTITION)) { + sb.append("|"); + sb.append(_partitionId == null ? "%" : _partitionId.stringify()); + } + if (_shardingScopes.contains(ScopeType.PARTICIPANT)) { + sb.append("|"); + sb.append(_host == null ? "%" : _host); + } + + return sb.toString(); + } + + // below are used for converting MonitoringEvent to Riemann EventDSL + + public String host() { + return _host; + } + + public String service() { + if (_clusterId == null) { + _clusterId = ClusterId.from("%"); + } + if (_resourceId == null) { + _resourceId = ResourceId.from("%"); + } + if (_partitionId == null) { + _partitionId = PartitionId.from("%"); + } + if (_name == null) { + _name = "%"; + } + return String.format("%s|%s|%s|%s", _clusterId, _resourceId, _partitionId, _name); + } + + public String eventState() { + return _eventState; + } + + public String description() { + return _description; + } + + public Long time() { + return _time; + } + + public Long longMetric() { + return _longMetric; + } + + public Float floatMetric() { + return _floatMetric; + } + + public Double doubleMetric() { + return _doubleMetric; + } + + public Float ttl() { + return _ttl; + } + + public List tags() { + return _tags; + } + + public Map attributes() { + return _attributes; + } +} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/package-info.java b/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringServer.java similarity index 63% rename from helix-core/src/main/java/org/apache/helix/alerts/package-info.java rename to helix-core/src/main/java/org/apache/helix/monitoring/MonitoringServer.java index bf1d9a69c1..b8c5a4c7bf 100644 --- a/helix-core/src/main/java/org/apache/helix/alerts/package-info.java +++ b/helix-core/src/main/java/org/apache/helix/monitoring/MonitoringServer.java @@ -1,3 +1,5 @@ +package org.apache.helix.monitoring; + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -16,7 +18,25 @@ * specific language governing permissions and limitations * under the License. */ + /** - * Classes for Helix alerts + * Generic interface for a monitoring service that should be attached to a controller. */ -package org.apache.helix.alerts; \ No newline at end of file +public interface MonitoringServer { + /** + * Start the monitoring service synchronously + */ + public void start(); + + /** + * Stop the monitoring service synchronously + */ + public void stop(); + + /** + * Check if the service has been started + * @return true if started, false otherwise + */ + public boolean isStarted(); + +} diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/StatusDumpTask.java b/helix-core/src/main/java/org/apache/helix/monitoring/StatusDumpTask.java new file mode 100644 index 0000000000..421aab078d --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/monitoring/StatusDumpTask.java @@ -0,0 +1,185 @@ +package org.apache.helix.monitoring; + +/* + * 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.Timer; +import java.util.TimerTask; + +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixTimerTask; +import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyType; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.store.ZNRecordJsonSerializer; +import org.apache.helix.util.HelixUtil; +import org.apache.log4j.Logger; +import org.apache.zookeeper.data.Stat; + +public class StatusDumpTask extends HelixTimerTask { + final static Logger LOG = Logger.getLogger(StatusDumpTask.class); + + Timer _timer = null; + final HelixDataAccessor _accessor; + final ClusterId _clusterId; + + class StatusDumpTimerTask extends TimerTask { + final HelixDataAccessor _accessor; + final PropertyKey.Builder _keyBuilder; + final BaseDataAccessor _baseAccessor; + final ZNRecordJsonSerializer _serializer; + final long _thresholdNoChangeInMs; + final ClusterId _clusterId; + + public StatusDumpTimerTask(ClusterId clusterId, HelixDataAccessor accessor, + long thresholdNoChangeInMs) { + _accessor = accessor; + _keyBuilder = accessor.keyBuilder(); + _baseAccessor = accessor.getBaseDataAccessor(); + _serializer = new ZNRecordJsonSerializer(); + _thresholdNoChangeInMs = thresholdNoChangeInMs; + _clusterId = clusterId; + } + + @Override + public void run() { + /** + * For each record in status-update and error znode + * TODO: for now the status updates are dumped to cluster controller's log. + * We need to think if we should create per-instance log files that contains + * per-instance status-updates and errors + */ + LOG.info("Scannning status updates ..."); + try { + List instanceNames = _accessor.getChildNames(_keyBuilder.instanceConfigs()); + for (String instanceName : instanceNames) { + + scanPath(_keyBuilder.statusUpdates(instanceName).getPath()); + scanPath(HelixUtil.getInstancePropertyPath(_clusterId.stringify(), instanceName, + PropertyType.ERRORS)); + } + + scanPath(HelixUtil.getControllerPropertyPath(_clusterId.stringify(), + PropertyType.STATUSUPDATES_CONTROLLER)); + scanPath(HelixUtil.getControllerPropertyPath(_clusterId.stringify(), + PropertyType.ERRORS_CONTROLLER)); + } catch (Exception e) { + LOG.error("Exception dumping status/errors, clusterId: " + _clusterId, e); + } + } + + // TODO: refactor this + void scanPath(String path) { + LOG.info("Scannning path: " + path); + List childs = _baseAccessor.getChildNames(path, 0); + if (childs == null || childs.isEmpty()) { + return; + } + + for (String child : childs) { + String childPath = path + "/" + child; + + try { + List grandChilds = _baseAccessor.getChildNames(childPath, 0); + if (grandChilds == null || grandChilds.isEmpty()) { + continue; + } + + for (String grandChild : grandChilds) { + String grandChildPath = childPath + "/" + grandChild; + try { + checkAndDump(grandChildPath); + } catch (Exception e) { + LOG.error("Exception in dumping status, path: " + grandChildPath, e); + } + } + } catch (Exception e) { + LOG.error("Exception in dumping status, path: " + childPath, e); + } + } + } + + void checkAndDump(String path) { + List paths = new ArrayList(); + paths.add(path); + + List childs = _baseAccessor.getChildNames(path, 0); + if (childs != null && !childs.isEmpty()) { + for (String child : childs) { + String childPath = path + "/" + child; + paths.add(childPath); + } + } + + long nowInMs = System.currentTimeMillis(); + + List stats = new ArrayList(); + List records = _baseAccessor.get(paths, stats, 0); + for (int i = 0; i < paths.size(); i++) { + String dumpPath = paths.get(i); + Stat stat = stats.get(i); + ZNRecord record = records.get(i); + long timePassedInMs = nowInMs - stat.getMtime(); + if (timePassedInMs > _thresholdNoChangeInMs) { + LOG.info("Dumping status update path: " + dumpPath + ", " + timePassedInMs + + "MS has passed"); + try { + LOG.info(new String(_serializer.serialize(record))); + } catch (Exception e) { + LOG.warn("Ignorable exception serializing path: " + dumpPath + ", record: " + record, e); + } + _baseAccessor.remove(dumpPath, 0); + } + } + } + } + + public StatusDumpTask(ClusterId clusterId, HelixDataAccessor accessor) { + _accessor = accessor; + _clusterId = clusterId; + } + + @Override + public void start() { + final long initialDelay = 30 * 60 * 1000; + final long period = 120 * 60 * 1000; + final long thresholdNoChangeInMs = 180 * 60 * 1000; + + if (_timer == null) { + LOG.info("Start StatusDumpTask"); + _timer = new Timer("StatusDumpTimerTask", true); + _timer.scheduleAtFixedRate(new StatusDumpTimerTask(_clusterId, _accessor, + thresholdNoChangeInMs), initialDelay, period); + } + + } + + @Override + public void stop() { + if (_timer != null) { + LOG.info("Stop StatusDumpTask"); + _timer.cancel(); + _timer = null; + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java b/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java index 84d6c08573..226b250e17 100644 --- a/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java +++ b/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java @@ -26,9 +26,9 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.PropertyType; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.util.HelixUtil; @@ -47,7 +47,7 @@ public class ZKPathDataDumpTask extends TimerTask { public ZKPathDataDumpTask(HelixManager manager, ZkClient zkClient, int thresholdNoChangeInMs) { _manager = manager; _zkClient = zkClient; - logger.info("Scannning cluster statusUpdate " + manager.getClusterName() + logger.info("Scanning cluster statusUpdate " + manager.getClusterName() + " thresholdNoChangeInMs: " + thresholdNoChangeInMs); _thresholdNoChangeInMs = thresholdNoChangeInMs; } @@ -59,7 +59,7 @@ public void run() { // We need to think if we should create per-instance log files that contains // per-instance statusUpdates // and errors - logger.info("Scannning status updates ..."); + logger.info("Scanning status updates ..."); try { HelixDataAccessor accessor = _manager.getHelixDataAccessor(); Builder keyBuilder = accessor.keyBuilder(); @@ -82,7 +82,7 @@ public void run() { } void scanPath(String path, int thresholdNoChangeInMs) { - logger.info("Scannning path " + path); + logger.info("Scanning path " + path); List subPaths = _zkClient.getChildren(path); for (String subPath : subPaths) { try { diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItem.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItem.java deleted file mode 100644 index 19634e0d7c..0000000000 --- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItem.java +++ /dev/null @@ -1,88 +0,0 @@ -package org.apache.helix.monitoring.mbeans; - -/* - * 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 org.apache.helix.alerts.AlertValueAndStatus; - -public class ClusterAlertItem implements ClusterAlertItemMBean { - String _alertItemName; - double _alertValue; - int _alertFired; - String _additionalInfo = ""; - AlertValueAndStatus _valueAndStatus; - long _lastUpdateTime = 0; - - public ClusterAlertItem(String name, AlertValueAndStatus valueAndStatus) { - _valueAndStatus = valueAndStatus; - _alertItemName = name; - refreshValues(); - } - - @Override - public String getSensorName() { - return _alertItemName; - } - - @Override - public double getAlertValue() { - return _alertValue; - } - - public void setValueMap(AlertValueAndStatus valueAndStatus) { - _valueAndStatus = valueAndStatus; - refreshValues(); - } - - void refreshValues() { - _lastUpdateTime = new Date().getTime(); - if (_valueAndStatus.getValue().getElements().size() > 0) { - _alertValue = Double.parseDouble(_valueAndStatus.getValue().getElements().get(0)); - } else { - _alertValue = 0; - } - _alertFired = _valueAndStatus.isFired() ? 1 : 0; - } - - @Override - public int getAlertFired() { - return _alertFired; - } - - public void setAdditionalInfo(String additionalInfo) { - _additionalInfo = additionalInfo; - } - - @Override - public String getAdditionalInfo() { - return _additionalInfo; - } - - public void reset() { - _alertFired = 0; - _additionalInfo = ""; - _alertValue = 0; - } - - public long getLastUpdateTime() { - return _lastUpdateTime; - } -} diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertMBeanCollection.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertMBeanCollection.java deleted file mode 100644 index 0d06ca3230..0000000000 --- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertMBeanCollection.java +++ /dev/null @@ -1,275 +0,0 @@ -package org.apache.helix.monitoring.mbeans; - -/* - * 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.StringWriter; -import java.lang.management.ManagementFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -import javax.management.MBeanServer; -import javax.management.ObjectName; - -import org.apache.helix.PropertyType; -import org.apache.helix.ZNRecord; -import org.apache.helix.alerts.AlertParser; -import org.apache.helix.alerts.AlertValueAndStatus; -import org.apache.helix.alerts.Tuple; -import org.apache.log4j.Logger; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.SerializationConfig; - -public class ClusterAlertMBeanCollection { - public static String DOMAIN_ALERT = "HelixAlerts"; - public static String ALERT_SUMMARY = "AlertSummary"; - - private static final Logger _logger = Logger.getLogger(ClusterAlertMBeanCollection.class); - ConcurrentHashMap _alertBeans = - new ConcurrentHashMap(); - - Map _recentAlertDelta; - ClusterAlertSummary _clusterAlertSummary; - ZNRecord _alertHistory = new ZNRecord(PropertyType.ALERT_HISTORY.toString()); - Set _previousFiredAlerts = new HashSet(); - // 5 min for mbean freshness threshold - public static final long ALERT_NOCHANGE_THRESHOLD = 5 * 60 * 1000; - - final MBeanServer _beanServer; - - public interface ClusterAlertSummaryMBean extends ClusterAlertItemMBean { - public String getAlertFiredHistory(); - } - - class ClusterAlertSummary extends ClusterAlertItem implements ClusterAlertSummaryMBean { - public ClusterAlertSummary(String name, AlertValueAndStatus valueAndStatus) { - super(name, valueAndStatus); - } - - /** - * Returns the previous 100 alert mbean turn on / off history - */ - @Override - public String getAlertFiredHistory() { - try { - ObjectMapper mapper = new ObjectMapper(); - SerializationConfig serializationConfig = mapper.getSerializationConfig(); - serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true); - StringWriter sw = new StringWriter(); - mapper.writeValue(sw, _alertHistory); - return sw.toString(); - } catch (Exception e) { - _logger.warn("", e); - return ""; - } - } - } - - public ClusterAlertMBeanCollection() { - _beanServer = ManagementFactory.getPlatformMBeanServer(); - } - - public Collection getCurrentAlertMBeans() { - ArrayList beans = new ArrayList(); - for (ClusterAlertItem item : _alertBeans.values()) { - beans.add(item); - } - return beans; - } - - void onNewAlertMbeanAdded(ClusterAlertItemMBean bean) { - try { - _logger.info("alert bean " + bean.getSensorName() + " exposed to jmx"); - System.out.println("alert bean " + bean.getSensorName() + " exposed to jmx"); - ObjectName objectName = new ObjectName(DOMAIN_ALERT + ":alert=" + bean.getSensorName()); - register(bean, objectName); - } catch (Exception e) { - _logger.error("", e); - e.printStackTrace(); - } - } - - public void setAlerts(String originAlert, Map alertResultMap, - String clusterName) { - if (alertResultMap == null) { - _logger.warn("null alertResultMap"); - return; - } - for (String alertName : alertResultMap.keySet()) { - String beanName = ""; - if (alertName.length() > 1) { - String comparator = AlertParser.getComponent(AlertParser.COMPARATOR_NAME, originAlert); - String constant = AlertParser.getComponent(AlertParser.CONSTANT_NAME, originAlert); - beanName = "(" + alertName + ")" + comparator + "(" + constant + ")"; - } else { - beanName = originAlert + "--(" + alertName + ")"; - } - // This is to make JMX happy; certain charaters cannot be in JMX bean name - beanName = beanName.replace('*', '%').replace('=', '#').replace(',', ';'); - if (!_alertBeans.containsKey(beanName)) { - ClusterAlertItem item = new ClusterAlertItem(beanName, alertResultMap.get(alertName)); - onNewAlertMbeanAdded(item); - _alertBeans.put(beanName, item); - } else { - _alertBeans.get(beanName).setValueMap(alertResultMap.get(alertName)); - } - } - refreshSummayAlert(clusterName); - } - - public void setAlertHistory(ZNRecord alertHistory) { - _alertHistory = alertHistory; - } - - /** - * The summary alert is a combination of all alerts, if it is on, something is wrong on this - * cluster. The additional info contains all alert mbean names that has been fired. - */ - void refreshSummayAlert(String clusterName) { - boolean fired = false; - String alertsFired = ""; - String summaryKey = ALERT_SUMMARY + "_" + clusterName; - for (String key : _alertBeans.keySet()) { - if (!key.equals(summaryKey)) { - ClusterAlertItem item = _alertBeans.get(key); - fired = (item.getAlertFired() == 1) | fired; - if (item.getAlertFired() == 1) { - alertsFired += item._alertItemName; - alertsFired += ";"; - } - } - } - Tuple t = new Tuple(); - t.add("0"); - AlertValueAndStatus summaryStatus = new AlertValueAndStatus(t, fired); - if (!_alertBeans.containsKey(summaryKey)) { - ClusterAlertSummary item = new ClusterAlertSummary(summaryKey, summaryStatus); - onNewAlertMbeanAdded(item); - item.setAdditionalInfo(alertsFired); - _alertBeans.put(summaryKey, item); - _clusterAlertSummary = item; - } else { - _alertBeans.get(summaryKey).setValueMap(summaryStatus); - _alertBeans.get(summaryKey).setAdditionalInfo(alertsFired); - } - } - - void register(Object bean, ObjectName name) { - try { - _beanServer.unregisterMBean(name); - } catch (Exception e) { - } - try { - _beanServer.registerMBean(bean, name); - } catch (Exception e) { - _logger.error("Could not register MBean", e); - } - } - - public void reset() { - for (String beanName : _alertBeans.keySet()) { - ClusterAlertItem item = _alertBeans.get(beanName); - item.reset(); - try { - ObjectName objectName = new ObjectName(DOMAIN_ALERT + ":alert=" + item.getSensorName()); - _beanServer.unregisterMBean(objectName); - } catch (Exception e) { - _logger.warn("", e); - } - } - _alertBeans.clear(); - } - - public void refreshAlertDelta(String clusterName) { - // Update the alert turn on/turn off history - String summaryKey = ALERT_SUMMARY + "_" + clusterName; - Set currentFiredAlerts = new HashSet(); - for (String key : _alertBeans.keySet()) { - if (!key.equals(summaryKey)) { - ClusterAlertItem item = _alertBeans.get(key); - if (item.getAlertFired() == 1) { - currentFiredAlerts.add(item._alertItemName); - } - } - } - - Map onOffAlertsMap = new HashMap(); - for (String alertName : currentFiredAlerts) { - if (!_previousFiredAlerts.contains(alertName)) { - onOffAlertsMap.put(alertName, "ON"); - _logger.info(alertName + " ON"); - _previousFiredAlerts.add(alertName); - } - } - for (String cachedAlert : _previousFiredAlerts) { - if (!currentFiredAlerts.contains(cachedAlert)) { - onOffAlertsMap.put(cachedAlert, "OFF"); - _logger.info(cachedAlert + " OFF"); - } - } - for (String key : onOffAlertsMap.keySet()) { - if (onOffAlertsMap.get(key).equals("OFF")) { - _previousFiredAlerts.remove(key); - } - } - if (onOffAlertsMap.size() == 0) { - _logger.info("No MBean change"); - } - _recentAlertDelta = onOffAlertsMap; - - checkMBeanFreshness(ALERT_NOCHANGE_THRESHOLD); - } - - public Map getRecentAlertDelta() { - return _recentAlertDelta; - } - - /** - * Remove mbeans that has not been changed for thresholdInMs MS - */ - void checkMBeanFreshness(long thresholdInMs) { - long now = new Date().getTime(); - Set oldBeanNames = new HashSet(); - // Get mbean items that has not been updated for thresholdInMs - for (String beanName : _alertBeans.keySet()) { - ClusterAlertItem item = _alertBeans.get(beanName); - if (now - item.getLastUpdateTime() > thresholdInMs) { - oldBeanNames.add(beanName); - _logger.info("bean " + beanName + " has not been updated for " + thresholdInMs + " MS"); - } - } - for (String beanName : oldBeanNames) { - ClusterAlertItem item = _alertBeans.get(beanName); - _alertBeans.remove(beanName); - try { - item.reset(); - ObjectName objectName = new ObjectName(DOMAIN_ALERT + ":alert=" + item.getSensorName()); - _beanServer.unregisterMBean(objectName); - } catch (Exception e) { - _logger.warn("", e); - } - } - } -} diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterMBeanObserver.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterMBeanObserver.java index 5922ea979c..d66c1f603d 100644 --- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterMBeanObserver.java +++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterMBeanObserver.java @@ -22,20 +22,13 @@ import java.io.IOException; import java.lang.management.ManagementFactory; -import javax.management.AttributeNotFoundException; import javax.management.InstanceNotFoundException; -import javax.management.IntrospectionException; -import javax.management.ListenerNotFoundException; -import javax.management.MBeanAttributeInfo; -import javax.management.MBeanException; -import javax.management.MBeanInfo; import javax.management.MBeanServerConnection; import javax.management.MBeanServerDelegate; import javax.management.MBeanServerNotification; import javax.management.MalformedObjectNameException; import javax.management.Notification; import javax.management.NotificationListener; -import javax.management.ReflectionException; import javax.management.relation.MBeanServerNotificationFilter; import org.apache.log4j.Logger; diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java index e24b41f955..afd2886c59 100644 --- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java +++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java @@ -22,8 +22,9 @@ import java.util.Map; import org.apache.helix.HelixDefinedState; -import org.apache.helix.HelixManager; -import org.apache.helix.PropertyType; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.log4j.Logger; @@ -88,30 +89,31 @@ public void updateExternalView(ExternalView externalView, IdealState idealState) // TODO fix this; IdealState shall have either map fields (CUSTOM mode) // or list fields (AUDO mode) - for (String partitionName : idealState.getRecord().getMapFields().keySet()) { - Map idealRecord = idealState.getInstanceStateMap(partitionName); - Map externalViewRecord = externalView.getStateMap(partitionName); + for (PartitionId partitionId : idealState.getPartitionIdSet()) { + Map idealRecord = idealState.getParticipantStateMap(partitionId); + Map externalViewRecord = externalView.getStateMap(partitionId); if (externalViewRecord == null) { numOfDiff += idealRecord.size(); continue; } - for (String host : idealRecord.keySet()) { + for (ParticipantId host : idealRecord.keySet()) { if (!externalViewRecord.containsKey(host) || !externalViewRecord.get(host).equals(idealRecord.get(host))) { numOfDiff++; } } - for (String host : externalViewRecord.keySet()) { - if (externalViewRecord.get(host).equalsIgnoreCase(HelixDefinedState.ERROR.toString())) { + for (ParticipantId host : externalViewRecord.keySet()) { + if (externalViewRecord.get(host).toString() + .equalsIgnoreCase(HelixDefinedState.ERROR.toString())) { numOfErrorPartitions++; } } } _numOfErrorPartitions = numOfErrorPartitions; _externalViewIdealStateDiff = numOfDiff; - _numOfPartitionsInExternalView = externalView.getPartitionSet().size(); + _numOfPartitionsInExternalView = externalView.getPartitionIdSet().size(); } @Override diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/StateTransitionStatMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/StateTransitionStatMonitor.java index 20a4a073a7..0198410005 100644 --- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/StateTransitionStatMonitor.java +++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/StateTransitionStatMonitor.java @@ -22,7 +22,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import org.apache.commons.math.stat.descriptive.DescriptiveStatistics; import org.apache.helix.monitoring.StatCollector; import org.apache.helix.monitoring.StateTransitionContext; import org.apache.helix.monitoring.StateTransitionDataPoint; diff --git a/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerElection.java b/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerElection.java index 25aada2846..d76e874776 100644 --- a/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerElection.java +++ b/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerElection.java @@ -19,8 +19,6 @@ * under the License. */ -import java.lang.management.ManagementFactory; - import org.apache.helix.ControllerChangeListener; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; @@ -31,9 +29,8 @@ import org.apache.helix.PropertyType; import org.apache.helix.controller.GenericHelixController; import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.controller.restlet.ZKPropertyTransferServer; +import org.apache.helix.manager.zk.ZkHelixLeaderElection; import org.apache.helix.model.LeaderHistory; -import org.apache.helix.model.LiveInstance; import org.apache.log4j.Logger; // TODO: merge with GenericHelixController @@ -108,49 +105,7 @@ public synchronized void onControllerChange(NotificationContext changeContext) { } private boolean tryUpdateController(HelixManager manager) { - // DataAccessor dataAccessor = manager.getDataAccessor(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - LiveInstance leader = new LiveInstance(manager.getInstanceName()); - try { - leader.setLiveInstance(ManagementFactory.getRuntimeMXBean().getName()); - // TODO: this session id is not the leader's session id in - // distributed mode - leader.setSessionId(manager.getSessionId()); - leader.setHelixVersion(manager.getVersion()); - if (ZKPropertyTransferServer.getInstance() != null) { - String zkPropertyTransferServiceUrl = - ZKPropertyTransferServer.getInstance().getWebserviceUrl(); - if (zkPropertyTransferServiceUrl != null) { - leader.setWebserviceUrl(zkPropertyTransferServiceUrl); - } - } else { - LOG.warn("ZKPropertyTransferServer instnace is null"); - } - boolean success = accessor.createProperty(keyBuilder.controllerLeader(), leader); - if (success) { - return true; - } else { - LOG.info("Unable to become leader probably because some other controller becames the leader"); - } - } catch (Exception e) { - LOG.error( - "Exception when trying to updating leader record in cluster:" + manager.getClusterName() - + ". Need to check again whether leader node has been created or not", e); - } - - leader = accessor.getProperty(keyBuilder.controllerLeader()); - if (leader != null) { - String leaderSessionId = leader.getSessionId(); - LOG.info("Leader exists for cluster: " + manager.getClusterName() + ", currentLeader: " - + leader.getInstanceName() + ", leaderSessionId: " + leaderSessionId); - - if (leaderSessionId != null && leaderSessionId.equals(manager.getSessionId())) { - return true; - } - } - return false; + return ZkHelixLeaderElection.tryBecomingLeader(manager); } private void updateHistory(HelixManager manager) { diff --git a/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerStateModel.java b/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerStateModel.java index 8c88b7c382..0c2eb7c674 100644 --- a/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerStateModel.java +++ b/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerStateModel.java @@ -53,7 +53,7 @@ public void onBecomeStandbyFromOffline(Message message, NotificationContext cont @Transition(to = "LEADER", from = "STANDBY") public void onBecomeLeaderFromStandby(Message message, NotificationContext context) throws Exception { - String clusterName = message.getPartitionName(); + String clusterName = message.getPartitionId().stringify(); String controllerName = message.getTgtName(); logger.info(controllerName + " becomes leader from standby for " + clusterName); @@ -74,7 +74,7 @@ public void onBecomeLeaderFromStandby(Message message, NotificationContext conte @Transition(to = "STANDBY", from = "LEADER") public void onBecomeStandbyFromLeader(Message message, NotificationContext context) { - String clusterName = message.getPartitionName(); + String clusterName = message.getPartitionId().stringify(); String controllerName = message.getTgtName(); logger.info(controllerName + " becoming standby from leader for " + clusterName); @@ -89,7 +89,7 @@ public void onBecomeStandbyFromLeader(Message message, NotificationContext conte @Transition(to = "OFFLINE", from = "STANDBY") public void onBecomeOfflineFromStandby(Message message, NotificationContext context) { - String clusterName = message.getPartitionName(); + String clusterName = message.getPartitionId().stringify(); String controllerName = message.getTgtName(); logger.info(controllerName + " becoming offline from standby for cluster:" + clusterName); @@ -109,7 +109,7 @@ public void onBecomeOfflineFromDropped(Message message, NotificationContext cont @Override public void rollbackOnError(Message message, NotificationContext context, StateTransitionError error) { - String clusterName = message.getPartitionName(); + String clusterName = message.getPartitionId().stringify(); String controllerName = message.getTgtName(); logger.error(controllerName + " rollbacks on error for " + clusterName); diff --git a/helix-core/src/main/java/org/apache/helix/participant/GenericLeaderStandbyModel.java b/helix-core/src/main/java/org/apache/helix/participant/GenericLeaderStandbyModel.java index 705d740370..aa21ee3235 100644 --- a/helix-core/src/main/java/org/apache/helix/participant/GenericLeaderStandbyModel.java +++ b/helix-core/src/main/java/org/apache/helix/participant/GenericLeaderStandbyModel.java @@ -21,9 +21,9 @@ import java.util.List; +import org.apache.helix.HelixConstants.ChangeType; import org.apache.helix.HelixManager; import org.apache.helix.NotificationContext; -import org.apache.helix.HelixConstants.ChangeType; import org.apache.helix.PropertyKey.Builder; import org.apache.helix.model.Message; import org.apache.helix.participant.statemachine.StateModel; diff --git a/helix-core/src/main/java/org/apache/helix/participant/HelixCustomCodeRunner.java b/helix-core/src/main/java/org/apache/helix/participant/HelixCustomCodeRunner.java index 6a2490ada4..bddd8d17a6 100644 --- a/helix-core/src/main/java/org/apache/helix/participant/HelixCustomCodeRunner.java +++ b/helix-core/src/main/java/org/apache/helix/participant/HelixCustomCodeRunner.java @@ -23,11 +23,14 @@ import java.util.Arrays; import java.util.List; -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; import org.apache.helix.HelixConstants.ChangeType; import org.apache.helix.HelixConstants.StateModelToken; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixManager; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkBaseDataAccessor; @@ -127,15 +130,16 @@ public void start() throws Exception { zkClient = new ZkClient(_zkAddr, ZkClient.DEFAULT_CONNECTION_TIMEOUT); zkClient.setZkSerializer(new ZNRecordSerializer()); HelixDataAccessor accessor = - new ZKHelixDataAccessor(_manager.getClusterName(), new ZkBaseDataAccessor(zkClient)); + new ZKHelixDataAccessor(_manager.getClusterName(), new ZkBaseDataAccessor( + zkClient)); Builder keyBuilder = accessor.keyBuilder(); IdealState idealState = new IdealState(_resourceName); idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setReplicas(StateModelToken.ANY_LIVEINSTANCE.toString()); idealState.setNumPartitions(1); - idealState.setStateModelDefRef(LEADER_STANDBY); - idealState.setStateModelFactoryName(_resourceName); + idealState.setStateModelDefId(StateModelDefId.from(LEADER_STANDBY)); + idealState.setStateModelFactoryId(StateModelFactoryId.from(_resourceName)); List prefList = new ArrayList(Arrays.asList(StateModelToken.ANY_LIVEINSTANCE.toString())); idealState.getRecord().setListField(_resourceName + "_0", prefList); diff --git a/helix-core/src/main/java/org/apache/helix/participant/HelixStateMachineEngine.java b/helix-core/src/main/java/org/apache/helix/participant/HelixStateMachineEngine.java index 31fcecf4a8..95afb70984 100644 --- a/helix-core/src/main/java/org/apache/helix/participant/HelixStateMachineEngine.java +++ b/helix-core/src/main/java/org/apache/helix/participant/HelixStateMachineEngine.java @@ -31,25 +31,35 @@ import org.apache.helix.NotificationContext; import org.apache.helix.NotificationContext.MapKey; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.messaging.handling.BatchMessageHandler; import org.apache.helix.messaging.handling.BatchMessageWrapper; import org.apache.helix.messaging.handling.HelixStateTransitionHandler; -import org.apache.helix.messaging.handling.HelixTaskExecutor; import org.apache.helix.messaging.handling.MessageHandler; import org.apache.helix.messaging.handling.TaskExecutor; import org.apache.helix.model.CurrentState; import org.apache.helix.model.Message; -import org.apache.helix.model.StateModelDefinition; import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.participant.statemachine.HelixStateModelFactory; +import org.apache.helix.participant.statemachine.HelixStateModelFactoryAdaptor; import org.apache.helix.participant.statemachine.StateModel; import org.apache.helix.participant.statemachine.StateModelFactory; import org.apache.helix.participant.statemachine.StateModelParser; import org.apache.log4j.Logger; public class HelixStateMachineEngine implements StateMachineEngine { - private static Logger logger = Logger.getLogger(HelixStateMachineEngine.class); + private static Logger LOG = Logger.getLogger(HelixStateMachineEngine.class); - // StateModelName->FactoryName->StateModelFactory + /** + * Map of StateModelDefId to map of FactoryName to StateModelFactory + * TODO change to use StateModelDefId and HelixStateModelFactory + */ private final Map>> _stateModelFactoryMap; private final StateModelParser _stateModelParser; private final HelixManager _manager; @@ -90,7 +100,7 @@ public boolean registerStateModelFactory(String stateModelName, throw new HelixException("stateModelDef|stateModelFactory|factoryName cannot be null"); } - logger.info("Register state model factory for state model " + stateModelName + LOG.info("Register state model factory for state model " + stateModelName + " using factory name " + factoryName + " with " + factory); if (!_stateModelFactoryMap.containsKey(stateModelName)) { @@ -99,7 +109,7 @@ public boolean registerStateModelFactory(String stateModelName, } if (_stateModelFactoryMap.get(stateModelName).containsKey(factoryName)) { - logger.warn("stateModelFactory for " + stateModelName + " using factoryName " + factoryName + LOG.warn("stateModelFactory for " + stateModelName + " using factoryName " + factoryName + " has already been registered."); return false; } @@ -113,7 +123,8 @@ public boolean registerStateModelFactory(String stateModelName, private void sendNopMessage() { if (_manager.isConnected()) { try { - Message nopMsg = new Message(MessageType.NO_OP, UUID.randomUUID().toString()); + Message nopMsg = + new Message(MessageType.NO_OP, MessageId.from(UUID.randomUUID().toString())); nopMsg.setSrcName(_manager.getInstanceName()); HelixDataAccessor accessor = _manager.getHelixDataAccessor(); @@ -130,9 +141,9 @@ private void sendNopMessage() { nopMsg.setTgtName(_manager.getInstanceName()); accessor.setProperty(keyBuilder.message(nopMsg.getTgtName(), nopMsg.getId()), nopMsg); } - logger.info("Send NO_OP message to " + nopMsg.getTgtName() + ", msgId: " + nopMsg.getId()); + LOG.info("Send NO_OP message to " + nopMsg.getTgtName() + ", msgId: " + nopMsg.getId()); } catch (Exception e) { - logger.error(e); + LOG.error(e); } } } @@ -160,19 +171,18 @@ public MessageHandler createHandler(Message message, NotificationContext context if (!type.equals(MessageType.STATE_TRANSITION.toString())) { throw new HelixException("Expect state-transition message type, but was " - + message.getMsgType() + ", msgId: " + message.getMsgId()); + + message.getMsgType() + ", msgId: " + message.getMessageId()); } - String partitionKey = message.getPartitionName(); - String stateModelName = message.getStateModelDef(); - String resourceName = message.getResourceName(); - String sessionId = message.getTgtSessionId(); + PartitionId partitionKey = message.getPartitionId(); + StateModelDefId stateModelId = message.getStateModelDefId(); + ResourceId resourceId = message.getResourceId(); + SessionId sessionId = message.getTypedTgtSessionId(); int bucketSize = message.getBucketSize(); - if (stateModelName == null) { - logger - .error("Fail to create msg-handler because message does not contain stateModelDef. msgId: " - + message.getId()); + if (stateModelId == null) { + LOG.error("Fail to create msg-handler because message does not contain stateModelDef. msgId: " + + message.getId()); return null; } @@ -182,59 +192,61 @@ public MessageHandler createHandler(Message message, NotificationContext context } StateModelFactory stateModelFactory = - getStateModelFactory(stateModelName, factoryName); + getStateModelFactory(stateModelId.stringify(), factoryName); if (stateModelFactory == null) { - logger.warn("Fail to create msg-handler because cannot find stateModelFactory for model: " - + stateModelName + " using factoryName: " + factoryName + " for resource: " - + resourceName); + LOG.warn("Fail to create msg-handler because cannot find stateModelFactory for model: " + + stateModelId + " using factoryName: " + factoryName + " for resource: " + resourceId); return null; } // check if the state model definition exists and cache it - if (!_stateModelDefs.containsKey(stateModelName)) { + if (!_stateModelDefs.containsKey(stateModelId.stringify())) { HelixDataAccessor accessor = _manager.getHelixDataAccessor(); Builder keyBuilder = accessor.keyBuilder(); StateModelDefinition stateModelDef = - accessor.getProperty(keyBuilder.stateModelDef(stateModelName)); + accessor.getProperty(keyBuilder.stateModelDef(stateModelId.stringify())); if (stateModelDef == null) { throw new HelixException("fail to create msg-handler because stateModelDef for " - + stateModelName + " does NOT exist"); + + stateModelId + " does NOT exist"); } - _stateModelDefs.put(stateModelName, stateModelDef); + _stateModelDefs.put(stateModelId.stringify(), stateModelDef); } if (message.getBatchMessageMode() == false) { // create currentStateDelta for this partition String initState = _stateModelDefs.get(message.getStateModelDef()).getInitialState(); - StateModel stateModel = stateModelFactory.getStateModel(partitionKey); + StateModel stateModel = stateModelFactory.getStateModel(partitionKey.stringify()); if (stateModel == null) { - stateModel = stateModelFactory.createAndAddStateModel(partitionKey); + stateModel = stateModelFactory.createAndAddStateModel(partitionKey.stringify()); stateModel.updateState(initState); } // TODO: move currentStateDelta to StateTransitionMsgHandler - CurrentState currentStateDelta = new CurrentState(resourceName); + CurrentState currentStateDelta = new CurrentState(resourceId.stringify()); currentStateDelta.setSessionId(sessionId); - currentStateDelta.setStateModelDefRef(stateModelName); + currentStateDelta.setStateModelDefRef(stateModelId.stringify()); currentStateDelta.setStateModelFactoryName(factoryName); currentStateDelta.setBucketSize(bucketSize); - currentStateDelta.setState(partitionKey, (stateModel.getCurrentState() == null) ? initState - : stateModel.getCurrentState()); + currentStateDelta.setState( + partitionKey, + (stateModel.getCurrentState() == null) ? State.from(initState) : State.from(stateModel + .getCurrentState())); return new HelixStateTransitionHandler(stateModelFactory, stateModel, message, context, currentStateDelta); } else { - BatchMessageWrapper wrapper = stateModelFactory.getBatchMessageWrapper(resourceName); + BatchMessageWrapper wrapper = + stateModelFactory.getBatchMessageWrapper(resourceId.stringify()); if (wrapper == null) { - wrapper = stateModelFactory.createAndAddBatchMessageWrapper(resourceName); + wrapper = stateModelFactory.createAndAddBatchMessageWrapper(resourceId.stringify()); } // get executor-service for the message TaskExecutor executor = (TaskExecutor) context.get(MapKey.TASK_EXECUTOR.toString()); if (executor == null) { - logger.error("fail to get executor-service for batch message: " + message.getId() - + ". msgType: " + message.getMsgType() + ", resource: " + message.getResourceName()); + LOG.error("fail to get executor-service for batch message: " + message.getId() + + ". msgType: " + message.getMsgType() + ", resource: " + message.getResourceId()); return null; } return new BatchMessageHandler(message, context, this, wrapper, executor); @@ -257,4 +269,86 @@ public boolean removeStateModelFactory(String stateModelDef, StateModelFactory factory, String factoryName) { throw new UnsupportedOperationException("Remove not yet supported"); } + + @Override + public boolean registerStateModelFactory(StateModelDefId stateModelDefId, + HelixStateModelFactory factory) { + return registerStateModelFactory(stateModelDefId, HelixConstants.DEFAULT_STATE_MODEL_FACTORY, + factory); + } + + @Override + public boolean registerStateModelFactory(StateModelDefId stateModelDefId, String factoryName, + HelixStateModelFactory factory) { + if (stateModelDefId == null || factoryName == null || factory == null) { + LOG.info("stateModelDefId|factoryName|stateModelFactory is null"); + return false; + } + + LOG.info("Registering state model factory for state-model-definition: " + stateModelDefId + + " using factory-name: " + factoryName + " with: " + factory); + + StateModelFactory factoryAdaptor = + new HelixStateModelFactoryAdaptor(factory); + + String stateModelDefName = stateModelDefId.stringify(); + if (!_stateModelFactoryMap.containsKey(stateModelDefName)) { + _stateModelFactoryMap.put(stateModelDefName, + new ConcurrentHashMap>()); + } + + if (_stateModelFactoryMap.get(stateModelDefName).containsKey(factoryName)) { + LOG.info("Skip register state model factory for " + stateModelDefId + " using factory-name " + + factoryName + ", since it has already been registered."); + return false; + } + + _stateModelFactoryMap.get(stateModelDefName).put(factoryName, factoryAdaptor); + + sendNopMessage(); + return true; + } + + @Override + public boolean removeStateModelFactory(StateModelDefId stateModelDefId) { + return removeStateModelFactory(stateModelDefId, HelixConstants.DEFAULT_STATE_MODEL_FACTORY); + } + + @Override + public boolean removeStateModelFactory(StateModelDefId stateModelDefId, String factoryName) { + if (stateModelDefId == null || factoryName == null) { + LOG.info("stateModelDefId|factoryName is null"); + return false; + } + + LOG.info("Removing state model factory for state-model-definition: " + stateModelDefId + + " using factory-name: " + factoryName); + + String stateModelDefName = stateModelDefId.stringify(); + Map> ftyMap = + _stateModelFactoryMap.get(stateModelDefName); + if (ftyMap == null) { + LOG.info("Skip remove state model factory " + stateModelDefId + ", since it does NOT exist"); + return false; + } + + StateModelFactory fty = ftyMap.remove(factoryName); + if (fty == null) { + LOG.info("Skip remove state model factory " + stateModelDefId + " using factory-name " + + factoryName + ", since it does NOT exist"); + return false; + } + + if (ftyMap.isEmpty()) { + _stateModelFactoryMap.remove(stateModelDefName); + } + + for (String partition : fty.getPartitionSet()) { + StateModel stateModel = fty.getStateModel(partition); + stateModel.reset(); + // TODO probably should remove the state from zookeeper + } + + return true; + } } diff --git a/helix-core/src/main/java/org/apache/helix/participant/StateMachineEngine.java b/helix-core/src/main/java/org/apache/helix/participant/StateMachineEngine.java index d11b3cc559..80c954512c 100644 --- a/helix-core/src/main/java/org/apache/helix/participant/StateMachineEngine.java +++ b/helix-core/src/main/java/org/apache/helix/participant/StateMachineEngine.java @@ -19,52 +19,84 @@ * under the License. */ +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.messaging.handling.MessageHandlerFactory; +import org.apache.helix.participant.statemachine.HelixStateModelFactory; import org.apache.helix.participant.statemachine.StateModel; import org.apache.helix.participant.statemachine.StateModelFactory; /** - * Helix participant manager uses this class to register/remove state model factory + * Helix participant uses this class to register/remove state model factory * State model factory creates state model that handles state transition messages */ public interface StateMachineEngine extends MessageHandlerFactory { + + /** + * Replaced by {@link #registerStateModelFactory(StateModelDefId, HelixStateModelFactory) + */ + @Deprecated + public boolean registerStateModelFactory(String stateModelDef, + StateModelFactory factory); + + /** + * Replaced by {@link #registerStateModelFactory(StateModelDefId, String, HelixStateModelFactory)} + */ + @Deprecated + public boolean registerStateModelFactory(String stateModelDef, + StateModelFactory factory, String factoryName); + + /** + * Replaced by {@link #removeStateModelFactory(StateModelDefId, HelixStateModelFactory)} + */ + @Deprecated + public boolean removeStateModelFactory(String stateModelDef, + StateModelFactory factory); + + /** + * Replaced by {@link #removeStateModelFactory(StateModelDefId, String, HelixStateModelFactory)} + */ + @Deprecated + public boolean removeStateModelFactory(String stateModelDef, + StateModelFactory factory, String factoryName); + /** * Register a default state model factory for a state model definition * A state model definition could be, for example: * "MasterSlave", "OnlineOffline", "LeaderStandby", etc. - * @param stateModelDef + * Replacing {@link #registerStateModelFactory(String, StateModelFactory)} + * @param stateModelDefId * @param factory * @return */ - public boolean registerStateModelFactory(String stateModelDef, - StateModelFactory factory); + public boolean registerStateModelFactory(StateModelDefId stateModelDefId, + HelixStateModelFactory factory); /** - * Register a state model factory with a name for a state model definition - * @param stateModelDef - * @param factory + * Register a state model factory with a factory name for a state model definition + * Replacing {@link #registerStateModelFactory(String, StateModelFactory, String)} + * @param stateModelDefId * @param factoryName + * @param factory * @return */ - public boolean registerStateModelFactory(String stateModelDef, - StateModelFactory factory, String factoryName); + public boolean registerStateModelFactory(StateModelDefId stateModelDefId, String factoryName, + HelixStateModelFactory factory); /** * Remove the default state model factory for a state model definition - * @param stateModelDef - * @param factory + * Replacing {@link #removeStateModelFactory(String, StateModelFactory) + * @param stateModelDefId * @return */ - public boolean removeStateModelFactory(String stateModelDef, - StateModelFactory factory); + public boolean removeStateModelFactory(StateModelDefId stateModelDefId); /** * Remove the state model factory with a name for a state model definition - * @param stateModelDef - * @param factory + * Replacing {@link #removeStateModelFactory(String, StateModelFactory, String)} + * @param stateModelDefId * @param factoryName * @return */ - public boolean removeStateModelFactory(String stateModelDef, - StateModelFactory factory, String factoryName); + public boolean removeStateModelFactory(StateModelDefId stateModelDefId, String factoryName); + } diff --git a/helix-core/src/main/java/org/apache/helix/participant/statemachine/HelixStateModelFactory.java b/helix-core/src/main/java/org/apache/helix/participant/statemachine/HelixStateModelFactory.java new file mode 100644 index 0000000000..45f56e5625 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/participant/statemachine/HelixStateModelFactory.java @@ -0,0 +1,118 @@ +package org.apache.helix.participant.statemachine; + +/* + * 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.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.messaging.handling.BatchMessageWrapper; + +/** + * State model factory that uses concrete id classes instead of strings. + * Replacing {@link org.apache.helix.participant.statemachine.StateModelFactory} + */ +public abstract class HelixStateModelFactory { + /** + * map from partitionId to stateModel + */ + private final ConcurrentMap _stateModelMap = + new ConcurrentHashMap(); + + /** + * map from resourceName to BatchMessageWrapper + */ + private final ConcurrentMap _batchMsgWrapperMap = + new ConcurrentHashMap(); + + /** + * This method will be invoked only once per partition per session + * @param partitionId + * @return + */ + public abstract T createNewStateModel(PartitionId partitionId); + + /** + * Create a state model for a partition + * @param partitionId + */ + public T createAndAddStateModel(PartitionId partitionId) { + T stateModel = createNewStateModel(partitionId); + _stateModelMap.put(partitionId, stateModel); + return stateModel; + } + + /** + * Get the state model for a partition + * @param partitionId + * @return state model if exists, null otherwise + */ + public T getStateModel(PartitionId partitionId) { + return _stateModelMap.get(partitionId); + } + + /** + * remove state model for a partition + * @param partitionId + * @return state model removed or null if not exist + */ + public T removeStateModel(PartitionId partitionId) { + return _stateModelMap.remove(partitionId); + } + + /** + * get partition set + * @return partitionId set + */ + public Set getPartitionSet() { + return _stateModelMap.keySet(); + } + + /** + * create a default batch-message-wrapper for a resource + * @param resourceId + * @return + */ + public BatchMessageWrapper createBatchMessageWrapper(ResourceId resourceId) { + return new BatchMessageWrapper(); + } + + /** + * create a batch-message-wrapper for a resource and put it into map + * @param resourceId + * @return + */ + public BatchMessageWrapper createAndAddBatchMessageWrapper(ResourceId resourceId) { + BatchMessageWrapper wrapper = createBatchMessageWrapper(resourceId); + _batchMsgWrapperMap.put(resourceId, wrapper); + return wrapper; + } + + /** + * get batch-message-wrapper for a resource + * @param resourceId + * @return + */ + public BatchMessageWrapper getBatchMessageWrapper(ResourceId resourceId) { + return _batchMsgWrapperMap.get(resourceId); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/DivideOperator.java b/helix-core/src/main/java/org/apache/helix/participant/statemachine/HelixStateModelFactoryAdaptor.java similarity index 64% rename from helix-core/src/main/java/org/apache/helix/alerts/DivideOperator.java rename to helix-core/src/main/java/org/apache/helix/participant/statemachine/HelixStateModelFactoryAdaptor.java index 122d4c175d..320275d37d 100644 --- a/helix-core/src/main/java/org/apache/helix/alerts/DivideOperator.java +++ b/helix-core/src/main/java/org/apache/helix/participant/statemachine/HelixStateModelFactoryAdaptor.java @@ -1,4 +1,4 @@ -package org.apache.helix.alerts; +package org.apache.helix.participant.statemachine; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,22 +19,18 @@ * under the License. */ -import java.util.Iterator; -import java.util.List; +import org.apache.helix.api.id.PartitionId; -public class DivideOperator extends Operator { +public class HelixStateModelFactoryAdaptor extends StateModelFactory { + final HelixStateModelFactory _factory; - public DivideOperator() { - minInputTupleLists = 2; - maxInputTupleLists = 2; - inputOutputTupleListsCountsEqual = false; - numOutputTupleLists = 1; + public HelixStateModelFactoryAdaptor(HelixStateModelFactory factory) { + _factory = factory; } @Override - public List>> execute(List>> input) { - // TODO Auto-generated method stub - return null; + public T createNewStateModel(String partitionName) { + return _factory.createNewStateModel(PartitionId.from(partitionName)); } } diff --git a/helix-core/src/main/java/org/apache/helix/participant/statemachine/ScheduledTaskStateModel.java b/helix-core/src/main/java/org/apache/helix/participant/statemachine/ScheduledTaskStateModel.java index 8b6a02c885..ca67d42e0e 100644 --- a/helix-core/src/main/java/org/apache/helix/participant/statemachine/ScheduledTaskStateModel.java +++ b/helix-core/src/main/java/org/apache/helix/participant/statemachine/ScheduledTaskStateModel.java @@ -52,6 +52,7 @@ public ScheduledTaskStateModel(ScheduledTaskStateModelFactory factory, public void onBecomeCompletedFromOffline(Message message, NotificationContext context) throws InterruptedException { logger.info(_partitionName + " onBecomeCompletedFromOffline"); + // System.err.println("\t\t" + _partitionName + " onBecomeCompletedFromOffline"); // Construct the inner task message from the mapfields of scheduledTaskQueue resource group Map messageInfo = diff --git a/helix-core/src/main/java/org/apache/helix/participant/statemachine/StateModelFactory.java b/helix-core/src/main/java/org/apache/helix/participant/statemachine/StateModelFactory.java index bd1a66895b..a74f67b938 100644 --- a/helix-core/src/main/java/org/apache/helix/participant/statemachine/StateModelFactory.java +++ b/helix-core/src/main/java/org/apache/helix/participant/statemachine/StateModelFactory.java @@ -19,13 +19,16 @@ * under the License. */ -import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.helix.messaging.handling.BatchMessageWrapper; +/** + * Replaced by {@link org.apache.helix.participant.statemachine.HelixStateModelFactory} + */ +@Deprecated public abstract class StateModelFactory { /** * mapping from partitionName to StateModel diff --git a/helix-core/src/main/java/org/apache/helix/tools/CLMLogFileAppender.java b/helix-core/src/main/java/org/apache/helix/tools/CLMLogFileAppender.java index 326607fad7..4f599254a1 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/CLMLogFileAppender.java +++ b/helix-core/src/main/java/org/apache/helix/tools/CLMLogFileAppender.java @@ -19,14 +19,9 @@ * under the License. */ -import java.io.File; import java.io.IOException; -import java.lang.management.ManagementFactory; -import java.lang.management.RuntimeMXBean; import java.text.SimpleDateFormat; import java.util.Calendar; -import java.util.Date; -import java.util.List; import org.apache.log4j.FileAppender; import org.apache.log4j.Layout; diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterExternalViewVerifier.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterExternalViewVerifier.java new file mode 100644 index 0000000000..a0959ccab9 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterExternalViewVerifier.java @@ -0,0 +1,188 @@ +package org.apache.helix.tools; + +/* + * 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.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.helix.api.Cluster; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.pipeline.Stage; +import org.apache.helix.controller.pipeline.StageContext; +import org.apache.helix.controller.stages.AttributeName; +import org.apache.helix.controller.stages.BestPossibleStateCalcStage; +import org.apache.helix.controller.stages.BestPossibleStateOutput; +import org.apache.helix.controller.stages.ClusterDataCache; +import org.apache.helix.controller.stages.ClusterEvent; +import org.apache.helix.controller.stages.CurrentStateComputationStage; +import org.apache.helix.controller.stages.ResourceComputationStage; +import org.apache.helix.manager.zk.ZkClient; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.Partition; +import org.apache.helix.model.ResourceAssignment; +import org.apache.log4j.Logger; + +import com.google.common.base.Functions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +/** + * given zk, cluster, and a list of expected live-instances + * check whether cluster's external-view reaches best-possible states + */ +public class ClusterExternalViewVerifier extends ClusterVerifier { + private static Logger LOG = Logger.getLogger(ClusterExternalViewVerifier.class); + + final List _expectSortedLiveNodes; // always sorted + + public ClusterExternalViewVerifier(ZkClient zkclient, String clusterName, + List expectLiveNodes) { + super(zkclient, clusterName); + _expectSortedLiveNodes = expectLiveNodes; + Collections.sort(_expectSortedLiveNodes); + } + + boolean verifyLiveNodes(List actualLiveNodes) { + Collections.sort(actualLiveNodes); + List rawActualLiveNodes = Lists.transform(actualLiveNodes, Functions.toStringFunction()); + return _expectSortedLiveNodes.equals(rawActualLiveNodes); + } + + /** + * @param externalView + * @param bestPossibleState map of partition to map of instance to state + * @return + */ + boolean verifyExternalView(ExternalView externalView, + Map> bestPossibleState) { + Map> bestPossibleStateMap = + convertBestPossibleState(bestPossibleState); + // trimBestPossibleState(bestPossibleStateMap); + + Map> externalViewMap = externalView.getRecord().getMapFields(); + return externalViewMap.equals(bestPossibleStateMap); + } + + static void runStage(ClusterEvent event, Stage stage) throws Exception { + StageContext context = new StageContext(); + stage.init(context); + stage.preProcess(); + stage.process(event); + stage.postProcess(); + } + + BestPossibleStateOutput calculateBestPossibleState(Cluster cluster) throws Exception { + ClusterEvent event = new ClusterEvent("event"); + event.addAttribute("ClusterDataCache", cluster); + + List stages = new ArrayList(); + stages.add(new ResourceComputationStage()); + stages.add(new CurrentStateComputationStage()); + stages.add(new BestPossibleStateCalcStage()); + + for (Stage stage : stages) { + runStage(event, stage); + } + + return event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); + } + + /** + * remove empty map and DROPPED state from best possible state + * @param bestPossibleState + */ + // static void trimBestPossibleState(Map> bestPossibleState) { + // Iterator>> iter = bestPossibleState.entrySet().iterator(); + // while (iter.hasNext()) { + // Map.Entry> entry = iter.next(); + // Map instanceStateMap = entry.getValue(); + // if (instanceStateMap.isEmpty()) { + // iter.remove(); + // } else { + // // remove instances with DROPPED state + // Iterator> insIter = instanceStateMap.entrySet().iterator(); + // while (insIter.hasNext()) { + // Map.Entry insEntry = insIter.next(); + // String state = insEntry.getValue(); + // if (state.equalsIgnoreCase(HelixDefinedState.DROPPED.toString())) { + // insIter.remove(); + // } + // } + // } + // } + // } + + static Map> convertBestPossibleState( + Map> bestPossibleState) { + Map> result = new HashMap>(); + for (PartitionId partition : bestPossibleState.keySet()) { + result.put(partition.stringify(), bestPossibleState.get(partition)); + } + return result; + } + + @Override + public boolean verify() throws Exception { + ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(_clusterName), _accessor); + Cluster cluster = clusterAccessor.readCluster(); + + List liveInstances = new ArrayList(); + liveInstances.addAll(cluster.getLiveParticipantMap().keySet()); + boolean success = verifyLiveNodes(liveInstances); + if (!success) { + LOG.info("liveNodes not match, expect: " + _expectSortedLiveNodes + ", actual: " + + liveInstances); + return false; + } + + BestPossibleStateOutput bestPossbileStates = calculateBestPossibleState(cluster); + Map externalViews = + _accessor.getChildValuesMap(_keyBuilder.externalViews()); + + // TODO all ideal-states should be included in external-views + + for (String resourceName : externalViews.keySet()) { + ExternalView externalView = externalViews.get(resourceName); + ResourceAssignment assignment = + bestPossbileStates.getResourceAssignment(ResourceId.from(resourceName)); + final Map> bestPossibleState = Maps.newHashMap(); + for (PartitionId partitionId : assignment.getMappedPartitionIds()) { + Map rawStateMap = + ResourceAssignment.stringMapFromReplicaMap(assignment.getReplicaMap(partitionId)); + bestPossibleState.put(partitionId, rawStateMap); + } + success = verifyExternalView(externalView, bestPossibleState); + if (!success) { + LOG.info("external-view for resource: " + resourceName + " not match"); + return false; + } + } + + return true; + } + +} diff --git a/helix-core/src/test/java/org/apache/helix/ZkHelixTestManager.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterLiveNodesVerifier.java similarity index 57% rename from helix-core/src/test/java/org/apache/helix/ZkHelixTestManager.java rename to helix-core/src/main/java/org/apache/helix/tools/ClusterLiveNodesVerifier.java index b660a1d316..b9de466a32 100644 --- a/helix-core/src/test/java/org/apache/helix/ZkHelixTestManager.java +++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterLiveNodesVerifier.java @@ -1,4 +1,4 @@ -package org.apache.helix; +package org.apache.helix.tools; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,26 +19,27 @@ * under the License. */ +import java.util.Collections; import java.util.List; -import org.apache.helix.manager.zk.CallbackHandler; -import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.manager.zk.ZkClient; -// ZkHelixManager used for test only. expose more class members -public class ZkHelixTestManager extends ZKHelixManager { +public class ClusterLiveNodesVerifier extends ClusterVerifier { - public ZkHelixTestManager(String clusterName, String instanceName, InstanceType instanceType, - String zkConnectString) throws Exception { - super(clusterName, instanceName, instanceType, zkConnectString); - // TODO Auto-generated constructor stub - } + final List _expectSortedLiveNodes; // always sorted - public ZkClient getZkClient() { - return _zkClient; + public ClusterLiveNodesVerifier(ZkClient zkclient, String clusterName, + List expectLiveNodes) { + super(zkclient, clusterName); + _expectSortedLiveNodes = expectLiveNodes; + Collections.sort(_expectSortedLiveNodes); } - public List getHandlers() { - return _handlers; + @Override + public boolean verify() throws Exception { + List actualLiveNodes = _accessor.getChildNames(_keyBuilder.liveInstances()); + Collections.sort(actualLiveNodes); + return _expectSortedLiveNodes.equals(actualLiveNodes); } + } diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java index a39e571963..6e2d5e0157 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java +++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java @@ -39,12 +39,9 @@ import org.apache.helix.HelixAdmin; import org.apache.helix.HelixConstants.StateModelToken; import org.apache.helix.HelixException; -import org.apache.helix.PropertyKey.Builder; import org.apache.helix.ZNRecord; 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.manager.zk.ZkClient; import org.apache.helix.model.ClusterConstraints; import org.apache.helix.model.ClusterConstraints.ConstraintType; @@ -55,7 +52,6 @@ import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.RebalanceMode; import org.apache.helix.model.InstanceConfig; -import org.apache.helix.model.LiveInstance; import org.apache.helix.model.StateModelDefinition; import org.apache.helix.model.builder.ConstraintItemBuilder; import org.apache.helix.model.builder.HelixConfigScopeBuilder; @@ -64,7 +60,7 @@ import org.apache.log4j.Logger; public class ClusterSetup { - private static Logger logger = Logger.getLogger(ClusterSetup.class); + private static Logger LOG = Logger.getLogger(ClusterSetup.class); public static final String zkServerAddress = "zkSvr"; // List info about the cluster / resource / Instances @@ -117,12 +113,6 @@ public class ClusterSetup { // help public static final String help = "help"; - // stats/alerts - public static final String addStat = "addStat"; - public static final String addAlert = "addAlert"; - public static final String dropStat = "dropStat"; - public static final String dropAlert = "dropAlert"; - // get/set/remove configs public static final String getConfig = "getConfig"; public static final String setConfig = "setConfig"; @@ -134,20 +124,19 @@ public class ClusterSetup { public static final String removeConstraint = "removeConstraint"; static Logger _logger = Logger.getLogger(ClusterSetup.class); - String _zkServerAddress; - ZkClient _zkClient; - HelixAdmin _admin; + final HelixAdmin _admin; public ClusterSetup(String zkServerAddress) { - _zkServerAddress = zkServerAddress; - _zkClient = ZKClientPool.getZkClient(_zkServerAddress); - _admin = new ZKHelixAdmin(_zkClient); + ZkClient zkClient = ZKClientPool.getZkClient(zkServerAddress); + _admin = new ZKHelixAdmin(zkClient); } public ClusterSetup(ZkClient zkClient) { - _zkServerAddress = zkClient.getServers(); - _zkClient = zkClient; - _admin = new ZKHelixAdmin(_zkClient); + _admin = new ZKHelixAdmin(zkClient); + } + + public ClusterSetup(HelixAdmin admin) { + _admin = admin; } public void addCluster(String clusterName, boolean overwritePrevious) { @@ -238,101 +227,12 @@ public void dropInstancesFromCluster(String clusterName, String[] instanceInfoAr } public void dropInstanceFromCluster(String clusterName, String instanceId) { - ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); - Builder keyBuilder = accessor.keyBuilder(); - InstanceConfig instanceConfig = toInstanceConfig(instanceId); - instanceId = instanceConfig.getInstanceName(); - - // ensure node is stopped - LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance(instanceId)); - if (liveInstance != null) { - throw new HelixException("Can't drop " + instanceId + ", please stop " + instanceId - + " before drop it"); - } - - InstanceConfig config = accessor.getProperty(keyBuilder.instanceConfig(instanceId)); - if (config == null) { - String error = "Node " + instanceId + " does not exist, cannot drop"; - _logger.warn(error); - throw new HelixException(error); - } - - // ensure node is disabled, otherwise fail - if (config.getInstanceEnabled()) { - String error = "Node " + instanceId + " is enabled, cannot drop"; - _logger.warn(error); - throw new HelixException(error); - } - _admin.dropInstance(clusterName, config); + _admin.dropInstance(clusterName, instanceConfig); } public void swapInstance(String clusterName, String oldInstanceName, String newInstanceName) { - ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); - Builder keyBuilder = accessor.keyBuilder(); - - InstanceConfig oldConfig = accessor.getProperty(keyBuilder.instanceConfig(oldInstanceName)); - if (oldConfig == null) { - String error = "Old instance " + oldInstanceName + " does not exist, cannot swap"; - _logger.warn(error); - throw new HelixException(error); - } - - InstanceConfig newConfig = accessor.getProperty(keyBuilder.instanceConfig(newInstanceName)); - if (newConfig == null) { - String error = "New instance " + newInstanceName + " does not exist, cannot swap"; - _logger.warn(error); - throw new HelixException(error); - } - - // ensure old instance is disabled, otherwise fail - if (oldConfig.getInstanceEnabled()) { - String error = - "Old instance " + oldInstanceName + " is enabled, it need to be disabled and turned off"; - _logger.warn(error); - throw new HelixException(error); - } - // ensure old instance is down, otherwise fail - List liveInstanceNames = accessor.getChildNames(accessor.keyBuilder().liveInstances()); - - if (liveInstanceNames.contains(oldInstanceName)) { - String error = - "Old instance " + oldInstanceName + " is still on, it need to be disabled and turned off"; - _logger.warn(error); - throw new HelixException(error); - } - - dropInstanceFromCluster(clusterName, oldInstanceName); - - List existingIdealStates = - accessor.getChildValues(accessor.keyBuilder().idealStates()); - for (IdealState idealState : existingIdealStates) { - swapInstanceInIdealState(idealState, oldInstanceName, newInstanceName); - accessor.setProperty(accessor.keyBuilder().idealStates(idealState.getResourceName()), - idealState); - } - } - - void swapInstanceInIdealState(IdealState idealState, String oldInstance, String newInstance) { - for (String partition : idealState.getRecord().getMapFields().keySet()) { - Map valMap = idealState.getRecord().getMapField(partition); - if (valMap.containsKey(oldInstance)) { - valMap.put(newInstance, valMap.get(oldInstance)); - valMap.remove(oldInstance); - } - } - - for (String partition : idealState.getRecord().getListFields().keySet()) { - List valList = idealState.getRecord().getListField(partition); - for (int i = 0; i < valList.size(); i++) { - if (valList.get(i).equals(oldInstance)) { - valList.remove(i); - valList.add(i, newInstance); - } - } - } + _admin.swapInstance(clusterName, oldInstanceName, newInstanceName); } public HelixAdmin getClusterManagementTool() { @@ -383,15 +283,15 @@ public void expandResource(String clusterName, String resourceName) { IdealState idealState = _admin.getResourceIdealState(clusterName, resourceName); if (idealState.getRebalanceMode() == RebalanceMode.FULL_AUTO || idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED) { - _logger.info("Skipping idealState " + idealState.getResourceName() + " " + _logger.info("Skipping idealState " + idealState.getResourceId() + " " + idealState.getRebalanceMode()); return; } boolean anyLiveInstance = false; for (List list : idealState.getRecord().getListFields().values()) { if (list.contains(StateModelToken.ANY_LIVEINSTANCE.toString())) { - _logger.info("Skipping idealState " + idealState.getResourceName() - + " with ANY_LIVEINSTANCE"); + _logger + .info("Skipping idealState " + idealState.getResourceId() + " with ANY_LIVEINSTANCE"); anyLiveInstance = true; continue; } @@ -590,18 +490,7 @@ public static void printUsage(Options cliOptions) { } @SuppressWarnings("static-access") - private static Options constructCommandLineOptions() { - Option helpOption = - OptionBuilder.withLongOpt(help).withDescription("Prints command-line options info") - .create(); - - Option zkServerOption = - OptionBuilder.withLongOpt(zkServerAddress).withDescription("Provide zookeeper address") - .create(); - zkServerOption.setArgs(1); - zkServerOption.setRequired(true); - zkServerOption.setArgName("ZookeeperServerAddress(Required)"); - + public static OptionGroup constructOptionGroup() { Option listClustersOption = OptionBuilder.withLongOpt(listClusters).withDescription("List existing clusters").create(); listClustersOption.setArgs(0); @@ -791,7 +680,7 @@ private static Options constructCommandLineOptions() { partitionInfoOption.setArgName("clusterName resourceName partitionName"); Option enableInstanceOption = - OptionBuilder.withLongOpt(enableInstance).withDescription("Enable/disable a Instance") + OptionBuilder.withLongOpt(enableInstance).withDescription("Enable/disable an instance") .create(); enableInstanceOption.setArgs(3); enableInstanceOption.setRequired(false); @@ -846,17 +735,6 @@ private static Options constructCommandLineOptions() { listStateModelOption.setRequired(false); listStateModelOption.setArgName("clusterName stateModelName"); - Option addStatOption = - OptionBuilder.withLongOpt(addStat).withDescription("Add a persistent stat").create(); - addStatOption.setArgs(2); - addStatOption.setRequired(false); - addStatOption.setArgName("clusterName statName"); - Option addAlertOption = - OptionBuilder.withLongOpt(addAlert).withDescription("Add an alert").create(); - addAlertOption.setArgs(2); - addAlertOption.setRequired(false); - addAlertOption.setArgName("clusterName alertName"); - Option addInstanceTagOption = OptionBuilder.withLongOpt(addInstanceTag).withDescription("Add a tag to instance").create(); addInstanceTagOption.setArgs(3); @@ -869,17 +747,6 @@ private static Options constructCommandLineOptions() { removeInstanceTagOption.setRequired(false); removeInstanceTagOption.setArgName("clusterName instanceName tag"); - Option dropStatOption = - OptionBuilder.withLongOpt(dropStat).withDescription("Drop a persistent stat").create(); - dropStatOption.setArgs(2); - dropStatOption.setRequired(false); - dropStatOption.setArgName("clusterName statName"); - Option dropAlertOption = - OptionBuilder.withLongOpt(dropAlert).withDescription("Drop an alert").create(); - dropAlertOption.setArgs(2); - dropAlertOption.setRequired(false); - dropAlertOption.setArgName("clusterName alertName"); - // TODO need deal with resource-names containing "," // set/get/remove configs options Option setConfOption = @@ -964,10 +831,6 @@ private static Options constructCommandLineOptions() { group.addOption(addStateModelDefOption); group.addOption(listStateModelsOption); group.addOption(listStateModelOption); - group.addOption(addStatOption); - group.addOption(addAlertOption); - group.addOption(dropStatOption); - group.addOption(dropAlertOption); group.addOption(addResourcePropertyOption); group.addOption(removeResourcePropertyOption); @@ -985,10 +848,26 @@ private static Options constructCommandLineOptions() { group.addOption(removeInstanceTagOption); group.addOption(instanceGroupTagOption); + return group; + } + + @SuppressWarnings("static-access") + private static Options constructCommandLineOptions() { + Option helpOption = + OptionBuilder.withLongOpt(help).withDescription("Prints command-line options info") + .create(); + + Option zkServerOption = + OptionBuilder.withLongOpt(zkServerAddress).withDescription("Provide zookeeper address") + .create(); + zkServerOption.setArgs(1); + zkServerOption.setRequired(true); + zkServerOption.setArgName("ZookeeperServerAddress(Required)"); + Options options = new Options(); options.addOption(helpOption); options.addOption(zkServerOption); - options.addOptionGroup(group); + options.addOptionGroup(constructOptionGroup()); return options; } @@ -1021,7 +900,18 @@ public static int processCommandLineArgs(String[] cliArgs) throws Exception { System.exit(1); } + if (cmd.hasOption(help)) { + printUsage(cliOptions); + return 0; + } + ClusterSetup setupTool = new ClusterSetup(cmd.getOptionValue(zkServerAddress)); + processCommandLineArgs(setupTool, cmd); + return 0; + } + + public static int processCommandLineArgs(ClusterSetup setupTool, CommandLine cmd) + throws IOException { if (cmd.hasOption(addCluster)) { String clusterName = cmd.getOptionValue(addCluster); @@ -1338,26 +1228,6 @@ public static int processCommandLineArgs(String[] cliArgs) throws Exception { setupTool.addIdealState(clusterName, resourceName, idealStateFile); return 0; - } else if (cmd.hasOption(addStat)) { - String clusterName = cmd.getOptionValues(addStat)[0]; - String statName = cmd.getOptionValues(addStat)[1]; - - setupTool.getClusterManagementTool().addStat(clusterName, statName); - } else if (cmd.hasOption(addAlert)) { - String clusterName = cmd.getOptionValues(addAlert)[0]; - String alertName = cmd.getOptionValues(addAlert)[1]; - - setupTool.getClusterManagementTool().addAlert(clusterName, alertName); - } else if (cmd.hasOption(dropStat)) { - String clusterName = cmd.getOptionValues(dropStat)[0]; - String statName = cmd.getOptionValues(dropStat)[1]; - - setupTool.getClusterManagementTool().dropStat(clusterName, statName); - } else if (cmd.hasOption(dropAlert)) { - String clusterName = cmd.getOptionValues(dropAlert)[0]; - String alertName = cmd.getOptionValues(dropAlert)[1]; - - setupTool.getClusterManagementTool().dropAlert(clusterName, alertName); } else if (cmd.hasOption(dropResource)) { String clusterName = cmd.getOptionValues(dropResource)[0]; String resourceName = cmd.getOptionValues(dropResource)[1]; @@ -1419,11 +1289,6 @@ else if (cmd.hasOption(setConstraint)) { String instanceName = cmd.getOptionValues(removeInstanceTag)[1]; String tag = cmd.getOptionValues(removeInstanceTag)[2]; setupTool.getClusterManagementTool().removeInstanceTag(clusterName, instanceName, tag); - } - // help option - else if (cmd.hasOption(help)) { - printUsage(cliOptions); - return 0; } else if (cmd.hasOption(addResourceProperty)) { String clusterName = cmd.getOptionValues(addResourceProperty)[0]; String resourceName = cmd.getOptionValues(addResourceProperty)[1]; diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java index 7ceee85e35..1563769987 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java +++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java @@ -19,9 +19,7 @@ * under the License. */ -import java.io.File; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -41,17 +39,24 @@ import org.apache.commons.cli.ParseException; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixDefinedState; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.PropertyPathConfig; import org.apache.helix.PropertyType; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.State; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; import org.apache.helix.controller.pipeline.Stage; import org.apache.helix.controller.pipeline.StageContext; import org.apache.helix.controller.stages.AttributeName; -import org.apache.helix.controller.stages.BestPossibleStateCalcStage; -import org.apache.helix.controller.stages.BestPossibleStateOutput; import org.apache.helix.controller.stages.ClusterDataCache; import org.apache.helix.controller.stages.ClusterEvent; +import org.apache.helix.controller.stages.BestPossibleStateCalcStage; +import org.apache.helix.controller.stages.BestPossibleStateOutput; import org.apache.helix.controller.stages.CurrentStateComputationStage; import org.apache.helix.controller.stages.ResourceComputationStage; import org.apache.helix.manager.zk.ZKHelixDataAccessor; @@ -59,11 +64,8 @@ import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; -import org.apache.helix.model.Partition; -import org.apache.helix.participant.statemachine.StateModel; -import org.apache.helix.participant.statemachine.StateModelFactory; -import org.apache.helix.store.PropertyJsonComparator; -import org.apache.helix.store.PropertyJsonSerializer; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.builder.ResourceAssignmentBuilder; import org.apache.helix.util.ZKClientPool; import org.apache.log4j.Logger; @@ -156,7 +158,7 @@ public boolean verify() { HelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); - return ClusterStateVerifier.verifyBestPossAndExtView(accessor, errStates); + return ClusterStateVerifier.verifyBestPossAndExtView(accessor, errStates, clusterName); } catch (Exception e) { LOG.error("exception in verification", e); } @@ -222,23 +224,18 @@ public String getClusterName() { } static boolean verifyBestPossAndExtView(HelixDataAccessor accessor, - Map> errStates) { + Map> errStates, String clusterName) { try { Builder keyBuilder = accessor.keyBuilder(); - // read cluster once and do verification - ClusterDataCache cache = new ClusterDataCache(); - cache.refresh(accessor); - Map idealStates = cache.getIdealStates(); - if (idealStates == null) // || idealStates.isEmpty()) - { + Map idealStates = accessor.getChildValuesMap(keyBuilder.idealStates()); + if (idealStates == null) { // ideal state is null because ideal state is dropped idealStates = Collections.emptyMap(); } Map extViews = accessor.getChildValuesMap(keyBuilder.externalViews()); - if (extViews == null) // || extViews.isEmpty()) - { + if (extViews == null) { extViews = Collections.emptyMap(); } @@ -250,28 +247,33 @@ static boolean verifyBestPossAndExtView(HelixDataAccessor accessor, } } + ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(clusterName), accessor); + Cluster cluster = clusterAccessor.readCluster(); // calculate best possible state - BestPossibleStateOutput bestPossOutput = ClusterStateVerifier.calcBestPossState(cache); - Map>> bestPossStateMap = - bestPossOutput.getStateMap(); + BestPossibleStateOutput bestPossOutput = ClusterStateVerifier.calcBestPossState(cluster); // set error states if (errStates != null) { for (String resourceName : errStates.keySet()) { + ResourceId resourceId = ResourceId.from(resourceName); Map partErrStates = errStates.get(resourceName); + ResourceAssignment resourceAssignment = bestPossOutput.getResourceAssignment(resourceId); + + ResourceAssignmentBuilder raBuilder = new ResourceAssignmentBuilder(resourceId); + List mappedPartitions = resourceAssignment.getMappedPartitionIds(); + for (PartitionId partitionId : mappedPartitions) { + raBuilder.addAssignments(partitionId, resourceAssignment.getReplicaMap(partitionId)); + } + for (String partitionName : partErrStates.keySet()) { String instanceName = partErrStates.get(partitionName); - - if (!bestPossStateMap.containsKey(resourceName)) { - bestPossStateMap.put(resourceName, new HashMap>()); - } - Partition partition = new Partition(partitionName); - if (!bestPossStateMap.get(resourceName).containsKey(partition)) { - bestPossStateMap.get(resourceName).put(partition, new HashMap()); - } - bestPossStateMap.get(resourceName).get(partition) - .put(instanceName, HelixDefinedState.ERROR.toString()); + PartitionId partitionId = PartitionId.from(partitionName); + ParticipantId participantId = ParticipantId.from(instanceName); + raBuilder.addAssignment(partitionId, participantId, + State.from(HelixDefinedState.ERROR.toString())); } + bestPossOutput.setResourceAssignment(resourceId, raBuilder.build()); + } } @@ -285,11 +287,12 @@ static boolean verifyBestPossAndExtView(HelixDataAccessor accessor, } // step 0: remove empty map and DROPPED state from best possible state - Map> bpStateMap = - bestPossOutput.getResourceMap(resourceName); - Iterator>> iter = bpStateMap.entrySet().iterator(); + Map> bpStateMap = + ResourceAssignment.stringMapsFromReplicaMaps(bestPossOutput.getResourceAssignment( + ResourceId.from(resourceName)).getResourceMap()); + Iterator>> iter = bpStateMap.entrySet().iterator(); while (iter.hasNext()) { - Map.Entry> entry = iter.next(); + Map.Entry> entry = iter.next(); Map instanceStateMap = entry.getValue(); if (instanceStateMap.isEmpty()) { iter.remove(); @@ -310,7 +313,9 @@ static boolean verifyBestPossAndExtView(HelixDataAccessor accessor, // step 1: externalView and bestPossibleState has equal size int extViewSize = extView.getRecord().getMapFields().size(); - int bestPossStateSize = bestPossOutput.getResourceMap(resourceName).size(); + int bestPossStateSize = + bestPossOutput.getResourceAssignment(ResourceId.from(resourceName)) + .getMappedPartitionIds().size(); if (extViewSize != bestPossStateSize) { LOG.info("exterView size (" + extViewSize + ") is different from bestPossState size (" + bestPossStateSize + ") for resource: " + resourceName); @@ -328,7 +333,8 @@ static boolean verifyBestPossAndExtView(HelixDataAccessor accessor, for (String partition : extView.getRecord().getMapFields().keySet()) { Map evInstanceStateMap = extView.getRecord().getMapField(partition); Map bpInstanceStateMap = - bestPossOutput.getInstanceStateMap(resourceName, new Partition(partition)); + ResourceAssignment.stringMapFromReplicaMap(bestPossOutput.getResourceAssignment( + ResourceId.from(resourceName)).getReplicaMap(PartitionId.from(partition))); boolean result = ClusterStateVerifier. compareMap(evInstanceStateMap, @@ -404,14 +410,15 @@ static void runStage(ClusterEvent event, Stage stage) throws Exception { /** * calculate the best possible state note that DROPPED states are not checked since when * kick off the BestPossibleStateCalcStage we are providing an empty current state map + * @param convertedDefs * @param cache * @return * @throws Exception */ - static BestPossibleStateOutput calcBestPossState(ClusterDataCache cache) throws Exception { + static BestPossibleStateOutput calcBestPossState(Cluster cluster) throws Exception { ClusterEvent event = new ClusterEvent("sampleEvent"); - event.addAttribute("ClusterDataCache", cache); + event.addAttribute("ClusterDataCache", cluster); ResourceComputationStage rcState = new ResourceComputationStage(); CurrentStateComputationStage csStage = new CurrentStateComputationStage(); @@ -424,7 +431,6 @@ static BestPossibleStateOutput calcBestPossState(ClusterDataCache cache) throws BestPossibleStateOutput output = event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); - // System.out.println("output:" + output); return output; } diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifier.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifier.java new file mode 100644 index 0000000000..98757c1bbe --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifier.java @@ -0,0 +1,147 @@ +package org.apache.helix.tools; + +/* + * 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.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import org.I0Itec.zkclient.IZkChildListener; +import org.I0Itec.zkclient.IZkDataListener; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.ZNRecord; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.manager.zk.ZkClient; +import org.apache.log4j.Logger; + +public abstract class ClusterVerifier implements IZkChildListener, IZkDataListener { + private static Logger LOG = Logger.getLogger(ClusterVerifier.class); + + protected final ZkClient _zkclient; + protected final String _clusterName; + protected final HelixDataAccessor _accessor; + protected final PropertyKey.Builder _keyBuilder; + private CountDownLatch _countdown; + + static class ClusterVerifyTrigger { + final PropertyKey _triggerKey; + final boolean _triggerOnChildDataChange; + + public ClusterVerifyTrigger(PropertyKey triggerKey, boolean triggerOnChildDataChange) { + _triggerKey = triggerKey; + _triggerOnChildDataChange = triggerOnChildDataChange; + } + } + + public ClusterVerifier(ZkClient zkclient, String clusterName) { + _zkclient = zkclient; + _clusterName = clusterName; + _accessor = new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkclient)); + _keyBuilder = _accessor.keyBuilder(); + } + + public boolean verifyByCallback(long timeout, List triggers) { + _countdown = new CountDownLatch(1); + + for (ClusterVerifyTrigger trigger : triggers) { + String path = trigger._triggerKey.getPath(); + _zkclient.subscribeChildChanges(path, this); + if (trigger._triggerOnChildDataChange) { + List childs = _zkclient.getChildren(path); + for (String child : childs) { + String childPath = String.format("%s/%s", path, child); + _zkclient.subscribeDataChanges(childPath, this); + } + } + } + + boolean success = false; + try { + success = verify(); + if (!success) { + + success = _countdown.await(timeout, TimeUnit.MILLISECONDS); + if (!success) { + // make a final try if timeout + success = verify(); + } + } + } catch (Exception e) { + LOG.error("Exception in verifier", e); + } + + // clean up + _zkclient.unsubscribeAll(); + + return success; + } + + @Override + public void handleDataChange(String dataPath, Object data) throws Exception { + boolean success = verify(); + if (success) { + _countdown.countDown(); + } + } + + @Override + public void handleDataDeleted(String dataPath) throws Exception { + _zkclient.unsubscribeDataChanges(dataPath, this); + } + + @Override + public void handleChildChange(String parentPath, List currentChilds) throws Exception { + for (String child : currentChilds) { + String childPath = String.format("%s/%s", parentPath, child); + _zkclient.subscribeDataChanges(childPath, this); + } + + boolean success = verify(); + if (success) { + _countdown.countDown(); + } + } + + public boolean verifyByPolling(long timeout) { + try { + long start = System.currentTimeMillis(); + boolean success; + do { + success = verify(); + if (success) { + return true; + } + TimeUnit.MILLISECONDS.sleep(500); + } while ((System.currentTimeMillis() - start) <= timeout); + } catch (Exception e) { + LOG.error("Exception in verifier", e); + } + return false; + } + + /** + * verify + * @return + * @throws Exception + */ + public abstract boolean verify() throws Exception; +} diff --git a/helix-core/src/main/java/org/apache/helix/tools/IntegrationTestUtil.java b/helix-core/src/main/java/org/apache/helix/tools/IntegrationTestUtil.java new file mode 100644 index 0000000000..ff23a16698 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/tools/IntegrationTestUtil.java @@ -0,0 +1,204 @@ +package org.apache.helix.tools; + +/* + * 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.List; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.OptionGroup; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.helix.PropertyKey; +import org.apache.helix.ZNRecord; +import org.apache.helix.manager.zk.ZNRecordSerializer; +import org.apache.helix.manager.zk.ZkClient; +import org.apache.log4j.Logger; + +/** + * collection of test utilities for integration tests + */ +public class IntegrationTestUtil { + private static Logger LOG = Logger.getLogger(IntegrationTestUtil.class); + + public static final long defaultTimeout = 30 * 1000; // in milliseconds + public static final String help = "help"; + public static final String zkSvr = "zkSvr"; + + public static final String verifyExternalView = "verifyExternalView"; + public static final String verifyLiveNodes = "verifyLiveNodes"; + public static final String readZNode = "readZNode"; + public static final String readLeader = "readLeader"; + + final ZkClient _zkclient; + final ZNRecordSerializer _serializer; + + public IntegrationTestUtil(ZkClient zkclient) { + _zkclient = zkclient; + _serializer = new ZNRecordSerializer(); + } + + public void verifyExternalView(String[] args) { + if (args == null || args.length == 0) { + System.err.println("Illegal arguments for " + verifyExternalView); + return; + } + + long timeoutValue = defaultTimeout; + + String clusterName = args[0]; + List liveNodes = new ArrayList(); + for (int i = 1; i < args.length; i++) { + liveNodes.add(args[i]); + } + + ClusterExternalViewVerifier verifier = + new ClusterExternalViewVerifier(_zkclient, clusterName, liveNodes); + boolean success = verifier.verifyByPolling(timeoutValue); + System.out.println(success ? "Successful" : "Failed"); + + } + + public void verifyLiveNodes(String[] args) { + if (args == null || args.length == 0) { + System.err.println("Illegal arguments for " + verifyLiveNodes); + return; + } + + long timeoutValue = defaultTimeout; + + String clusterName = args[0]; + List liveNodes = new ArrayList(); + for (int i = 1; i < args.length; i++) { + liveNodes.add(args[i]); + } + + ClusterLiveNodesVerifier verifier = + new ClusterLiveNodesVerifier(_zkclient, clusterName, liveNodes); + boolean success = verifier.verifyByPolling(timeoutValue); + System.out.println(success ? "Successful" : "Failed"); + } + + public void readZNode(String path) { + ZNRecord record = _zkclient.readData(path, true); + if (record == null) { + System.out.println("null"); + } else { + System.out.println(new String(_serializer.serialize(record))); + } + } + + @SuppressWarnings("static-access") + static Options constructCommandLineOptions() { + Option helpOption = + OptionBuilder.withLongOpt(help).withDescription("Prints command-line options information") + .create(); + + Option zkSvrOption = + OptionBuilder.hasArgs(1).isRequired(true).withArgName("zookeeperAddress") + .withLongOpt(zkSvr).withDescription("Provide zookeeper-address").create(); + + Option verifyExternalViewOption = + OptionBuilder.hasArgs().isRequired(false).withArgName("clusterName node1 node2..") + .withLongOpt(verifyExternalView).withDescription("Verify external-view").create(); + + Option verifyLiveNodesOption = + OptionBuilder.hasArg().isRequired(false).withArgName("clusterName node1, node2..") + .withLongOpt(verifyLiveNodes).withDescription("Verify live-nodes").create(); + + Option readZNodeOption = + OptionBuilder.hasArgs(1).isRequired(false).withArgName("zkPath").withLongOpt(readZNode) + .withDescription("Read znode").create(); + + Option readLeaderOption = + OptionBuilder.hasArgs(1).isRequired(false).withArgName("clusterName") + .withLongOpt(readLeader).withDescription("Read cluster controller").create(); + + OptionGroup optGroup = new OptionGroup(); + optGroup.setRequired(true); + optGroup.addOption(verifyExternalViewOption); + optGroup.addOption(verifyLiveNodesOption); + optGroup.addOption(readZNodeOption); + optGroup.addOption(readLeaderOption); + + Options options = new Options(); + options.addOption(helpOption); + options.addOption(zkSvrOption); + options.addOptionGroup(optGroup); + + return options; + } + + static void printUsage(Options cliOptions) { + HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setWidth(1000); + helpFormatter.printHelp("java " + ClusterExternalViewVerifier.class.getName(), cliOptions); + } + + static void processCommandLineArgs(String[] cliArgs) { + CommandLineParser cliParser = new GnuParser(); + Options cliOptions = constructCommandLineOptions(); + CommandLine cmd = null; + try { + cmd = cliParser.parse(cliOptions, cliArgs); + } catch (ParseException pe) { + System.err.println("failed to parse command-line args: " + Arrays.asList(cliArgs) + + ", exception: " + pe.toString()); + printUsage(cliOptions); + System.exit(1); + } + + String zkServer = cmd.getOptionValue(zkSvr); + + ZkClient zkclient = + new ZkClient(zkServer, ZkClient.DEFAULT_SESSION_TIMEOUT, + ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer()); + IntegrationTestUtil util = new IntegrationTestUtil(zkclient); + + if (cmd != null) { + if (cmd.hasOption(verifyExternalView)) { + String[] args = cmd.getOptionValues(verifyExternalView); + util.verifyExternalView(args); + } else if (cmd.hasOption(verifyLiveNodes)) { + String[] args = cmd.getOptionValues(verifyLiveNodes); + util.verifyLiveNodes(args); + } else if (cmd.hasOption(readZNode)) { + String path = cmd.getOptionValue(readZNode); + util.readZNode(path); + } else if (cmd.hasOption(readLeader)) { + String clusterName = cmd.getOptionValue(readLeader); + PropertyKey.Builder keyBuilder = new PropertyKey.Builder(clusterName); + util.readZNode(keyBuilder.controllerLeader().getPath()); + } else { + printUsage(cliOptions); + } + } + } + + public static void main(String[] args) { + processCommandLineArgs(args); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/tools/MessagePoster.java b/helix-core/src/main/java/org/apache/helix/tools/MessagePoster.java index f0114d05b5..8120981064 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/MessagePoster.java +++ b/helix-core/src/main/java/org/apache/helix/tools/MessagePoster.java @@ -22,10 +22,14 @@ import java.util.UUID; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkClient; -import org.apache.helix.model.Message; import org.apache.helix.model.LiveInstance.LiveInstanceProperty; +import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageState; import org.apache.helix.model.Message.MessageType; import org.apache.helix.util.HelixUtil; @@ -37,8 +41,8 @@ public void post(String zkServer, Message message, String clusterName, String in String path = HelixUtil.getMessagePath(clusterName, instanceName) + "/" + message.getId(); client.delete(path); ZNRecord record = client.readData(HelixUtil.getLiveInstancePath(clusterName, instanceName)); - message.setTgtSessionId(record.getSimpleField(LiveInstanceProperty.SESSION_ID.toString()) - .toString()); + message.setTgtSessionId(SessionId.from(record.getSimpleField( + LiveInstanceProperty.SESSION_ID.toString()).toString())); message.setTgtName(record.getId()); // System.out.println(message); client.createPersistent(path, message.getRecord()); @@ -46,12 +50,12 @@ public void post(String zkServer, Message message, String clusterName, String in public void postFaultInjectionMessage(String zkServer, String clusterName, String instanceName, String payloadString, String partition) { - Message message = new Message("FaultInjection", UUID.randomUUID().toString()); + Message message = new Message("FaultInjection", MessageId.from(UUID.randomUUID().toString())); if (payloadString != null) { message.getRecord().setSimpleField("faultType", payloadString); } if (partition != null) { - message.setPartitionName(partition); + message.setPartitionId(PartitionId.from(partition)); } post(zkServer, message, clusterName, instanceName); @@ -59,16 +63,16 @@ public void postFaultInjectionMessage(String zkServer, String clusterName, Strin public void postTestMessage(String zkServer, String clusterName, String instanceName) { String msgSrc = "cm-instance-0"; - String msgId = "TestMessageId-2"; + MessageId msgId = MessageId.from("TestMessageId-2"); Message message = new Message(MessageType.STATE_TRANSITION, msgId); - message.setMsgId(msgId); + message.setMessageId(msgId); message.setSrcName(msgSrc); message.setTgtName(instanceName); message.setMsgState(MessageState.NEW); - message.setFromState("Slave"); - message.setToState("Master"); - message.setPartitionName("EspressoDB.partition-0." + instanceName); + message.setFromState(State.from("Slave")); + message.setToState(State.from("Master")); + message.setPartitionId(PartitionId.from("EspressoDB.partition-0." + instanceName)); post(zkServer, message, clusterName, instanceName); } diff --git a/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java b/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java new file mode 100644 index 0000000000..2ea08083e4 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java @@ -0,0 +1,1154 @@ +package org.apache.helix.tools; + +/* + * 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.DataInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +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 org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.OptionGroup; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Resource; +import org.apache.helix.api.RunningInstance; +import org.apache.helix.api.Scope; +import org.apache.helix.api.Scope.ScopeType; +import org.apache.helix.api.State; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.accessor.ParticipantAccessor; +import org.apache.helix.api.accessor.ResourceAccessor; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ConstraintId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.rebalancer.context.CustomRebalancerContext; +import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext; +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.manager.zk.ZkClient; +import org.apache.helix.model.ClusterConstraints; +import org.apache.helix.model.ClusterConstraints.ConstraintType; +import org.apache.helix.model.ConstraintItem; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.util.HelixUtil; +import org.apache.log4j.Logger; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +/** + * Parse command line and call helix-admin + */ +public class NewClusterSetup { + + private static Logger LOG = Logger.getLogger(NewClusterSetup.class); + + /** + * List all helix cluster setup options + */ + public enum HelixOption { + // help + help(0, "", "Print command-line options"), + + // zookeeper address + zkSvr(1, true, "zookeeperServerAddress", "Zookeeper address (host:port, required)"), + + // list cluster/resource/instances + listClusters(0, "", "List clusters"), + listResources(1, "clusterId", "List resources in a cluster"), + listInstances(1, "clusterId", "List instances in a cluster"), + + // add, drop, and rebalance cluster + addCluster(1, "clusterId", "Add a new cluster"), + activateCluster(3, "clusterId grandClusterId true/false", + "Enable/disable a cluster in distributed controller mode"), + dropCluster(1, "clusterId", "Delete a cluster"), + dropResource(2, "clusterId resourceId", "Drop a resource from a cluster"), + addInstance(2, "clusterId instanceId", "Add an instance to a cluster"), + addResource(4, "clusterId resourceId partitionNumber stateModelDefId", + "Add a resource to a cluster"), + addStateModelDef(2, "clusterId jsonFileName", "Add a state model definition to a cluster"), + addIdealState(2, "clusterId resourceId jsonfileName", + "Add an ideal state of a resource in cluster"), + swapInstance(3, "clusterId oldInstanceId newInstanceId", + "Swap an old instance in cluster with a new instance"), + dropInstance(2, "clusterId instanceId", "Drop an instance from a cluster"), + rebalance(3, "clusterId resourceId replicas", "Rebalance a resource in cluster"), + expandCluster(1, "clusterId", "Expand a cluster"), + expandResource(2, "clusterId resourceId", "Expand resource to additional nodes"), + @Deprecated + mode(1, "rebalancerMode", "Specify rebalancer mode, used with " + addResource + " command"), + rebalancerMode(1, "rebalancerMode", "Specify rebalancer mode, used with " + addResource + + " command"), + instanceGroupTag(1, "instanceGroupTag", "Specify instance group tag, used with " + rebalance + + " command"), + bucketSize(1, "bucketSize", "Specify bucket size, used with " + addResource + " command"), + resourceKeyPrefix(1, "resourceKeyPrefix", "Specify resource key prefix, used with " + rebalance + + " command"), + maxPartitionsPerNode(1, "maxPartitionsPerNode", "Specify max partitions per node, used with " + + addResource + " command"), + addResourceProperty(4, "clusterId resourceId propertyName propertyValue", + "Add a resource property"), + removeResourceProperty(3, "clusterId resourceId propertyName", "Remove a resource property"), + addInstanceTag(3, "clusterId instanceId tag", "Add a tag to instance"), + removeInstanceTag(3, "clusterId instanceId tag", "Remove a tag from instance"), + + // query info + listClusterInfo(1, "clusterId", "Query informaton of a cluster"), + listInstanceInfo(2, "clusterId instanceId", "Query information of an instance in cluster"), + listResourceInfo(2, "clusterId resourceId", "Query information of a resource"), + listPartitionInfo(3, "clusterId resourceId partitionId", "Query information of a partition"), + listStateModels(1, "clusterId", "Query information of state models in a cluster"), + listStateModel(2, "clusterId stateModelDefId", "Query information of a state model in cluster"), + + // enable/disable/reset instances/cluster/resource/partition + enableInstance(3, "clusterId instanceId true/false", "Enable/disable an instance"), + enablePartition(-1, "true/false clusterId instanceId resourceId partitionId...", + "Enable/disable partitions"), + enableCluster(2, "clusterId true/false", "Pause/resume the controller of a cluster"), + resetPartition(4, "clusterId instanceId resourceId partitionName", + "Reset a partition in error state"), + resetInstance(2, "clusterId instanceId", "Reset all partitions in error state for an instance"), + resetResource(2, "clusterId resourceId", "Reset all partitions in error state for a resource"), + + // set/set/remove configs + getConfig(3, "scope(e.g. RESOURCE) configScopeArgs(e.g. myCluster,testDB) keys(e.g. k1,k2)", + "Get configs"), + setConfig(3, + "scope(e.g. RESOURCE) configScopeArgs(e.g. myCluster,testDB) keyValues(e.g. k1=v1,k2=v2)", + "Set configs"), + removeConfig(3, "scope(e.g. RESOURCE) configScopeArgs(e.g. myCluster,testDB) keys(e.g. k1,k2)", + "Remove configs"), + + // get/set/remove constraints + getConstraints(2, "clusterId constraintType(e.g. MESSAGE_CONSTRAINT)", "Get constraints"), + setConstraint( + 4, + "clusterId constraintType(e.g. MESSAGE_CONSTRAINT) constraintId keyValues(e.g. k1=v1,k2=v2)", + "Set a constraint, create if not exist"), + removeConstraint(3, "clusterId constraintType(e.g. MESSAGE_CONSTRAINT) constraintId", + "Remove a constraint"); + + final int _argNum; + final boolean _isRequired; + final String _argName; + final String _description; + + private HelixOption(int argNum, boolean isRequired, String argName, String description) { + _argNum = argNum; + _isRequired = isRequired; + _argName = argName; + _description = description; + } + + private HelixOption(int argNum, String argName, String description) { + this(argNum, false, argName, description); + } + } + + private final ZkClient _zkclient; + private final BaseDataAccessor _baseAccessor; + + private NewClusterSetup(ZkClient zkclient) { + _zkclient = zkclient; + _baseAccessor = new ZkBaseDataAccessor(_zkclient); + } + + @SuppressWarnings("static-access") + static Options constructCommandLineOptions() { + Options options = new Options(); + + OptionGroup optionGroup = new OptionGroup(); + for (HelixOption option : HelixOption.values()) { + Option opt = + OptionBuilder.withLongOpt(option.name()).hasArgs(option._argNum) + .isRequired(option._isRequired).withArgName(option._argName) + .withDescription(option._description).create(); + if (option == HelixOption.help || option == HelixOption.zkSvr) { + options.addOption(opt); + } else { + optionGroup.addOption(opt); + } + } + options.addOptionGroup(optionGroup); + return options; + } + + /** + * Check if we have the right number of arguments + * @param opt + * @param optValues + */ + static void checkArgNum(HelixOption opt, String[] optValues) { + + if (opt._argNum != -1 && opt._argNum < optValues.length) { + throw new IllegalArgumentException(opt + " should have no less than " + opt._argNum + + " arguments, but was: " + optValues.length + ", " + Arrays.asList(optValues)); + } + } + + static void printUsage(Options cliOptions) { + HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setWidth(1000); + helpFormatter.printHelp("java " + NewClusterSetup.class.getName(), cliOptions); + } + + ClusterAccessor clusterAccessor(String clusterName) { + HelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, _baseAccessor); + return new ClusterAccessor(ClusterId.from(clusterName), accessor); + } + + ParticipantAccessor participantAccessor(String clusterName) { + return new ParticipantAccessor(new ZKHelixDataAccessor(clusterName, _baseAccessor)); + } + + ResourceAccessor resourceAccessor(String clusterName) { + return new ResourceAccessor(new ZKHelixDataAccessor(clusterName, _baseAccessor)); + } + + void addCluster(String[] optValues) { + String clusterName = optValues[0]; + + List defaultStateModelDefs = new ArrayList(); + defaultStateModelDefs.add(new StateModelDefinition(StateModelConfigGenerator + .generateConfigForMasterSlave())); + + ClusterConfig.Builder builder = + new ClusterConfig.Builder(ClusterId.from(clusterName)) + .addStateModelDefinitions(defaultStateModelDefs); + + ClusterAccessor accessor = clusterAccessor(clusterName); + accessor.createCluster(builder.build()); + } + + void addResource(String[] optValues, String[] rebalancerModeValues, String[] bucketSizeValues, + String[] maxPartitionsPerNodeValues) { + String clusterName = optValues[0]; + String resourceName = optValues[1]; + int partitionNumber = Integer.parseInt(optValues[2]); + String stateModelDefName = optValues[3]; + RebalanceMode rebalancerMode = + rebalancerModeValues == null ? RebalanceMode.SEMI_AUTO : RebalanceMode + .valueOf(rebalancerModeValues[0]); + + int bucketSize = bucketSizeValues == null ? 0 : Integer.parseInt(bucketSizeValues[0]); + + int maxPartitionsPerNode = + maxPartitionsPerNodeValues == null ? -1 : Integer.parseInt(maxPartitionsPerNodeValues[0]); + + ResourceId resourceId = ResourceId.from(resourceName); + StateModelDefId stateModelDefId = StateModelDefId.from(stateModelDefName); + + IdealState idealState = new IdealState(resourceName); + idealState.setRebalanceMode(rebalancerMode); + idealState.setNumPartitions(partitionNumber); + idealState.setMaxPartitionsPerInstance(maxPartitionsPerNode); + idealState.setStateModelDefId(stateModelDefId); + + RebalancerContext rebalancerCtx = PartitionedRebalancerContext.from(idealState); + ResourceConfig.Builder builder = + new ResourceConfig.Builder(resourceId).rebalancerContext(rebalancerCtx).bucketSize( + bucketSize); + + ClusterAccessor accessor = clusterAccessor(clusterName); + accessor.addResourceToCluster(builder.build()); + + } + + void rebalance(String[] optValues, String[] groupTagValues) { + String clusterName = optValues[0]; + String resourceName = optValues[1]; + int replicaCount = Integer.parseInt(optValues[2]); + String groupTag = null; + if (groupTagValues != null && groupTagValues.length > 0) { + groupTag = groupTagValues[0]; + } + ResourceAccessor accessor = resourceAccessor(clusterName); + accessor.generateDefaultAssignment(ResourceId.from(resourceName), replicaCount, groupTag); + } + + void addInstance(String[] optValues) { + String clusterName = optValues[0]; + String[] instanceIds = optValues[1].split(";"); + + ClusterAccessor accessor = clusterAccessor(clusterName); + for (String instanceId : instanceIds) { + ParticipantConfig.Builder builder = + new ParticipantConfig.Builder(ParticipantId.from(instanceId)); + + accessor.addParticipantToCluster(builder.build()); + } + } + + void dropCluster(String[] optValues) { + String clusterName = optValues[0]; + ClusterAccessor accessor = clusterAccessor(clusterName); + accessor.dropCluster(); + } + + void dropResource(String[] optValues) { + String clusterName = optValues[0]; + String resourceName = optValues[1]; + + ClusterAccessor accessor = clusterAccessor(clusterName); + accessor.dropResourceFromCluster(ResourceId.from(resourceName)); + } + + void dropInstance(String[] optValues) { + String clusterName = optValues[0]; + String[] instanceIds = optValues[1].split(";"); + ClusterAccessor accessor = clusterAccessor(clusterName); + for (String instanceId : instanceIds) { + accessor.dropParticipantFromCluster(ParticipantId.from(instanceId)); + } + + } + + private static byte[] readFile(String filePath) throws IOException { + File file = new File(filePath); + + int size = (int) file.length(); + byte[] bytes = new byte[size]; + DataInputStream dis = null; + try { + dis = new DataInputStream(new FileInputStream(file)); + int read = 0; + int numRead = 0; + while (read < bytes.length && (numRead = dis.read(bytes, read, bytes.length - read)) >= 0) { + read = read + numRead; + } + return bytes; + } finally { + if (dis != null) { + dis.close(); + } + } + } + + void addStateModelDef(String[] optValues) { + String clusterName = optValues[0]; + String stateModelDefJsonFile = optValues[1]; + + try { + StateModelDefinition stateModelDef = + new StateModelDefinition( + (ZNRecord) (new ZNRecordSerializer().deserialize(readFile(stateModelDefJsonFile)))); + ClusterAccessor accessor = clusterAccessor(clusterName); + accessor.addStateModelDefinitionToCluster(stateModelDef); + + } catch (IOException e) { + LOG.error("Could not parse the state model", e); + } + + } + + void addIdealState(String[] optValues) { + String clusterName = optValues[0]; + String resourceName = optValues[1]; + String idealStateJsonFile = optValues[2]; + + try { + IdealState idealState = + new IdealState( + (ZNRecord) (new ZNRecordSerializer().deserialize(readFile(idealStateJsonFile)))); + + RebalancerContext rebalancerCtx = PartitionedRebalancerContext.from(idealState); + ResourceConfig.Builder builder = + new ResourceConfig.Builder(ResourceId.from(resourceName)) + .rebalancerContext(rebalancerCtx).bucketSize(idealState.getBucketSize()); + + ClusterAccessor accessor = clusterAccessor(clusterName); + accessor.addResourceToCluster(builder.build()); + } catch (IOException e) { + e.printStackTrace(); + } + + } + + void addInstanceTag(String[] optValues) { + String clusterName = optValues[0]; + String participantName = optValues[1]; + String tag = optValues[2]; + + ParticipantAccessor accessor = participantAccessor(clusterName); + ParticipantId participantId = ParticipantId.from(participantName); + + ParticipantConfig.Delta delta = new ParticipantConfig.Delta(participantId); + delta.addTag(tag); + accessor.updateParticipant(participantId, delta); + } + + void removeInstanceTag(String[] optValues) { + String clusterName = optValues[0]; + String participantName = optValues[1]; + String tag = optValues[2]; + + ParticipantAccessor accessor = participantAccessor(clusterName); + ParticipantId participantId = ParticipantId.from(participantName); + + ParticipantConfig.Delta delta = new ParticipantConfig.Delta(participantId); + delta.removeTag(tag); + accessor.updateParticipant(participantId, delta); + } + + void listPartitionInfo(String[] optValues) { + String clusterName = optValues[0]; + String resourceName = optValues[1]; + String partitionName = optValues[2]; + + ResourceId resourceId = ResourceId.from(resourceName); + PartitionId partitionId = PartitionId.from(partitionName); + ResourceAccessor accessor = resourceAccessor(clusterName); + Resource resource = accessor.readResource(resourceId); + + StringBuilder sb = new StringBuilder(); + Map stateMap = resource.getExternalView().getStateMap(partitionId); + sb.append(resourceName + "/" + partitionName + ", externalView: " + stateMap); + PartitionedRebalancerContext partitionedContext = + resource.getRebalancerConfig().getRebalancerContext(PartitionedRebalancerContext.class); + if (partitionedContext != null) { + // for partitioned contexts, check the mode and apply mode-specific information if possible + if (partitionedContext.getRebalanceMode() == RebalanceMode.SEMI_AUTO) { + SemiAutoRebalancerContext semiAutoContext = + resource.getRebalancerConfig().getRebalancerContext(SemiAutoRebalancerContext.class); + sb.append(", preferenceList: " + semiAutoContext.getPreferenceList(partitionId)); + } else if (partitionedContext.getRebalanceMode() == RebalanceMode.CUSTOMIZED) { + CustomRebalancerContext customContext = + resource.getRebalancerConfig().getRebalancerContext(CustomRebalancerContext.class); + sb.append(", preferenceMap: " + customContext.getPreferenceMap(partitionId)); + } + if (partitionedContext.anyLiveParticipant()) { + sb.append(", anyLiveParticipant: " + partitionedContext.anyLiveParticipant()); + } else { + sb.append(", replicaCount: " + partitionedContext.getReplicaCount()); + } + } + + System.out.println(sb.toString()); + } + + void enableInstance(String[] optValues) { + String clusterName = optValues[0]; + String instanceId = optValues[1]; + if (instanceId.indexOf(":") != -1) { + instanceId = instanceId.replaceAll(":", "_"); + } + boolean enabled = Boolean.parseBoolean(optValues[2].toLowerCase()); + + ParticipantAccessor accessor = participantAccessor(clusterName); + if (enabled) { + accessor.enableParticipant(ParticipantId.from(instanceId)); + } else { + accessor.disableParticipant(ParticipantId.from(instanceId)); + } + } + + void enablePartition(String[] optValues) { + boolean enabled = Boolean.parseBoolean(optValues[0].toLowerCase()); + String clusterName = optValues[1]; + ParticipantId participantId = ParticipantId.from(optValues[2]); + ResourceId resourceId = ResourceId.from(optValues[3]); + + Set partitionIdSet = new HashSet(); + for (int i = 4; i < optValues.length; i++) { + partitionIdSet.add(PartitionId.from(optValues[i])); + } + + ParticipantAccessor accessor = participantAccessor(clusterName); + if (enabled) { + accessor.enablePartitionsForParticipant(participantId, resourceId, partitionIdSet); + } else { + accessor.disablePartitionsForParticipant(participantId, resourceId, partitionIdSet); + } + } + + void enableCluster(String[] optValues) { + String clusterName = optValues[0]; + boolean enabled = Boolean.parseBoolean(optValues[1].toLowerCase()); + + ClusterAccessor accessor = clusterAccessor(clusterName); + if (enabled) { + accessor.resumeCluster(); + } else { + accessor.pauseCluster(); + } + } + + /** + * Convert user config to key value map + * @param userConfig + * @param mapKey + * @param keys + * @return + */ + private Map keyValueMap(UserConfig userConfig, String mapKey, String[] keys) { + Map results = new HashMap(); + + for (String key : keys) { + if (mapKey == null) { + results.put(key, userConfig.getSimpleField(key)); + } else { + results.put(key, userConfig.getMapField(mapKey).get(key)); + } + } + return results; + } + + void getConfig(String[] optValues) { + ScopeType scopeType = ScopeType.valueOf(optValues[0].toUpperCase()); + String[] scopeArgs = optValues[1].split("[\\s,]"); + String[] keys = optValues[2].split("[\\s,]"); + + String clusterName = scopeArgs[0]; + Map results = null; + switch (scopeType) { + case CLUSTER: { + ClusterAccessor accessor = clusterAccessor(clusterName); + results = keyValueMap(accessor.readUserConfig(), null, keys); + break; + } + case PARTICIPANT: { + ParticipantId participantId = ParticipantId.from(scopeArgs[1]); + ParticipantAccessor accessor = participantAccessor(clusterName); + results = keyValueMap(accessor.readUserConfig(participantId), null, keys); + break; + } + case RESOURCE: { + ResourceId resourceId = ResourceId.from(scopeArgs[1]); + ResourceAccessor accessor = resourceAccessor(clusterName); + results = keyValueMap(accessor.readUserConfig(resourceId), null, keys); + break; + } + case PARTITION: { + ResourceId resourceId = ResourceId.from(scopeArgs[1]); + String partitionId = scopeArgs[2]; + ResourceAccessor accessor = resourceAccessor(clusterName); + results = keyValueMap(accessor.readUserConfig(resourceId), partitionId, keys); + break; + } + default: + System.err.println("Non-recognized scopeType: " + scopeType); + break; + } + + System.out.println(results); + } + + /** + * Convert key-value map to user-config + * @param scope + * @param mapKey + * @param keyValues + * @return + */ + private UserConfig userConfig(Scope scope, String mapKey, String[] keyValues) { + UserConfig userConfig = new UserConfig(scope); + + for (String keyValue : keyValues) { + String[] splits = keyValue.split("="); + String key = splits[0]; + String value = splits[1]; + if (mapKey == null) { + userConfig.setSimpleField(key, value); + } else { + if (userConfig.getMapField(mapKey) == null) { + userConfig.setMapField(mapKey, new TreeMap()); + } + userConfig.getMapField(mapKey).put(key, value); + } + } + return userConfig; + } + + void setConfig(String[] optValues) { + ScopeType scopeType = ScopeType.valueOf(optValues[0].toUpperCase()); + String[] scopeArgs = optValues[1].split("[\\s,]"); + String[] keyValues = optValues[2].split("[\\s,]"); + + String clusterName = scopeArgs[0]; + Map results = new HashMap(); + switch (scopeType) { + case CLUSTER: { + ClusterAccessor accessor = clusterAccessor(clusterName); + Scope scope = Scope.cluster(ClusterId.from(clusterName)); + UserConfig userConfig = userConfig(scope, null, keyValues); + accessor.setUserConfig(userConfig); + break; + } + case PARTICIPANT: { + ParticipantId participantId = ParticipantId.from(scopeArgs[1]); + ParticipantAccessor accessor = participantAccessor(clusterName); + Scope scope = Scope.participant(participantId); + UserConfig userConfig = userConfig(scope, null, keyValues); + accessor.setUserConfig(participantId, userConfig); + break; + } + case RESOURCE: { + ResourceId resourceId = ResourceId.from(scopeArgs[1]); + ResourceAccessor accessor = resourceAccessor(clusterName); + Scope scope = Scope.resource(resourceId); + UserConfig userConfig = userConfig(scope, null, keyValues); + accessor.setUserConfig(resourceId, userConfig); + break; + } + case PARTITION: { + ResourceId resourceId = ResourceId.from(scopeArgs[1]); + String partitionId = scopeArgs[2]; + ResourceAccessor accessor = resourceAccessor(clusterName); + Scope scope = Scope.resource(resourceId); + UserConfig userConfig = userConfig(scope, partitionId, keyValues); + accessor.setUserConfig(resourceId, userConfig); + break; + } + default: + System.err.println("Non-recognized scopeType: " + scopeType); + break; + } + + System.out.println(results); + } + + void setConstraint(String[] optValues) { + String clusterName = optValues[0]; + String constraintType = optValues[1]; + String constraintId = optValues[2]; + String constraintAttributesMap = optValues[3]; + if (clusterName == null || constraintType == null || constraintId == null + || constraintAttributesMap == null) { + System.err + .println("fail to set constraint. missing clusterName|constraintType|constraintId|constraintAttributesMap"); + return; + } + ClusterId clusterId = ClusterId.from(clusterName); + ClusterAccessor accessor = clusterAccessor(clusterName); + Map constraintAttributes = + HelixUtil.parseCsvFormatedKeyValuePairs(constraintAttributesMap); + ConstraintItem item = new ConstraintItem(constraintAttributes); + ClusterConfig.Delta delta = + new ClusterConfig.Delta(clusterId).addConstraintItem( + ConstraintType.valueOf(constraintType), ConstraintId.from(constraintId), item); + accessor.updateCluster(delta); + } + + void getConstraints(String[] optValues) { + String clusterName = optValues[0]; + ConstraintType constraintType = ConstraintType.valueOf(optValues[1]); + ClusterAccessor accessor = clusterAccessor(clusterName); + ClusterConstraints constraints = accessor.readConstraints(constraintType); + System.out.println(constraints.toString()); + } + + void removeConstraint(String[] optValues) { + String clusterName = optValues[0]; + ConstraintType constraintType = ConstraintType.valueOf(optValues[1]); + ConstraintId constraintId = ConstraintId.from(optValues[2]); + ClusterAccessor accessor = clusterAccessor(clusterName); + accessor.removeConstraint(constraintType, constraintId); + } + + void listClusterInfo(String[] optValues) { + String clusterName = optValues[0]; + ClusterAccessor accessor = clusterAccessor(clusterName); + Set resources = accessor.readResources().keySet(); + StringBuilder sb = + new StringBuilder("Existing resources in cluster ").append(clusterName).append(":\n"); + for (ResourceId resourceId : resources) { + sb.append(resourceId.stringify()).append('\n'); + } + Set participants = accessor.readParticipants().keySet(); + sb.append("Participants in cluster ").append(clusterName).append(":\n"); + for (ParticipantId participantId : participants) { + sb.append(participantId.stringify()).append('\n'); + } + System.out.print(sb.toString()); + } + + void listParticipantInfo(String[] optValues) { + String clusterName = optValues[0]; + String participantName = optValues[1]; + ParticipantAccessor accessor = participantAccessor(clusterName); + ParticipantId participantId = ParticipantId.from(participantName); + Participant participant = accessor.readParticipant(participantId); + StringBuilder sb = + new StringBuilder("Participant ").append(participantName).append(" in cluster ") + .append(clusterName).append(":\n").append("hostName: ") + .append(participant.getHostName()).append(", port: ").append(participant.getPort()) + .append(", enabled: ").append(participant.isEnabled()).append(", disabledPartitions: ") + .append(participant.getDisabledPartitionIds().toString()).append(", tags:") + .append(participant.getTags().toString()).append(", currentState: ") + .append(", messages: ").append(participant.getMessageMap().toString()) + .append(participant.getCurrentStateMap().toString()).append(", alive: ") + .append(participant.isAlive()).append(", userConfig: ") + .append(participant.getUserConfig().toString()); + if (participant.isAlive()) { + RunningInstance runningInstance = participant.getRunningInstance(); + sb.append(", sessionId: ").append(runningInstance.getSessionId().stringify()) + .append(", processId: ").append(runningInstance.getPid().stringify()) + .append(", helixVersion: ").append(runningInstance.getVersion().toString()); + } + System.out.println(sb.toString()); + } + + void listResourceInfo(String[] optValues) { + String clusterName = optValues[0]; + String resourceName = optValues[1]; + ResourceAccessor accessor = resourceAccessor(clusterName); + ResourceId resourceId = ResourceId.from(resourceName); + Resource resource = accessor.readResource(resourceId); + StringBuilder sb = + new StringBuilder("Resource ").append(resourceName).append(" in cluster ") + .append(clusterName).append(":\n").append("externalView: ") + .append(resource.getExternalView()).append(", userConfig: ") + .append(resource.getUserConfig()).append(", rebalancerContext: ") + .append(resource.getRebalancerConfig().getSerializedContext()); + System.out.println(sb.toString()); + } + + void listResources(String[] optValues) { + String clusterName = optValues[0]; + ClusterAccessor accessor = clusterAccessor(clusterName); + Set resources = accessor.readResources().keySet(); + StringBuilder sb = + new StringBuilder("Existing resources in cluster ").append(clusterName).append(":\n"); + for (ResourceId resourceId : resources) { + sb.append(resourceId.stringify()).append('\n'); + } + System.out.print(sb.toString()); + } + + void listParticipants(String[] optValues) { + String clusterName = optValues[0]; + ClusterAccessor accessor = clusterAccessor(clusterName); + Set participants = accessor.readParticipants().keySet(); + StringBuilder sb = + new StringBuilder("Participants in cluster ").append(clusterName).append(":\n"); + for (ParticipantId participantId : participants) { + sb.append(participantId.stringify()).append('\n'); + } + System.out.print(sb.toString()); + } + + void listStateModels(String[] optValues) { + String clusterName = optValues[0]; + ClusterAccessor accessor = clusterAccessor(clusterName); + Set stateModelDefs = accessor.readStateModelDefinitions().keySet(); + StringBuilder sb = + new StringBuilder("State models in cluster ").append(clusterName).append(":\n"); + for (StateModelDefId stateModelDefId : stateModelDefs) { + sb.append(stateModelDefId.stringify()).append('\n'); + } + System.out.print(sb.toString()); + } + + void listStateModel(String[] optValues) { + String clusterName = optValues[0]; + String stateModel = optValues[1]; + StateModelDefId stateModelDefId = StateModelDefId.from(stateModel); + ClusterAccessor accessor = clusterAccessor(clusterName); + Map stateModelDefs = + accessor.readStateModelDefinitions(); + StateModelDefinition stateModelDef = stateModelDefs.get(stateModelDefId); + StringBuilder sb = new StringBuilder("StateModelDefinition: ").append(stateModelDef.toString()); + System.out.println(sb.toString()); + } + + void listClusters(String[] optValues) { + List result = Lists.newArrayList(); + List clusterNames = _baseAccessor.getChildNames("/", 0); + for (String clusterName : clusterNames) { + ClusterAccessor accessor = clusterAccessor(clusterName); + if (accessor.isClusterStructureValid()) { + result.add(ClusterId.from(clusterName)); + } + } + System.out.println("Existing clusters: " + result); + } + + void removeConfig(String[] optValues) { + ScopeType type = ScopeType.valueOf(optValues[0].toUpperCase()); + String[] scopeArgs = optValues[1].split("[\\s,]"); + String[] keys = optValues[2].split("[\\s,]"); + String clusterName = scopeArgs[0]; + UserConfig userConfig; + switch (type) { + case CLUSTER: + ClusterAccessor clusterAccessor = clusterAccessor(clusterName); + userConfig = clusterAccessor.readUserConfig(); + removeKeysFromUserConfig(userConfig, keys); + clusterAccessor.setUserConfig(userConfig); + break; + case RESOURCE: + ResourceAccessor resourceAccessor = resourceAccessor(clusterName); + ResourceId resourceId = ResourceId.from(scopeArgs[1]); + userConfig = resourceAccessor.readUserConfig(resourceId); + removeKeysFromUserConfig(userConfig, keys); + resourceAccessor.setUserConfig(resourceId, userConfig); + break; + case PARTICIPANT: + ParticipantAccessor participantAccessor = participantAccessor(clusterName); + ParticipantId participantId = ParticipantId.from(scopeArgs[1]); + userConfig = participantAccessor.readUserConfig(participantId); + removeKeysFromUserConfig(userConfig, keys); + participantAccessor.setUserConfig(participantId, userConfig); + break; + case PARTITION: + ResourceAccessor resourcePartitionAccessor = resourceAccessor(clusterName); + PartitionId partitionId = PartitionId.from(scopeArgs[1]); + userConfig = resourcePartitionAccessor.readUserConfig(partitionId.getResourceId()); + removePartitionFromResourceUserConfig(userConfig, partitionId, keys); + resourcePartitionAccessor.setUserConfig(partitionId.getResourceId(), userConfig); + break; + } + } + + private void removeKeysFromUserConfig(UserConfig userConfig, String[] keys) { + Map simpleFields = Maps.newHashMap(userConfig.getSimpleFields()); + for (String key : keys) { + simpleFields.remove(key); + } + userConfig.setSimpleFields(simpleFields); + } + + private void removePartitionFromResourceUserConfig(UserConfig userConfig, + PartitionId partitionId, String[] keys) { + Map fields = Maps.newHashMap(userConfig.getMapField(partitionId.stringify())); + for (String key : keys) { + fields.remove(key); + } + userConfig.setMapField(partitionId.stringify(), fields); + } + + void swapParticipants(String[] optValues) { + String clusterName = optValues[0]; + String oldParticipantName = optValues[1]; + String newParticipantName = optValues[2]; + ParticipantAccessor accessor = participantAccessor(clusterName); + accessor.swapParticipants(ParticipantId.from(oldParticipantName), + ParticipantId.from(newParticipantName)); + } + + void resetPartition(String[] optValues) { + String clusterName = optValues[0]; + String participantName = optValues[1]; + String resourceName = optValues[2]; + String partitionName = optValues[3]; + + Set partitionIds = ImmutableSet.of(PartitionId.from(partitionName)); + ParticipantAccessor accessor = participantAccessor(clusterName); + accessor.resetPartitionsForParticipant(ParticipantId.from(participantName), + ResourceId.from(resourceName), partitionIds); + } + + void resetResource(String[] optValues) { + String clusterName = optValues[0]; + String resourceName = optValues[1]; + Set resourceIds = ImmutableSet.of(ResourceId.from(resourceName)); + ResourceAccessor accessor = resourceAccessor(clusterName); + accessor.resetResources(resourceIds); + } + + void resetParticipant(String[] optValues) { + String clusterName = optValues[0]; + String participantName = optValues[1]; + Set participantIds = ImmutableSet.of(ParticipantId.from(participantName)); + ParticipantAccessor accessor = participantAccessor(clusterName); + accessor.resetParticipants(participantIds); + } + + void expandResource(String[] optValues) { + String clusterName = optValues[0]; + String resourceName = optValues[1]; + expandResource(ClusterId.from(clusterName), ResourceId.from(resourceName)); + } + + void expandCluster(String[] optValues) { + String clusterName = optValues[0]; + ClusterAccessor accessor = clusterAccessor(clusterName); + Cluster cluster = accessor.readCluster(); + for (ResourceId resourceId : cluster.getResourceMap().keySet()) { + expandResource(ClusterId.from(clusterName), resourceId); + } + } + + private void expandResource(ClusterId clusterId, ResourceId resourceId) { + ResourceAccessor accessor = resourceAccessor(clusterId.stringify()); + Resource resource = accessor.readResource(resourceId); + SemiAutoRebalancerContext context = + resource.getRebalancerConfig().getRebalancerContext(SemiAutoRebalancerContext.class); + if (context == null) { + LOG.info("Only SEMI_AUTO mode supported for resource expansion"); + return; + } + if (context.anyLiveParticipant()) { + LOG.info("Resource uses ANY_LIVE_PARTICIPANT, skipping default assignment"); + return; + } + if (context.getPreferenceLists().size() == 0) { + LOG.info("No preference lists have been set yet, skipping default assignment"); + return; + } + accessor.generateDefaultAssignment(resourceId, -1, null); + } + + static int processCommandLineArgs(String[] cliArgs) { + CommandLineParser cliParser = new GnuParser(); + Options cliOptions = constructCommandLineOptions(); + CommandLine cmd = null; + + try { + cmd = cliParser.parse(cliOptions, cliArgs); + } catch (ParseException pe) { + System.err.println("CommandLineClient: failed to parse command-line options: " + pe); + printUsage(cliOptions); + System.exit(1); + } + + String zkAddr = cmd.getOptionValue(HelixOption.zkSvr.name()); + ZkClient zkclient = null; + + try { + zkclient = + new ZkClient(zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT, ZkClient.DEFAULT_SESSION_TIMEOUT, + new ZNRecordSerializer()); + + NewClusterSetup setup = new NewClusterSetup(zkclient); + + Option[] options = cmd.getOptions(); + + for (Option option : options) { + if (option.getLongOpt().equals(HelixOption.zkSvr.name())) { + continue; + } + + HelixOption opt = HelixOption.valueOf(option.getLongOpt()); + String[] optValues = cmd.getOptionValues(option.getLongOpt()); + + checkArgNum(opt, optValues); + + switch (opt) { + case listClusters: + setup.listClusters(optValues); + break; + case listResources: + setup.listResources(optValues); + break; + case listInstances: + setup.listParticipants(optValues); + break; + case addCluster: + setup.addCluster(optValues); + break; + case activateCluster: + break; + case dropCluster: + setup.dropCluster(optValues); + break; + case dropResource: + setup.dropResource(optValues); + break; + case addInstance: + setup.addInstance(optValues); + break; + case addResource: + String[] rebalancerModeValues = null; + if (cmd.hasOption(HelixOption.rebalancerMode.name())) { + rebalancerModeValues = cmd.getOptionValues(HelixOption.rebalancerMode.name()); + checkArgNum(HelixOption.rebalancerMode, rebalancerModeValues); + } + String[] bucketSizeValues = null; + if (cmd.hasOption(HelixOption.bucketSize.name())) { + bucketSizeValues = cmd.getOptionValues(HelixOption.bucketSize.name()); + checkArgNum(HelixOption.bucketSize, bucketSizeValues); + } + String[] maxPartitionsPerNodeValues = null; + if (cmd.hasOption(HelixOption.maxPartitionsPerNode.name())) { + maxPartitionsPerNodeValues = + cmd.getOptionValues(HelixOption.maxPartitionsPerNode.name()); + checkArgNum(HelixOption.maxPartitionsPerNode, maxPartitionsPerNodeValues); + } + setup.addResource(optValues, rebalancerModeValues, bucketSizeValues, + maxPartitionsPerNodeValues); + break; + case addStateModelDef: + setup.addStateModelDef(optValues); + break; + case addIdealState: + setup.addIdealState(optValues); + break; + case swapInstance: + setup.swapParticipants(optValues); + break; + case dropInstance: + setup.dropInstance(optValues); + break; + case rebalance: + String[] groupTagValues = null; + if (cmd.hasOption(HelixOption.instanceGroupTag.name())) { + groupTagValues = cmd.getOptionValues(HelixOption.instanceGroupTag.name()); + checkArgNum(HelixOption.instanceGroupTag, groupTagValues); + } + setup.rebalance(optValues, groupTagValues); + break; + case expandCluster: + setup.expandCluster(optValues); + break; + case expandResource: + setup.expandResource(optValues); + break; + case mode: + case rebalancerMode: + case bucketSize: + case maxPartitionsPerNode: + // always used with addResource command + continue; + case instanceGroupTag: + // always used with rebalance command + continue; + case resourceKeyPrefix: + throw new UnsupportedOperationException(HelixOption.resourceKeyPrefix + + " is not supported, please set partition names directly"); + case addResourceProperty: + throw new UnsupportedOperationException(HelixOption.addResourceProperty + + " is not supported, please use setConfig"); + case removeResourceProperty: + throw new UnsupportedOperationException(HelixOption.removeResourceProperty + + " is not supported, please use removeConfig"); + case addInstanceTag: + setup.addInstanceTag(optValues); + break; + case removeInstanceTag: + setup.removeInstanceTag(optValues); + break; + case listClusterInfo: + setup.listClusterInfo(optValues); + break; + case listInstanceInfo: + setup.listParticipantInfo(optValues); + break; + case listResourceInfo: + setup.listResourceInfo(optValues); + break; + case listPartitionInfo: + setup.listPartitionInfo(optValues); + break; + case listStateModels: + setup.listStateModels(optValues); + break; + case listStateModel: + setup.listStateModel(optValues); + break; + case enableInstance: + setup.enableInstance(optValues); + break; + case enablePartition: + setup.enablePartition(optValues); + break; + case enableCluster: + setup.enableCluster(optValues); + break; + case resetPartition: + setup.resetPartition(optValues); + break; + case resetInstance: + setup.resetParticipant(optValues); + break; + case resetResource: + setup.resetResource(optValues); + break; + case getConfig: + setup.getConfig(optValues); + break; + case setConfig: + setup.setConfig(optValues); + break; + case removeConfig: + setup.removeConfig(optValues); + break; + case getConstraints: + setup.getConstraints(optValues); + break; + case setConstraint: + setup.setConstraint(optValues); + break; + case removeConstraint: + setup.removeConstraint(optValues); + break; + default: + System.err.println("Non-recognized option: " + opt); + break; + } + + // process 1 option only + break; + } + + return 0; + } finally { + if (zkclient != null) { + zkclient.close(); + } + } + } + + public static void main(String[] args) { + // if (args.length == 1 && args[0].equals("setup-test-cluster")) { + // System.out + // .println("By default setting up TestCluster with 6 instances, 10 partitions, Each partition will have 3 replicas"); + // new ClusterSetup("localhost:2181").setupTestCluster("TestCluster"); + // System.exit(0); + // } + + int ret = processCommandLineArgs(args); + System.exit(ret); + + } +} diff --git a/helix-core/src/main/java/org/apache/helix/tools/StateModelConfigGenerator.java b/helix-core/src/main/java/org/apache/helix/tools/StateModelConfigGenerator.java index 508e447ee9..81276266f2 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/StateModelConfigGenerator.java +++ b/helix-core/src/main/java/org/apache/helix/tools/StateModelConfigGenerator.java @@ -25,10 +25,11 @@ import java.util.Map; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory; import org.apache.helix.manager.zk.ZNRecordSerializer; -import org.apache.helix.model.Transition; import org.apache.helix.model.StateModelDefinition.StateModelDefinitionProperty; +import org.apache.helix.model.Transition; import org.apache.helix.model.builder.StateTransitionTableBuilder; // TODO refactor to use StateModelDefinition.Builder @@ -36,8 +37,8 @@ public class StateModelConfigGenerator { public static void main(String[] args) { ZNRecordSerializer serializer = new ZNRecordSerializer(); - StateModelConfigGenerator generator = new StateModelConfigGenerator(); - System.out.println(new String(serializer.serialize(generator.generateConfigForMasterSlave()))); + System.out.println(new String(serializer.serialize(StateModelConfigGenerator + .generateConfigForMasterSlave()))); } /** @@ -328,9 +329,9 @@ public static ZNRecord generateConfigForScheduledTaskQueue() { states.add("OFFLINE"); List transitions = new ArrayList(); - transitions.add(new Transition("OFFLINE", "COMPLETED")); - transitions.add(new Transition("OFFLINE", "DROPPED")); - transitions.add(new Transition("COMPLETED", "DROPPED")); + transitions.add(Transition.from(State.from("OFFLINE"), State.from("COMPLETED"))); + transitions.add(Transition.from(State.from("OFFLINE"), State.from("DROPPED"))); + transitions.add(Transition.from(State.from("COMPLETED"), State.from("DROPPED"))); StateTransitionTableBuilder builder = new StateTransitionTableBuilder(); Map> next = builder.buildTransitionTable(states, transitions); diff --git a/helix-core/src/main/java/org/apache/helix/tools/YAISCalculator.java b/helix-core/src/main/java/org/apache/helix/tools/YAISCalculator.java deleted file mode 100644 index 4292baaf52..0000000000 --- a/helix-core/src/main/java/org/apache/helix/tools/YAISCalculator.java +++ /dev/null @@ -1,174 +0,0 @@ -package org.apache.helix.tools; - -/* - * 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.LinkedList; -import java.util.List; -import java.util.Random; - -public class YAISCalculator { - static class Assignment { - private final int numNodes; - private final int replication; - Partition[] partitions; - Node[] nodes; - - public Assignment(int numNodes, int numPartitions, int replication) { - this.numNodes = numNodes; - this.replication = replication; - partitions = new Partition[numPartitions]; - for (int i = 0; i < numPartitions; i++) { - partitions[i] = new Partition(i, replication); - } - nodes = new Node[numNodes]; - for (int i = 0; i < numNodes; i++) { - nodes[i] = new Node(replication); - } - } - - public void assign(int partitionId, int replicaId, int nodeId) { - System.out.println("Assigning (" + partitionId + "," + replicaId + ") to " + nodeId); - partitions[partitionId].nodeIds[replicaId] = nodeId; - nodes[nodeId].partitionLists.get(replicaId).push(partitionId); - } - - public void unassign(int partitionId, int replicaId) { - - } - - Integer[] getPartitionsPerNode(int nodeId, int replicaId) { - List partitionsList = new ArrayList(); - for (Partition p : partitions) { - if (p.nodeIds[replicaId] == nodeId) { - partitionsList.add(p.partionId); - } - } - Integer[] array = new Integer[partitionsList.size()]; - partitionsList.toArray(array); - return array; - } - - public void printPerNode() { - for (int nodeId = 0; nodeId < numNodes; nodeId++) { - for (int r = 0; r < replication; r++) { - StringBuilder sb = new StringBuilder(); - sb.append("(").append(nodeId).append(",").append(r).append("):\t"); - Node node = nodes[nodeId]; - LinkedList linkedList = node.partitionLists.get(r); - for (int partitionId : linkedList) { - sb.append(partitionId).append(","); - } - System.out.println(sb.toString()); - } - - } - } - } - - static class Partition { - - final int partionId; - - public Partition(int partionId, int replication) { - this.partionId = partionId; - nodeIds = new int[replication]; - Arrays.fill(nodeIds, -1); - } - - int nodeIds[]; - } - - static class Node { - private final int replication; - ArrayList> partitionLists; - - public Node(int replication) { - this.replication = replication; - partitionLists = new ArrayList>(replication); - for (int i = 0; i < replication; i++) { - partitionLists.add(new LinkedList()); - } - } - - } - - public static void main(String[] args) { - doAssignment(new int[] { - 5 - }, 120, 3); - } - - private static void doAssignment(int[] nodes, int partitions, int replication) { - int N = nodes[0]; - int totalNodes = 0; - for (int temp : nodes) { - totalNodes += temp; - } - Assignment assignment = new Assignment(totalNodes, partitions, replication); - int nodeId = 0; - for (int i = 0; i < partitions; i++) { - assignment.assign(i, 0, nodeId); - nodeId = (nodeId + 1) % N; - } - Random random = new Random(); - for (int r = 1; r < replication; r++) { - for (int id = 0; id < N; id++) { - Integer[] partitionsPerNode = assignment.getPartitionsPerNode(id, 0); - boolean[] used = new boolean[partitionsPerNode.length]; - Arrays.fill(used, false); - System.out.println(id + "-" + partitionsPerNode.length); - nodeId = (id + r) % N; - int count = partitionsPerNode.length; - boolean done = false; - do { - if (nodeId != id) { - int nextInt = random.nextInt(count); - int temp = 0; - for (int b = 0; b < used.length; b++) { - if (!used[b] && temp == nextInt) { - assignment.assign(partitionsPerNode[b], r, nodeId); - used[b] = true; - break; - } - } - } - nodeId = (nodeId + 1) % N; - } while (count > 0); - - } - } - if (nodes.length > 1) { - int prevNodeCount = nodes[0]; - for (int i = 1; i < nodes.length; i++) { - int newNodeCount = prevNodeCount + nodes[i]; - int masterPartitionsToMove = - (int) ((partitions * 1.0 / prevNodeCount - partitions * 1.0 / newNodeCount) * 1 * prevNodeCount); - while (masterPartitionsToMove > 0) { - - } - - } - } - assignment.printPerNode(); - } - -} diff --git a/helix-core/src/main/java/org/apache/helix/tools/YAMLClusterSetup.java b/helix-core/src/main/java/org/apache/helix/tools/YAMLClusterSetup.java new file mode 100644 index 0000000000..9ec1ef9c81 --- /dev/null +++ b/helix-core/src/main/java/org/apache/helix/tools/YAMLClusterSetup.java @@ -0,0 +1,322 @@ +package org.apache.helix.tools; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.HelixAdmin; +import org.apache.helix.HelixException; +import org.apache.helix.manager.zk.ZKHelixAdmin; +import org.apache.helix.manager.zk.ZKHelixManager; +import org.apache.helix.model.HelixConfigScope; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.model.builder.HelixConfigScopeBuilder; +import org.apache.helix.tools.YAMLClusterSetup.YAMLClusterConfig.ParticipantConfig; +import org.apache.helix.tools.YAMLClusterSetup.YAMLClusterConfig.ResourceConfig; +import org.apache.helix.tools.YAMLClusterSetup.YAMLClusterConfig.ResourceConfig.ConstraintsConfig; +import org.apache.helix.tools.YAMLClusterSetup.YAMLClusterConfig.ResourceConfig.StateModelConfig; +import org.apache.log4j.Logger; +import org.yaml.snakeyaml.Yaml; + +/* + * 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. + */ + +/** + * Supports HelixAdmin operations specified by a YAML configuration file defining a cluster, + * resources, participants, etc. + * See the user-rebalanced-lock-manager recipe for an annotated example file. + */ +public class YAMLClusterSetup { + private static final Logger LOG = Logger.getLogger(YAMLClusterSetup.class); + + private final String _zkAddress; + + /** + * Start the YAML parser for a given zookeeper instance + * @param zkAddress + */ + public YAMLClusterSetup(String zkAddress) { + _zkAddress = zkAddress; + } + + /** + * Set up the cluster by parsing a YAML file. + * @param input InputStream representing the file + * @return ClusterConfig Java wrapper of the configuration file + */ + public YAMLClusterConfig setupCluster(InputStream input) { + // parse the YAML + Yaml yaml = new Yaml(); + YAMLClusterConfig cfg = yaml.loadAs(input, YAMLClusterConfig.class); + + // create the cluster + HelixAdmin helixAdmin = new ZKHelixAdmin(_zkAddress); + if (cfg.clusterName == null) { + throw new HelixException("Cluster name is required!"); + } + helixAdmin.addCluster(cfg.clusterName); + + // add each participant + if (cfg.participants != null) { + for (ParticipantConfig participant : cfg.participants) { + helixAdmin.addInstance(cfg.clusterName, getInstanceCfg(participant)); + } + } + + // add each resource + if (cfg.resources != null) { + for (ResourceConfig resource : cfg.resources) { + if (resource.name == null) { + throw new HelixException("Resources must be named!"); + } + if (resource.stateModel == null || resource.stateModel.name == null) { + throw new HelixException("Resource must specify a named state model!"); + } + // if states is null, assume using a built-in or already-added state model + if (resource.stateModel.states != null) { + StateModelDefinition stateModelDef = + getStateModelDef(resource.stateModel, resource.constraints); + helixAdmin.addStateModelDef(cfg.clusterName, resource.stateModel.name, stateModelDef); + } else { + StateModelDefinition stateModelDef = null; + if (resource.stateModel.name.equals("MasterSlave")) { + stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + } else if (resource.stateModel.name.equals("OnlineOffline")) { + stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForOnlineOffline()); + } else if (resource.stateModel.name.equals("LeaderStandby")) { + stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForLeaderStandby()); + } + if (stateModelDef != null) { + try { + helixAdmin.addStateModelDef(cfg.clusterName, resource.stateModel.name, stateModelDef); + } catch (HelixException e) { + LOG.warn("State model definition " + resource.stateModel.name + + " could not be added."); + } + } + } + int partitions = 1; + int replicas = 1; + if (resource.partitions != null) { + if (resource.partitions.containsKey("count")) { + partitions = resource.partitions.get("count"); + } + if (resource.partitions.containsKey("replicas")) { + replicas = resource.partitions.get("replicas"); + } + } + + if (resource.rebalancer == null || !resource.rebalancer.containsKey("mode")) { + throw new HelixException("Rebalance mode is required!"); + } + helixAdmin.addResource(cfg.clusterName, resource.name, partitions, + resource.stateModel.name, resource.rebalancer.get("mode")); + // user-defined rebalancer + if (resource.rebalancer.containsKey("class") + && resource.rebalancer.get("mode").equals(RebalanceMode.USER_DEFINED.toString())) { + IdealState idealState = helixAdmin.getResourceIdealState(cfg.clusterName, resource.name); + idealState.setRebalancerClassName(resource.rebalancer.get("class")); + helixAdmin.setResourceIdealState(cfg.clusterName, resource.name, idealState); + } + helixAdmin.rebalance(cfg.clusterName, resource.name, replicas); + } + } + + // enable auto join if this option is set + if (cfg.autoJoinAllowed != null && cfg.autoJoinAllowed) { + HelixConfigScope scope = + new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(cfg.clusterName) + .build(); + Map properties = new HashMap(); + properties.put(ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, cfg.autoJoinAllowed.toString()); + helixAdmin.setConfig(scope, properties); + } + return cfg; + } + + private static InstanceConfig getInstanceCfg(ParticipantConfig participant) { + if (participant == null || participant.name == null || participant.host == null + || participant.port == null) { + throw new HelixException("Participant must have a specified name, host, and port!"); + } + InstanceConfig instanceCfg = new InstanceConfig(participant.name); + instanceCfg.setHostName(participant.host); + instanceCfg.setPort(participant.port.toString()); + return instanceCfg; + } + + private static StateModelDefinition getStateModelDef(StateModelConfig stateModel, + ConstraintsConfig constraints) { + // Use a builder to define the state model + StateModelDefinition.Builder builder = new StateModelDefinition.Builder(stateModel.name); + if (stateModel.states == null || stateModel.states.size() == 0) { + throw new HelixException("List of states are required in a state model!"); + } + Set stateSet = new HashSet(stateModel.states); + if (stateModel.initialState == null) { + throw new HelixException("Initial state is required in a state model!"); + } else if (!stateSet.contains(stateModel.initialState)) { + throw new HelixException("Initial state is not a valid state"); + } + builder.initialState(stateModel.initialState); + + // Build a helper for state priorities + Map statePriorities = new HashMap(); + if (constraints != null && constraints.state != null && constraints.state.priorityList != null) { + int statePriority = 0; + for (String state : constraints.state.priorityList) { + if (!stateSet.contains(state)) { + throw new HelixException("State " + state + + " in the state priority list is not in the state list!"); + } + statePriorities.put(state, statePriority); + statePriority++; + } + } + + // Add states, set state priorities + for (String state : stateModel.states) { + if (statePriorities.containsKey(state)) { + builder.addState(state, statePriorities.get(state)); + } else { + builder.addState(state); + } + } + + // Set state counts + for (Map counts : constraints.state.counts) { + String state = counts.get("name"); + if (!stateSet.contains(state)) { + throw new HelixException("State " + state + " has a count, but not in the state list!"); + } + builder.dynamicUpperBound(state, counts.get("count")); + } + + // Build a helper for transition priorities + Map transitionPriorities = new HashMap(); + if (constraints != null && constraints.transition != null + && constraints.transition.priorityList != null) { + int transitionPriority = 0; + for (String transition : constraints.transition.priorityList) { + transitionPriorities.put(transition, transitionPriority); + transitionPriority++; + } + } + + // Add the transitions + if (stateModel.transitions == null || stateModel.transitions.size() == 0) { + throw new HelixException("Transitions are required!"); + } + for (Map transitions : stateModel.transitions) { + String name = transitions.get("name"); + String from = transitions.get("from"); + String to = transitions.get("to"); + if (name == null || from == null || to == null) { + throw new HelixException("All transitions must have a name, a from state, and a to state"); + } + if (transitionPriorities.containsKey(name)) { + builder.addTransition(from, to, transitionPriorities.get(name)); + } else { + builder.addTransition(from, to); + } + } + + return builder.build(); + } + + /** + * Java wrapper for the YAML input file + */ + public static class YAMLClusterConfig { + public String clusterName; + public List resources; + public List participants; + public Boolean autoJoinAllowed; + + public static class ResourceConfig { + public String name; + public Map rebalancer; + public Map partitions; + public StateModelConfig stateModel; + public ConstraintsConfig constraints; + + public static class StateModelConfig { + public String name; + public List states; + public List> transitions; + public String initialState; + } + + public static class ConstraintsConfig { + public StateConstraintsConfig state; + public TransitionConstraintsConfig transition; + + public static class StateConstraintsConfig { + public List> counts; + public List priorityList; + } + + public static class TransitionConstraintsConfig { + public List priorityList; + } + } + } + + public static class ParticipantConfig { + public String name; + public String host; + public Integer port; + } + } + + /** + * Start a cluster defined by a YAML file + * @param args zkAddr, yamlFile + */ + public static void main(String[] args) { + if (args.length < 2) { + LOG.error("USAGE: YAMLClusterSetup zkAddr yamlFile"); + return; + } + String zkAddress = args[0]; + String yamlFile = args[1]; + + InputStream input; + try { + input = new FileInputStream(new File(yamlFile)); + } catch (FileNotFoundException e) { + LOG.error("Could not open " + yamlFile); + return; + } + new YAMLClusterSetup(zkAddress).setupCluster(input); + } +} diff --git a/helix-core/src/main/java/org/apache/helix/tools/ZkLogAnalyzer.java b/helix-core/src/main/java/org/apache/helix/tools/ZkLogAnalyzer.java index 11e1b6681a..8b32ddc9f1 100644 --- a/helix-core/src/main/java/org/apache/helix/tools/ZkLogAnalyzer.java +++ b/helix-core/src/main/java/org/apache/helix/tools/ZkLogAnalyzer.java @@ -342,11 +342,14 @@ public boolean accept(File file) { // sendMessageLines.add(inputLine); stats.msgSentCount++; - if (msg.getFromState().equals("OFFLINE") && msg.getToState().equals("SLAVE")) { + if (msg.getTypedFromState().toString().equals("OFFLINE") + && msg.getTypedToState().toString().equals("SLAVE")) { stats.msgSentCount_O2S++; - } else if (msg.getFromState().equals("SLAVE") && msg.getToState().equals("MASTER")) { + } else if (msg.getTypedFromState().toString().equals("SLAVE") + && msg.getTypedToState().toString().equals("MASTER")) { stats.msgSentCount_S2M++; - } else if (msg.getFromState().equals("MASTER") && msg.getToState().equals("SLAVE")) { + } else if (msg.getTypedFromState().toString().equals("MASTER") + && msg.getTypedToState().toString().equals("SLAVE")) { stats.msgSentCount_M2S++; } // System.out.println("Message create:"+new diff --git a/helix-core/src/main/java/org/apache/helix/util/RebalanceUtil.java b/helix-core/src/main/java/org/apache/helix/util/RebalanceUtil.java index 3f8c406397..d304a878c1 100644 --- a/helix-core/src/main/java/org/apache/helix/util/RebalanceUtil.java +++ b/helix-core/src/main/java/org/apache/helix/util/RebalanceUtil.java @@ -27,9 +27,13 @@ import java.util.TreeMap; import org.apache.helix.HelixException; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.model.IdealState; import org.apache.helix.model.StateModelDefinition; +import com.google.common.base.Functions; +import com.google.common.collect.Lists; + public class RebalanceUtil { public static Map buildInternalIdealState(IdealState state) { // Try parse the partition number from name DB_n. If not, sort the partitions and @@ -37,17 +41,19 @@ public static Map buildInternalIdealState(IdealState state) { Map partitionIndex = new HashMap(); Map reversePartitionIndex = new HashMap(); boolean indexInPartitionName = true; - for (String partitionId : state.getPartitionSet()) { - int lastPos = partitionId.lastIndexOf("_"); + for (PartitionId partitionId : state.getPartitionIdSet()) { + String partitionName = partitionId.stringify(); + int lastPos = partitionName.lastIndexOf("_"); if (lastPos < 0) { indexInPartitionName = false; break; } try { - String idStr = partitionId.substring(lastPos + 1); + String idStr = partitionName.substring(lastPos + 1); int partition = Integer.parseInt(idStr); - partitionIndex.put(partitionId, partition); - reversePartitionIndex.put(state.getResourceName() + "_" + partition, partitionId); + partitionIndex.put(partitionName, partition); + reversePartitionIndex.put(state.getResourceId().stringify() + "_" + partition, + partitionName); } catch (Exception e) { indexInPartitionName = false; partitionIndex.clear(); @@ -58,19 +64,20 @@ public static Map buildInternalIdealState(IdealState state) { if (indexInPartitionName == false) { List partitions = new ArrayList(); - partitions.addAll(state.getPartitionSet()); + partitions.addAll(Lists.transform(Lists.newArrayList(state.getPartitionIdSet()), + Functions.toStringFunction())); Collections.sort(partitions); for (int i = 0; i < partitions.size(); i++) { partitionIndex.put(partitions.get(i), i); - reversePartitionIndex.put(state.getResourceName() + "_" + i, partitions.get(i)); + reversePartitionIndex.put(state.getResourceId().stringify() + "_" + i, partitions.get(i)); } } Map> nodeMasterAssignmentMap = new TreeMap>(); Map>> combinedNodeSlaveAssignmentMap = new TreeMap>>(); - for (String partition : state.getPartitionSet()) { - List instances = state.getRecord().getListField(partition); + for (PartitionId partition : state.getPartitionIdSet()) { + List instances = state.getRecord().getListField(partition.stringify()); String master = instances.get(0); if (!nodeMasterAssignmentMap.containsKey(master)) { nodeMasterAssignmentMap.put(master, new ArrayList()); @@ -90,8 +97,8 @@ public static Map buildInternalIdealState(IdealState state) { } Map result = new TreeMap(); - result.put("MasterAssignmentMap", nodeMasterAssignmentMap); - result.put("SlaveAssignmentMap", combinedNodeSlaveAssignmentMap); + result.put("PrimaryAssignmentMap", nodeMasterAssignmentMap); + result.put("SecondaryAssignmentMap", combinedNodeSlaveAssignmentMap); result.put("replicas", Integer.parseInt(state.getReplicas())); result.put("partitions", new Integer(state.getRecord().getListFields().size())); result.put("reversePartitionIndex", reversePartitionIndex); diff --git a/helix-core/src/main/java/org/apache/helix/util/StatusUpdateUtil.java b/helix-core/src/main/java/org/apache/helix/util/StatusUpdateUtil.java index 02c39d110a..e05ad96687 100644 --- a/helix-core/src/main/java/org/apache/helix/util/StatusUpdateUtil.java +++ b/helix-core/src/main/java/org/apache/helix/util/StatusUpdateUtil.java @@ -36,12 +36,13 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixProperty; import org.apache.helix.PropertyKey; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.SessionId; import org.apache.helix.model.Error; import org.apache.helix.model.Message; -import org.apache.helix.model.StatusUpdate; import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StatusUpdate; import org.apache.log4j.Logger; /** @@ -254,7 +255,7 @@ public ZNRecord createEmptyStatusUpdateRecord(String id) { */ ZNRecord createMessageLogRecord(Message message) { ZNRecord result = new ZNRecord(getStatusUpdateRecordName(message)); - String mapFieldKey = "MESSAGE " + message.getMsgId(); + String mapFieldKey = "MESSAGE " + message.getMessageId(); result.setMapField(mapFieldKey, new TreeMap()); // Store all the simple fields of the message in the new ZNRecord's map @@ -282,7 +283,7 @@ ZNRecord createMessageLogRecord(Message message) { * @param additional * info the additional debug information */ - public ZNRecord createMessageStatusUpdateRecord(Message message, Level level, Class classInfo, + public ZNRecord createMessageStatusUpdateRecord(Message message, Level level, Class classInfo, String additionalInfo) { ZNRecord result = createEmptyStatusUpdateRecord(getStatusUpdateRecordName(message)); Map contentMap = new TreeMap(); @@ -290,7 +291,7 @@ public ZNRecord createMessageStatusUpdateRecord(Message message, Level level, Cl contentMap.put("Message state", message.getMsgState().toString()); contentMap.put("AdditionalInfo", additionalInfo); contentMap.put("Class", classInfo.toString()); - contentMap.put("MSG_ID", message.getMsgId()); + contentMap.put("MSG_ID", message.getMessageId().stringify()); DateFormat formatter = new SimpleDateFormat("yyyyMMdd-HHmmss.SSSSSS"); String time = formatter.format(new Date()); @@ -304,8 +305,8 @@ public ZNRecord createMessageStatusUpdateRecord(Message message, Level level, Cl String getRecordIdForMessage(Message message) { if (message.getMsgType().equals(MessageType.STATE_TRANSITION)) { - return message.getPartitionName() + " Trans:" + message.getFromState().charAt(0) + "->" - + message.getToState().charAt(0) + " " + UUID.randomUUID().toString(); + return message.getPartitionId() + " Trans:" + message.getTypedFromState().toString().charAt(0) + + "->" + message.getTypedToState().toString().charAt(0) + " " + UUID.randomUUID().toString(); } else { return message.getMsgType() + " " + UUID.randomUUID().toString(); } @@ -325,7 +326,7 @@ String getRecordIdForMessage(Message message) { * @param accessor * the zookeeper data accessor that writes the status update to zookeeper */ - public void logMessageStatusUpdateRecord(Message message, Level level, Class classInfo, + public void logMessageStatusUpdateRecord(Message message, Level level, Class classInfo, String additionalInfo, HelixDataAccessor accessor) { try { ZNRecord record = createMessageStatusUpdateRecord(message, level, classInfo, additionalInfo); @@ -335,12 +336,12 @@ public void logMessageStatusUpdateRecord(Message message, Level level, Class cla } } - public void logError(Message message, Class classInfo, String additionalInfo, + public void logError(Message message, Class classInfo, String additionalInfo, HelixDataAccessor accessor) { logMessageStatusUpdateRecord(message, Level.HELIX_ERROR, classInfo, additionalInfo, accessor); } - public void logError(Message message, Class classInfo, Exception e, String additionalInfo, + public void logError(Message message, Class classInfo, Exception e, String additionalInfo, HelixDataAccessor accessor) { StringWriter sw = new StringWriter(); PrintWriter pw = new PrintWriter(sw); @@ -349,12 +350,12 @@ public void logError(Message message, Class classInfo, Exception e, String addit additionalInfo + sw.toString(), accessor); } - public void logInfo(Message message, Class classInfo, String additionalInfo, + public void logInfo(Message message, Class classInfo, String additionalInfo, HelixDataAccessor accessor) { logMessageStatusUpdateRecord(message, Level.HELIX_INFO, classInfo, additionalInfo, accessor); } - public void logWarning(Message message, Class classInfo, String additionalInfo, + public void logWarning(Message message, Class classInfo, String additionalInfo, HelixDataAccessor accessor) { logMessageStatusUpdateRecord(message, Level.HELIX_WARNING, classInfo, additionalInfo, accessor); } @@ -375,16 +376,16 @@ void publishStatusUpdateRecord(ZNRecord record, Message message, Level level, String instanceName = message.getTgtName(); String statusUpdateSubPath = getStatusUpdateSubPath(message); String statusUpdateKey = getStatusUpdateKey(message); - String sessionId = message.getExecutionSessionId(); + SessionId sessionId = message.getTypedExecutionSessionId(); if (sessionId == null) { - sessionId = message.getTgtSessionId(); + sessionId = message.getTypedTgtSessionId(); } if (sessionId == null) { - sessionId = "*"; + sessionId = SessionId.from("*"); } Builder keyBuilder = accessor.keyBuilder(); - if (!_recordedMessages.containsKey(message.getMsgId())) { + if (!_recordedMessages.containsKey(message.getMessageId().stringify())) { // TODO instanceName of a controller might be any string if (instanceName.equalsIgnoreCase("Controller")) { accessor.updateProperty( @@ -394,8 +395,8 @@ void publishStatusUpdateRecord(ZNRecord record, Message message, Level level, } else { PropertyKey propertyKey = - keyBuilder.stateTransitionStatus(instanceName, sessionId, statusUpdateSubPath, - statusUpdateKey); + keyBuilder.stateTransitionStatus(instanceName, sessionId.stringify(), + statusUpdateSubPath, statusUpdateKey); ZNRecord statusUpdateRecord = createMessageLogRecord(message); @@ -408,7 +409,7 @@ void publishStatusUpdateRecord(ZNRecord record, Message message, Level level, accessor.updateProperty(propertyKey, new StatusUpdate(statusUpdateRecord)); } - _recordedMessages.put(message.getMsgId(), message.getMsgId()); + _recordedMessages.put(message.getMessageId().stringify(), message.getMessageId().stringify()); } if (instanceName.equalsIgnoreCase("Controller")) { @@ -418,8 +419,8 @@ void publishStatusUpdateRecord(ZNRecord record, Message message, Level level, } else { PropertyKey propertyKey = - keyBuilder.stateTransitionStatus(instanceName, sessionId, statusUpdateSubPath, - statusUpdateKey); + keyBuilder.stateTransitionStatus(instanceName, sessionId.stringify(), + statusUpdateSubPath, statusUpdateKey); // For now write participant StatusUpdates to log4j. // we are using restlet as another data channel to report to controller. if (_logger.isTraceEnabled()) { @@ -436,9 +437,9 @@ void publishStatusUpdateRecord(ZNRecord record, Message message, Level level, private String getStatusUpdateKey(Message message) { if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.toString())) { - return message.getPartitionName(); + return message.getPartitionId().stringify(); } - return message.getMsgId(); + return message.getMessageId().stringify(); } /** @@ -446,7 +447,7 @@ private String getStatusUpdateKey(Message message) { */ String getStatusUpdateSubPath(Message message) { if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.toString())) { - return message.getResourceName(); + return message.getResourceId().stringify(); } else { return message.getMsgType(); } @@ -454,9 +455,9 @@ String getStatusUpdateSubPath(Message message) { String getStatusUpdateRecordName(Message message) { if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.toString())) { - return message.getTgtSessionId() + "__" + message.getResourceName(); + return message.getTypedTgtSessionId() + "__" + message.getResourceId(); } - return message.getMsgId(); + return message.getMessageId().stringify(); } /** @@ -472,12 +473,12 @@ void publishErrorRecord(ZNRecord record, Message message, HelixDataAccessor acce String instanceName = message.getTgtName(); String statusUpdateSubPath = getStatusUpdateSubPath(message); String statusUpdateKey = getStatusUpdateKey(message); - String sessionId = message.getExecutionSessionId(); + SessionId sessionId = message.getTypedExecutionSessionId(); if (sessionId == null) { - sessionId = message.getTgtSessionId(); + sessionId = message.getTypedTgtSessionId(); } if (sessionId == null) { - sessionId = "*"; + sessionId = SessionId.from("*"); } Builder keyBuilder = accessor.keyBuilder(); @@ -488,7 +489,7 @@ void publishErrorRecord(ZNRecord record, Message message, HelixDataAccessor acce // ../{sessionId}/{subPath} // accessor.setProperty(PropertyType.ERRORS_CONTROLLER, record, // statusUpdateSubPath); - accessor.setProperty(keyBuilder.controllerTaskError(statusUpdateSubPath), new Error(record)); + accessor.updateProperty(keyBuilder.controllerTaskError(statusUpdateSubPath), new Error(record)); } else { // accessor.updateProperty(PropertyType.ERRORS, // record, @@ -496,7 +497,7 @@ void publishErrorRecord(ZNRecord record, Message message, HelixDataAccessor acce // sessionId, // statusUpdateSubPath, // statusUpdateKey); - accessor.updateProperty(keyBuilder.stateTransitionError(instanceName, sessionId, + accessor.updateProperty(keyBuilder.stateTransitionError(instanceName, sessionId.stringify(), statusUpdateSubPath, statusUpdateKey), new Error(record)); } diff --git a/helix-core/src/main/java/org/apache/helix/util/ZKClientPool.java b/helix-core/src/main/java/org/apache/helix/util/ZKClientPool.java index 0c45020c76..0980e480b8 100644 --- a/helix-core/src/main/java/org/apache/helix/util/ZKClientPool.java +++ b/helix-core/src/main/java/org/apache/helix/util/ZKClientPool.java @@ -22,7 +22,6 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import org.apache.helix.HelixException; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkClient; import org.apache.zookeeper.ZooKeeper.States; diff --git a/helix-core/src/test/java/org/apache/helix/Mocks.java b/helix-core/src/test/java/org/apache/helix/Mocks.java index 7bafe1b053..0b16992313 100644 --- a/helix-core/src/test/java/org/apache/helix/Mocks.java +++ b/helix-core/src/test/java/org/apache/helix/Mocks.java @@ -31,8 +31,6 @@ import org.I0Itec.zkclient.IZkChildListener; import org.I0Itec.zkclient.IZkDataListener; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.healthcheck.HealthReportProvider; -import org.apache.helix.healthcheck.ParticipantHealthReportCollector; import org.apache.helix.messaging.AsyncCallback; import org.apache.helix.messaging.handling.HelixTaskExecutor; import org.apache.helix.messaging.handling.HelixTaskResult; @@ -40,13 +38,13 @@ import org.apache.helix.messaging.handling.MessageTask; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; import org.apache.helix.model.Message; +import org.apache.helix.monitoring.MonitoringServer; import org.apache.helix.participant.StateMachineEngine; import org.apache.helix.participant.statemachine.StateModel; import org.apache.helix.participant.statemachine.StateModelInfo; import org.apache.helix.participant.statemachine.Transition; import org.apache.helix.store.zk.ZkHelixPropertyStore; import org.apache.zookeeper.data.Stat; -import org.omg.CORBA._PolicyStub; public class Mocks { public static class MockBaseDataAccessor implements BaseDataAccessor { @@ -371,12 +369,6 @@ public ClusterMessagingService getMessagingService() { return _msgSvc; } - @Override - public ParticipantHealthReportCollector getHealthReportCollector() { - // TODO Auto-generated method stub - return null; - } - @Override public InstanceType getInstanceType() { return InstanceType.PARTICIPANT; @@ -393,13 +385,6 @@ public void setVersion(String version) { } - @Override - public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName) - throws Exception { - // TODO Auto-generated method stub - - } - @Override public StateMachineEngine getStateMachineEngine() { // TODO Auto-generated method stub @@ -473,6 +458,12 @@ public HelixManagerProperties getProperties() { return _properties; } + @Override + public void addControllerMessageListener(MessageListener listener) { + // TODO Auto-generated method stub + + } + } public static class MockAccessor implements HelixDataAccessor // DataAccessor @@ -675,7 +666,7 @@ public Builder keyBuilder() { } @Override - public BaseDataAccessor getBaseDataAccessor() { + public BaseDataAccessor getBaseDataAccessor() { // TODO Auto-generated method stub return null; } @@ -699,22 +690,6 @@ public List getProperty(List keys) { } } - public static class MockHealthReportProvider extends HealthReportProvider { - - @Override - public Map getRecentHealthReport() { - // TODO Auto-generated method stub - return null; - } - - @Override - public void resetStats() { - // TODO Auto-generated method stub - - } - - } - public static class MockClusterMessagingService implements ClusterMessagingService { @Override diff --git a/helix-core/src/test/java/org/apache/helix/TestConfigAccessor.java b/helix-core/src/test/java/org/apache/helix/TestConfigAccessor.java index c8514ef418..d58389d09d 100644 --- a/helix-core/src/test/java/org/apache/helix/TestConfigAccessor.java +++ b/helix-core/src/test/java/org/apache/helix/TestConfigAccessor.java @@ -19,7 +19,6 @@ * under the License. */ -import java.util.Arrays; import java.util.Date; import java.util.List; diff --git a/helix-core/src/test/java/org/apache/helix/TestGroupCommit.java b/helix-core/src/test/java/org/apache/helix/TestGroupCommit.java index 9e8f943699..3413627310 100644 --- a/helix-core/src/test/java/org/apache/helix/TestGroupCommit.java +++ b/helix-core/src/test/java/org/apache/helix/TestGroupCommit.java @@ -22,10 +22,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import org.apache.helix.BaseDataAccessor; -import org.apache.helix.GroupCommit; -import org.apache.helix.ZNRecord; - public class TestGroupCommit { // @Test public void testGroupCommit() throws InterruptedException { diff --git a/helix-core/src/test/java/org/apache/helix/TestHelixConfigAccessor.java b/helix-core/src/test/java/org/apache/helix/TestHelixConfigAccessor.java index 337bcfe1ea..9e8b079315 100644 --- a/helix-core/src/test/java/org/apache/helix/TestHelixConfigAccessor.java +++ b/helix-core/src/test/java/org/apache/helix/TestHelixConfigAccessor.java @@ -24,8 +24,8 @@ import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.helix.model.HelixConfigScope; -import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.builder.HelixConfigScopeBuilder; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java b/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java index 4aa16ebf77..a3b16e5c66 100644 --- a/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java +++ b/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java @@ -23,6 +23,12 @@ import org.apache.helix.Mocks.MockManager; import org.apache.helix.Mocks.MockStateModel; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.messaging.handling.AsyncCallbackService; import org.apache.helix.messaging.handling.HelixStateTransitionHandler; import org.apache.helix.messaging.handling.HelixTask; @@ -40,18 +46,18 @@ public class TestHelixTaskExecutor { @Test() public void testCMTaskExecutor() throws Exception { System.out.println("START TestCMTaskExecutor"); - String msgId = "TestMessageId"; + MessageId msgId = MessageId.from("TestMessageId"); Message message = new Message(MessageType.TASK_REPLY, msgId); - message.setMsgId(msgId); + message.setMessageId(msgId); message.setSrcName("cm-instance-0"); message.setTgtName("cm-instance-1"); - message.setTgtSessionId("1234"); - message.setFromState("Offline"); - message.setToState("Slave"); - message.setPartitionName("TestDB_0"); - message.setResourceName("TestDB"); - message.setStateModelDef("MasterSlave"); + message.setTgtSessionId(SessionId.from("1234")); + message.setFromState(State.from("Offline")); + message.setToState(State.from("Slave")); + message.setPartitionId(PartitionId.from("TestDB_0")); + message.setResourceId(ResourceId.from("TestDB")); + message.setStateModelDef(StateModelDefId.from("MasterSlave")); MockManager manager = new MockManager("clusterName"); HelixDataAccessor accessor = manager.getHelixDataAccessor(); @@ -67,7 +73,7 @@ public void testCMTaskExecutor() throws Exception { NotificationContext context = new NotificationContext(manager); CurrentState currentStateDelta = new CurrentState("TestDB"); - currentStateDelta.setState("TestDB_0", "OFFLINE"); + currentStateDelta.setState(PartitionId.from("TestDB_0"), State.from("OFFLINE")); StateModelFactory stateModelFactory = new StateModelFactory() { diff --git a/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java b/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java index 3c7f2afd05..43b4407dbc 100644 --- a/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java +++ b/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java @@ -21,20 +21,23 @@ import java.util.Date; -import org.apache.helix.HelixConstants; -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.NotificationContext; import org.apache.helix.Mocks.MockManager; import org.apache.helix.Mocks.MockStateModel; import org.apache.helix.Mocks.MockStateModelAnnotated; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.messaging.handling.HelixStateTransitionHandler; import org.apache.helix.messaging.handling.HelixTask; import org.apache.helix.messaging.handling.HelixTaskExecutor; import org.apache.helix.model.CurrentState; import org.apache.helix.model.Message; -import org.apache.helix.model.StateModelDefinition; import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StateModelDefinition; import org.apache.helix.participant.statemachine.StateModelFactory; import org.apache.helix.tools.StateModelConfigGenerator; import org.testng.AssertJUnit; @@ -45,17 +48,17 @@ public class TestHelixTaskHandler { public void testInvocation() throws Exception { HelixTaskExecutor executor = new HelixTaskExecutor(); System.out.println("START TestCMTaskHandler.testInvocation()"); - Message message = new Message(MessageType.STATE_TRANSITION, "Some unique id"); + Message message = new Message(MessageType.STATE_TRANSITION, MessageId.from("Some unique id")); message.setSrcName("cm-instance-0"); - message.setTgtSessionId("1234"); - message.setFromState("Offline"); - message.setToState("Slave"); - message.setPartitionName("TestDB_0"); - message.setMsgId("Some unique message id"); - message.setResourceName("TestDB"); + message.setTgtSessionId(SessionId.from("1234")); + message.setFromState(State.from("Offline")); + message.setToState(State.from("Slave")); + message.setPartitionId(PartitionId.from("TestDB_0")); + message.setMessageId(MessageId.from("Some unique message id")); + message.setResourceId(ResourceId.from("TestDB")); message.setTgtName("localhost"); - message.setStateModelDef("MasterSlave"); + message.setStateModelDef(StateModelDefId.from("MasterSlave")); message.setStateModelFactoryName(HelixConstants.DEFAULT_STATE_MODEL_FACTORY); MockStateModel stateModel = new MockStateModel(); NotificationContext context; @@ -68,7 +71,7 @@ public void testInvocation() throws Exception { context = new NotificationContext(manager); CurrentState currentStateDelta = new CurrentState("TestDB"); - currentStateDelta.setState("TestDB_0", "OFFLINE"); + currentStateDelta.setState(PartitionId.from("TestDB_0"), State.from("OFFLINE")); HelixStateTransitionHandler stHandler = new HelixStateTransitionHandler(null, stateModel, message, context, currentStateDelta); @@ -85,16 +88,16 @@ public void testInvocationAnnotated() throws Exception { System.out.println("START TestCMTaskHandler.testInvocationAnnotated() at " + new Date(System.currentTimeMillis())); HelixTaskExecutor executor = new HelixTaskExecutor(); - Message message = new Message(MessageType.STATE_TRANSITION, "Some unique id"); + Message message = new Message(MessageType.STATE_TRANSITION, MessageId.from("Some unique id")); message.setSrcName("cm-instance-0"); - message.setTgtSessionId("1234"); - message.setFromState("Offline"); - message.setToState("Slave"); - message.setPartitionName("TestDB_0"); - message.setMsgId("Some unique message id"); - message.setResourceName("TestDB"); + message.setTgtSessionId(SessionId.from("1234")); + message.setFromState(State.from("Offline")); + message.setToState(State.from("Slave")); + message.setPartitionId(PartitionId.from("TestDB_0")); + message.setMessageId(MessageId.from("Some unique message id")); + message.setResourceId(ResourceId.from("TestDB")); message.setTgtName("localhost"); - message.setStateModelDef("MasterSlave"); + message.setStateModelDef(StateModelDefId.from("MasterSlave")); message.setStateModelFactoryName(HelixConstants.DEFAULT_STATE_MODEL_FACTORY); MockStateModelAnnotated stateModel = new MockStateModelAnnotated(); NotificationContext context; @@ -110,7 +113,7 @@ public void testInvocationAnnotated() throws Exception { context = new NotificationContext(manager); CurrentState currentStateDelta = new CurrentState("TestDB"); - currentStateDelta.setState("TestDB_0", "OFFLINE"); + currentStateDelta.setState(PartitionId.from("TestDB_0"), State.from("OFFLINE")); StateModelFactory stateModelFactory = new StateModelFactory() { diff --git a/helix-core/src/test/java/org/apache/helix/TestHelixVersionCompare.java b/helix-core/src/test/java/org/apache/helix/TestHelixVersionCompare.java new file mode 100644 index 0000000000..489369ec35 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/TestHelixVersionCompare.java @@ -0,0 +1,74 @@ +package org.apache.helix; + +/* + * 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 TestHelixVersionCompare { + + @Test + public void testNullVersionCompare() { + boolean isNoLessThan = HelixManagerProperties.versionNoLessThan(null, null); + Assert.assertTrue(isNoLessThan, "Skip version compare if no version is specified"); + + isNoLessThan = HelixManagerProperties.versionNoLessThan("1.0", null); + Assert.assertTrue(isNoLessThan, "Skip version compare if no version is specified"); + + isNoLessThan = HelixManagerProperties.versionNoLessThan(null, "1.0"); + Assert.assertTrue(isNoLessThan, "Skip version compare if no version is specified"); + } + + @Test + public void testEmptyVersionCompare() { + boolean isNoLessThan = HelixManagerProperties.versionNoLessThan("", ""); + Assert.assertTrue(isNoLessThan, "Skip version compare if empty version is specified"); + + } + + @Test + public void testNonNumericalVersionCompare() { + boolean isNoLessThan = + HelixManagerProperties.versionNoLessThan("project.version1", "project.version2"); + Assert.assertTrue(isNoLessThan, "Skip version compare if non-numerical version is specified"); + + } + + @Test + public void testNumericalVersionCompare() { + boolean isNoLessThan = HelixManagerProperties.versionNoLessThan("0.7.0", "0.6.1"); + Assert.assertTrue(isNoLessThan); + + isNoLessThan = HelixManagerProperties.versionNoLessThan("0.5.31", "0.6.1"); + Assert.assertFalse(isNoLessThan); + + isNoLessThan = HelixManagerProperties.versionNoLessThan("0.5.31-SNAPSHOT", "0.6.1"); + Assert.assertFalse(isNoLessThan); + + isNoLessThan = HelixManagerProperties.versionNoLessThan("0.5.31-incubating", "0.6.1"); + Assert.assertFalse(isNoLessThan); + + isNoLessThan = HelixManagerProperties.versionNoLessThan("0.7.0", "0.6.1-SNAPSHOT"); + Assert.assertTrue(isNoLessThan); + + isNoLessThan = HelixManagerProperties.versionNoLessThan("0.7.0", "0.6.1-incubating"); + Assert.assertTrue(isNoLessThan); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/TestHelper.java b/helix-core/src/test/java/org/apache/helix/TestHelper.java index 5c4749edf0..871d717557 100644 --- a/helix-core/src/test/java/org/apache/helix/TestHelper.java +++ b/helix-core/src/test/java/org/apache/helix/TestHelper.java @@ -45,7 +45,11 @@ import org.I0Itec.zkclient.exception.ZkNoNodeException; import org.apache.commons.io.FileUtils; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.integration.manager.ZkTestManager; import org.apache.helix.manager.zk.CallbackHandler; import org.apache.helix.manager.zk.ZKHelixAdmin; @@ -60,8 +64,6 @@ import org.apache.helix.model.Message.MessageType; import org.apache.helix.model.StateModelDefinition; import org.apache.helix.model.StateModelDefinition.StateModelDefinitionProperty; -import org.apache.helix.participant.DistClusterControllerStateModelFactory; -import org.apache.helix.participant.StateMachineEngine; import org.apache.helix.store.zk.ZNode; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.util.ZKClientPool; @@ -126,7 +128,7 @@ public void createDefaultNameSpace(org.I0Itec.zkclient.ZkClient zkClient) { try { zkClient.deleteRecursive(rootNamespace); } catch (Exception e) { - LOG.error("fail to deleteRecursive path:" + rootNamespace + "\nexception:" + e); + LOG.error("fail to deleteRecursive path:" + rootNamespace, e); } } } @@ -147,90 +149,6 @@ static public void stopZkServer(ZkServer zkServer) { } } - public static StartCMResult startDummyProcess(final String zkAddr, final String clusterName, - final String instanceName) throws Exception { - StartCMResult result = new StartCMResult(); - ZkHelixTestManager manager = null; - manager = new ZkHelixTestManager(clusterName, instanceName, InstanceType.PARTICIPANT, zkAddr); - result._manager = manager; - Thread thread = new Thread(new DummyProcessThread(manager, instanceName)); - result._thread = thread; - thread.start(); - - return result; - } - - private static ZkHelixTestManager startHelixController(final String zkConnectString, - final String clusterName, final String controllerName, final String controllerMode) { - ZkHelixTestManager manager = null; - try { - if (controllerMode.equalsIgnoreCase(HelixControllerMain.STANDALONE)) { - manager = - new ZkHelixTestManager(clusterName, controllerName, InstanceType.CONTROLLER, - zkConnectString); - manager.connect(); - } else if (controllerMode.equalsIgnoreCase(HelixControllerMain.DISTRIBUTED)) { - manager = - new ZkHelixTestManager(clusterName, controllerName, - InstanceType.CONTROLLER_PARTICIPANT, zkConnectString); - - DistClusterControllerStateModelFactory stateModelFactory = - new DistClusterControllerStateModelFactory(zkConnectString); - - StateMachineEngine stateMach = manager.getStateMachineEngine(); - stateMach.registerStateModelFactory("LeaderStandby", stateModelFactory); - manager.connect(); - } else { - LOG.error("cluster controller mode:" + controllerMode + " NOT supported"); - } - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - - return manager; - } - - // TODO refactor this - public static StartCMResult startController(final String clusterName, - final String controllerName, final String zkConnectString, final String controllerMode) - throws Exception { - final StartCMResult result = new StartCMResult(); - final ZkHelixTestManager manager = - startHelixController(zkConnectString, clusterName, controllerName, controllerMode); - result._manager = manager; - - Thread thread = new Thread(new Runnable() { - @Override - public void run() { - // ClusterManager manager = null; - - try { - - Thread.currentThread().join(); - } catch (InterruptedException e) { - String msg = - "controller:" + controllerName + ", " + Thread.currentThread().getName() - + " interrupted"; - LOG.info(msg); - // System.err.println(msg); - } catch (Exception e) { - e.printStackTrace(); - } - } - }); - - thread.start(); - result._thread = thread; - return result; - } - - public static class StartCMResult { - public Thread _thread; - public ZkHelixTestManager _manager; - - } - public static void setupEmptyCluster(ZkClient zkClient, String clusterName) { ZKHelixAdmin admin = new ZKHelixAdmin(zkClient); admin.addCluster(clusterName, true); @@ -246,11 +164,6 @@ public static Set setOf(T... s) { return set; } - // public static void verifyWithTimeout(String verifierName, Object... args) - // { - // verifyWithTimeout(verifierName, 30 * 1000, args); - // } - /** * generic method for verification with a timeout * @param verifierName @@ -287,8 +200,7 @@ public static void verifyWithTimeout(String verifierName, long timeout, Object.. Assert.assertTrue(result); } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Exception in verify: " + verifierName, e); } } @@ -309,7 +221,7 @@ public static boolean verifyEmptyCurStateAndExtView(String clusterName, String r try { ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); Builder keyBuilder = accessor.keyBuilder(); for (String instanceName : instanceNames) { @@ -392,7 +304,7 @@ public static void verifyState(String clusterName, String zkAddr, try { ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); Builder keyBuilder = accessor.keyBuilder(); for (String resGroupPartitionKey : stateMap.keySet()) { @@ -494,15 +406,15 @@ public void run() { return resultsMap; } - public static Message createMessage(String msgId, String fromState, String toState, + public static Message createMessage(MessageId 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.setFromState(State.from(fromState)); + msg.setToState(State.from(toState)); msg.setTgtName(tgtName); - msg.setResourceName(resourceName); - msg.setPartitionName(partitionName); - msg.setStateModelDef("MasterSlave"); + msg.setResourceId(ResourceId.from(resourceName)); + msg.setPartitionId(PartitionId.from(partitionName)); + msg.setStateModelDef(StateModelDefId.from("MasterSlave")); return msg; } @@ -800,7 +712,7 @@ public static StateModelDefinition generateStateModelDefForBootstrap() { stateTransitionPriorityList.add("OFFLINE-IDLE"); stateTransitionPriorityList.add("IDLE-OFFLINE"); stateTransitionPriorityList.add("IDLE-DROPPED"); - stateTransitionPriorityList.add("ERROR-IDLED"); + stateTransitionPriorityList.add("ERROR-IDLE"); record.setListField(StateModelDefinitionProperty.STATE_TRANSITION_PRIORITYLIST.toString(), stateTransitionPriorityList); return new StateModelDefinition(record); diff --git a/helix-core/src/test/java/org/apache/helix/TestHierarchicalDataStore.java b/helix-core/src/test/java/org/apache/helix/TestHierarchicalDataStore.java index da6add2842..af953e9a7a 100644 --- a/helix-core/src/test/java/org/apache/helix/TestHierarchicalDataStore.java +++ b/helix-core/src/test/java/org/apache/helix/TestHierarchicalDataStore.java @@ -19,14 +19,11 @@ * under the License. */ -import org.apache.helix.ZNRecord; +import java.io.FileFilter; + import org.apache.helix.controller.HierarchicalDataHolder; import org.apache.helix.manager.zk.ZkClient; -import org.testng.annotations.Test; import org.testng.AssertJUnit; -import java.io.FileFilter; - -import org.testng.Assert; import org.testng.annotations.Test; public class TestHierarchicalDataStore extends ZkUnitTestBase { diff --git a/helix-core/src/test/java/org/apache/helix/TestParticipantHealthReportCollectorImpl.java b/helix-core/src/test/java/org/apache/helix/TestParticipantHealthReportCollectorImpl.java deleted file mode 100644 index fe4c6885cc..0000000000 --- a/helix-core/src/test/java/org/apache/helix/TestParticipantHealthReportCollectorImpl.java +++ /dev/null @@ -1,78 +0,0 @@ -package org.apache.helix; - -/* - * 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.Mocks.MockHealthReportProvider; -import org.apache.helix.Mocks.MockManager; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.healthcheck.ParticipantHealthReportTask; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -public class TestParticipantHealthReportCollectorImpl { - - protected ParticipantHealthReportCollectorImpl _providerImpl; - protected ParticipantHealthReportTask _providerTask; - protected HelixManager _manager; - protected MockHealthReportProvider _mockProvider; - - @BeforeMethod(groups = { - "unitTest" - }) - public void setup() { - _providerImpl = new ParticipantHealthReportCollectorImpl(new MockManager(), "instance_123"); - _providerTask = new ParticipantHealthReportTask(_providerImpl); - _mockProvider = new MockHealthReportProvider(); - } - - @Test(groups = { - "unitTest" - }) - public void testStart() throws Exception { - _providerTask.start(); - _providerTask.start(); - } - - @Test(groups = { - "unitTest" - }) - public void testStop() throws Exception { - _providerTask.stop(); - _providerTask.stop(); - } - - @Test(groups = { - "unitTest" - }) - public void testAddProvider() throws Exception { - _providerImpl.removeHealthReportProvider(_mockProvider); - _providerImpl.addHealthReportProvider(_mockProvider); - _providerImpl.addHealthReportProvider(_mockProvider); - } - - @Test(groups = { - "unitTest" - }) - public void testRemoveProvider() throws Exception { - _providerImpl.addHealthReportProvider(_mockProvider); - _providerImpl.removeHealthReportProvider(_mockProvider); - _providerImpl.removeHealthReportProvider(_mockProvider); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/TestPerfCounters.java b/helix-core/src/test/java/org/apache/helix/TestPerfCounters.java deleted file mode 100644 index d95bff83e4..0000000000 --- a/helix-core/src/test/java/org/apache/helix/TestPerfCounters.java +++ /dev/null @@ -1,69 +0,0 @@ -package org.apache.helix; - -/* - * 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.healthcheck.DefaultPerfCounters; -import org.testng.AssertJUnit; -import org.testng.annotations.BeforeTest; -import org.testng.annotations.Test; - -public class TestPerfCounters { - - final String INSTANCE_NAME = "instance_123"; - final long AVAILABLE_CPUS = 1; - final long FREE_PHYSICAL_MEMORY = 2; - final long FREE_JVM_MEMORY = 3; - final long TOTAL_JVM_MEMORY = 4; - final double AVERAGE_SYSTEM_LOAD = 5; - - DefaultPerfCounters _perfCounters; - - @BeforeTest() - public void setup() { - _perfCounters = - new DefaultPerfCounters(INSTANCE_NAME, AVAILABLE_CPUS, FREE_PHYSICAL_MEMORY, - FREE_JVM_MEMORY, TOTAL_JVM_MEMORY, AVERAGE_SYSTEM_LOAD); - } - - @Test() - public void testGetAvailableCpus() { - AssertJUnit.assertEquals(AVAILABLE_CPUS, _perfCounters.getAvailableCpus()); - } - - @Test() - public void testGetAverageSystemLoad() { - AssertJUnit.assertEquals(AVERAGE_SYSTEM_LOAD, _perfCounters.getAverageSystemLoad()); - } - - @Test() - public void testGetTotalJvmMemory() { - AssertJUnit.assertEquals(TOTAL_JVM_MEMORY, _perfCounters.getTotalJvmMemory()); - } - - @Test() - public void testGetFreeJvmMemory() { - AssertJUnit.assertEquals(FREE_JVM_MEMORY, _perfCounters.getFreeJvmMemory()); - } - - @Test() - public void testGetFreePhysicalMemory() { - AssertJUnit.assertEquals(FREE_PHYSICAL_MEMORY, _perfCounters.getFreePhysicalMemory()); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/TestPerformanceHealthReportProvider.java b/helix-core/src/test/java/org/apache/helix/TestPerformanceHealthReportProvider.java deleted file mode 100644 index 60e1bcb5b9..0000000000 --- a/helix-core/src/test/java/org/apache/helix/TestPerformanceHealthReportProvider.java +++ /dev/null @@ -1,160 +0,0 @@ -package org.apache.helix; - -/* - * 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 org.apache.helix.HelixDataAccessor; -import org.apache.helix.ZNRecord; -import org.apache.helix.Mocks.MockManager; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.healthcheck.PerformanceHealthReportProvider; -import org.apache.helix.model.HealthStat; -import org.testng.AssertJUnit; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -public class TestPerformanceHealthReportProvider { - - protected static final String CLUSTER_NAME = "TestCluster"; - protected final String STAT_NAME = "Stat_123"; - protected final String PARTITION_NAME = "Partition_456"; - protected final String FAKE_STAT_NAME = "Stat_ABC"; - protected final String FAKE_PARTITION_NAME = "Partition_DEF"; - protected final String STORED_STAT = "789"; - protected final String INSTANCE_NAME = "instance:1"; - - PerformanceHealthReportProvider _healthProvider; - MockManager _helixManager; - - public void incrementPartitionStat() throws Exception { - _helixManager = new MockManager(CLUSTER_NAME); - _healthProvider.incrementPartitionStat(STAT_NAME, PARTITION_NAME); - } - - public void transmitReport() throws Exception { - _helixManager = new MockManager(CLUSTER_NAME); - Map> partitionReport = - _healthProvider.getRecentPartitionHealthReport(); - ZNRecord record = new ZNRecord(_healthProvider.getReportName()); - if (partitionReport != null) { - record.setMapFields(partitionReport); - } - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - - Builder keyBuilder = accessor.keyBuilder(); - accessor.setProperty(keyBuilder.healthReport(INSTANCE_NAME, record.getId()), new HealthStat( - record)); - } - - @BeforeMethod() - public void setup() { - _healthProvider = new PerformanceHealthReportProvider(); - } - - @Test() - public void testGetRecentHealthReports() throws Exception { - _healthProvider.getRecentHealthReport(); - _healthProvider.getRecentPartitionHealthReport(); - } - - @Test() - public void testIncrementPartitionStat() throws Exception { - // stat does not exist yet - _healthProvider.incrementPartitionStat(STAT_NAME, PARTITION_NAME); - transmitReport(); - // stat does exist - _healthProvider.incrementPartitionStat(STAT_NAME, PARTITION_NAME); - transmitReport(); - String retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME); - AssertJUnit.assertEquals(2.0, Double.parseDouble(retrievedStat)); - - // set to some other value - _healthProvider.submitPartitionStat(STAT_NAME, PARTITION_NAME, STORED_STAT); - transmitReport(); - _healthProvider.incrementPartitionStat(STAT_NAME, PARTITION_NAME); - transmitReport(); - retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME); - AssertJUnit - .assertEquals(Double.parseDouble(retrievedStat), Double.parseDouble(STORED_STAT) + 1); - } - - @Test() - public void testSetGetPartitionStat() throws Exception { - _healthProvider.submitPartitionStat(STAT_NAME, PARTITION_NAME, STORED_STAT); - transmitReport(); - String retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME); - // check on correct retrieval for real stat, real partition - AssertJUnit.assertEquals(STORED_STAT, retrievedStat); - - // real stat, fake partition - retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, FAKE_PARTITION_NAME); - AssertJUnit.assertNull(retrievedStat); - - // fake stat, real partition - retrievedStat = _healthProvider.getPartitionStat(FAKE_STAT_NAME, PARTITION_NAME); - AssertJUnit.assertNull(retrievedStat); - - // fake stat, fake partition - retrievedStat = _healthProvider.getPartitionStat(FAKE_STAT_NAME, FAKE_PARTITION_NAME); - AssertJUnit.assertNull(retrievedStat); - } - - @Test() - public void testGetPartitionHealthReport() throws Exception { - // test empty map case - Map> resultMap = _healthProvider.getRecentPartitionHealthReport(); - AssertJUnit.assertEquals(resultMap.size(), 0); - - // test non-empty case - testSetGetPartitionStat(); - resultMap = _healthProvider.getRecentPartitionHealthReport(); - // check contains 1 stat - AssertJUnit.assertEquals(1, resultMap.size()); - // check contains STAT_NAME STAT - AssertJUnit.assertTrue(resultMap.keySet().contains(STAT_NAME)); - Map statMap = resultMap.get(STAT_NAME); - // check statMap has size 1 - AssertJUnit.assertEquals(1, statMap.size()); - // check contains PARTITION_NAME - AssertJUnit.assertTrue(statMap.keySet().contains(PARTITION_NAME)); - // check stored val - String statVal = statMap.get(PARTITION_NAME); - AssertJUnit.assertEquals(statVal, STORED_STAT); - } - - @Test() - public void testPartitionStatReset() throws Exception { - incrementPartitionStat(); - // ensure stat appears - String retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME); - AssertJUnit.assertEquals(1.0, Double.parseDouble(retrievedStat)); - // reset partition stats - _healthProvider.resetStats(); - transmitReport(); - retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME); - AssertJUnit.assertEquals(null, retrievedStat); - } - - @Test() - public void testGetReportName() throws Exception { - _healthProvider.getReportName(); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/TestZKCallback.java b/helix-core/src/test/java/org/apache/helix/TestZKCallback.java index 74d2987711..da438bad2d 100644 --- a/helix-core/src/test/java/org/apache/helix/TestZKCallback.java +++ b/helix-core/src/test/java/org/apache/helix/TestZKCallback.java @@ -24,19 +24,13 @@ import java.util.List; import java.util.UUID; -import org.apache.helix.ConfigChangeListener; -import org.apache.helix.CurrentStateChangeListener; -import org.apache.helix.ExternalViewChangeListener; -import org.apache.helix.HelixConstants; -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.HelixManagerFactory; -import org.apache.helix.IdealStateChangeListener; -import org.apache.helix.InstanceType; -import org.apache.helix.LiveInstanceChangeListener; -import org.apache.helix.MessageListener; -import org.apache.helix.NotificationContext; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.model.CurrentState; @@ -155,7 +149,7 @@ public void testInvocation() throws Exception { testListener.Reset(); CurrentState curState = new CurrentState("db-12345"); - curState.setSessionId("sessionId"); + curState.setSessionId(SessionId.from("sessionId")); curState.setStateModelDefRef("StateModelDef"); accessor.setProperty(keyBuilder.currentState("localhost_8900", testHelixManager.getSessionId(), curState.getId()), curState); @@ -166,7 +160,7 @@ public void testInvocation() throws Exception { IdealState idealState = new IdealState("db-1234"); idealState.setNumPartitions(400); idealState.setReplicas(Integer.toString(2)); - idealState.setStateModelDefRef("StateModeldef"); + idealState.setStateModelDefId(StateModelDefId.from("StateModeldef")); accessor.setProperty(keyBuilder.idealStates("db-1234"), idealState); Thread.sleep(100); AssertJUnit.assertTrue(testListener.idealStateChangeReceived); @@ -184,13 +178,14 @@ public void testInvocation() throws Exception { // recList.add(dummyRecord); testListener.Reset(); - Message message = new Message(MessageType.STATE_TRANSITION, UUID.randomUUID().toString()); - message.setTgtSessionId("*"); - message.setResourceName("testResource"); - message.setPartitionName("testPartitionKey"); - message.setStateModelDef("MasterSlave"); - message.setToState("toState"); - message.setFromState("fromState"); + Message message = + new Message(MessageType.STATE_TRANSITION, MessageId.from(UUID.randomUUID().toString())); + message.setTgtSessionId(SessionId.from("*")); + message.setResourceId(ResourceId.from("testResource")); + message.setPartitionId(PartitionId.from("testPartitionKey")); + message.setStateModelDef(StateModelDefId.from("MasterSlave")); + message.setToState(State.from("toState")); + message.setFromState(State.from("fromState")); message.setTgtName("testTarget"); message.setStateModelFactoryName(HelixConstants.DEFAULT_STATE_MODEL_FACTORY); diff --git a/helix-core/src/test/java/org/apache/helix/TestZKRoutingInfoProvider.java b/helix-core/src/test/java/org/apache/helix/TestZKRoutingInfoProvider.java index 314d3deed0..1d45467c7e 100644 --- a/helix-core/src/test/java/org/apache/helix/TestZKRoutingInfoProvider.java +++ b/helix-core/src/test/java/org/apache/helix/TestZKRoutingInfoProvider.java @@ -27,10 +27,9 @@ import java.util.Set; import java.util.TreeMap; -import org.apache.helix.ZNRecord; import org.apache.helix.controller.ExternalViewGenerator; -import org.apache.helix.model.Message; import org.apache.helix.model.CurrentState.CurrentStateProperty; +import org.apache.helix.model.Message; import org.testng.AssertJUnit; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/TestZNRecord.java b/helix-core/src/test/java/org/apache/helix/TestZNRecord.java index 860d1801d6..9ff48496e8 100644 --- a/helix-core/src/test/java/org/apache/helix/TestZNRecord.java +++ b/helix-core/src/test/java/org/apache/helix/TestZNRecord.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; -import org.apache.helix.ZNRecord; import org.testng.Assert; import org.testng.AssertJUnit; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/TestZNRecordBucketizer.java b/helix-core/src/test/java/org/apache/helix/TestZNRecordBucketizer.java index 654f145c73..86f3cc2a55 100644 --- a/helix-core/src/test/java/org/apache/helix/TestZNRecordBucketizer.java +++ b/helix-core/src/test/java/org/apache/helix/TestZNRecordBucketizer.java @@ -19,7 +19,6 @@ * under the License. */ -import org.apache.helix.ZNRecordBucketizer; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/TestZkBasis.java b/helix-core/src/test/java/org/apache/helix/TestZkBasis.java index e8d7549991..8b315d8194 100644 --- a/helix-core/src/test/java/org/apache/helix/TestZkBasis.java +++ b/helix-core/src/test/java/org/apache/helix/TestZkBasis.java @@ -19,16 +19,6 @@ * under the License. */ -import org.I0Itec.zkclient.IZkChildListener; -import org.I0Itec.zkclient.IZkDataListener; -import org.apache.helix.TestHelper; -import org.apache.helix.ZkTestHelper; -import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.manager.zk.ZNRecordSerializer; -import org.apache.helix.manager.zk.ZkClient; -import org.testng.Assert; -import org.testng.annotations.Test; - import java.util.Collections; import java.util.List; import java.util.Map; @@ -36,6 +26,13 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import org.I0Itec.zkclient.IZkChildListener; +import org.I0Itec.zkclient.IZkDataListener; +import org.apache.helix.manager.zk.ZNRecordSerializer; +import org.apache.helix.manager.zk.ZkClient; +import org.testng.Assert; +import org.testng.annotations.Test; + /** * test zookeeper basis */ diff --git a/helix-core/src/test/java/org/apache/helix/TestZkClientWrapper.java b/helix-core/src/test/java/org/apache/helix/TestZkClientWrapper.java index f3b0ac9bff..0decbd824b 100644 --- a/helix-core/src/test/java/org/apache/helix/TestZkClientWrapper.java +++ b/helix-core/src/test/java/org/apache/helix/TestZkClientWrapper.java @@ -21,9 +21,9 @@ import org.I0Itec.zkclient.IZkStateListener; import org.I0Itec.zkclient.ZkConnection; -import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkClient; +import org.apache.log4j.Logger; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher.Event.KeeperState; @@ -35,6 +35,8 @@ import org.testng.annotations.Test; public class TestZkClientWrapper extends ZkUnitTestBase { + private static Logger LOG = Logger.getLogger(TestZkClientWrapper.class); + ZkClient _zkClient; @BeforeClass @@ -70,7 +72,7 @@ void testGetStat() { } @Test() - void testSessioExpire() { + void testSessioExpire() throws Exception { IZkStateListener listener = new IZkStateListener() { @Override @@ -83,31 +85,27 @@ public void handleNewSession() throws Exception { System.out.println("In Old connection New session"); } }; + _zkClient.subscribeStateChanges(listener); ZkConnection connection = ((ZkConnection) _zkClient.getConnection()); ZooKeeper zookeeper = connection.getZookeeper(); System.out.println("old sessionId= " + zookeeper.getSessionId()); - try { - Watcher watcher = new Watcher() { - @Override - public void process(WatchedEvent event) { - System.out.println("In New connection In process event:" + event); - } - }; - ZooKeeper newZookeeper = - new ZooKeeper(connection.getServers(), zookeeper.getSessionTimeout(), watcher, - zookeeper.getSessionId(), zookeeper.getSessionPasswd()); - Thread.sleep(3000); - System.out.println("New sessionId= " + newZookeeper.getSessionId()); - Thread.sleep(3000); - newZookeeper.close(); - Thread.sleep(10000); - connection = ((ZkConnection) _zkClient.getConnection()); - zookeeper = connection.getZookeeper(); - System.out.println("After session expiry sessionId= " + zookeeper.getSessionId()); - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } + Watcher watcher = new Watcher() { + @Override + public void process(WatchedEvent event) { + System.out.println("In New connection In process event:" + event); + } + }; + ZooKeeper newZookeeper = + new ZooKeeper(connection.getServers(), zookeeper.getSessionTimeout(), watcher, + zookeeper.getSessionId(), zookeeper.getSessionPasswd()); + Thread.sleep(3000); + System.out.println("New sessionId= " + newZookeeper.getSessionId()); + Thread.sleep(3000); + newZookeeper.close(); + Thread.sleep(10000); + connection = ((ZkConnection) _zkClient.getConnection()); + zookeeper = connection.getZookeeper(); + System.out.println("After session expiry sessionId= " + zookeeper.getSessionId()); } } diff --git a/helix-core/src/test/java/org/apache/helix/TestZnodeModify.java b/helix-core/src/test/java/org/apache/helix/TestZnodeModify.java index 7d66ebdf80..fdf6e72a76 100644 --- a/helix-core/src/test/java/org/apache/helix/TestZnodeModify.java +++ b/helix-core/src/test/java/org/apache/helix/TestZnodeModify.java @@ -25,17 +25,16 @@ import java.util.List; import java.util.Map; -import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.model.IdealState.IdealStateProperty; import org.apache.helix.model.IdealState.RebalanceMode; import org.apache.helix.tools.TestCommand; +import org.apache.helix.tools.TestCommand.CommandType; import org.apache.helix.tools.TestExecutor; +import org.apache.helix.tools.TestExecutor.ZnodePropertyType; import org.apache.helix.tools.TestTrigger; import org.apache.helix.tools.ZnodeOpArg; -import org.apache.helix.tools.TestCommand.CommandType; -import org.apache.helix.tools.TestExecutor.ZnodePropertyType; import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.AssertJUnit; @@ -216,8 +215,7 @@ public void run() { zkClient.createPersistent(pathChild1, true); zkClient.writeData(pathChild1, record); } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); + logger.error("Interrupted sleep", e); } } }.start(); diff --git a/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java b/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java index db2a6d031f..7aa2351779 100644 --- a/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java +++ b/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java @@ -20,10 +20,17 @@ */ import java.io.BufferedReader; +import java.io.IOException; import java.io.InputStreamReader; import java.io.PrintWriter; import java.net.Socket; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; @@ -31,21 +38,19 @@ import org.I0Itec.zkclient.IZkDataListener; import org.I0Itec.zkclient.IZkStateListener; import org.I0Itec.zkclient.ZkConnection; -import org.apache.helix.InstanceType; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.manager.zk.CallbackHandler; 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.manager.zk.ZkClient; import org.apache.helix.model.ExternalView; import org.apache.log4j.Logger; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.Event.KeeperState; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooKeeper.States; +import org.testng.Assert; public class ZkTestHelper { private static Logger LOG = Logger.getLogger(ZkTestHelper.class); @@ -102,13 +107,37 @@ public void process(WatchedEvent event) { LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId())); } + /** + * Simulate a zk state change by calling {@link ZkClient#process(WatchedEvent)} directly + */ + public static void simulateZkStateDisconnected(ZkClient client) { + WatchedEvent event = new WatchedEvent(EventType.None, KeeperState.Disconnected, null); + client.process(event); + } + + /** + * Get zk connection session id + * @param client + * @return + */ + public static String getSessionId(ZkClient client) { + ZkConnection connection = ((ZkConnection) client.getConnection()); + ZooKeeper curZookeeper = connection.getZookeeper(); + return Long.toHexString(curZookeeper.getSessionId()); + } + + /** + * Expire current zk session and wait for {@link IZkStateListener#handleNewSession()} invoked + * @param zkClient + * @throws Exception + */ public static void expireSession(final ZkClient zkClient) throws Exception { - final CountDownLatch waitExpire = new CountDownLatch(1); + final CountDownLatch waitNewSession = new CountDownLatch(1); IZkStateListener listener = new IZkStateListener() { @Override public void handleStateChanged(KeeperState state) throws Exception { - // System.err.println("handleStateChanged. state: " + state); + LOG.info("IZkStateListener#handleStateChanged, state: " + state); } @Override @@ -120,7 +149,7 @@ public void handleNewSession() throws Exception { ZooKeeper curZookeeper = connection.getZookeeper(); LOG.info("handleNewSession. sessionId: " + Long.toHexString(curZookeeper.getSessionId())); - waitExpire.countDown(); + waitNewSession.countDown(); } }; @@ -128,12 +157,13 @@ public void handleNewSession() throws Exception { ZkConnection connection = ((ZkConnection) zkClient.getConnection()); ZooKeeper curZookeeper = connection.getZookeeper(); - LOG.info("Before expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId())); + String oldSessionId = Long.toHexString(curZookeeper.getSessionId()); + LOG.info("Before session expiry. sessionId: " + oldSessionId + ", zk: " + curZookeeper); Watcher watcher = new Watcher() { @Override public void process(WatchedEvent event) { - LOG.info("Process watchEvent: " + event); + LOG.info("Watcher#process, event: " + event); } }; @@ -144,17 +174,19 @@ public void process(WatchedEvent event) { while (dupZookeeper.getState() != States.CONNECTED) { Thread.sleep(10); } + Assert.assertEquals(dupZookeeper.getState(), States.CONNECTED, "Fail to connect to zk using current session info"); dupZookeeper.close(); // make sure session expiry really happens - waitExpire.await(); + waitNewSession.await(); zkClient.unsubscribeStateChanges(listener); connection = (ZkConnection) zkClient.getConnection(); curZookeeper = connection.getZookeeper(); - // System.err.println("zk: " + oldZookeeper); - LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId())); + String newSessionId = Long.toHexString(curZookeeper.getSessionId()); + LOG.info("After session expiry. sessionId: " + newSessionId + ", zk: " + curZookeeper); + Assert.assertNotSame(newSessionId, oldSessionId, "Fail to expire current session, zk: " + curZookeeper); } /** @@ -323,12 +355,12 @@ public static Map> getListenersBySession(String zkAddr) thro return listenerMapBySession; } - static java.lang.reflect.Field getField(Class clazz, String fieldName) + static java.lang.reflect.Field getField(Class clazz, String fieldName) throws NoSuchFieldException { try { return clazz.getDeclaredField(fieldName); } catch (NoSuchFieldException e) { - Class superClass = clazz.getSuperclass(); + Class superClass = clazz.getSuperclass(); if (superClass == null) { throw e; } else { diff --git a/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java index abf75becdd..33968f7806 100644 --- a/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java +++ b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java @@ -28,6 +28,10 @@ import org.I0Itec.zkclient.ZkConnection; import org.I0Itec.zkclient.ZkServer; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.pipeline.Pipeline; import org.apache.helix.controller.pipeline.Stage; import org.apache.helix.controller.pipeline.StageContext; @@ -43,9 +47,9 @@ import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; -import org.apache.helix.model.StateModelDefinition; import org.apache.helix.model.Message.Attributes; import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.StateModelDefinition; import org.apache.helix.tools.StateModelConfigGenerator; import org.apache.helix.util.HelixUtil; import org.apache.log4j.Logger; @@ -164,7 +168,7 @@ public void verifyResource(ZkClient zkClient, String clusterName, String resourc public void verifyEnabled(ZkClient zkClient, String clusterName, String instance, boolean wantEnabled) { ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); Builder keyBuilder = accessor.keyBuilder(); InstanceConfig config = accessor.getProperty(keyBuilder.instanceConfig(instance)); @@ -173,15 +177,15 @@ public void verifyEnabled(ZkClient zkClient, String clusterName, String instance public void verifyReplication(ZkClient zkClient, String clusterName, String resource, int repl) { ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); Builder keyBuilder = accessor.keyBuilder(); IdealState idealState = accessor.getProperty(keyBuilder.idealStates(resource)); - for (String partitionName : idealState.getPartitionSet()) { + for (PartitionId partitionId : idealState.getPartitionIdSet()) { if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO) { - AssertJUnit.assertEquals(repl, idealState.getPreferenceList(partitionName).size()); + AssertJUnit.assertEquals(repl, idealState.getPreferenceList(partitionId).size()); } else if (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED) { - AssertJUnit.assertEquals(repl, idealState.getInstanceStateMap(partitionName).size()); + AssertJUnit.assertEquals(repl, idealState.getParticipantStateMap(partitionId).size()); } } } @@ -247,20 +251,19 @@ public void process(WatchedEvent event) { protected void setupStateModel(String clusterName) { ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); - StateModelConfigGenerator generator = new StateModelConfigGenerator(); StateModelDefinition masterSlave = - new StateModelDefinition(generator.generateConfigForMasterSlave()); + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); accessor.setProperty(keyBuilder.stateModelDef(masterSlave.getId()), masterSlave); StateModelDefinition leaderStandby = - new StateModelDefinition(generator.generateConfigForLeaderStandby()); + new StateModelDefinition(StateModelConfigGenerator.generateConfigForLeaderStandby()); accessor.setProperty(keyBuilder.stateModelDef(leaderStandby.getId()), leaderStandby); StateModelDefinition onlineOffline = - new StateModelDefinition(generator.generateConfigForOnlineOffline()); + new StateModelDefinition(StateModelConfigGenerator.generateConfigForOnlineOffline()); accessor.setProperty(keyBuilder.stateModelDef(onlineOffline.getId()), onlineOffline); } @@ -268,7 +271,7 @@ protected void setupStateModel(String clusterName) { protected List setupIdealState(String clusterName, int[] nodes, String[] resources, int partitions, int replicas) { ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); List idealStates = new ArrayList(); @@ -289,7 +292,7 @@ protected List setupIdealState(String clusterName, int[] nodes, Stri } idealState.setReplicas(Integer.toString(replicas)); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setNumPartitions(partitions); idealStates.add(idealState); @@ -302,14 +305,14 @@ protected List setupIdealState(String clusterName, int[] nodes, Stri protected void setupLiveInstances(String clusterName, int[] liveInstances) { ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); for (int i = 0; i < liveInstances.length; i++) { String instance = "localhost_" + liveInstances[i]; LiveInstance liveInstance = new LiveInstance(instance); liveInstance.setSessionId("session_" + liveInstances[i]); - liveInstance.setHelixVersion("0.0.0"); + liveInstance.setHelixVersion("0.4.0"); accessor.setProperty(keyBuilder.liveInstance(instance), liveInstance); } } @@ -344,11 +347,11 @@ protected void runStage(ClusterEvent event, Stage stage) throws Exception { stage.postProcess(); } - protected Message createMessage(MessageType type, String msgId, String fromState, String toState, - String resourceName, String tgtName) { + protected Message createMessage(MessageType type, MessageId msgId, String fromState, + String toState, String resourceName, String tgtName) { Message msg = new Message(type.toString(), msgId); - msg.setFromState(fromState); - msg.setToState(toState); + msg.setFromState(State.from(fromState)); + msg.setToState(State.from(toState)); msg.getRecord().setSimpleField(Attributes.RESOURCE_NAME.toString(), resourceName); msg.setTgtName(tgtName); return msg; diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestAddAlerts.java b/helix-core/src/test/java/org/apache/helix/alerts/TestAddAlerts.java deleted file mode 100644 index 4263ddca20..0000000000 --- a/helix-core/src/test/java/org/apache/helix/alerts/TestAddAlerts.java +++ /dev/null @@ -1,116 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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 org.apache.helix.HelixDataAccessor; -import org.apache.helix.ZNRecord; -import org.apache.helix.Mocks.MockManager; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.alerts.AlertParser; -import org.apache.helix.alerts.AlertsHolder; -import org.apache.helix.controller.stages.HealthDataCache; -import org.testng.AssertJUnit; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -public class TestAddAlerts { - - protected static final String CLUSTER_NAME = "TestCluster"; - - MockManager _helixManager; - AlertsHolder _alertsHolder; - - public final String EXP = AlertParser.EXPRESSION_NAME; - public final String CMP = AlertParser.COMPARATOR_NAME; - public final String CON = AlertParser.CONSTANT_NAME; - - @BeforeMethod() - public void setup() { - _helixManager = new MockManager(CLUSTER_NAME); - _alertsHolder = new AlertsHolder(_helixManager, new HealthDataCache()); - } - - public boolean alertRecordContains(ZNRecord rec, String alertName) { - Map> alerts = rec.getMapFields(); - return alerts.containsKey(alertName); - } - - public int alertsSize(ZNRecord rec) { - Map> alerts = rec.getMapFields(); - return alerts.size(); - } - - @Test() - public void testAddAlert() throws Exception { - String alert = - EXP + "(accumulate()(dbFoo.partition10.latency))" + CMP + "(GREATER)" + CON + "(10)"; - _alertsHolder.addAlert(alert); - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.alerts()).getRecord(); - System.out.println("alert: " + alert); - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(alertRecordContains(rec, alert)); - AssertJUnit.assertEquals(1, alertsSize(rec)); - } - - @Test() - public void testAddTwoAlerts() throws Exception { - String alert1 = - EXP + "(accumulate()(dbFoo.partition10.latency))" + CMP + "(GREATER)" + CON + "(10)"; - String alert2 = - EXP + "(accumulate()(dbFoo.partition10.latency))" + CMP + "(GREATER)" + CON + "(100)"; - _alertsHolder.addAlert(alert1); - _alertsHolder.addAlert(alert2); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.alerts()).getRecord(); - // System.out.println("alert: "+alert1); - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(alertRecordContains(rec, alert1)); - AssertJUnit.assertTrue(alertRecordContains(rec, alert2)); - AssertJUnit.assertEquals(2, alertsSize(rec)); - } - - @Test(groups = { - "unitTest" - }) - public void testAddTwoWildcardAlert() throws Exception { - String alert1 = - EXP + "(accumulate()(dbFoo.partition*.put*))" + CMP + "(GREATER)" + CON + "(10)"; - _alertsHolder.addAlert(alert1); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.alerts()).getRecord(); - // System.out.println("alert: "+alert1); - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(alertRecordContains(rec, alert1)); - AssertJUnit.assertEquals(1, alertsSize(rec)); - } - - // add 2 wildcard alert here -} diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestAddPersistentStats.java b/helix-core/src/test/java/org/apache/helix/alerts/TestAddPersistentStats.java deleted file mode 100644 index e9e5e32e5b..0000000000 --- a/helix-core/src/test/java/org/apache/helix/alerts/TestAddPersistentStats.java +++ /dev/null @@ -1,211 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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 org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixException; -import org.apache.helix.ZNRecord; -import org.apache.helix.Mocks.MockManager; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.alerts.StatsHolder; -import org.apache.helix.controller.stages.HealthDataCache; -import org.testng.AssertJUnit; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -public class TestAddPersistentStats { - - protected static final String CLUSTER_NAME = "TestCluster"; - - MockManager _helixManager; - StatsHolder _statsHolder; - - @BeforeMethod(groups = { - "unitTest" - }) - public void setup() { - _helixManager = new MockManager(CLUSTER_NAME); - _statsHolder = new StatsHolder(_helixManager, new HealthDataCache()); - } - - public boolean statRecordContains(ZNRecord rec, String statName) { - Map> stats = rec.getMapFields(); - return stats.containsKey(statName); - } - - public int statsSize(ZNRecord rec) { - Map> stats = rec.getMapFields(); - return stats.size(); - } - - @Test(groups = { - "unitTest" - }) - public void testAddStat() throws Exception { - String stat = "window(5)(dbFoo.partition10.latency)"; - _statsHolder.addStat(stat); - _statsHolder.persistStats(); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordContains(rec, stat)); - AssertJUnit.assertEquals(1, statsSize(rec)); - } - - @Test(groups = { - "unitTest" - }) - public void testAddTwoStats() throws Exception { - String stat1 = "window(5)(dbFoo.partition10.latency)"; - _statsHolder.addStat(stat1); - _statsHolder.persistStats(); - String stat2 = "window(5)(dbFoo.partition11.latency)"; - _statsHolder.addStat(stat2); - _statsHolder.persistStats(); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordContains(rec, stat1)); - AssertJUnit.assertTrue(statRecordContains(rec, stat2)); - AssertJUnit.assertEquals(2, statsSize(rec)); - } - - @Test(groups = { - "unitTest" - }) - public void testAddDuplicateStat() throws Exception { - String stat = "window(5)(dbFoo.partition10.latency)"; - _statsHolder.addStat(stat); - _statsHolder.addStat(stat); - _statsHolder.persistStats(); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordContains(rec, stat)); - AssertJUnit.assertEquals(1, statsSize(rec)); - } - - @Test(groups = { - "unitTest" - }) - public void testAddPairOfStats() throws Exception { - String exp = "accumulate()(dbFoo.partition10.latency, dbFoo.partition10.count)"; - _statsHolder.addStat(exp); - _statsHolder.persistStats(); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordContains(rec, "accumulate()(dbFoo.partition10.latency)")); - AssertJUnit.assertTrue(statRecordContains(rec, "accumulate()(dbFoo.partition10.count)")); - AssertJUnit.assertEquals(2, statsSize(rec)); - } - - @Test(groups = { - "unitTest" - }) - public void testAddStatsWithOperators() throws Exception { - String exp = - "accumulate()(dbFoo.partition10.latency, dbFoo.partition10.count)|EACH|ACCUMULATE|DIVIDE"; - _statsHolder.addStat(exp); - _statsHolder.persistStats(); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordContains(rec, "accumulate()(dbFoo.partition10.latency)")); - AssertJUnit.assertTrue(statRecordContains(rec, "accumulate()(dbFoo.partition10.count)")); - AssertJUnit.assertEquals(2, statsSize(rec)); - } - - @Test(groups = { - "unitTest" - }) - public void testAddNonExistentAggregator() throws Exception { - String exp = "fakeagg()(dbFoo.partition10.latency)"; - boolean caughtException = false; - try { - _statsHolder.addStat(exp); - } catch (HelixException e) { - caughtException = true; - } - AssertJUnit.assertTrue(caughtException); - } - - @Test(groups = { - "unitTest" - }) - public void testGoodAggregatorBadArgs() throws Exception { - String exp = "accumulate(10)(dbFoo.partition10.latency)"; - boolean caughtException = false; - try { - _statsHolder.addStat(exp); - } catch (HelixException e) { - caughtException = true; - } - AssertJUnit.assertTrue(caughtException); - } - - @Test(groups = { - "unitTest" - }) - public void testAddBadNestingStat1() throws Exception { - String exp = "window((5)(dbFoo.partition10.latency)"; - boolean caughtException = false; - try { - _statsHolder.addStat(exp); - } catch (HelixException e) { - caughtException = true; - } - AssertJUnit.assertTrue(caughtException); - } - - @Test(groups = { - "unitTest" - }) - public void testAddBadNestingStat2() throws Exception { - String exp = "window(5)(dbFoo.partition10.latency))"; - boolean caughtException = false; - try { - _statsHolder.addStat(exp); - } catch (HelixException e) { - caughtException = true; - } - AssertJUnit.assertTrue(caughtException); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestAlertValidation.java b/helix-core/src/test/java/org/apache/helix/alerts/TestAlertValidation.java deleted file mode 100644 index 09a9d93530..0000000000 --- a/helix-core/src/test/java/org/apache/helix/alerts/TestAlertValidation.java +++ /dev/null @@ -1,169 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.HelixException; -import org.apache.helix.alerts.AlertParser; -import org.testng.annotations.Test; -import org.testng.AssertJUnit; -import org.testng.Assert; -import org.testng.annotations.Test; - -@Test -public class TestAlertValidation { - - public final String EXP = AlertParser.EXPRESSION_NAME; - public final String CMP = AlertParser.COMPARATOR_NAME; - public final String CON = AlertParser.CONSTANT_NAME; - - @Test - public void testSimple() { - String alertName = - EXP + "(accumulate()(dbFoo.partition10.latency)) " + CMP + "(GREATER) " + CON + "(10)"; - boolean caughtException = false; - try { - AlertParser.validateAlert(alertName); - } catch (HelixException e) { - caughtException = true; - e.printStackTrace(); - } - AssertJUnit.assertFalse(caughtException); - } - - @Test - public void testSingleInSingleOut() { - String alertName = - EXP + "(accumulate()(dbFoo.partition10.latency)|EXPAND) " + CMP + "(GREATER) " + CON - + "(10)"; - boolean caughtException = false; - try { - AlertParser.validateAlert(alertName); - } catch (HelixException e) { - caughtException = true; - e.printStackTrace(); - } - AssertJUnit.assertFalse(caughtException); - } - - @Test - public void testDoubleInDoubleOut() { - String alertName = - EXP + "(accumulate()(dbFoo.partition10.latency, dbFoo.partition11.latency)|EXPAND) " + CMP - + "(GREATER) " + CON + "(10)"; - boolean caughtException = false; - try { - AlertParser.validateAlert(alertName); - } catch (HelixException e) { - caughtException = true; - e.printStackTrace(); - } - AssertJUnit.assertTrue(caughtException); - } - - @Test - public void testTwoStageOps() { - String alertName = - EXP + "(accumulate()(dbFoo.partition*.latency, dbFoo.partition*.count)|EXPAND|DIVIDE) " - + CMP + "(GREATER) " + CON + "(10)"; - boolean caughtException = false; - try { - AlertParser.validateAlert(alertName); - } catch (HelixException e) { - caughtException = true; - e.printStackTrace(); - } - AssertJUnit.assertFalse(caughtException); - } - - @Test - public void testTwoListsIntoOne() { - String alertName = - EXP + "(accumulate()(dbFoo.partition10.latency, dbFoo.partition11.count)|SUM) " + CMP - + "(GREATER) " + CON + "(10)"; - boolean caughtException = false; - try { - AlertParser.validateAlert(alertName); - } catch (HelixException e) { - caughtException = true; - e.printStackTrace(); - } - AssertJUnit.assertFalse(caughtException); - } - - @Test - public void testSumEach() { - String alertName = - EXP - + "(accumulate()(dbFoo.partition*.latency, dbFoo.partition*.count)|EXPAND|SUMEACH|DIVIDE) " - + CMP + "(GREATER) " + CON + "(10)"; - boolean caughtException = false; - try { - AlertParser.validateAlert(alertName); - } catch (HelixException e) { - caughtException = true; - e.printStackTrace(); - } - AssertJUnit.assertFalse(caughtException); - } - - @Test - public void testNeedTwoTuplesGetOne() { - String alertName = - EXP + "(accumulate()(dbFoo.partition*.latency)|EXPAND|DIVIDE) " + CMP + "(GREATER) " + CON - + "(10)"; - boolean caughtException = false; - try { - AlertParser.validateAlert(alertName); - } catch (HelixException e) { - caughtException = true; - e.printStackTrace(); - } - AssertJUnit.assertTrue(caughtException); - } - - @Test - public void testExtraPipe() { - String alertName = - EXP + "(accumulate()(dbFoo.partition10.latency)|) " + CMP + "(GREATER) " + CON + "(10)"; - boolean caughtException = false; - try { - AlertParser.validateAlert(alertName); - } catch (HelixException e) { - caughtException = true; - e.printStackTrace(); - } - AssertJUnit.assertTrue(caughtException); - } - - @Test - public void testAlertUnknownOp() { - String alertName = - EXP + "(accumulate()(dbFoo.partition10.latency)|BADOP) " + CMP + "(GREATER) " + CON - + "(10)"; - boolean caughtException = false; - try { - AlertParser.validateAlert(alertName); - } catch (HelixException e) { - caughtException = true; - e.printStackTrace(); - } - AssertJUnit.assertTrue(caughtException); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestArrivingParticipantStats.java b/helix-core/src/test/java/org/apache/helix/alerts/TestArrivingParticipantStats.java deleted file mode 100644 index 848ef5abe9..0000000000 --- a/helix-core/src/test/java/org/apache/helix/alerts/TestArrivingParticipantStats.java +++ /dev/null @@ -1,498 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.apache.helix.HelixDataAccessor; -import org.apache.helix.ZNRecord; -import org.apache.helix.Mocks.MockManager; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.alerts.StatsHolder; -import org.apache.helix.alerts.Tuple; -import org.apache.helix.controller.stages.HealthDataCache; -import org.testng.AssertJUnit; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -public class TestArrivingParticipantStats { - protected static final String CLUSTER_NAME = "TestCluster"; - - MockManager _helixManager; - StatsHolder _statsHolder; - - @BeforeMethod(groups = { - "unitTest" - }) - public void setup() { - _helixManager = new MockManager(CLUSTER_NAME); - _statsHolder = new StatsHolder(_helixManager, new HealthDataCache()); - } - - public Map getStatFields(String value, String timestamp) { - Map statMap = new HashMap(); - statMap.put(StatsHolder.VALUE_NAME, value); - statMap.put(StatsHolder.TIMESTAMP_NAME, timestamp); - return statMap; - } - - public boolean statRecordContains(ZNRecord rec, String statName) { - Map> stats = rec.getMapFields(); - return stats.containsKey(statName); - } - - public boolean statRecordHasValue(ZNRecord rec, String statName, String value) { - Map> stats = rec.getMapFields(); - Map statFields = stats.get(statName); - return (statFields.get(StatsHolder.VALUE_NAME).equals(value)); - } - - public boolean statRecordHasTimestamp(ZNRecord rec, String statName, String timestamp) { - Map> stats = rec.getMapFields(); - Map statFields = stats.get(statName); - return (statFields.get(StatsHolder.TIMESTAMP_NAME).equals(timestamp)); - } - - // Exact matching persistent stat, but has no values yet - @Test(groups = { - "unitTest" - }) - public void testAddFirstParticipantStat() throws Exception { - // add a persistent stat - String persistentStat = "accumulate()(dbFoo.partition10.latency)"; - _statsHolder.addStat(persistentStat); - - // generate incoming stat - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("0", "0"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0")); - } - - // Exact matching persistent stat, but has no values yet - @Test(groups = { - "unitTest" - }) - public void testAddRepeatParticipantStat() throws Exception { - // add a persistent stat - String persistentStat = "accumulate()(dbFoo.partition10.latency)"; - _statsHolder.addStat(persistentStat); - - // generate incoming stat - String incomingStatName = "dbFoo.partition10.latency"; - // apply stat once and then again - Map statFields = getStatFields("0", "0"); - _statsHolder.applyStat(incomingStatName, statFields); - statFields = getStatFields("1", "10"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "1.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0")); - } - - // test to ensure backdated stats not applied - @Test(groups = { - "unitTest" - }) - public void testBackdatedParticipantStat() throws Exception { - // add a persistent stat - String persistentStat = "accumulate()(dbFoo.partition10.latency)"; - _statsHolder.addStat(persistentStat); - - // generate incoming stat - String incomingStatName = "dbFoo.partition10.latency"; - // apply stat once and then again - Map statFields = getStatFields("0", "0"); - _statsHolder.applyStat(incomingStatName, statFields); - statFields = getStatFields("1", "10"); - _statsHolder.applyStat(incomingStatName, statFields); - statFields = getStatFields("5", "15"); - _statsHolder.applyStat(incomingStatName, statFields); - statFields = getStatFields("1", "10"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "6.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "15.0")); - } - - // Exact matching persistent stat, but has no values yet - @Test(groups = { - "unitTest" - }) - public void testAddFirstParticipantStatToWildCard() throws Exception { - // add a persistent stat - String persistentWildcardStat = "accumulate()(dbFoo.partition*.latency)"; - _statsHolder.addStat(persistentWildcardStat); - - // generate incoming stat - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("0", "0"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - String persistentStat = "accumulate()(dbFoo.partition10.latency)"; - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0")); - } - - // test to add 2nd report to same stat - @Test(groups = { - "unitTest" - }) - public void testAddSecondParticipantStatToWildCard() throws Exception { - // add a persistent stat - String persistentWildcardStat = "accumulate()(dbFoo.partition*.latency)"; - _statsHolder.addStat(persistentWildcardStat); - - // generate incoming stat - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("1", "0"); - _statsHolder.applyStat(incomingStatName, statFields); - statFields = getStatFields("1", "10"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - String persistentStat = "accumulate()(dbFoo.partition10.latency)"; - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "2.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0")); - } - - // Exact matching persistent stat, but has no values yet - @Test(groups = { - "unitTest" - }) - public void testAddParticipantStatToDoubleWildCard() throws Exception { - // add a persistent stat - String persistentWildcardStat = "accumulate()(db*.partition*.latency)"; - _statsHolder.addStat(persistentWildcardStat); - - // generate incoming stat - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("0", "0"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - String persistentStat = "accumulate()(dbFoo.partition10.latency)"; - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0")); - } - - @Test(groups = { - "unitTest" - }) - public void testAddWildcardInFirstStatToken() throws Exception { - String persistentWildcardStat = "accumulate()(instance*.reportingage)"; - _statsHolder.addStat(persistentWildcardStat); - - // generate incoming stat - String incomingStatName = "instance10.reportingage"; - Map statFields = getStatFields("1", "10"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - String persistentStat = "accumulate()(instance10.reportingage)"; - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "1.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0")); - - } - - // test to add report to same wildcard stat, different actual stat - @Test(groups = { - "unitTest" - }) - public void testAddTwoDistinctParticipantStatsToSameWildCard() throws Exception { - // add a persistent stat - String persistentWildcardStat = "accumulate()(dbFoo.partition*.latency)"; - _statsHolder.addStat(persistentWildcardStat); - - // generate incoming stat - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("1", "10"); - _statsHolder.applyStat(incomingStatName, statFields); - incomingStatName = "dbFoo.partition11.latency"; - statFields = getStatFields("5", "10"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - String persistentStat = "accumulate()(dbFoo.partition10.latency)"; - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "1.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0")); - persistentStat = "accumulate()(dbFoo.partition11.latency)"; - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "5.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0")); - } - - // Exact matching persistent stat, but has no values yet - @Test(groups = { - "unitTest" - }) - public void testWindowStat() throws Exception { - // add a persistent stat - String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)"; - _statsHolder.addStat(persistentWildcardStat); - - // generate incoming stat - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("0", "0"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - String persistentStat = "window(3)(dbFoo.partition10.latency)"; - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0")); - - // add 2nd stat - statFields = getStatFields("10", "1"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0,10.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0,1.0")); - - // add 3rd stat - statFields = getStatFields("20", "2"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0,10.0,20.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0,1.0,2.0")); - - } - - @Test(groups = { - "unitTest" - }) - public void testWindowStatExpiration() throws Exception { - String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)"; - String persistentStat = "window(3)(dbFoo.partition10.latency)"; - // init with 3 elements - testWindowStat(); - - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("30", "3"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "10.0,20.0,30.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "1.0,2.0,3.0")); - } - - @Test(groups = { - "unitTest" - }) - public void testWindowStatStale() throws Exception { - String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)"; - String persistentStat = "window(3)(dbFoo.partition10.latency)"; - // init with 3 elements - testWindowStat(); - - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("10", "1"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0,10.0,20.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0,1.0,2.0")); - } - - // test that has 2 agg stats for same raw stat - // Exact matching persistent stat, but has no values yet - @Test(groups = { - "unitTest" - }) - public void testAddStatForTwoAggTypes() throws Exception { - // add a persistent stat - String persistentStatOne = "accumulate()(dbFoo.partition10.latency)"; - String persistentStatTwo = "window(3)(dbFoo.partition10.latency)"; - _statsHolder.addStat(persistentStatOne); - _statsHolder.persistStats(); - _statsHolder.addStat(persistentStatTwo); - _statsHolder.persistStats(); - - // generate incoming stat - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("0", "0"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStatOne, "0.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStatOne, "0.0")); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStatTwo, "0.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStatTwo, "0.0")); - } - - // test merging 2 window stats, new is applied - @Test(groups = { - "unitTest" - }) - public void testMergeTwoWindowsYesMerge() throws Exception { - String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)"; - String persistentStat = "window(3)(dbFoo.partition10.latency)"; - String incomingStatName = "dbFoo.partition10.latency"; - // init with 3 elements - testWindowStat(); - - // create a two tuples, value and time - Tuple valTuple = new Tuple(); - Tuple timeTuple = new Tuple(); - valTuple.add("30.0"); - valTuple.add("40.0"); - timeTuple.add("3.0"); - timeTuple.add("4.0"); - Map statFields = getStatFields(valTuple.toString(), timeTuple.toString()); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "20.0,30.0,40.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "2.0,3.0,4.0")); - } - - // test merging 2 window stats, new is ignored - @Test(groups = { - "unitTest" - }) - public void testMergeTwoWindowsNoMerge() throws Exception { - String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)"; - String persistentStat = "window(3)(dbFoo.partition10.latency)"; - String incomingStatName = "dbFoo.partition10.latency"; - // init with 3 elements - testWindowStat(); - - // create a two tuples, value and time - Tuple valTuple = new Tuple(); - Tuple timeTuple = new Tuple(); - valTuple.add("0.0"); - valTuple.add("40.0"); - timeTuple.add("0.0"); - timeTuple.add("4.0"); - Map statFields = getStatFields(valTuple.toString(), timeTuple.toString()); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - // check persistent stats - HelixDataAccessor accessor = _helixManager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord(); - System.out.println("rec: " + rec.toString()); - AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0,10.0,20.0")); - AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0,1.0,2.0")); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestBaseStatsValidation.java b/helix-core/src/test/java/org/apache/helix/alerts/TestBaseStatsValidation.java deleted file mode 100644 index 51831822d3..0000000000 --- a/helix-core/src/test/java/org/apache/helix/alerts/TestBaseStatsValidation.java +++ /dev/null @@ -1,173 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.HelixException; -import org.apache.helix.alerts.ExpressionOperatorType; -import org.apache.helix.alerts.ExpressionParser; -import org.testng.annotations.Test; -import org.testng.AssertJUnit; -import org.testng.Assert; -import org.testng.annotations.Test; - -@Test -public class TestBaseStatsValidation { - - @Test - public void testParseSingletonExpression() { - String[] actual = null; - - String statName = "window(5)(dbFoo.partition10.latency)"; - try { - actual = ExpressionParser.getBaseStats(statName); - } catch (HelixException e) { - e.printStackTrace(); - } - AssertJUnit.assertEquals(statName, actual[0]); - } - - @Test - public void testExtraParen() { - String[] actual = null; - - String statName = "window(5)(dbFoo.partition10.latency)()"; - boolean caughtException = false; - try { - actual = ExpressionParser.getBaseStats(statName); - } catch (HelixException e) { - caughtException = true; - // e.printStackTrace(); - } - AssertJUnit.assertEquals(true, caughtException); - } - - @Test - public void testParseSingletonWildcardExpression() { - String[] actual = null; - - String statName = "accumulate()(dbFoo.partition*.latency)"; - try { - actual = ExpressionParser.getBaseStats(statName); - } catch (HelixException e) { - e.printStackTrace(); - } - AssertJUnit.assertEquals(statName, actual[0]); - } - - @Test - public void testParsePairOfExpressions() { - String[] actual = null; - - String expression = "accumulate()(dbFoo.partition10.latency, dbFoo.partition10.count)"; - try { - actual = ExpressionParser.getBaseStats(expression); - } catch (HelixException e) { - e.printStackTrace(); - } - AssertJUnit.assertEquals("accumulate()(dbFoo.partition10.latency)", actual[0]); - AssertJUnit.assertEquals("accumulate()(dbFoo.partition10.count)", actual[1]); - } - - /* - * SUM is not to be persisted, so pull out the pieces - */ - @Test - public void testSUMExpression() { - String[] actual = null; - - String expression = "accumulate()(dbFoo.partition*.latency)|SUM"; - try { - actual = ExpressionParser.getBaseStats(expression); - } catch (HelixException e) { - e.printStackTrace(); - } - AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.latency)", actual[0]); - } - - @Test - public void testSumPairExpression() { - String[] actual = null; - - String expression = "window(5)(dbFoo.partition10.latency, dbFoo.partition11.latency)|SUM"; - try { - actual = ExpressionParser.getBaseStats(expression); - } catch (HelixException e) { - e.printStackTrace(); - } - AssertJUnit.assertEquals("window(5)(dbFoo.partition10.latency)", actual[0]); - AssertJUnit.assertEquals("window(5)(dbFoo.partition11.latency)", actual[1]); - } - - @Test - public void testEachPairExpression() { - String[] actual = null; - - String expression = "accumulate()(dbFoo.partition*.latency, dbFoo.partition*.count)|EACH"; - try { - actual = ExpressionParser.getBaseStats(expression); - } catch (HelixException e) { - e.printStackTrace(); - } - AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.latency)", actual[0]); - AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.count)", actual[1]); - } - - @Test - public void testAccumulateExpression() { - String[] actual = null; - - String expression = "accumulate()(dbFoo.partition10.latency)|ACCUMULATE"; - try { - actual = ExpressionParser.getBaseStats(expression); - } catch (HelixException e) { - e.printStackTrace(); - } - AssertJUnit.assertEquals("accumulate()(dbFoo.partition10.latency)", actual[0]); - } - - @Test - public void testAccumulateEachExpression() { - String[] actual = null; - - String expression = "window(5)(dbFoo.partition*.latency)|EACH|ACCUMULATE"; - try { - actual = ExpressionParser.getBaseStats(expression); - } catch (HelixException e) { - e.printStackTrace(); - } - AssertJUnit.assertEquals("window(5)(dbFoo.partition*.latency)", actual[0]); - } - - @Test - public void testAccumulateEachPairExpression() { - String[] actual = null; - - String expression = - "accumulate()(dbFoo.partition*.latency, dbFoo.partition*.count)|EACH|ACCUMULATE|DIVIDE"; - try { - actual = ExpressionParser.getBaseStats(expression); - } catch (HelixException e) { - e.printStackTrace(); - } - AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.latency)", actual[0]); - AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.count)", actual[1]); - } - -} diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestEvaluateAlerts.java b/helix-core/src/test/java/org/apache/helix/alerts/TestEvaluateAlerts.java deleted file mode 100644 index 403dfdb1eb..0000000000 --- a/helix-core/src/test/java/org/apache/helix/alerts/TestEvaluateAlerts.java +++ /dev/null @@ -1,393 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.apache.helix.HelixException; -import org.apache.helix.Mocks.MockManager; -import org.apache.helix.alerts.AlertParser; -import org.apache.helix.alerts.AlertProcessor; -import org.apache.helix.alerts.AlertValueAndStatus; -import org.apache.helix.alerts.AlertsHolder; -import org.apache.helix.alerts.StatsHolder; -import org.apache.helix.controller.stages.HealthDataCache; -import org.testng.AssertJUnit; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -public class TestEvaluateAlerts { - protected static final String CLUSTER_NAME = "TestCluster"; - - MockManager _helixManager; - AlertsHolder _alertsHolder; - StatsHolder _statsHolder; - - public final String EXP = AlertParser.EXPRESSION_NAME; - public final String CMP = AlertParser.COMPARATOR_NAME; - public final String CON = AlertParser.CONSTANT_NAME; - - @BeforeMethod(groups = { - "unitTest" - }) - public void setup() { - HealthDataCache cache = new HealthDataCache(); - _helixManager = new MockManager(CLUSTER_NAME); - _alertsHolder = new AlertsHolder(_helixManager, cache); - _statsHolder = _alertsHolder._statsHolder; - } - - public Map getStatFields(String value, String timestamp) { - Map statMap = new HashMap(); - statMap.put(StatsHolder.VALUE_NAME, value); - statMap.put(StatsHolder.TIMESTAMP_NAME, timestamp); - return statMap; - } - - public String getSimpleStat() throws HelixException { - String stat = "accumulate()(dbFoo.partition10.latency)"; - // _statsHolder.addStat(stat); - return stat; - } - - public String addPairOfStats() throws HelixException { - String stat = "accumulate()(dbFoo.partition10.latency, dbFoo.partition11.latency)"; - _statsHolder.addStat(stat); - _statsHolder.persistStats(); - return stat; - } - - public String getWildcardStat() throws HelixException { - String stat = "accumulate()(dbFoo.partition*.latency)"; - // _statsHolder.addStat(stat); - return stat; - } - - public String addSimpleAlert() throws HelixException { - String alert = - EXP + "(accumulate()(dbFoo.partition10.latency))" + CMP + "(GREATER)" + CON + "(100)"; - _alertsHolder.addAlert(alert); - return alert; - } - - public String addWildcardAlert() throws HelixException { - String alert = - EXP + "(accumulate()(dbFoo.partition*.latency))" + CMP + "(GREATER)" + CON + "(100)"; - _alertsHolder.addAlert(alert); - return alert; - } - - public String addTwoWildcardAlert() throws HelixException { - String alert = - EXP + "(accumulate()(dbFoo.partition*.put*))" + CMP + "(GREATER)" + CON + "(100)"; - _alertsHolder.addAlert(alert); - return alert; - } - - public String addExpandWildcardAlert() throws HelixException { - String alert = - EXP + "(accumulate()(dbFoo.partition*.latency)|EXPAND)" + CMP + "(GREATER)" + CON + "(100)"; - _alertsHolder.addAlert(alert); - return alert; - } - - public String addExpandSumAlert() throws HelixException { - String alert = - EXP + "(accumulate()(dbFoo.partition10.latency,dbFoo.partition11.latency)|EXPAND|SUM)" - + CMP + "(GREATER)" + CON + "(100)"; - _alertsHolder.addAlert(alert); - return alert; - } - - public String addExpandSumWildcardAlert() throws HelixException { - String alert = - EXP + "(accumulate()(dbFoo.partition*.success,dbFoo.partition*.failure)|EXPAND|SUM)" + CMP - + "(GREATER)" + CON + "(100)"; - _alertsHolder.addAlert(alert); - return alert; - } - - public String addExpandSumEachWildcardAlert() throws HelixException { - String alert = - EXP + "(accumulate()(dbFoo.partition*.success,dbFoo.partition*.failure)|EXPAND|SUMEACH)" - + CMP + "(GREATER)" + CON + "(100)"; - _alertsHolder.addAlert(alert); - return alert; - } - - public String addExpandSumEachSumWildcardAlert() throws HelixException { - String alert = - EXP - + "(accumulate()(dbFoo.partition*.success,dbFoo.partition*.failure)|EXPAND|SUMEACH|SUM)" - + CMP + "(GREATER)" + CON + "(100)"; - _alertsHolder.addAlert(alert); - return alert; - } - - public String addArrivingSimpleStat() throws HelixException { - _statsHolder.refreshStats(); - String incomingStatName = "dbFoo.partition10.latency"; - Map statFields = getStatFields("110", "0"); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - return incomingStatName; - } - - public String addArrivingPairOfStats() throws HelixException { - _statsHolder.refreshStats(); - String incomingStatName1 = "dbFoo.partition10.latency"; - String incomingStatName2 = "dbFoo.partition11.latency"; - Map statFields = getStatFields("50", "0"); - _statsHolder.applyStat(incomingStatName1, statFields); - statFields = getStatFields("51", "0"); - _statsHolder.applyStat(incomingStatName2, statFields); - _statsHolder.persistStats(); - return null; - } - - @Test(groups = { - "unitTest" - }) - public void testSimpleAlertFires() { - String alert = addSimpleAlert(); - String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert); - _statsHolder.refreshStats(); // need to refresh since not triggered by stats aggregation stage - addArrivingSimpleStat(); - Map> alertResult = - AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - boolean alertFired = alertResult.get(alert).get(AlertProcessor.noWildcardAlertKey).isFired(); - AssertJUnit.assertTrue(alertFired); - } - - @Test(groups = { - "unitTest" - }) - public void testSimpleAlertNoStatArrivesFires() { - String alert = addSimpleAlert(); - String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert); - Map> alertResult = - AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - AssertJUnit.assertEquals(null, alertResult.get(AlertProcessor.noWildcardAlertKey)); - } - - @Test(groups = { - "unitTest" - }) - public void testWildcardAlertFires() { - String alert = addWildcardAlert(); - String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert); - String incomingStatName = addArrivingSimpleStat(); - - Map> alertResult = - AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - String wildcardBinding = incomingStatName; - boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired(); - AssertJUnit.assertTrue(alertFired); - } - - @Test(groups = { - "unitTest" - }) - public void testExpandOperatorWildcardAlertFires() { - String alert = addExpandWildcardAlert(); - String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert); - String incomingStatName = addArrivingSimpleStat(); - Map> alertResult = - AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - String wildcardBinding = incomingStatName; - boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired(); - AssertJUnit.assertTrue(alertFired); - } - - @Test(groups = { - "unitTest" - }) - public void testExpandSumOperatorAlertFires() { - String alert = addExpandSumAlert(); - String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert); - addArrivingPairOfStats(); - Map> alertResult = - AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - boolean alertFired = alertResult.get(alert).get(AlertProcessor.noWildcardAlertKey).isFired(); - AssertJUnit.assertTrue(alertFired); - } - - /** - * We need to re-decide how to support the feature to specify more than one stats in - * an alert. - * Probabaly instead of - * "(dbFoo.partition*.success,dbFoo.partition*.failure)", use the form - * "(dbFoo.partition*.(success, failure))" as it seems that the stat source is always the - * same. - * - //@Test (groups = {"unitTest"}) - * public void testExpandSumOperatorWildcardAlert() - * { - * String alert = addExpandSumWildcardAlert(); - * String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert); - * String part10SuccStat = "dbFoo.partition10.success"; - * String part10FailStat = "dbFoo.partition10.failure"; - * String part11SuccStat = "dbFoo.partition11.success"; - * String part11FailStat = "dbFoo.partition11.failure"; - * Map statFields = getStatFields("50","0"); - * _statsHolder.applyStat(part10SuccStat, statFields); - * statFields = getStatFields("51","0"); - * _statsHolder.applyStat(part10FailStat, statFields); - * statFields = getStatFields("50","0"); - * _statsHolder.applyStat(part11SuccStat, statFields); - * statFields = getStatFields("49","0"); - * _statsHolder.applyStat(part11FailStat, statFields); - * Map> alertResult = - * AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - * boolean alertFired = alertResult.get(alert).get("10").isFired(); //10 should fire - * AssertJUnit.assertTrue(alertFired); - * alertFired = alertResult.get(alert).get("11").isFired(); //11 should not fire - * AssertJUnit.assertFalse(alertFired); - * } - * //@Test (groups = {"unitTest"}) - * public void testExpandSumEachSumOperatorWildcardAlert() - * { - * String alert = addExpandSumEachSumWildcardAlert(); - * String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert); - * String part10SuccStat = "dbFoo.partition10.success"; - * String part10FailStat = "dbFoo.partition10.failure"; - * String part11SuccStat = "dbFoo.partition11.success"; - * String part11FailStat = "dbFoo.partition11.failure"; - * Map statFields = getStatFields("50","0"); - * _statsHolder.applyStat(part10SuccStat, statFields); - * statFields = getStatFields("51","0"); - * _statsHolder.applyStat(part10FailStat, statFields); - * statFields = getStatFields("50","0"); - * _statsHolder.applyStat(part11SuccStat, statFields); - * statFields = getStatFields("49","0"); - * _statsHolder.applyStat(part11FailStat, statFields); - * Map> alertResult = - * AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - * boolean alertFired = alertResult.get(alert).get(_statsHolder.getStatsList().get(0)).isFired(); - * //10 should fire - * AssertJUnit.assertTrue(alertFired); - * } - * //@Test (groups = {"unitTest"}) - * public void testTwoAlerts() - * { - * //alert 1 - * String alert1 = addSimpleAlert(); - * String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert1); - * addArrivingSimpleStat(); - * //alert 2 - * String alert2 = addExpandSumWildcardAlert(); - * stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert2); - * String part10SuccStat = "dbFoo.partition10.success"; - * String part10FailStat = "dbFoo.partition10.failure"; - * String part11SuccStat = "dbFoo.partition11.success"; - * String part11FailStat = "dbFoo.partition11.failure"; - * Map statFields = getStatFields("50","0"); - * _statsHolder.applyStat(part10SuccStat, statFields); - * statFields = getStatFields("51","0"); - * _statsHolder.applyStat(part10FailStat, statFields); - * statFields = getStatFields("50","0"); - * _statsHolder.applyStat(part11SuccStat, statFields); - * statFields = getStatFields("49","0"); - * _statsHolder.applyStat(part11FailStat, statFields); - * Map> alertResult = - * AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - * //alert 1 check - * boolean alertFired = alertResult.get(alert1).get(AlertProcessor.noWildcardAlertKey).isFired(); - * AssertJUnit.assertTrue(alertFired); - * //alert 2 check - * alertFired = alertResult.get(alert2).get("10").isFired(); //10 should fire - * AssertJUnit.assertTrue(alertFired); - * alertFired = alertResult.get(alert2).get("11").isFired(); //11 should not fire - * AssertJUnit.assertFalse(alertFired); - * } - */ - @Test(groups = { - "unitTest" - }) - public void testAddWildcardInFirstStatToken() throws Exception { - String alert = "EXP(decay(1)(instance*.reportingage))CMP(GREATER)CON(300)"; - _alertsHolder.addAlert(alert); - _statsHolder.persistStats(); - - _statsHolder.refreshStats(); - // generate incoming stat - String incomingStatName = "instance10.reportingage"; - Map statFields = getStatFields("301", "10"); - _statsHolder.refreshStats(); - - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - - Map> alertResult = - AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - String wildcardBinding = incomingStatName; - boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired(); - AssertJUnit.assertTrue(alertFired); - } - - @Test(groups = { - "unitTest" - }) - public void testTwoWildcardAlertFires() { - // error is with * and ) - String alert = addTwoWildcardAlert(); - String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert); - String incomingStatName = "dbFoo.partition10.putCount"; - Map statFields = getStatFields("110", "0"); - _statsHolder.refreshStats(); - _statsHolder.applyStat(incomingStatName, statFields); - _statsHolder.persistStats(); - Map> alertResult = - AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - String wildcardBinding = incomingStatName; // XXX: this is not going to work...need "Count" in - // here too. - boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired(); - AssertJUnit.assertTrue(alertFired); - } - - /* - * only supporting wildcards at end of components right now - * @Test (groups = {"unitTest"}) - * public void testTwoWildcardsNotAtEndFires() - * { - * String alert = EXP + "(accumulate()(dbFoo.partition*.*Count))" - * + CMP + "(GREATER)" + CON + "(100)"; - * _alertsHolder.addAlert(alert); - * String incomingStatName = "dbFoo.partition10.putCount"; - * Map statFields = getStatFields("110","0"); - * _statsHolder.applyStat(incomingStatName, statFields); - * Map> alertResult = - * AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList()); - * String wildcardBinding = "10,put"; //XXX: this is not going to work...need "Count" in here too. - * boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired(); - * AssertJUnit.assertTrue(alertFired); - * } - */ - - // test using sumall - // test using rows where some tuples are null (no stat sent) - // test with window tuples where some windows are different lengths - // anything else, look around at the code - - // next: review all older tests - // next: actually write the fired alerts to ZK - -} diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestOperators.java b/helix-core/src/test/java/org/apache/helix/alerts/TestOperators.java deleted file mode 100644 index 2d54a27183..0000000000 --- a/helix-core/src/test/java/org/apache/helix/alerts/TestOperators.java +++ /dev/null @@ -1,329 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.Iterator; -import java.util.List; -import java.util.Map; - -import org.apache.helix.alerts.SumEachOperator; -import org.apache.helix.alerts.SumOperator; -import org.apache.helix.alerts.Tuple; -import org.testng.AssertJUnit; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -public class TestOperators { - - SumOperator _sumOp; - SumEachOperator _sumEachOp; - - @BeforeMethod(groups = { - "unitTest" - }) - public void setup() { - _sumOp = new SumOperator(); - _sumEachOp = new SumEachOperator(); - } - - @Test(groups = { - "unitTest" - }) - public void testTwoNulls() { - Tuple tup1 = null; - Tuple tup2 = null; - List> tup1List = new ArrayList>(); - List> tup2List = new ArrayList>(); - tup1List.add(tup1); - tup2List.add(tup2); - List>> tupsList = new ArrayList>>(); - tupsList.add(tup1List.iterator()); - tupsList.add(tup2List.iterator()); - List>> result = _sumOp.execute(tupsList); - AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter - Iterator> resultIter = result.get(0); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup = resultIter.next(); - AssertJUnit.assertEquals(null, resultTup); - } - - @Test(groups = { - "unitTest" - }) - public void testOneNullLeft() { - Tuple tup1 = null; - Tuple tup2 = new Tuple(); - tup2.add("1.0"); - List> tup1List = new ArrayList>(); - List> tup2List = new ArrayList>(); - tup1List.add(tup1); - tup2List.add(tup2); - List>> tupsList = new ArrayList>>(); - tupsList.add(tup1List.iterator()); - tupsList.add(tup2List.iterator()); - List>> result = _sumOp.execute(tupsList); - AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter - Iterator> resultIter = result.get(0); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup = resultIter.next(); - AssertJUnit.assertEquals("1.0", resultTup.toString()); - } - - @Test(groups = { - "unitTest" - }) - public void testOneNullRight() { - Tuple tup1 = new Tuple(); - Tuple tup2 = null; - tup1.add("1.0"); - List> tup1List = new ArrayList>(); - List> tup2List = new ArrayList>(); - tup1List.add(tup1); - tup2List.add(tup2); - List>> tupsList = new ArrayList>>(); - tupsList.add(tup1List.iterator()); - tupsList.add(tup2List.iterator()); - List>> result = _sumOp.execute(tupsList); - AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter - Iterator> resultIter = result.get(0); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup = resultIter.next(); - AssertJUnit.assertEquals("1.0", resultTup.toString()); - } - - @Test(groups = { - "unitTest" - }) - public void testTwoSingeltons() { - Tuple tup1 = new Tuple(); - Tuple tup2 = new Tuple(); - tup1.add("1.0"); - tup2.add("2.0"); - List> tup1List = new ArrayList>(); - List> tup2List = new ArrayList>(); - tup1List.add(tup1); - tup2List.add(tup2); - List>> tupsList = new ArrayList>>(); - tupsList.add(tup1List.iterator()); - tupsList.add(tup2List.iterator()); - List>> result = _sumOp.execute(tupsList); - AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter - Iterator> resultIter = result.get(0); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup = resultIter.next(); - AssertJUnit.assertEquals("3.0", resultTup.toString()); - } - - @Test(groups = { - "unitTest" - }) - public void testThreeSingeltons() { - Tuple tup1 = new Tuple(); - Tuple tup2 = new Tuple(); - Tuple tup3 = new Tuple(); - tup1.add("1.0"); - tup2.add("2.0"); - tup3.add("3.0"); - List> tup1List = new ArrayList>(); - List> tup2List = new ArrayList>(); - List> tup3List = new ArrayList>(); - tup1List.add(tup1); - tup2List.add(tup2); - tup3List.add(tup3); - List>> tupsList = new ArrayList>>(); - tupsList.add(tup1List.iterator()); - tupsList.add(tup2List.iterator()); - tupsList.add(tup3List.iterator()); - List>> result = _sumOp.execute(tupsList); - AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter - Iterator> resultIter = result.get(0); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup = resultIter.next(); - AssertJUnit.assertEquals("6.0", resultTup.toString()); - } - - @Test(groups = { - "unitTest" - }) - public void testThreeTriples() { - Tuple tup1 = new Tuple(); - Tuple tup2 = new Tuple(); - Tuple tup3 = new Tuple(); - tup1.add("1.0"); - tup1.add("2.0"); - tup1.add("3.0"); - tup2.add("4.0"); - tup2.add("5.0"); - tup2.add("6.0"); - tup3.add("7.0"); - tup3.add("8.0"); - tup3.add("9.0"); - List> tup1List = new ArrayList>(); - List> tup2List = new ArrayList>(); - List> tup3List = new ArrayList>(); - tup1List.add(tup1); - tup2List.add(tup2); - tup3List.add(tup3); - List>> tupsList = new ArrayList>>(); - tupsList.add(tup1List.iterator()); - tupsList.add(tup2List.iterator()); - tupsList.add(tup3List.iterator()); - List>> result = _sumOp.execute(tupsList); - AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter - Iterator> resultIter = result.get(0); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup = resultIter.next(); - AssertJUnit.assertEquals("12.0,15.0,18.0", resultTup.toString()); - } - - @Test(groups = { - "unitTest" - }) - public void testThreeTriplesOneMissing() { - Tuple tup1 = new Tuple(); - Tuple tup2 = new Tuple(); - Tuple tup3 = new Tuple(); - tup1.add("1.0"); - tup1.add("2.0"); - tup1.add("3.0"); - tup2.add("5.0"); - tup2.add("6.0"); - tup3.add("7.0"); - tup3.add("8.0"); - tup3.add("9.0"); - List> tup1List = new ArrayList>(); - List> tup2List = new ArrayList>(); - List> tup3List = new ArrayList>(); - tup1List.add(tup1); - tup2List.add(tup2); - tup3List.add(tup3); - List>> tupsList = new ArrayList>>(); - tupsList.add(tup1List.iterator()); - tupsList.add(tup2List.iterator()); - tupsList.add(tup3List.iterator()); - List>> result = _sumOp.execute(tupsList); - AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter - Iterator> resultIter = result.get(0); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup = resultIter.next(); - // tuple 2 missing 1 entry, other 2 get bumped to right - AssertJUnit.assertEquals("8.0,15.0,18.0", resultTup.toString()); - } - - // test multiple rows - @Test(groups = { - "unitTest" - }) - public void testThreeTriplesOneMissingTwoRows() { - Tuple tup1Dot1 = new Tuple(); - Tuple tup2Dot1 = new Tuple(); - Tuple tup3Dot1 = new Tuple(); - Tuple tup1Dot2 = new Tuple(); - Tuple tup2Dot2 = new Tuple(); - Tuple tup3Dot2 = new Tuple(); - tup1Dot1.add("1.0"); - tup1Dot1.add("2.0"); - tup1Dot1.add("3.0"); - tup2Dot1.add("5.0"); - tup2Dot1.add("6.0"); - tup3Dot1.add("7.0"); - tup3Dot1.add("8.0"); - tup3Dot1.add("9.0"); - tup1Dot2.add("10.0"); - tup1Dot2.add("11.0"); - tup1Dot2.add("12.0"); - tup2Dot2.add("13.0"); - tup2Dot2.add("14.0"); - tup2Dot2.add("15.0"); - tup3Dot2.add("16.0"); - tup3Dot2.add("17.0"); - tup3Dot2.add("18.0"); - List> tup1List = new ArrayList>(); - List> tup2List = new ArrayList>(); - List> tup3List = new ArrayList>(); - tup1List.add(tup1Dot1); - tup2List.add(tup2Dot1); - tup3List.add(tup3Dot1); - tup1List.add(tup1Dot2); - tup2List.add(tup2Dot2); - tup3List.add(tup3Dot2); - List>> tupsList = new ArrayList>>(); - tupsList.add(tup1List.iterator()); - tupsList.add(tup2List.iterator()); - tupsList.add(tup3List.iterator()); - List>> result = _sumOp.execute(tupsList); - AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter - Iterator> resultIter = result.get(0); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup1 = resultIter.next(); - // tuple 2 missing 1 entry, other 2 get bumped to right - AssertJUnit.assertEquals("8.0,15.0,18.0", resultTup1.toString()); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup2 = resultIter.next(); - AssertJUnit.assertEquals("39.0,42.0,45.0", resultTup2.toString()); - } - - @Test(groups = { - "unitTest" - }) - public void testSumAll() { - Tuple tup1 = new Tuple(); - Tuple tup2 = new Tuple(); - Tuple tup3 = new Tuple(); - Tuple tup4 = new Tuple(); - Tuple tup5 = new Tuple(); - Tuple tup6 = new Tuple(); - tup1.add("1.0"); - tup2.add("2.0"); - tup3.add("3.0"); - tup4.add("4.0"); - tup5.add("5.0"); - tup6.add("6.0"); - List> list1 = new ArrayList>(); - List> list2 = new ArrayList>(); - List> list3 = new ArrayList>(); - list1.add(tup1); - list1.add(tup4); - list2.add(tup2); - list2.add(tup5); - list3.add(tup3); - list3.add(tup6); - - List>> tupsList = new ArrayList>>(); - tupsList.add(list1.iterator()); - tupsList.add(list2.iterator()); - tupsList.add(list3.iterator()); - List>> result = _sumEachOp.execute(tupsList); - AssertJUnit.assertEquals(3, result.size()); // should be just 1 iter - Iterator> resultIter = result.get(0); - AssertJUnit.assertTrue(resultIter.hasNext()); - Tuple resultTup1 = resultIter.next(); - AssertJUnit.assertEquals("5.0", resultTup1.toString()); - resultIter = result.get(1); - AssertJUnit.assertTrue(resultIter.hasNext()); - resultTup1 = resultIter.next(); - AssertJUnit.assertEquals("7.0", resultTup1.toString()); - resultIter = result.get(2); - AssertJUnit.assertTrue(resultIter.hasNext()); - resultTup1 = resultIter.next(); - AssertJUnit.assertEquals("9.0", resultTup1.toString()); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestStatsMatch.java b/helix-core/src/test/java/org/apache/helix/alerts/TestStatsMatch.java deleted file mode 100644 index 094f347fb6..0000000000 --- a/helix-core/src/test/java/org/apache/helix/alerts/TestStatsMatch.java +++ /dev/null @@ -1,107 +0,0 @@ -package org.apache.helix.alerts; - -/* - * 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.HelixException; -import org.apache.helix.alerts.ExpressionParser; -import org.testng.annotations.Test; -import org.testng.AssertJUnit; -import org.testng.Assert; -import org.testng.annotations.Test; - -@Test -public class TestStatsMatch { - - @Test - public void testExactMatch() { - - String persistedStatName = "window(5)(dbFoo.partition10.latency)"; - String incomingStatName = "dbFoo.partition10.latency"; - AssertJUnit.assertTrue(ExpressionParser.isIncomingStatExactMatch(persistedStatName, - incomingStatName)); - } - - @Test - public void testSingleWildcardMatch() { - - String persistedStatName = "window(5)(dbFoo.partition*.latency)"; - String incomingStatName = "dbFoo.partition10.latency"; - AssertJUnit.assertTrue(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName, - incomingStatName)); - } - - @Test - public void testDoubleWildcardMatch() { - - String persistedStatName = "window(5)(db*.partition*.latency)"; - String incomingStatName = "dbFoo.partition10.latency"; - AssertJUnit.assertTrue(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName, - incomingStatName)); - } - - @Test - public void testWildcardMatchNoWildcard() { - - String persistedStatName = "window(5)(dbFoo.partition10.latency)"; - String incomingStatName = "dbFoo.partition10.latency"; - AssertJUnit.assertFalse(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName, - incomingStatName)); - } - - @Test - public void testWildcardMatchTooManyFields() { - - String persistedStatName = "window(5)(dbFoo.partition*.latency)"; - String incomingStatName = "dbFoo.tableBar.partition10.latency"; - AssertJUnit.assertFalse(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName, - incomingStatName)); - } - - @Test - public void testWildcardMatchTooFewFields() { - - String persistedStatName = "window(5)(dbFoo.partition*.latency)"; - String incomingStatName = "dbFoo.latency"; - AssertJUnit.assertFalse(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName, - incomingStatName)); - } - - @Test - public void testBadWildcardRepeated() { - - String persistedStatName = "window(5)(dbFoo.partition**4.latency)"; - String incomingStatName = "dbFoo.partition10.latency"; - boolean match = - ExpressionParser.isIncomingStatWildcardMatch(persistedStatName, incomingStatName); - - AssertJUnit.assertFalse(match); - } - - @Test - public void testBadWildcardNotAtEnd() { - - String persistedStatName = "window(5)(dbFoo.*partition.latency)"; - String incomingStatName = "dbFoo.partition10.latency"; - boolean match = - ExpressionParser.isIncomingStatWildcardMatch(persistedStatName, incomingStatName); - - AssertJUnit.assertFalse(match); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/api/TestId.java b/helix-core/src/test/java/org/apache/helix/api/TestId.java new file mode 100644 index 0000000000..488e03770b --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/api/TestId.java @@ -0,0 +1,109 @@ +package org.apache.helix.api; + +/* + * 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.api.id.ClusterId; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ProcId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.api.id.StateModelFactoryId; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestId { + /** + * Make sure that a partition serializes and deserializes properly + */ + @Test + public void testPartitionId() { + final String partitionName = "Resource_3"; + final String resourceName = "Resource"; + final String partitionSuffix = "3"; + PartitionId partitionId = PartitionId.from(partitionName); + Assert.assertEquals(partitionId.stringify(), partitionName); + PartitionId partitionId2 = PartitionId.from(ResourceId.from(resourceName), partitionSuffix); + Assert.assertEquals(partitionId2.stringify(), partitionName); + Assert.assertEquals(partitionId, partitionId2); + Assert.assertEquals(partitionId.toString(), partitionId2.toString()); + } + + /** + * Check that PartitionId doesn't need to be of the form resource_partition for compatibility + */ + @Test + public void testPartitionIdCompatibility() { + final String partitionName = "Resource--3"; + PartitionId partitionId = PartitionId.from(partitionName); + Assert.assertEquals(partitionId.stringify(), partitionName); + } + + /** + * Check that ids can be converted back and forth between strings and concrete classes + */ + @Test + public void basicIdTest() { + final String resourceName = "Resource"; + final String clusterName = "Cluster"; + final String participantName = "Participant"; + final String sessionName = "Session"; + final String processName = "Process"; + final String stateModelName = "StateModel"; + final String stateModelFactoryName = "StateModelFactory"; + final String messageName = "Message"; + Assert.assertEquals(ResourceId.from(resourceName).stringify(), resourceName); + Assert.assertEquals(ClusterId.from(clusterName).stringify(), clusterName); + Assert.assertEquals(ParticipantId.from(participantName).stringify(), participantName); + Assert.assertEquals(SessionId.from(sessionName).stringify(), sessionName); + Assert.assertEquals(ProcId.from(processName).stringify(), processName); + Assert.assertEquals(StateModelDefId.from(stateModelName).stringify(), stateModelName); + Assert.assertEquals(StateModelFactoryId.from(stateModelFactoryName).stringify(), + stateModelFactoryName); + Assert.assertEquals(MessageId.from(messageName).stringify(), messageName); + } + + /** + * Check that equality with string works + */ + @Test + public void testStringEquality() { + final String resourceName = "Resource"; + Assert.assertTrue(ResourceId.from(resourceName).equals(resourceName)); + } + + /** + * Ensure that trying to create an id with null yields null + */ + @Test + public void testNull() { + Assert.assertNull(ClusterId.from(null)); + Assert.assertNull(ResourceId.from(null)); + Assert.assertNull(PartitionId.from(null)); + Assert.assertNull(ParticipantId.from(null)); + Assert.assertNull(SessionId.from(null)); + Assert.assertNull(ProcId.from(null)); + Assert.assertNull(StateModelDefId.from(null)); + Assert.assertNull(StateModelFactoryId.from(null)); + Assert.assertNull(MessageId.from(null)); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/api/TestNamespacedConfig.java b/helix-core/src/test/java/org/apache/helix/api/TestNamespacedConfig.java new file mode 100644 index 0000000000..761ffe2199 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/api/TestNamespacedConfig.java @@ -0,0 +1,177 @@ +package org.apache.helix.api; + +import java.util.List; +import java.util.Map; + +import org.apache.helix.api.config.ResourceConfig.ResourceType; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.manager.zk.ZKHelixManager; +import org.apache.helix.model.ClusterConfiguration; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.InstanceConfig.InstanceConfigProperty; +import org.apache.helix.model.ResourceConfiguration; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +/* + * 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. + */ + +/** + * A user config is a namespaced subset in the physical model and a separate entity in the logical + * model. These tests ensure that that separation is honored. + */ +public class TestNamespacedConfig { + /** + * Ensure that user configs are separated from helix configs in properties that hold both + */ + @Test + public void testUserConfigUpdates() { + final String testKey = "testKey"; + final String prefixedKey = UserConfig.class.getSimpleName() + "!testKey"; + final String testSimpleValue = "testValue"; + final List testListValue = ImmutableList.of("testValue"); + final Map testMapValue = ImmutableMap.of("testInnerKey", "testValue"); + + // first, add Helix configuration to an InstanceConfig + ParticipantId participantId = ParticipantId.from("testParticipant"); + InstanceConfig instanceConfig = new InstanceConfig(participantId); + instanceConfig.setHostName("localhost"); + + // now, add user configuration + UserConfig userConfig = new UserConfig(Scope.participant(participantId)); + userConfig.setSimpleField(testKey, testSimpleValue); + userConfig.setListField(testKey, testListValue); + userConfig.setMapField(testKey, testMapValue); + + // add the user configuration to the Helix configuration + instanceConfig.addNamespacedConfig(userConfig); + + // get the user configuration back from the property + UserConfig retrievedConfig = UserConfig.from(instanceConfig); + + // check that the property still has the host name + Assert.assertTrue(instanceConfig.getHostName().equals("localhost")); + + // check that the retrieved config does not contain the host name + Assert.assertEquals(retrievedConfig.getStringField( + InstanceConfigProperty.HELIX_HOST.toString(), "not localhost"), "not localhost"); + + // check that both the retrieved config and the original config have the added properties + Assert.assertEquals(userConfig.getSimpleField(testKey), testSimpleValue); + Assert.assertEquals(userConfig.getListField(testKey), testListValue); + Assert.assertEquals(userConfig.getMapField(testKey), testMapValue); + Assert.assertEquals(retrievedConfig.getSimpleField(testKey), testSimpleValue); + Assert.assertEquals(retrievedConfig.getListField(testKey), testListValue); + Assert.assertEquals(retrievedConfig.getMapField(testKey), testMapValue); + + // test that the property has the user config, but prefixed + Assert.assertEquals(instanceConfig.getRecord().getSimpleField(prefixedKey), testSimpleValue); + Assert.assertEquals(instanceConfig.getRecord().getListField(prefixedKey), testListValue); + Assert.assertEquals(instanceConfig.getRecord().getMapField(prefixedKey), testMapValue); + } + + @Test + public void testResourceUserConfigCompatibility() { + final String KEY1 = "key1"; + final String VALUE1 = "value1"; + final String KEY2 = "key2"; + final String VALUE2 = "value2"; + final String KEY3 = "key3"; + final String VALUE3 = "value3"; + + // add key1 through user config, key2 through resource config, key3 through ideal state, + // resource type through resource config, rebalance mode through ideal state + ResourceId resourceId = ResourceId.from("resourceId"); + UserConfig userConfig = new UserConfig(Scope.resource(resourceId)); + userConfig.setSimpleField(KEY1, VALUE1); + ResourceConfiguration resourceConfig = new ResourceConfiguration(resourceId); + resourceConfig.setType(ResourceType.DATA); + resourceConfig.addNamespacedConfig(userConfig); + resourceConfig.getRecord().setSimpleField(KEY2, VALUE2); + IdealState idealState = new IdealState(resourceId); + idealState.setRebalanceMode(RebalanceMode.USER_DEFINED); + idealState.getRecord().setSimpleField(KEY3, VALUE3); + + // should have key1, key2, and key3, not type or rebalance mode + UserConfig result = resourceConfig.getUserConfig(); + idealState.updateUserConfig(result); + Assert.assertEquals(result.getSimpleField(KEY1), VALUE1); + Assert.assertEquals(result.getSimpleField(KEY2), VALUE2); + Assert.assertEquals(result.getSimpleField(KEY3), VALUE3); + Assert.assertNull(result.getSimpleField(ResourceConfiguration.Fields.TYPE.toString())); + Assert + .assertNull(result.getSimpleField(IdealState.IdealStateProperty.REBALANCE_MODE.toString())); + } + + @Test + public void testParticipantUserConfigCompatibility() { + final String KEY1 = "key1"; + final String VALUE1 = "value1"; + final String KEY2 = "key2"; + final String VALUE2 = "value2"; + + // add key1 through user config, key2 through instance config, hostname through user config + ParticipantId participantId = ParticipantId.from("participantId"); + UserConfig userConfig = new UserConfig(Scope.participant(participantId)); + userConfig.setSimpleField(KEY1, VALUE1); + InstanceConfig instanceConfig = new InstanceConfig(participantId); + instanceConfig.setHostName("localhost"); + instanceConfig.addNamespacedConfig(userConfig); + instanceConfig.getRecord().setSimpleField(KEY2, VALUE2); + + // should have key1 and key2, not hostname + UserConfig result = instanceConfig.getUserConfig(); + Assert.assertEquals(result.getSimpleField(KEY1), VALUE1); + Assert.assertEquals(result.getSimpleField(KEY2), VALUE2); + Assert.assertNull(result.getSimpleField(InstanceConfig.InstanceConfigProperty.HELIX_HOST + .toString())); + } + + @Test + public void testClusterUserConfigCompatibility() { + final String KEY1 = "key1"; + final String VALUE1 = "value1"; + final String KEY2 = "key2"; + final String VALUE2 = "value2"; + + // add the following: key1 straight to user config, key2 to cluster configuration, + // allow auto join to cluster configuration + ClusterId clusterId = ClusterId.from("clusterId"); + UserConfig userConfig = new UserConfig(Scope.cluster(clusterId)); + userConfig.setSimpleField(KEY1, VALUE1); + ClusterConfiguration clusterConfiguration = new ClusterConfiguration(clusterId); + clusterConfiguration.addNamespacedConfig(userConfig); + clusterConfiguration.getRecord().setSimpleField(KEY2, VALUE2); + clusterConfiguration.setAutoJoinAllowed(true); + + // there should be key1 and key2, but not auto join + UserConfig result = clusterConfiguration.getUserConfig(); + Assert.assertEquals(result.getSimpleField(KEY1), VALUE1); + Assert.assertEquals(result.getSimpleField(KEY2), VALUE2); + Assert.assertNull(result.getSimpleField(ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN)); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java b/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java new file mode 100644 index 0000000000..0d597f6f04 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java @@ -0,0 +1,250 @@ +package org.apache.helix.api; + +/* + * 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.List; +import java.util.Map; + +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext; +import org.apache.helix.controller.stages.AttributeName; +import org.apache.helix.controller.stages.ClusterEvent; +import org.apache.helix.controller.stages.BestPossibleStateCalcStage; +import org.apache.helix.controller.stages.BestPossibleStateOutput; +import org.apache.helix.controller.stages.ResourceCurrentState; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.tools.ClusterStateVerifier; +import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import com.google.common.base.Function; +import com.google.common.collect.Maps; + +public class TestNewStages extends ZkUnitTestBase { + final int n = 2; + final int p = 8; + final int r = 2; + MockParticipantManager[] _participants = new MockParticipantManager[n]; + ClusterControllerManager _controller; + + ClusterId _clusterId; + HelixDataAccessor _dataAccessor; + + @Test + public void testReadClusterDataStage() { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String testName = className + "_" + methodName; + + System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis())); + + ClusterAccessor clusterAccessor = new ClusterAccessor(_clusterId, _dataAccessor); + Cluster cluster = clusterAccessor.readCluster(); + + ClusterId id = cluster.getId(); + Assert.assertEquals(id, _clusterId); + Map liveParticipantMap = cluster.getLiveParticipantMap(); + Assert.assertEquals(liveParticipantMap.size(), n); + + for (ParticipantId participantId : liveParticipantMap.keySet()) { + Participant participant = liveParticipantMap.get(participantId); + Map curStateMap = participant.getCurrentStateMap(); + Assert.assertEquals(curStateMap.size(), 1); + + ResourceId resourceId = ResourceId.from("TestDB0"); + Assert.assertTrue(curStateMap.containsKey(resourceId)); + CurrentState curState = curStateMap.get(resourceId); + Map partitionStateMap = curState.getTypedPartitionStateMap(); + Assert.assertEquals(partitionStateMap.size(), p); + } + + Map resourceMap = cluster.getResourceMap(); + Assert.assertEquals(resourceMap.size(), 1); + + ResourceId resourceId = ResourceId.from("TestDB0"); + Assert.assertTrue(resourceMap.containsKey(resourceId)); + Resource resource = resourceMap.get(resourceId); + Assert.assertNotNull(resource.getRebalancerConfig().getRebalancerContext( + SemiAutoRebalancerContext.class)); + + System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis())); + } + + @Test + public void testBasicBestPossibleStateCalcStage() { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String testName = className + "_" + methodName; + + System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis())); + + // Set up the event + ClusterAccessor clusterAccessor = new ClusterAccessor(_clusterId, _dataAccessor); + Cluster cluster = clusterAccessor.readCluster(); + ClusterEvent event = new ClusterEvent(testName); + event.addAttribute(AttributeName.CURRENT_STATE.toString(), new ResourceCurrentState()); + Map resourceConfigMap = + Maps.transformValues(cluster.getResourceMap(), new Function() { + @Override + public ResourceConfig apply(Resource resource) { + return resource.getConfig(); + } + }); + event.addAttribute(AttributeName.RESOURCES.toString(), resourceConfigMap); + event.addAttribute("ClusterDataCache", cluster); + + // Run the stage + try { + new BestPossibleStateCalcStage().process(event); + } catch (Exception e) { + Assert.fail(e.toString()); + } + + // Verify the result + BestPossibleStateOutput bestPossibleStateOutput = + event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); + Assert.assertNotNull(bestPossibleStateOutput); + ResourceId resourceId = ResourceId.from("TestDB0"); + ResourceAssignment assignment = bestPossibleStateOutput.getResourceAssignment(resourceId); + Assert.assertNotNull(assignment); + Resource resource = cluster.getResource(resourceId); + verifySemiAutoRebalance(resource, assignment); + + System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis())); + } + + @Test + public void testClusterRebalancers() { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String testName = className + "_" + methodName; + + System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis())); + + ClusterAccessor clusterAccessor = new ClusterAccessor(_clusterId, _dataAccessor); + Cluster cluster = clusterAccessor.readCluster(); + + ResourceId resourceId = ResourceId.from("TestDB0"); + Resource resource = cluster.getResource(resourceId); + ResourceCurrentState currentStateOutput = new ResourceCurrentState(); + ResourceAssignment semiAutoResult = + resource.getRebalancerConfig().getRebalancer() + .computeResourceMapping(resource.getRebalancerConfig(), cluster, currentStateOutput); + verifySemiAutoRebalance(resource, semiAutoResult); + + System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis())); + } + + /** + * Check that a semi auto rebalance is run, and all partitions are mapped by preference list + * @param resource the resource to verify + * @param assignment the assignment to verify + */ + private void verifySemiAutoRebalance(Resource resource, ResourceAssignment assignment) { + Assert.assertEquals(assignment.getMappedPartitionIds().size(), resource.getSubUnitSet().size()); + SemiAutoRebalancerContext context = + resource.getRebalancerConfig().getRebalancerContext(SemiAutoRebalancerContext.class); + for (PartitionId partitionId : assignment.getMappedPartitionIds()) { + List preferenceList = context.getPreferenceList(partitionId); + Map replicaMap = assignment.getReplicaMap(partitionId); + Assert.assertEquals(replicaMap.size(), preferenceList.size()); + Assert.assertEquals(replicaMap.size(), r); + boolean hasMaster = false; + for (ParticipantId participant : preferenceList) { + Assert.assertTrue(replicaMap.containsKey(participant)); + State state = replicaMap.get(participant); + if (state.equals(State.from("MASTER"))) { + Assert.assertFalse(hasMaster); + hasMaster = true; + } + } + Assert.assertEquals(replicaMap.get(preferenceList.get(0)), State.from("MASTER")); + } + } + + @BeforeClass + public void beforeClass() throws Exception { + // set up a running class + // Logger.getRootLogger().setLevel(Level.INFO); + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + _clusterId = ClusterId.from(clusterName); + + System.out.println("START " + _clusterId + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + p, // partitions per resource + n, // number of nodes + r, // replicas + "MasterSlave", true); // do rebalance + + _controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + _controller.syncStart(); + + // start participants + for (int i = 0; i < n; i++) { + String instanceName = "localhost_" + (12918 + i); + + _participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + _participants[i].syncStart(); + } + + boolean result = + ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(result); + + _dataAccessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + } + + @AfterClass + public void afterClass() { + // tear down the cluster + _controller.syncStop(); + for (int i = 0; i < n; i++) { + _participants[i].syncStop(); + } + + System.out.println("END " + _clusterId + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/api/TestUpdateConfig.java b/helix-core/src/test/java/org/apache/helix/api/TestUpdateConfig.java new file mode 100644 index 0000000000..74781cd985 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/api/TestUpdateConfig.java @@ -0,0 +1,158 @@ +package org.apache.helix.api; + +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.rebalancer.context.FullAutoRebalancerContext; +import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext; +import org.testng.Assert; +import org.testng.annotations.Test; + +/* + * 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. + */ + +/** + * Testing the deltas of the various config classes. They should be able to update corresponding + * configs correctly + */ +public class TestUpdateConfig { + @Test + public void testParticipantConfigUpdate() { + final String ORIG_HOSTNAME = "host1"; + final String NEW_HOSTNAME = "host2"; + final int PORT = 1234; + final String TAG1 = "tag1"; + final String TAG2 = "tag2"; + final String TAG3 = "tag3"; + final PartitionId partition1 = PartitionId.from("resource_1"); + final PartitionId partition2 = PartitionId.from("resource_2"); + final PartitionId partition3 = PartitionId.from("resource_3"); + final ParticipantId participantId = ParticipantId.from("participant"); + + // start: add a user config, set host & port, add 2 tags and 2 disabled partition, start + // disabled + UserConfig userConfig = new UserConfig(Scope.participant(participantId)); + userConfig.setSimpleField("key1", "value1"); + ParticipantConfig config = + new ParticipantConfig.Builder(participantId).hostName(ORIG_HOSTNAME).port(PORT) + .enabled(false).addTag(TAG1).addTag(TAG2).addDisabledPartition(partition1) + .addDisabledPartition(partition2).userConfig(userConfig).build(); + UserConfig newUserConfig = new UserConfig(Scope.participant(participantId)); + newUserConfig.setSimpleField("key2", "value2"); + + // update: change host, remove a tag, add a tag, remove a disabled partition, add a disabled + // partition, change user config + ParticipantConfig updated = + new ParticipantConfig.Delta(participantId).setHostName(NEW_HOSTNAME).removeTag(TAG1) + .addTag(TAG3).removeDisabledPartition(partition1).addDisabledPartition(partition3) + .setUserConfig(newUserConfig).mergeInto(config); + Assert.assertEquals(updated.getHostName(), NEW_HOSTNAME); + Assert.assertEquals(updated.getPort(), PORT); + Assert.assertFalse(updated.hasTag(TAG1)); + Assert.assertTrue(updated.hasTag(TAG2)); + Assert.assertTrue(updated.hasTag(TAG3)); + Assert.assertFalse(updated.getDisabledPartitions().contains(partition1)); + Assert.assertTrue(updated.getDisabledPartitions().contains(partition2)); + Assert.assertTrue(updated.getDisabledPartitions().contains(partition3)); + Assert.assertNull(updated.getUserConfig().getSimpleField("key1")); + Assert.assertEquals(updated.getUserConfig().getSimpleField("key2"), "value2"); + Assert.assertFalse(updated.isEnabled()); + } + + @Test + public void testResourceConfigUpdate() { + final int OLD_BUCKET_SIZE = 0; + final int NEW_BUCKET_SIZE = 1; + final ResourceId resourceId = ResourceId.from("resource"); + + // start: add a user config, a semi auto rebalancer context, and set bucket size and batch + // message mode + UserConfig userConfig = new UserConfig(Scope.resource(resourceId)); + userConfig.setSimpleField("key1", "value1"); + SemiAutoRebalancerContext rebalancerContext = + new SemiAutoRebalancerContext.Builder(resourceId).build(); + ResourceConfig config = + new ResourceConfig.Builder(resourceId).userConfig(userConfig) + .rebalancerContext(rebalancerContext).bucketSize(OLD_BUCKET_SIZE) + .batchMessageMode(true).build(); + + // update: overwrite user config, change to full auto rebalancer context, and change the bucket + // size + UserConfig newUserConfig = new UserConfig(Scope.resource(resourceId)); + newUserConfig.setSimpleField("key2", "value2"); + FullAutoRebalancerContext newRebalancerContext = + new FullAutoRebalancerContext.Builder(resourceId).build(); + ResourceConfig updated = + new ResourceConfig.Delta(resourceId).setBucketSize(NEW_BUCKET_SIZE) + .setUserConfig(newUserConfig).setRebalancerContext(newRebalancerContext) + .mergeInto(config); + Assert.assertEquals(updated.getBucketSize(), NEW_BUCKET_SIZE); + Assert.assertTrue(updated.getBatchMessageMode()); + Assert.assertNull(updated.getRebalancerConfig().getRebalancerContext( + SemiAutoRebalancerContext.class)); + Assert.assertNotNull(updated.getRebalancerConfig().getRebalancerContext( + FullAutoRebalancerContext.class)); + Assert.assertNull(updated.getUserConfig().getSimpleField("key1")); + Assert.assertEquals(updated.getUserConfig().getSimpleField("key2"), "value2"); + } + + @Test + public void testClusterConfigUpdate() { + final ClusterId clusterId = ClusterId.from("cluster"); + final StateModelDefId masterSlave = StateModelDefId.from("MasterSlave"); + final State master = State.from("MASTER"); + final State slave = State.from("SLAVE"); + final State offline = State.from("OFFLINE"); + + // start: add a user config, add master and slave constraints + UserConfig userConfig = new UserConfig(Scope.cluster(clusterId)); + userConfig.setSimpleField("key1", "value1"); + ClusterConfig config = + new ClusterConfig.Builder(clusterId) + .addStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, master, 2) + .addStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, slave, 3) + .userConfig(userConfig).autoJoin(true).build(); + + // update: overwrite user config, change master constraint, remove slave constraint, add offline + // constraint, change auto join + UserConfig newUserConfig = new UserConfig(Scope.cluster(clusterId)); + newUserConfig.setSimpleField("key2", "value2"); + ClusterConfig updated = + new ClusterConfig.Delta(clusterId) + .addStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, master, 1) + .removeStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, slave) + .addStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, offline, "R") + .setUserConfig(newUserConfig).setAutoJoin(false).mergeInto(config); + Assert.assertEquals( + updated.getStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, master), "1"); + Assert.assertEquals( + updated.getStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, slave), "-1"); + Assert.assertEquals( + updated.getStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, offline), "R"); + Assert.assertNull(updated.getUserConfig().getSimpleField("key1")); + Assert.assertEquals(updated.getUserConfig().getSimpleField("key2"), "value2"); + Assert.assertFalse(updated.autoJoinAllowed()); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/api/accessor/TestAccessorRecreate.java b/helix-core/src/test/java/org/apache/helix/api/accessor/TestAccessorRecreate.java new file mode 100644 index 0000000000..4eebbc6ced --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/api/accessor/TestAccessorRecreate.java @@ -0,0 +1,170 @@ +package org.apache.helix.api.accessor; + +import java.util.concurrent.TimeUnit; + +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Scope; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.log4j.Logger; +import org.testng.Assert; +import org.testng.annotations.Test; + +/* + * 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 TestAccessorRecreate extends ZkUnitTestBase { + private static final Logger LOG = Logger.getLogger(TestAccessorRecreate.class); + + /** + * This test just makes sure that a cluster is only recreated if it is incomplete. This is not + * directly testing atomicity, but rather a use case where a machine died while creating the + * cluster. + */ + @Test + public void testRecreateCluster() { + final String MODIFIER = "modifier"; + final ClusterId clusterId = ClusterId.from("TestAccessorRecreate!testCluster"); + + // connect + boolean connected = _gZkClient.waitUntilConnected(30000, TimeUnit.MILLISECONDS); + if (!connected) { + LOG.warn("Connection not established"); + return; + } + BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); + HelixDataAccessor helixAccessor = new ZKHelixDataAccessor(clusterId.stringify(), baseAccessor); + ClusterAccessor accessor = new ClusterAccessor(clusterId, helixAccessor); + + // create a cluster + boolean created = createCluster(clusterId, accessor, MODIFIER, 1); + Assert.assertTrue(created); + + // read the cluster + Cluster clusterSnapshot = accessor.readCluster(); + Assert.assertEquals(clusterSnapshot.getUserConfig().getIntField(MODIFIER, -1), 1); + + // create a cluster with the same id + boolean created2 = createCluster(clusterId, accessor, MODIFIER, 2); + Assert.assertFalse(created2); // should fail since cluster exists + + // remove a required property + helixAccessor.removeProperty(helixAccessor.keyBuilder().liveInstances()); + + // try again, should work this time + created2 = createCluster(clusterId, accessor, MODIFIER, 2); + Assert.assertTrue(created2); + + // read the cluster again + clusterSnapshot = accessor.readCluster(); + Assert.assertEquals(clusterSnapshot.getUserConfig().getIntField(MODIFIER, -1), 2); + + accessor.dropCluster(); + } + + /** + * This test just makes sure that a participant is only recreated if it is incomplete. This is not + * directly testing atomicity, but rather a use case where a machine died while creating the + * participant. + */ + @Test + public void testRecreateParticipant() { + final String MODIFIER = "modifier"; + final ClusterId clusterId = ClusterId.from("testCluster"); + final ParticipantId participantId = ParticipantId.from("testParticipant"); + + // connect + boolean connected = _gZkClient.waitUntilConnected(30000, TimeUnit.MILLISECONDS); + if (!connected) { + LOG.warn("Connection not established"); + return; + } + BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); + HelixDataAccessor helixAccessor = new ZKHelixDataAccessor(clusterId.stringify(), baseAccessor); + ClusterAccessor accessor = new ClusterAccessor(clusterId, helixAccessor); + + // create the cluster + boolean clusterCreated = createCluster(clusterId, accessor, MODIFIER, 0); + Assert.assertTrue(clusterCreated); + + // create the participant + boolean created = createParticipant(participantId, accessor, MODIFIER, 1); + Assert.assertTrue(created); + + // read the participant + ParticipantAccessor participantAccessor = new ParticipantAccessor(helixAccessor); + Participant participantSnapshot = participantAccessor.readParticipant(participantId); + Assert.assertEquals(participantSnapshot.getUserConfig().getIntField(MODIFIER, -1), 1); + + // create a participant with the same id + boolean created2 = createParticipant(participantId, accessor, MODIFIER, 2); + Assert.assertFalse(created2); // should fail since participant exists + + // remove a required property + helixAccessor.removeProperty(helixAccessor.keyBuilder().messages(participantId.stringify())); + + // try again, should work this time + created2 = createParticipant(participantId, accessor, MODIFIER, 2); + Assert.assertTrue(created2); + + // read the cluster again + participantSnapshot = participantAccessor.readParticipant(participantId); + Assert.assertEquals(participantSnapshot.getUserConfig().getIntField(MODIFIER, -1), 2); + + accessor.dropCluster(); + } + + private boolean createCluster(ClusterId clusterId, ClusterAccessor accessor, String modifierName, + int modifierValue) { + // create a cluster + UserConfig userConfig = new UserConfig(Scope.cluster(clusterId)); + userConfig.setIntField(modifierName, modifierValue); + ClusterConfig cluster = new ClusterConfig.Builder(clusterId).userConfig(userConfig).build(); + return accessor.createCluster(cluster); + } + + private boolean createParticipant(ParticipantId participantId, ClusterAccessor accessor, + String modifierName, int modifierValue) { + // create a participant + UserConfig userConfig = new UserConfig(Scope.participant(participantId)); + userConfig.setIntField(modifierName, modifierValue); + ParticipantConfig participant = + new ParticipantConfig.Builder(participantId).hostName("host").port(0) + .userConfig(userConfig).build(); + return accessor.addParticipantToCluster(participant); + } + // private HelixLockable lockProvider() { + // return new HelixLockable() { + // @Override + // public HelixLock getLock(ClusterId clusterId, Scope scope) { + // return new ZKHelixLock(clusterId, scope, _gZkClient); + // } + // }; + // } +} diff --git a/helix-core/src/test/java/org/apache/helix/api/accessor/TestAtomicAccessors.java b/helix-core/src/test/java/org/apache/helix/api/accessor/TestAtomicAccessors.java new file mode 100644 index 0000000000..443c3db047 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/api/accessor/TestAtomicAccessors.java @@ -0,0 +1,200 @@ +package org.apache.helix.api.accessor; + +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.lock.HelixLock; +import org.apache.helix.lock.HelixLockable; +import org.apache.helix.lock.zk.ZKHelixLock; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.manager.zk.ZkClient; +import org.testng.Assert; +import org.testng.annotations.Test; + +/* + * 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. + */ + +/** + * Test that the atomic accessors behave atomically in response to interwoven updates. + */ +public class TestAtomicAccessors extends ZkUnitTestBase { + private static final long TIMEOUT = 30000L; + private static final long EXTRA_WAIT = 10000L; + + @Test + public void testClusterUpdates() { + final ClusterId clusterId = ClusterId.from("TestAtomicAccessors!testCluster"); + final BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); + final HelixDataAccessor helixAccessor = + new ZKHelixDataAccessor(clusterId.stringify(), baseAccessor); + final LockProvider lockProvider = new LockProvider(); + final StateModelDefId stateModelDefId = StateModelDefId.from("FakeModel"); + final State state = State.from("fake"); + final int constraint1 = 10; + final int constraint2 = 11; + final String key1 = "key1"; + final String key2 = "key2"; + + // set up the cluster (non-atomically since this concurrency comes later) + ClusterAccessor accessor = new ClusterAccessor(clusterId, helixAccessor); + ClusterConfig config = new ClusterConfig.Builder(clusterId).build(); + boolean created = accessor.createCluster(config); + Assert.assertTrue(created); + + // thread that will update the cluster in one way + Thread t1 = new Thread() { + @Override + public void run() { + UserConfig userConfig = new UserConfig(Scope.cluster(clusterId)); + userConfig.setBooleanField(key1, true); + ClusterConfig.Delta delta = + new ClusterConfig.Delta(clusterId).addStateUpperBoundConstraint( + Scope.cluster(clusterId), stateModelDefId, state, constraint1).setUserConfig( + userConfig); + ClusterAccessor accessor = + new AtomicClusterAccessor(clusterId, helixAccessor, lockProvider); + accessor.updateCluster(delta); + } + }; + + // thread that will update the cluster in another way + Thread t2 = new Thread() { + @Override + public void run() { + UserConfig userConfig = new UserConfig(Scope.cluster(clusterId)); + userConfig.setBooleanField(key2, true); + ClusterConfig.Delta delta = + new ClusterConfig.Delta(clusterId).addStateUpperBoundConstraint( + Scope.cluster(clusterId), stateModelDefId, state, constraint2).setUserConfig( + userConfig); + ClusterAccessor accessor = + new AtomicClusterAccessor(clusterId, helixAccessor, lockProvider); + accessor.updateCluster(delta); + } + }; + + // start the threads + t1.start(); + t2.start(); + + // make sure the threads are done + long startTime = System.currentTimeMillis(); + try { + t1.join(TIMEOUT); + t2.join(TIMEOUT); + } catch (InterruptedException e) { + Assert.fail(e.getMessage()); + t1.interrupt(); + t2.interrupt(); + } + long endTime = System.currentTimeMillis(); + if (endTime - startTime > TIMEOUT - EXTRA_WAIT) { + Assert.fail("Test timed out"); + t1.interrupt(); + t2.interrupt(); + } + + Assert.assertTrue(lockProvider.hasLockBlocked()); + + accessor.dropCluster(); + } + + /** + * A HelixLockable that returns an instrumented ZKHelixLock + */ + private class LockProvider implements HelixLockable { + private HelixLock _firstLock = null; + private AtomicBoolean _hasSecondBlocked = new AtomicBoolean(false); + + @Override + public synchronized HelixLock getLock(ClusterId clusterId, Scope scope) { + return new MyLock(clusterId, scope, _gZkClient); + } + + /** + * Check if a lock object has blocked + * @return true if a block happened, false otherwise + */ + public synchronized boolean hasLockBlocked() { + return _hasSecondBlocked.get(); + } + + /** + * An instrumented ZKHelixLock + */ + private class MyLock extends ZKHelixLock { + /** + * Instantiate a lock that instruments a ZKHelixLock + * @param clusterId the cluster to lock + * @param scope the scope to lock on + * @param zkClient an active ZooKeeper client + */ + public MyLock(ClusterId clusterId, Scope scope, ZkClient zkClient) { + super(clusterId, scope, zkClient); + } + + @Override + public synchronized boolean lock() { + // synchronize here to ensure atomic set and so that the first lock is the first one who + // gets to lock + if (_firstLock == null) { + _firstLock = this; + } + return super.lock(); + } + + @Override + public boolean unlock() { + if (_firstLock == this) { + // wait to unlock until another thread has blocked + synchronized (_hasSecondBlocked) { + if (!_hasSecondBlocked.get()) { + try { + _hasSecondBlocked.wait(TIMEOUT); + } catch (InterruptedException e) { + } + } + } + } + return super.unlock(); + } + + @Override + protected void setBlocked(boolean isBlocked) { + if (isBlocked) { + synchronized (_hasSecondBlocked) { + _hasSecondBlocked.set(true); + _hasSecondBlocked.notify(); + } + } + super.setBlocked(isBlocked); + } + } + } +} diff --git a/helix-core/src/test/java/org/apache/helix/controller/alert/TestAlertAction.java b/helix-core/src/test/java/org/apache/helix/controller/alert/TestAlertAction.java new file mode 100644 index 0000000000..9a6d636bf5 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/controller/alert/TestAlertAction.java @@ -0,0 +1,95 @@ +package org.apache.helix.controller.alert; + +/* + * 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 TestAlertAction { + @Test + public void testBasic() throws Exception { + AlertAction.Builder builder = new AlertAction.Builder(); + + AlertAction action = + builder.cmd("enableInstance").args("TestCluster", "{node}", "false").build(); + Assert.assertNotNull(action.getCliArgs()); + Assert.assertEquals(action.getCliArgs().length, 4); + Assert.assertEquals(action.getCliArgs()[0], "enableInstance"); + Assert.assertEquals(action.getCliArgs()[1], "TestCluster"); + Assert.assertEquals(action.getCliArgs()[2], "{node}"); + Assert.assertEquals(action.getCliArgs()[3], "false"); + Assert.assertEquals(action.toString(), "(enableInstance)(TestCluster {node} false)"); + + action = AlertAction.from("(enableInstance)(TestCluster {node} true)"); + Assert.assertNotNull(action.getCliArgs()); + Assert.assertEquals(action.getCliArgs().length, 4); + Assert.assertEquals(action.getCliArgs()[0], "enableInstance"); + Assert.assertEquals(action.getCliArgs()[1], "TestCluster"); + Assert.assertEquals(action.getCliArgs()[2], "{node}"); + Assert.assertEquals(action.getCliArgs()[3], "true"); + } + + @Test + public void testInvalidActionStr() { + try { + AlertAction.from("(invalidCmd)(TestCluster {node} true)"); + Assert.fail("Should fail on invalid HelixAdminCmd"); + } catch (IllegalArgumentException e) { + // ok + } + + try { + AlertAction.from("(enableInstance)(TestCluster {invalidScope} true)"); + Assert.fail("Should fail on invalid alertScope"); + } catch (IllegalArgumentException e) { + // ok + } + + try { + AlertAction.from("(enableInstance)()(TestCluster {node} true)"); + Assert.fail("Should fail on invalid alertAction format"); + } catch (IllegalArgumentException e) { + // ok + } + + } + + @Test + public void testInvaidConstructorArgument() { + try { + new AlertAction(null, new String[] { + "TestCluster", "{node}", "true)" + }); + Assert.fail("Should fail on null HelixAdminCmd"); + } catch (NullPointerException e) { + // ok + } + + try { + new AlertAction("enableInstance", new String[] { + "TestCluster", "{invalidScope}", "true)" + }); + Assert.fail("Should fail on invalid alert scope"); + } catch (IllegalArgumentException e) { + // ok + } + + } +} diff --git a/helix-core/src/test/java/org/apache/helix/controller/alert/TestAlertName.java b/helix-core/src/test/java/org/apache/helix/controller/alert/TestAlertName.java new file mode 100644 index 0000000000..1478be51fa --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/controller/alert/TestAlertName.java @@ -0,0 +1,110 @@ +package org.apache.helix.controller.alert; + +/* + * 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.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.alert.AlertScope.AlertScopeField; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestAlertName { + + @Test + public void testBasic() { + AlertName.Builder builder = new AlertName.Builder(); + AlertName name = + builder.cluster(ClusterId.from("TestCluster")).node(ParticipantId.from("localhost_12918")) + .resource(ResourceId.from("TestDB")).partitionId(PartitionId.from("TestDB_0")) + .metric("latency95").largerThan("1000").build(); + Assert.assertEquals(name.toString(), + "(TestCluster.%.localhost_12918.TestDB.TestDB_0)(latency95)>(1000)"); + + name = AlertName.from("(TestCluster.%.localhost_12918.TestDB.TestDB_0)(latency95)>(1000)"); + Assert.assertEquals(name._scope.getClusterId(), ClusterId.from("TestCluster")); + Assert.assertNull(name._scope.get(AlertScopeField.tenant)); + Assert.assertEquals(name._scope.get(AlertScopeField.node), + ParticipantId.from("localhost_12918")); + Assert.assertEquals(name._scope.get(AlertScopeField.resource), ParticipantId.from("TestDB")); + Assert.assertEquals(name._scope.get(AlertScopeField.partition), ParticipantId.from("TestDB_0")); + Assert.assertEquals(name._comparator, AlertComparator.LARGER); + Assert.assertEquals(name._metric, "latency95"); + Assert.assertEquals(name._value, "1000"); + } + + @Test + public void testInvalidAlertNameString() { + try { + AlertName.from("(TestCluster.%.localhost_12918.TestDB.TestDB_0)(latency95)><(1000)"); + Assert.fail("Should fail on invalid comparator ><"); + } catch (IllegalArgumentException e) { + // ok + } + + try { + AlertName.from("(TestCluster.%.localhost_12918.TestDB.TestDB_0)()(latency95)>()(1000)"); + Assert.fail("Should fail on invalid alert name format"); + } catch (IllegalArgumentException e) { + // ok + } + + try { + AlertName.from("(TestCluster.%.localhost_12918.TestDB.TestDB_0)>(1000)"); + Assert.fail("Should fail on missing metric field"); + } catch (IllegalArgumentException e) { + // ok + } + + try { + AlertName.from("(TestCluster.%.localhost_12918.TestDB.TestDB_0.blabla)(latency95)>(1000)"); + Assert.fail("Should fail on invalid too many scope arguments"); + } catch (IllegalArgumentException e) { + // ok + } + } + + @Test + public void testMatch() { + AlertName.Builder builder; + AlertName matchingName; + boolean match; + + builder = new AlertName.Builder(); + AlertName name = + builder.cluster(ClusterId.from("TestCluster")).resource(ResourceId.from("TestDB")) + .metric("latency95").largerThan("1000").build(); + + builder = new AlertName.Builder(); + matchingName = + builder.cluster(ClusterId.from("TestCluster")).node(ParticipantId.from("localhost_12918")) + .resource(ResourceId.from("TestDB")).metric("latency95").largerThan("1000").build(); + match = name.match(matchingName); + Assert.assertTrue(match); + + builder = new AlertName.Builder(); + matchingName = + builder.cluster(ClusterId.from("TestCluster")).node(ParticipantId.from("localhost_12918")) + .resource(ResourceId.from("MyDB")).metric("latency95").largerThan("1000").build(); + match = name.match(matchingName); + Assert.assertFalse(match); + } +} \ No newline at end of file diff --git a/helix-core/src/test/java/org/apache/helix/controller/alert/TestAlertScope.java b/helix-core/src/test/java/org/apache/helix/controller/alert/TestAlertScope.java new file mode 100644 index 0000000000..45e85301dd --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/controller/alert/TestAlertScope.java @@ -0,0 +1,78 @@ +package org.apache.helix.controller.alert; + +/* + * 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.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.alert.AlertScope.AlertScopeField; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestAlertScope { + + @Test + public void testBasic() { + AlertScope scope = + new AlertScope(ClusterId.from("TestCluster"), null, ParticipantId.from("localhost_12918"), + ResourceId.from("TestDB"), null); + + Assert.assertEquals(scope.getClusterId(), ClusterId.from("TestCluster")); + Assert.assertEquals(scope.get(AlertScopeField.node), ParticipantId.from("localhost_12918")); + Assert.assertEquals(scope.get(AlertScopeField.resource), ResourceId.from("TestDB")); + Assert.assertEquals(scope.toString(), "TestCluster.%.localhost_12918.TestDB.%"); + + AlertScope matchingScope = new AlertScope("TestCluster", "Tenant1", "localhost_12918"); + boolean match = scope.match(matchingScope); + Assert.assertTrue(match); + + } + + @Test + public void testEmptyClusterId() { + AlertScope scope = new AlertScope((String[]) null); + Assert.assertEquals(scope.toString(), "%.%.%.%.%"); + + scope = new AlertScope(new String[] {}); + Assert.assertEquals(scope.toString(), "%.%.%.%.%"); + + scope = new AlertScope(null, "Tenant1", "localhost_12918"); + Assert.assertEquals(scope.toString(), "%.Tenant1.localhost_12918.%.%"); + } + + @Test + public void testWildcard() { + AlertScope scope = new AlertScope("%"); + Assert.assertEquals(scope.toString(), "%.%.%.%.%"); + + scope = new AlertScope("TestCluster", "%", "localhost_12918"); + Assert.assertEquals(scope.toString(), "TestCluster.%.localhost_12918.%.%"); + } + + @Test + public void testTooManyArguments() { + try { + new AlertScope("TestCluster", null, "localhost_12918", "TestDB", "TestDB_0", "blabla"); + Assert.fail("Should fail on too many arguments"); + } catch (IllegalArgumentException e) { + // ok + } + } +} diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/context/TestSerializeRebalancerContext.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/context/TestSerializeRebalancerContext.java new file mode 100644 index 0000000000..5bbe54f4ca --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/context/TestSerializeRebalancerContext.java @@ -0,0 +1,103 @@ +package org.apache.helix.controller.rebalancer.context; + +import java.util.Map; + +import org.apache.helix.api.Partition; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.model.ResourceConfiguration; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + +/* + * 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. + */ + +/** + * Ensure that a RebalancerContext of a specified type is able to be serialized and deserialized. + */ +public class TestSerializeRebalancerContext { + @Test + public void basicTest() { + // populate a context + CustomRebalancerContext context = new CustomRebalancerContext(); + context.setAnyLiveParticipant(false); + context.setMaxPartitionsPerParticipant(Integer.MAX_VALUE); + Map partitionMap = Maps.newHashMap(); + ResourceId resourceId = ResourceId.from("testResource"); + PartitionId partitionId = PartitionId.from(resourceId, "0"); + partitionMap.put(partitionId, new Partition(partitionId)); + context.setPartitionMap(partitionMap); + Map> preferenceMaps = Maps.newHashMap(); + ParticipantId participant1 = ParticipantId.from("participant1"); + ParticipantId participant2 = ParticipantId.from("participant2"); + Map preferenceMap = + ImmutableMap.of(participant1, State.from("MASTER"), participant2, State.from("SLAVE")); + preferenceMaps.put(partitionId, preferenceMap); + context.setPreferenceMaps(preferenceMaps); + context.setReplicaCount(3); + context.setStateModelDefId(StateModelDefId.from("MasterSlave")); + context.setResourceId(resourceId); + + // serialize and deserialize by wrapping in a config + RebalancerConfig config = new RebalancerConfig(context); + CustomRebalancerContext deserialized = + config.getRebalancerContext(CustomRebalancerContext.class); + + // check to make sure that the two objects contain the same data + Assert.assertNotNull(deserialized); + Assert.assertEquals(deserialized.anyLiveParticipant(), context.anyLiveParticipant()); + Assert.assertEquals(deserialized.getPreferenceMap(partitionId).get(participant1), context + .getPreferenceMap(partitionId).get(participant1)); + Assert.assertEquals(deserialized.getPreferenceMap(partitionId).get(participant2), context + .getPreferenceMap(partitionId).get(participant2)); + Assert.assertEquals(deserialized.getReplicaCount(), context.getReplicaCount()); + Assert.assertEquals(deserialized.getStateModelDefId(), context.getStateModelDefId()); + Assert.assertEquals(deserialized.getResourceId(), context.getResourceId()); + + // wrap in a physical config and then unwrap it + ResourceConfiguration physicalConfig = new ResourceConfiguration(resourceId); + physicalConfig.addNamespacedConfig(config.toNamespacedConfig()); + RebalancerConfig extractedConfig = new RebalancerConfig(physicalConfig); + CustomRebalancerContext extractedContext = + extractedConfig.getRebalancerContext(CustomRebalancerContext.class); + + // make sure the unwrapped data hasn't changed + Assert.assertNotNull(extractedContext); + Assert.assertEquals(extractedContext.anyLiveParticipant(), context.anyLiveParticipant()); + Assert.assertEquals(extractedContext.getPreferenceMap(partitionId).get(participant1), context + .getPreferenceMap(partitionId).get(participant1)); + Assert.assertEquals(extractedContext.getPreferenceMap(partitionId).get(participant2), context + .getPreferenceMap(partitionId).get(participant2)); + Assert.assertEquals(extractedContext.getReplicaCount(), context.getReplicaCount()); + Assert.assertEquals(extractedContext.getStateModelDefId(), context.getStateModelDefId()); + Assert.assertEquals(extractedContext.getResourceId(), context.getResourceId()); + + // make sure that it's legal to use a base rebalancer context + RebalancerContext rebalancerContext = + extractedConfig.getRebalancerContext(RebalancerContext.class); + Assert.assertNotNull(rebalancerContext); + Assert.assertEquals(rebalancerContext.getResourceId(), context.getResourceId()); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/BaseStageTest.java b/helix-core/src/test/java/org/apache/helix/controller/stages/BaseStageTest.java index 64e378ed3e..e53530c7e2 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/BaseStageTest.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/BaseStageTest.java @@ -29,15 +29,24 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; import org.apache.helix.Mocks; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.Resource; +import org.apache.helix.api.Scope; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.ResourceConfig.ResourceType; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.pipeline.Stage; import org.apache.helix.controller.pipeline.StageContext; -import org.apache.helix.controller.stages.ClusterEvent; +import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.Resource; import org.apache.helix.model.StateModelDefinition; import org.apache.helix.tools.StateModelConfigGenerator; import org.testng.annotations.AfterClass; @@ -90,7 +99,7 @@ protected List setupIdealState(int nodes, String[] resources, int pa record.setListField(resourceName + "_" + p, value); } IdealState idealState = new IdealState(record); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setRebalanceMode(rebalanceMode); idealState.setNumPartitions(partitions); idealStates.add(idealState); @@ -107,11 +116,16 @@ protected List setupIdealState(int nodes, String[] resources, int pa protected void setupLiveInstances(int numLiveInstances) { // setup liveInstances for (int i = 0; i < numLiveInstances; i++) { - LiveInstance liveInstance = new LiveInstance("localhost_" + i); + String instanceName = "localhost_" + i; + InstanceConfig instanceConfig = new InstanceConfig(ParticipantId.from(instanceName)); + instanceConfig.setHostName("localhost"); + instanceConfig.setPort(Integer.toString(i)); + LiveInstance liveInstance = new LiveInstance(instanceName); liveInstance.setSessionId("session_" + i); Builder keyBuilder = accessor.keyBuilder(); - accessor.setProperty(keyBuilder.liveInstance("localhost_" + i), liveInstance); + accessor.setProperty(keyBuilder.instanceConfig(instanceName), instanceConfig); + accessor.setProperty(keyBuilder.liveInstance(instanceName), liveInstance); } } @@ -128,32 +142,40 @@ protected void runStage(ClusterEvent event, Stage stage) { stage.postProcess(); } - protected void setupStateModel() { - ZNRecord masterSlave = new StateModelConfigGenerator().generateConfigForMasterSlave(); - + protected Map setupStateModel() { Builder keyBuilder = accessor.keyBuilder(); - accessor.setProperty(keyBuilder.stateModelDef(masterSlave.getId()), new StateModelDefinition( - masterSlave)); + Map defs = + new HashMap(); - ZNRecord leaderStandby = new StateModelConfigGenerator().generateConfigForLeaderStandby(); - accessor.setProperty(keyBuilder.stateModelDef(leaderStandby.getId()), new StateModelDefinition( - leaderStandby)); + ZNRecord masterSlave = StateModelConfigGenerator.generateConfigForMasterSlave(); + StateModelDefinition masterSlaveDef = new StateModelDefinition(masterSlave); + defs.put(StateModelDefId.from(masterSlaveDef.getId()), masterSlaveDef); + accessor.setProperty(keyBuilder.stateModelDef(masterSlave.getId()), masterSlaveDef); - ZNRecord onlineOffline = new StateModelConfigGenerator().generateConfigForOnlineOffline(); - accessor.setProperty(keyBuilder.stateModelDef(onlineOffline.getId()), new StateModelDefinition( - onlineOffline)); + ZNRecord leaderStandby = StateModelConfigGenerator.generateConfigForLeaderStandby(); + StateModelDefinition leaderStandbyDef = new StateModelDefinition(leaderStandby); + defs.put(StateModelDefId.from(leaderStandbyDef.getId()), leaderStandbyDef); + accessor.setProperty(keyBuilder.stateModelDef(leaderStandby.getId()), leaderStandbyDef); + + ZNRecord onlineOffline = StateModelConfigGenerator.generateConfigForOnlineOffline(); + StateModelDefinition onlineOfflineDef = new StateModelDefinition(onlineOffline); + defs.put(StateModelDefId.from(onlineOfflineDef.getId()), onlineOfflineDef); + accessor.setProperty(keyBuilder.stateModelDef(onlineOffline.getId()), onlineOfflineDef); + + return defs; } - protected Map getResourceMap() { - Map resourceMap = new HashMap(); - Resource testResource = new Resource("testResourceName"); - testResource.setStateModelDefRef("MasterSlave"); - testResource.addPartition("testResourceName_0"); - testResource.addPartition("testResourceName_1"); - testResource.addPartition("testResourceName_2"); - testResource.addPartition("testResourceName_3"); - testResource.addPartition("testResourceName_4"); - resourceMap.put("testResourceName", testResource); + protected Map getResourceMap(List idealStates) { + Map resourceMap = new HashMap(); + for (IdealState idealState : idealStates) { + ResourceId resourceId = idealState.getResourceId(); + RebalancerContext context = PartitionedRebalancerContext.from(idealState); + Resource resource = + new Resource(resourceId, ResourceType.DATA, idealState, null, null, context, + new UserConfig(Scope.resource(resourceId)), idealState.getBucketSize(), + idealState.getBatchMessageMode()); + resourceMap.put(resourceId, resource.getConfig()); + } return resourceMap; } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java b/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java index 26da8ee906..d0305fbccf 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java @@ -25,7 +25,6 @@ import org.apache.helix.ControllerChangeListener; import org.apache.helix.CurrentStateChangeListener; import org.apache.helix.ExternalViewChangeListener; -import org.apache.helix.HealthStateChangeListener; import org.apache.helix.HelixAdmin; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; @@ -40,8 +39,8 @@ import org.apache.helix.PropertyKey; import org.apache.helix.ScopedConfigChangeListener; import org.apache.helix.ZNRecord; -import org.apache.helix.healthcheck.ParticipantHealthReportCollector; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.monitoring.MonitoringServer; import org.apache.helix.participant.StateMachineEngine; import org.apache.helix.store.zk.ZkHelixPropertyStore; @@ -157,12 +156,6 @@ public ClusterMessagingService getMessagingService() { return null; } - @Override - public ParticipantHealthReportCollector getHealthReportCollector() { - // TODO Auto-generated method stub - return null; - } - @Override public InstanceType getInstanceType() { // TODO Auto-generated method stub @@ -175,13 +168,6 @@ public String getVersion() { return null; } - @Override - public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName) - throws Exception { - // TODO Auto-generated method stub - - } - @Override public StateMachineEngine getStateMachineEngine() { // TODO Auto-generated method stub @@ -254,4 +240,11 @@ public HelixManagerProperties getProperties() { // TODO Auto-generated method stub return null; } + + @Override + public void addControllerMessageListener(MessageListener listener) { + // TODO Auto-generated method stub + + } + } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleCalcStageCompatibility.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleCalcStageCompatibility.java index 33570a074e..b2581a37c4 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleCalcStageCompatibility.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleCalcStageCompatibility.java @@ -21,15 +21,19 @@ import java.util.ArrayList; import java.util.Date; +import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.IdealStateModeProperty; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; import org.testng.AssertJUnit; import org.testng.annotations.Test; @@ -41,18 +45,20 @@ public class TestBestPossibleCalcStageCompatibility extends BaseStageTest { @Test public void testSemiAutoModeCompatibility() { - System.out.println("START TestBestPossibleStateCalcStage at " - + new Date(System.currentTimeMillis())); + System.out + .println("START TestBestPossibleStateCalcStageCompatibility_testSemiAutoModeCompatibility at " + + new Date(System.currentTimeMillis())); String[] resources = new String[] { "testResourceName" }; - setupIdealStateDeprecated(5, resources, 10, 1, IdealStateModeProperty.AUTO); + List idealStates = + setupIdealStateDeprecated(5, resources, 10, 1, IdealStateModeProperty.AUTO); setupLiveInstances(5); setupStateModel(); - Map resourceMap = getResourceMap(); - CurrentStateOutput currentStateOutput = new CurrentStateOutput(); + Map resourceMap = getResourceMap(idealStates); + ResourceCurrentState currentStateOutput = new ResourceCurrentState(); event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap); event.addAttribute(AttributeName.CURRENT_STATE.toString(), currentStateOutput); @@ -64,28 +70,33 @@ public void testSemiAutoModeCompatibility() { BestPossibleStateOutput output = event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); for (int p = 0; p < 5; p++) { - Partition resource = new Partition("testResourceName_" + p); - AssertJUnit.assertEquals("MASTER", output.getInstanceStateMap("testResourceName", resource) - .get("localhost_" + (p + 1) % 5)); + Map replicaMap = + output.getResourceAssignment(ResourceId.from("testResourceName")).getReplicaMap( + PartitionId.from("testResourceName_" + p)); + AssertJUnit.assertEquals(State.from("MASTER"), + replicaMap.get(ParticipantId.from("localhost_" + (p + 1) % 5))); } - System.out.println("END TestBestPossibleStateCalcStage at " - + new Date(System.currentTimeMillis())); + System.out + .println("END TestBestPossibleStateCalcStageCompatibility_testSemiAutoModeCompatibility at " + + new Date(System.currentTimeMillis())); } @Test public void testCustomModeCompatibility() { - System.out.println("START TestBestPossibleStateCalcStage at " - + new Date(System.currentTimeMillis())); + System.out + .println("START TestBestPossibleStateCalcStageCompatibility_testCustomModeCompatibility at " + + new Date(System.currentTimeMillis())); String[] resources = new String[] { "testResourceName" }; - setupIdealStateDeprecated(5, resources, 10, 1, IdealStateModeProperty.CUSTOMIZED); + List idealStates = + setupIdealStateDeprecated(5, resources, 10, 1, IdealStateModeProperty.CUSTOMIZED); setupLiveInstances(5); setupStateModel(); - Map resourceMap = getResourceMap(); - CurrentStateOutput currentStateOutput = new CurrentStateOutput(); + Map resourceMap = getResourceMap(idealStates); + ResourceCurrentState currentStateOutput = new ResourceCurrentState(); event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap); event.addAttribute(AttributeName.CURRENT_STATE.toString(), currentStateOutput); @@ -97,12 +108,15 @@ public void testCustomModeCompatibility() { BestPossibleStateOutput output = event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); for (int p = 0; p < 5; p++) { - Partition resource = new Partition("testResourceName_" + p); - AssertJUnit.assertNull(output.getInstanceStateMap("testResourceName", resource).get( - "localhost_" + (p + 1) % 5)); + Map replicaMap = + output.getResourceAssignment(ResourceId.from("testResourceName")).getReplicaMap( + PartitionId.from("testResourceName_" + p)); + AssertJUnit.assertEquals(State.from("MASTER"), + replicaMap.get(ParticipantId.from("localhost_" + (p + 1) % 5))); } - System.out.println("END TestBestPossibleStateCalcStage at " - + new Date(System.currentTimeMillis())); + System.out + .println("END TestBestPossibleStateCalcStageCompatibility_testCustomModeCompatibility at " + + new Date(System.currentTimeMillis())); } protected List setupIdealStateDeprecated(int nodes, String[] resources, @@ -115,16 +129,19 @@ protected List setupIdealStateDeprecated(int nodes, String[] resourc for (int i = 0; i < resources.length; i++) { String resourceName = resources[i]; - ZNRecord record = new ZNRecord(resourceName); + IdealState idealState = new IdealState(resourceName); for (int p = 0; p < partitions; p++) { - List value = new ArrayList(); + List value = new ArrayList(); for (int r = 0; r < replicas; r++) { - value.add("localhost_" + (p + r + 1) % nodes); + value.add(ParticipantId.from("localhost_" + (p + r + 1) % nodes)); } - record.setListField(resourceName + "_" + p, value); + idealState.setPreferenceList(PartitionId.from(resourceName + "_" + p), value); + Map preferenceMap = new HashMap(); + preferenceMap.put(ParticipantId.from("localhost_" + (p + 1) % 5), State.from("MASTER")); + idealState.setParticipantStateMap( + PartitionId.from(ResourceId.from(resourceName), Integer.toString(p)), preferenceMap); } - IdealState idealState = new IdealState(record); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setIdealStateMode(mode.toString()); idealState.setNumPartitions(partitions); idealStates.add(idealState); diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleStateCalcStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleStateCalcStage.java index 82c7b37823..d116182eb0 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleStateCalcStage.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleStateCalcStage.java @@ -20,15 +20,15 @@ */ import java.util.Date; +import java.util.List; import java.util.Map; -import org.apache.helix.controller.stages.AttributeName; -import org.apache.helix.controller.stages.BestPossibleStateCalcStage; -import org.apache.helix.controller.stages.BestPossibleStateOutput; -import org.apache.helix.controller.stages.CurrentStateOutput; -import org.apache.helix.controller.stages.ReadClusterDataStage; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.RebalanceMode; import org.testng.AssertJUnit; import org.testng.annotations.Test; @@ -43,12 +43,12 @@ public void testSimple() { String[] resources = new String[] { "testResourceName" }; - setupIdealState(5, resources, 10, 1, RebalanceMode.SEMI_AUTO); + List idealStates = setupIdealState(5, resources, 10, 1, RebalanceMode.SEMI_AUTO); setupLiveInstances(5); setupStateModel(); - Map resourceMap = getResourceMap(); - CurrentStateOutput currentStateOutput = new CurrentStateOutput(); + Map resourceMap = getResourceMap(idealStates); + ResourceCurrentState currentStateOutput = new ResourceCurrentState(); event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap); event.addAttribute(AttributeName.CURRENT_STATE.toString(), currentStateOutput); @@ -60,9 +60,11 @@ public void testSimple() { BestPossibleStateOutput output = event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString()); for (int p = 0; p < 5; p++) { - Partition resource = new Partition("testResourceName_" + p); - AssertJUnit.assertEquals("MASTER", output.getInstanceStateMap("testResourceName", resource) - .get("localhost_" + (p + 1) % 5)); + Map replicaMap = + output.getResourceAssignment(ResourceId.from("testResourceName")).getReplicaMap( + PartitionId.from("testResourceName_" + p)); + AssertJUnit.assertEquals(State.from("MASTER"), + replicaMap.get(ParticipantId.from("localhost_" + (p + 1) % 5))); } System.out.println("END TestBestPossibleStateCalcStage at " + new Date(System.currentTimeMillis())); diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestCompatibilityCheckStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestCompatibilityCheckStage.java index 391d1af881..2c1aedf483 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestCompatibilityCheckStage.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestCompatibilityCheckStage.java @@ -23,15 +23,15 @@ import java.util.List; import org.apache.helix.Mocks; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.pipeline.StageContext; -import org.apache.helix.controller.stages.CompatibilityCheckStage; -import org.apache.helix.controller.stages.ReadClusterDataStage; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; import org.apache.helix.model.IdealState; +import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.LiveInstance.LiveInstanceProperty; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.testng.Assert; import org.testng.annotations.Test; @@ -50,10 +50,10 @@ private void prepare(String controllerVersion, String participantVersion, // set ideal state String resourceName = "testResource"; ZNRecord record = - DefaultIdealStateCalculator.calculateIdealState(instances, partitions, replicas, - resourceName, "MASTER", "SLAVE"); + DefaultTwoStateStrategy.calculateIdealState(instances, partitions, replicas, resourceName, + "MASTER", "SLAVE"); IdealState idealState = new IdealState(record); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); Builder keyBuilder = accessor.keyBuilder(); accessor.setProperty(keyBuilder.idealStates(resourceName), idealState); @@ -66,6 +66,8 @@ record = new ZNRecord("localhost_0"); LiveInstance liveInstance = new LiveInstance(record); liveInstance.setSessionId("session_0"); accessor.setProperty(keyBuilder.liveInstance("localhost_0"), liveInstance); + InstanceConfig config = new InstanceConfig(liveInstance.getInstanceName()); + accessor.setProperty(keyBuilder.instanceConfig(config.getInstanceName()), config); if (controllerVersion != null) { ((Mocks.MockManager) manager).setVersion(controllerVersion); diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestCurrentStateComputationStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestCurrentStateComputationStage.java index 7687e189f2..65d551d2fd 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestCurrentStateComputationStage.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestCurrentStateComputationStage.java @@ -19,18 +19,22 @@ * under the License. */ +import java.util.List; import java.util.Map; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.stages.AttributeName; -import org.apache.helix.controller.stages.CurrentStateComputationStage; -import org.apache.helix.controller.stages.CurrentStateOutput; -import org.apache.helix.controller.stages.ReadClusterDataStage; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.model.CurrentState; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; -import org.apache.helix.model.Resource; import org.testng.AssertJUnit; import org.testng.annotations.Test; @@ -38,21 +42,29 @@ public class TestCurrentStateComputationStage extends BaseStageTest { @Test public void testEmptyCS() { - Map resourceMap = getResourceMap(); + String[] resources = new String[] { + "testResourceName" + }; + List idealStates = setupIdealState(5, resources, 10, 1, RebalanceMode.SEMI_AUTO); + Map resourceMap = getResourceMap(idealStates); event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap); CurrentStateComputationStage stage = new CurrentStateComputationStage(); runStage(event, new ReadClusterDataStage()); runStage(event, stage); - CurrentStateOutput output = event.getAttribute(AttributeName.CURRENT_STATE.toString()); + ResourceCurrentState output = event.getAttribute(AttributeName.CURRENT_STATE.toString()); AssertJUnit.assertEquals( - output.getCurrentStateMap("testResourceName", new Partition("testResourceName_0")).size(), - 0); + output.getCurrentStateMap(ResourceId.from("testResourceName"), + PartitionId.from("testResourceName_0")).size(), 0); } @Test public void testSimpleCS() { // setup resource - Map resourceMap = getResourceMap(); + String[] resources = new String[] { + "testResourceName" + }; + List idealStates = setupIdealState(5, resources, 10, 1, RebalanceMode.SEMI_AUTO); + Map resourceMap = getResourceMap(idealStates); setupLiveInstances(5); @@ -60,42 +72,42 @@ public void testSimpleCS() { CurrentStateComputationStage stage = new CurrentStateComputationStage(); runStage(event, new ReadClusterDataStage()); runStage(event, stage); - CurrentStateOutput output1 = event.getAttribute(AttributeName.CURRENT_STATE.toString()); + ResourceCurrentState output1 = event.getAttribute(AttributeName.CURRENT_STATE.toString()); AssertJUnit.assertEquals( - output1.getCurrentStateMap("testResourceName", new Partition("testResourceName_0")).size(), - 0); + output1.getCurrentStateMap(ResourceId.from("testResourceName"), + PartitionId.from("testResourceName_0")).size(), 0); // Add a state transition messages - Message message = new Message(Message.MessageType.STATE_TRANSITION, "msg1"); - message.setFromState("OFFLINE"); - message.setToState("SLAVE"); - message.setResourceName("testResourceName"); - message.setPartitionName("testResourceName_1"); + Message message = new Message(Message.MessageType.STATE_TRANSITION, MessageId.from("msg1")); + message.setFromState(State.from("OFFLINE")); + message.setToState(State.from("SLAVE")); + message.setResourceId(ResourceId.from("testResourceName")); + message.setPartitionId(PartitionId.from("testResourceName_1")); message.setTgtName("localhost_3"); - message.setTgtSessionId("session_3"); + message.setTgtSessionId(SessionId.from("session_3")); Builder keyBuilder = accessor.keyBuilder(); accessor.setProperty(keyBuilder.message("localhost_" + 3, message.getId()), message); runStage(event, new ReadClusterDataStage()); runStage(event, stage); - CurrentStateOutput output2 = event.getAttribute(AttributeName.CURRENT_STATE.toString()); - String pendingState = - output2.getPendingState("testResourceName", new Partition("testResourceName_1"), - "localhost_3"); - AssertJUnit.assertEquals(pendingState, "SLAVE"); + ResourceCurrentState output2 = event.getAttribute(AttributeName.CURRENT_STATE.toString()); + State pendingState = + output2.getPendingState(ResourceId.from("testResourceName"), + PartitionId.from("testResourceName_1"), ParticipantId.from("localhost_3")); + AssertJUnit.assertEquals(pendingState, State.from("SLAVE")); ZNRecord record1 = new ZNRecord("testResourceName"); // Add a current state that matches sessionId and one that does not match CurrentState stateWithLiveSession = new CurrentState(record1); - stateWithLiveSession.setSessionId("session_3"); + stateWithLiveSession.setSessionId(SessionId.from("session_3")); stateWithLiveSession.setStateModelDefRef("MasterSlave"); - stateWithLiveSession.setState("testResourceName_1", "OFFLINE"); + stateWithLiveSession.setState(PartitionId.from("testResourceName_1"), State.from("OFFLINE")); ZNRecord record2 = new ZNRecord("testResourceName"); CurrentState stateWithDeadSession = new CurrentState(record2); - stateWithDeadSession.setSessionId("session_dead"); + stateWithDeadSession.setSessionId(SessionId.from("session_dead")); stateWithDeadSession.setStateModelDefRef("MasterSlave"); - stateWithDeadSession.setState("testResourceName_1", "MASTER"); + stateWithDeadSession.setState(PartitionId.from("testResourceName_1"), State.from("MASTER")); accessor.setProperty(keyBuilder.currentState("localhost_3", "session_3", "testResourceName"), stateWithLiveSession); @@ -104,11 +116,11 @@ public void testSimpleCS() { stateWithDeadSession); runStage(event, new ReadClusterDataStage()); runStage(event, stage); - CurrentStateOutput output3 = event.getAttribute(AttributeName.CURRENT_STATE.toString()); - String currentState = - output3.getCurrentState("testResourceName", new Partition("testResourceName_1"), - "localhost_3"); - AssertJUnit.assertEquals(currentState, "OFFLINE"); + ResourceCurrentState output3 = event.getAttribute(AttributeName.CURRENT_STATE.toString()); + State currentState = + output3.getCurrentState(ResourceId.from("testResourceName"), + PartitionId.from("testResourceName_1"), ParticipantId.from("localhost_3")); + AssertJUnit.assertEquals(currentState, State.from("OFFLINE")); } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java index cdf11febd8..dd5f441577 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java @@ -20,6 +20,7 @@ */ import java.util.ArrayList; +import java.util.Collections; import java.util.Date; import java.util.List; import java.util.Map; @@ -28,32 +29,31 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.ZNRecord; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; import org.apache.helix.controller.pipeline.Pipeline; -import org.apache.helix.controller.stages.AttributeName; -import org.apache.helix.controller.stages.ClusterEvent; -import org.apache.helix.controller.stages.MessageSelectionStageOutput; -import org.apache.helix.controller.stages.MessageThrottleStage; -import org.apache.helix.controller.stages.MessageThrottleStageOutput; -import org.apache.helix.controller.stages.ReadClusterDataStage; -import org.apache.helix.controller.stages.ResourceComputationStage; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.model.ClusterConstraints; -import org.apache.helix.model.ConstraintItem; -import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; import org.apache.helix.model.ClusterConstraints.ConstraintAttribute; import org.apache.helix.model.ClusterConstraints.ConstraintType; +import org.apache.helix.model.ConstraintItem; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageType; -import org.apache.log4j.Logger; +import org.apache.helix.model.ResourceAssignment; import org.testng.Assert; import org.testng.annotations.Test; public class TestMessageThrottleStage extends ZkUnitTestBase { - private static final Logger LOG = Logger.getLogger(TestMessageThrottleStage.class.getName()); final String _className = getShortClassName(); @Test @@ -62,12 +62,12 @@ public void testMsgThrottleBasic() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); HelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); HelixManager manager = new DummyClusterManager(clusterName, accessor); // ideal state: node0 is MASTER, node1 is SLAVE // replica=2 means 1 master and 1 slave - setupIdealState(clusterName, new int[] { + List idealStates = setupIdealState(clusterName, new int[] { 0, 1 }, new String[] { "TestDB" @@ -77,9 +77,16 @@ public void testMsgThrottleBasic() throws Exception { }); setupStateModel(clusterName); + ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(clusterName), accessor); + clusterAccessor.initClusterStructure(); + ClusterEvent event = new ClusterEvent("testEvent"); event.addAttribute("helixmanager", manager); + // get an empty best possible output for the partitions + BestPossibleStateOutput bestPossOutput = getEmptyBestPossibleStateOutput(idealStates); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.toString(), bestPossOutput); + MessageThrottleStage throttleStage = new MessageThrottleStage(); try { runStage(event, throttleStage); @@ -106,22 +113,24 @@ public void testMsgThrottleBasic() throws Exception { } catch (Exception e) { // OK } - MessageSelectionStageOutput msgSelectOutput = new MessageSelectionStageOutput(); + MessageOutput msgSelectOutput = new MessageOutput(); List selectMessages = new ArrayList(); Message msg = - createMessage(MessageType.STATE_TRANSITION, "msgId-001", "OFFLINE", "SLAVE", "TestDB", - "localhost_0"); + createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-001"), "OFFLINE", + "SLAVE", "TestDB", "localhost_0"); selectMessages.add(msg); - msgSelectOutput.addMessages("TestDB", new Partition("TestDB_0"), selectMessages); + msgSelectOutput.setMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0"), + selectMessages); event.addAttribute(AttributeName.MESSAGES_SELECTED.toString(), msgSelectOutput); runStage(event, throttleStage); - MessageThrottleStageOutput msgThrottleOutput = + MessageOutput msgThrottleOutput = event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString()); - Assert.assertEquals(msgThrottleOutput.getMessages("TestDB", new Partition("TestDB_0")).size(), - 1); + Assert.assertEquals( + msgThrottleOutput.getMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0")) + .size(), 1); System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); @@ -133,12 +142,12 @@ public void testMsgThrottleConstraints() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); HelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); HelixManager manager = new DummyClusterManager(clusterName, accessor); // ideal state: node0 is MASTER, node1 is SLAVE // replica=2 means 1 master and 1 slave - setupIdealState(clusterName, new int[] { + List idealStates = setupIdealState(clusterName, new int[] { 0, 1 }, new String[] { "TestDB" @@ -148,6 +157,9 @@ public void testMsgThrottleConstraints() throws Exception { }); setupStateModel(clusterName); + ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(clusterName), accessor); + clusterAccessor.initClusterStructure(); + // setup constraints ZNRecord record = new ZNRecord(ConstraintType.MESSAGE_CONSTRAINT.toString()); @@ -223,8 +235,8 @@ public void testMsgThrottleConstraints() throws Exception { // test constraintSelection // message1: hit contraintSelection rule1 and rule2 Message msg1 = - createMessage(MessageType.STATE_TRANSITION, "msgId-001", "OFFLINE", "SLAVE", "TestDB", - "localhost_0"); + createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-001"), "OFFLINE", + "SLAVE", "TestDB", "localhost_0"); Map msgAttr = ClusterConstraints.toConstraintAttributes(msg1); Set matches = constraint.match(msgAttr); @@ -244,8 +256,8 @@ public void testMsgThrottleConstraints() throws Exception { // message2: hit contraintSelection rule1, rule2, and rule3 Message msg2 = - createMessage(MessageType.STATE_TRANSITION, "msgId-002", "OFFLINE", "SLAVE", "TestDB", - "localhost_1"); + createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-002"), "OFFLINE", + "SLAVE", "TestDB", "localhost_1"); msgAttr = ClusterConstraints.toConstraintAttributes(msg2); matches = constraint.match(msgAttr); @@ -267,27 +279,31 @@ public void testMsgThrottleConstraints() throws Exception { ClusterEvent event = new ClusterEvent("testEvent"); event.addAttribute("helixmanager", manager); + // get an empty best possible output for the partitions + BestPossibleStateOutput bestPossOutput = getEmptyBestPossibleStateOutput(idealStates); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.toString(), bestPossOutput); + Pipeline dataRefresh = new Pipeline(); dataRefresh.addStage(new ReadClusterDataStage()); runPipeline(event, dataRefresh); runStage(event, new ResourceComputationStage()); - MessageSelectionStageOutput msgSelectOutput = new MessageSelectionStageOutput(); + MessageOutput msgSelectOutput = new MessageOutput(); Message msg3 = - createMessage(MessageType.STATE_TRANSITION, "msgId-003", "OFFLINE", "SLAVE", "TestDB", - "localhost_0"); + createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-003"), "OFFLINE", + "SLAVE", "TestDB", "localhost_0"); Message msg4 = - createMessage(MessageType.STATE_TRANSITION, "msgId-004", "OFFLINE", "SLAVE", "TestDB", - "localhost_0"); + createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-004"), "OFFLINE", + "SLAVE", "TestDB", "localhost_0"); Message msg5 = - createMessage(MessageType.STATE_TRANSITION, "msgId-005", "OFFLINE", "SLAVE", "TestDB", - "localhost_0"); + createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-005"), "OFFLINE", + "SLAVE", "TestDB", "localhost_0"); Message msg6 = - createMessage(MessageType.STATE_TRANSITION, "msgId-006", "OFFLINE", "SLAVE", "TestDB", - "localhost_1"); + createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-006"), "OFFLINE", + "SLAVE", "TestDB", "localhost_1"); List selectMessages = new ArrayList(); selectMessages.add(msg1); @@ -297,15 +313,16 @@ public void testMsgThrottleConstraints() throws Exception { selectMessages.add(msg5); // should be throttled selectMessages.add(msg6); // should be throttled - msgSelectOutput.addMessages("TestDB", new Partition("TestDB_0"), selectMessages); + msgSelectOutput.setMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0"), + selectMessages); event.addAttribute(AttributeName.MESSAGES_SELECTED.toString(), msgSelectOutput); runStage(event, throttleStage); - MessageThrottleStageOutput msgThrottleOutput = + MessageOutput msgThrottleOutput = event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString()); List throttleMessages = - msgThrottleOutput.getMessages("TestDB", new Partition("TestDB_0")); + msgThrottleOutput.getMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0")); Assert.assertEquals(throttleMessages.size(), 4); Assert.assertTrue(throttleMessages.contains(msg1)); Assert.assertTrue(throttleMessages.contains(msg2)); @@ -325,5 +342,18 @@ private boolean containsConstraint(Set constraints, ConstraintIt return false; } + private BestPossibleStateOutput getEmptyBestPossibleStateOutput(List idealStates) { + BestPossibleStateOutput output = new BestPossibleStateOutput(); + for (IdealState idealState : idealStates) { + ResourceId resourceId = idealState.getResourceId(); + ResourceAssignment assignment = new ResourceAssignment(resourceId); + for (PartitionId partitionId : idealState.getPartitionIdSet()) { + Map emptyMap = Collections.emptyMap(); + assignment.addReplicaMap(partitionId, emptyMap); + } + output.setResourceAssignment(resourceId, assignment); + } + return output; + } // add pending message test case } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java index 820abbebd6..4b33d8c97a 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java @@ -20,15 +20,28 @@ */ import java.util.ArrayList; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.helix.TestHelper; -import org.apache.helix.controller.stages.MessageSelectionStage; +import org.apache.helix.api.HelixVersion; +import org.apache.helix.api.Participant; +import org.apache.helix.api.RunningInstance; +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ProcId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.controller.stages.MessageSelectionStage.Bounds; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.CurrentState; import org.apache.helix.model.Message; import org.testng.Assert; import org.testng.annotations.Test; @@ -38,25 +51,41 @@ public class TestMsgSelectionStage { public void testMasterXfer() { System.out.println("START testMasterXfer at " + new Date(System.currentTimeMillis())); - Map liveInstances = new HashMap(); - liveInstances.put("localhost_0", new LiveInstance("localhost_0")); - liveInstances.put("localhost_1", new LiveInstance("localhost_1")); - - Map currentStates = new HashMap(); - currentStates.put("localhost_0", "SLAVE"); - currentStates.put("localhost_1", "MASTER"); - - Map pendingStates = new HashMap(); + Map liveInstances = new HashMap(); + Set disabledPartitions = Collections.emptySet(); + Set tags = Collections.emptySet(); + Map currentStateMap = Collections.emptyMap(); + Map messageMap = Collections.emptyMap(); + RunningInstance runningInstance0 = + new RunningInstance(SessionId.from("session_0"), HelixVersion.from("1.2.3.4"), + ProcId.from("0")); + RunningInstance runningInstance1 = + new RunningInstance(SessionId.from("session_1"), HelixVersion.from("1.2.3.4"), + ProcId.from("1")); + liveInstances.put(ParticipantId.from("localhost_0"), + new Participant(ParticipantId.from("localhost_0"), "localhost", 0, true, + disabledPartitions, tags, runningInstance0, currentStateMap, messageMap, + new UserConfig(Scope.participant(ParticipantId.from("localhost_0"))))); + liveInstances.put(ParticipantId.from("localhost_1"), + new Participant(ParticipantId.from("localhost_1"), "localhost", 1, true, + disabledPartitions, tags, runningInstance1, currentStateMap, messageMap, + new UserConfig(Scope.participant(ParticipantId.from("localhost_1"))))); + + Map currentStates = new HashMap(); + currentStates.put(ParticipantId.from("localhost_0"), State.from("SLAVE")); + currentStates.put(ParticipantId.from("localhost_1"), State.from("MASTER")); + + Map pendingStates = new HashMap(); List messages = new ArrayList(); - messages.add(TestHelper.createMessage("msgId_0", "SLAVE", "MASTER", "localhost_0", "TestDB", - "TestDB_0")); - messages.add(TestHelper.createMessage("msgId_1", "MASTER", "SLAVE", "localhost_1", "TestDB", - "TestDB_0")); + messages.add(TestHelper.createMessage(MessageId.from("msgId_0"), "SLAVE", "MASTER", + "localhost_0", "TestDB", "TestDB_0")); + messages.add(TestHelper.createMessage(MessageId.from("msgId_1"), "MASTER", "SLAVE", + "localhost_1", "TestDB", "TestDB_0")); - Map stateConstraints = new HashMap(); - stateConstraints.put("MASTER", new Bounds(0, 1)); - stateConstraints.put("SLAVE", new Bounds(0, 2)); + Map stateConstraints = new HashMap(); + stateConstraints.put(State.from("MASTER"), new Bounds(0, 1)); + stateConstraints.put(State.from("SLAVE"), new Bounds(0, 2)); Map stateTransitionPriorities = new HashMap(); stateTransitionPriorities.put("MASTER-SLAVE", 0); @@ -64,10 +93,10 @@ public void testMasterXfer() { List selectedMsg = new MessageSelectionStage().selectMessages(liveInstances, currentStates, pendingStates, - messages, stateConstraints, stateTransitionPriorities, "OFFLINE"); + messages, stateConstraints, stateTransitionPriorities, State.from("OFFLINE")); Assert.assertEquals(selectedMsg.size(), 1); - Assert.assertEquals(selectedMsg.get(0).getMsgId(), "msgId_1"); + Assert.assertEquals(selectedMsg.get(0).getMessageId(), MessageId.from("msgId_1")); System.out.println("END testMasterXfer at " + new Date(System.currentTimeMillis())); } @@ -76,24 +105,40 @@ public void testMasterXferAfterMasterResume() { System.out.println("START testMasterXferAfterMasterResume at " + new Date(System.currentTimeMillis())); - Map liveInstances = new HashMap(); - liveInstances.put("localhost_0", new LiveInstance("localhost_0")); - liveInstances.put("localhost_1", new LiveInstance("localhost_1")); - - Map currentStates = new HashMap(); - currentStates.put("localhost_0", "SLAVE"); - currentStates.put("localhost_1", "SLAVE"); - - Map pendingStates = new HashMap(); - pendingStates.put("localhost_1", "MASTER"); + Map liveInstances = new HashMap(); + Set disabledPartitions = Collections.emptySet(); + Set tags = Collections.emptySet(); + Map currentStateMap = Collections.emptyMap(); + Map messageMap = Collections.emptyMap(); + RunningInstance runningInstance0 = + new RunningInstance(SessionId.from("session_0"), HelixVersion.from("1.2.3.4"), + ProcId.from("0")); + RunningInstance runningInstance1 = + new RunningInstance(SessionId.from("session_1"), HelixVersion.from("1.2.3.4"), + ProcId.from("1")); + liveInstances.put(ParticipantId.from("localhost_0"), + new Participant(ParticipantId.from("localhost_0"), "localhost", 0, true, + disabledPartitions, tags, runningInstance0, currentStateMap, messageMap, + new UserConfig(Scope.participant(ParticipantId.from("localhost_0"))))); + liveInstances.put(ParticipantId.from("localhost_1"), + new Participant(ParticipantId.from("localhost_1"), "localhost", 1, true, + disabledPartitions, tags, runningInstance1, currentStateMap, messageMap, + new UserConfig(Scope.participant(ParticipantId.from("localhost_1"))))); + + Map currentStates = new HashMap(); + currentStates.put(ParticipantId.from("localhost_0"), State.from("SLAVE")); + currentStates.put(ParticipantId.from("localhost_1"), State.from("SLAVE")); + + Map pendingStates = new HashMap(); + pendingStates.put(ParticipantId.from("localhost_1"), State.from("MASTER")); List messages = new ArrayList(); - messages.add(TestHelper.createMessage("msgId_0", "SLAVE", "MASTER", "localhost_0", "TestDB", - "TestDB_0")); + messages.add(TestHelper.createMessage(MessageId.from("msgId_0"), "SLAVE", "MASTER", + "localhost_0", "TestDB", "TestDB_0")); - Map stateConstraints = new HashMap(); - stateConstraints.put("MASTER", new Bounds(0, 1)); - stateConstraints.put("SLAVE", new Bounds(0, 2)); + Map stateConstraints = new HashMap(); + stateConstraints.put(State.from("MASTER"), new Bounds(0, 1)); + stateConstraints.put(State.from("SLAVE"), new Bounds(0, 2)); Map stateTransitionPriorities = new HashMap(); stateTransitionPriorities.put("MASTER-SLAVE", 0); @@ -101,7 +146,7 @@ public void testMasterXferAfterMasterResume() { List selectedMsg = new MessageSelectionStage().selectMessages(liveInstances, currentStates, pendingStates, - messages, stateConstraints, stateTransitionPriorities, "OFFLINE"); + messages, stateConstraints, stateTransitionPriorities, State.from("OFFLINE")); Assert.assertEquals(selectedMsg.size(), 0); System.out.println("END testMasterXferAfterMasterResume at " diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java deleted file mode 100644 index d1913458a9..0000000000 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java +++ /dev/null @@ -1,54 +0,0 @@ -package org.apache.helix.controller.stages; - -/* - * 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.HelixManager; -import org.apache.helix.controller.stages.StatsAggregationStage; -import org.apache.helix.integration.ZkStandAloneCMTestBase; -import org.testng.Assert; -import org.testng.annotations.Test; - -public class TestParseInfoFromAlert extends ZkStandAloneCMTestBase { - @Test - public void TestParse() { - StatsAggregationStage stage = new StatsAggregationStage(); - String controllerName = CONTROLLER_PREFIX + "_0"; - HelixManager manager = _startCMResultMap.get(controllerName)._manager; - - String instanceName = - stage.parseInstanceName("localhost_12918.TestStat@DB=123.latency", manager); - Assert.assertTrue(instanceName.equals("localhost_12918")); - - instanceName = stage.parseInstanceName("localhost_12955.TestStat@DB=123.latency", manager); - Assert.assertTrue(instanceName == null); - - instanceName = stage.parseInstanceName("localhost_12922.TestStat@DB=123.latency", manager); - Assert.assertTrue(instanceName.equals("localhost_12922")); - - String resourceName = - stage.parseResourceName("localhost_12918.TestStat@DB=TestDB.latency", manager); - Assert.assertTrue(resourceName.equals("TestDB")); - - String partitionName = - stage.parsePartitionName("localhost_12918.TestStat@DB=TestDB;Partition=TestDB_22.latency", - manager); - Assert.assertTrue(partitionName.equals("TestDB_22")); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java index fccd0c796a..2e17ad3722 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java @@ -25,36 +25,39 @@ import org.apache.helix.HelixAdmin; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.controller.HelixControllerMain; import org.apache.helix.controller.pipeline.Pipeline; import org.apache.helix.controller.stages.AttributeName; import org.apache.helix.controller.stages.BestPossibleStateCalcStage; import org.apache.helix.controller.stages.ClusterEvent; import org.apache.helix.controller.stages.CurrentStateComputationStage; -import org.apache.helix.controller.stages.MessageGenerationPhase; import org.apache.helix.controller.stages.MessageSelectionStage; -import org.apache.helix.controller.stages.MessageSelectionStageOutput; import org.apache.helix.controller.stages.MessageThrottleStage; import org.apache.helix.controller.stages.ReadClusterDataStage; import org.apache.helix.controller.stages.ResourceComputationStage; import org.apache.helix.controller.stages.TaskAssignmentStage; +import org.apache.helix.integration.manager.ClusterControllerManager; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.model.CurrentState; +import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.Message; -import org.apache.helix.model.Partition; import org.apache.helix.model.Message.Attributes; -import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.annotations.Test; public class TestRebalancePipeline extends ZkUnitTestBase { - private static final Logger LOG = Logger.getLogger(TestRebalancePipeline.class.getName()); final String _className = getShortClassName(); @Test @@ -63,7 +66,7 @@ public void testDuplicateMsg() { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); HelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); HelixManager manager = new DummyClusterManager(clusterName, accessor); ClusterEvent event = new ClusterEvent("testEvent"); @@ -78,11 +81,17 @@ public void testDuplicateMsg() { setupIdealState(clusterName, new int[] { 0, 1 }, resourceGroups, 1, 2); + setupInstances(clusterName, new int[] { + 0, 1 + }); setupLiveInstances(clusterName, new int[] { 0, 1 }); setupStateModel(clusterName); + ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(clusterName), accessor); + clusterAccessor.initClusterStructure(); + // cluster data cache refresh pipeline Pipeline dataRefresh = new Pipeline(); dataRefresh.addStage(new ReadClusterDataStage()); @@ -92,7 +101,7 @@ public void testDuplicateMsg() { rebalancePipeline.addStage(new ResourceComputationStage()); rebalancePipeline.addStage(new CurrentStateComputationStage()); rebalancePipeline.addStage(new BestPossibleStateCalcStage()); - rebalancePipeline.addStage(new MessageGenerationPhase()); + rebalancePipeline.addStage(new MessageGenerationStage()); rebalancePipeline.addStage(new MessageSelectionStage()); rebalancePipeline.addStage(new MessageThrottleStage()); rebalancePipeline.addStage(new TaskAssignmentStage()); @@ -105,14 +114,14 @@ public void testDuplicateMsg() { runPipeline(event, dataRefresh); runPipeline(event, rebalancePipeline); - MessageSelectionStageOutput msgSelOutput = - event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); + MessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); List messages = - msgSelOutput.getMessages(resourceName, new Partition(resourceName + "_0")); + msgSelOutput.getMessages(ResourceId.from(resourceName), + PartitionId.from(resourceName + "_0")); Assert.assertEquals(messages.size(), 1, "Should output 1 message: OFFLINE-SLAVE for node0"); Message message = messages.get(0); - Assert.assertEquals(message.getFromState(), "OFFLINE"); - Assert.assertEquals(message.getToState(), "SLAVE"); + Assert.assertEquals(message.getTypedFromState().toString(), "OFFLINE"); + Assert.assertEquals(message.getTypedToState().toString(), "SLAVE"); Assert.assertEquals(message.getTgtName(), "localhost_0"); // round2: updates node0 currentState to SLAVE but keep the @@ -123,7 +132,9 @@ public void testDuplicateMsg() { runPipeline(event, dataRefresh); runPipeline(event, rebalancePipeline); msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); - messages = msgSelOutput.getMessages(resourceName, new Partition(resourceName + "_0")); + messages = + msgSelOutput.getMessages(ResourceId.from(resourceName), + PartitionId.from(resourceName + "_0")); Assert.assertEquals(messages.size(), 0, "Should NOT output 1 message: SLAVE-MASTER for node1"); System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); @@ -136,7 +147,7 @@ public void testMsgTriggeredRebalance() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); HelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); HelixManager manager = new DummyClusterManager(clusterName, accessor); ClusterEvent event = new ClusterEvent("testEvent"); @@ -160,8 +171,13 @@ public void testMsgTriggeredRebalance() throws Exception { 0, 1 }); - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(clusterName), accessor); + clusterAccessor.initClusterStructure(); + + + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); // round1: controller sends O->S to both node0 and node1 Thread.sleep(1000); @@ -211,7 +227,7 @@ public void testChangeIdealStateWithPendingMsg() { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); HelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); HelixManager manager = new DummyClusterManager(clusterName, accessor); ClusterEvent event = new ClusterEvent("testEvent"); event.addAttribute("helixmanager", manager); @@ -225,11 +241,17 @@ public void testChangeIdealStateWithPendingMsg() { setupIdealState(clusterName, new int[] { 0, 1 }, resourceGroups, 1, 2); + setupInstances(clusterName, new int[] { + 0, 1 + }); setupLiveInstances(clusterName, new int[] { 0, 1 }); setupStateModel(clusterName); + ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(clusterName), accessor); + clusterAccessor.initClusterStructure(); + // cluster data cache refresh pipeline Pipeline dataRefresh = new Pipeline(); dataRefresh.addStage(new ReadClusterDataStage()); @@ -239,7 +261,7 @@ public void testChangeIdealStateWithPendingMsg() { rebalancePipeline.addStage(new ResourceComputationStage()); rebalancePipeline.addStage(new CurrentStateComputationStage()); rebalancePipeline.addStage(new BestPossibleStateCalcStage()); - rebalancePipeline.addStage(new MessageGenerationPhase()); + rebalancePipeline.addStage(new MessageGenerationStage()); rebalancePipeline.addStage(new MessageSelectionStage()); rebalancePipeline.addStage(new MessageThrottleStage()); rebalancePipeline.addStage(new TaskAssignmentStage()); @@ -252,14 +274,14 @@ public void testChangeIdealStateWithPendingMsg() { runPipeline(event, dataRefresh); runPipeline(event, rebalancePipeline); - MessageSelectionStageOutput msgSelOutput = - event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); + MessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); List messages = - msgSelOutput.getMessages(resourceName, new Partition(resourceName + "_0")); + msgSelOutput.getMessages(ResourceId.from(resourceName), + PartitionId.from(resourceName + "_0")); Assert.assertEquals(messages.size(), 1, "Should output 1 message: OFFLINE-SLAVE for node0"); Message message = messages.get(0); - Assert.assertEquals(message.getFromState(), "OFFLINE"); - Assert.assertEquals(message.getToState(), "SLAVE"); + Assert.assertEquals(message.getTypedFromState().toString(), "OFFLINE"); + Assert.assertEquals(message.getTypedToState().toString(), "SLAVE"); Assert.assertEquals(message.getTgtName(), "localhost_0"); // round2: drop resource, but keep the @@ -270,13 +292,15 @@ public void testChangeIdealStateWithPendingMsg() { runPipeline(event, dataRefresh); runPipeline(event, rebalancePipeline); msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); - messages = msgSelOutput.getMessages(resourceName, new Partition(resourceName + "_0")); + messages = + msgSelOutput.getMessages(ResourceId.from(resourceName), + PartitionId.from(resourceName + "_0")); Assert.assertEquals(messages.size(), 1, "Should output only 1 message: OFFLINE->DROPPED for localhost_1"); message = messages.get(0); - Assert.assertEquals(message.getFromState(), "SLAVE"); - Assert.assertEquals(message.getToState(), "OFFLINE"); + Assert.assertEquals(message.getTypedFromState().toString(), "SLAVE"); + Assert.assertEquals(message.getTypedToState().toString(), "OFFLINE"); Assert.assertEquals(message.getTgtName(), "localhost_1"); // round3: remove O->S for localhost_0, controller should now send O->DROPPED to @@ -287,12 +311,14 @@ public void testChangeIdealStateWithPendingMsg() { runPipeline(event, dataRefresh); runPipeline(event, rebalancePipeline); msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); - messages = msgSelOutput.getMessages(resourceName, new Partition(resourceName + "_0")); + messages = + msgSelOutput.getMessages(ResourceId.from(resourceName), + PartitionId.from(resourceName + "_0")); Assert.assertEquals(messages.size(), 1, "Should output 1 message: OFFLINE->DROPPED for localhost_0"); message = messages.get(0); - Assert.assertEquals(message.getFromState(), "OFFLINE"); - Assert.assertEquals(message.getToState(), "DROPPED"); + Assert.assertEquals(message.getTypedFromState().toString(), "OFFLINE"); + Assert.assertEquals(message.getTypedToState().toString(), "DROPPED"); Assert.assertEquals(message.getTgtName(), "localhost_0"); System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); @@ -306,7 +332,7 @@ public void testMasterXfer() { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); HelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); HelixManager manager = new DummyClusterManager(clusterName, accessor); ClusterEvent event = new ClusterEvent("testEvent"); event.addAttribute("helixmanager", manager); @@ -320,11 +346,17 @@ public void testMasterXfer() { setupIdealState(clusterName, new int[] { 0, 1 }, resourceGroups, 1, 2); + setupInstances(clusterName, new int[] { + 1 + }); setupLiveInstances(clusterName, new int[] { 1 }); setupStateModel(clusterName); + ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(clusterName), accessor); + clusterAccessor.initClusterStructure(); + // cluster data cache refresh pipeline Pipeline dataRefresh = new Pipeline(); dataRefresh.addStage(new ReadClusterDataStage()); @@ -334,7 +366,7 @@ public void testMasterXfer() { rebalancePipeline.addStage(new ResourceComputationStage()); rebalancePipeline.addStage(new CurrentStateComputationStage()); rebalancePipeline.addStage(new BestPossibleStateCalcStage()); - rebalancePipeline.addStage(new MessageGenerationPhase()); + rebalancePipeline.addStage(new MessageGenerationStage()); rebalancePipeline.addStage(new MessageSelectionStage()); rebalancePipeline.addStage(new MessageThrottleStage()); rebalancePipeline.addStage(new TaskAssignmentStage()); @@ -345,18 +377,21 @@ public void testMasterXfer() { runPipeline(event, dataRefresh); runPipeline(event, rebalancePipeline); - MessageSelectionStageOutput msgSelOutput = - event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); + MessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); List messages = - msgSelOutput.getMessages(resourceName, new Partition(resourceName + "_0")); + msgSelOutput.getMessages(ResourceId.from(resourceName), + PartitionId.from(resourceName + "_0")); Assert.assertEquals(messages.size(), 1, "Should output 1 message: SLAVE-MASTER for node1"); Message message = messages.get(0); - Assert.assertEquals(message.getFromState(), "SLAVE"); - Assert.assertEquals(message.getToState(), "MASTER"); + Assert.assertEquals(message.getTypedFromState().toString(), "SLAVE"); + Assert.assertEquals(message.getTypedToState().toString(), "MASTER"); Assert.assertEquals(message.getTgtName(), "localhost_1"); // round2: updates node0 currentState to SLAVE but keep the // message, make sure controller should not send S->M until removal is done + setupInstances(clusterName, new int[] { + 0 + }); setupLiveInstances(clusterName, new int[] { 0 }); @@ -366,7 +401,9 @@ public void testMasterXfer() { runPipeline(event, dataRefresh); runPipeline(event, rebalancePipeline); msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString()); - messages = msgSelOutput.getMessages(resourceName, new Partition(resourceName + "_0")); + messages = + msgSelOutput.getMessages(ResourceId.from(resourceName), + PartitionId.from(resourceName + "_0")); Assert.assertEquals(messages.size(), 0, "Should NOT output 1 message: SLAVE-MASTER for node0"); System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); @@ -376,13 +413,28 @@ public void testMasterXfer() { protected void setCurrentState(String clusterName, String instance, String resourceGroupName, String resourceKey, String sessionId, String state) { ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); CurrentState curState = new CurrentState(resourceGroupName); - curState.setState(resourceKey, state); - curState.setSessionId(sessionId); + curState.setState(PartitionId.from(resourceKey), State.from(state)); + curState.setSessionId(SessionId.from(sessionId)); curState.setStateModelDefRef("MasterSlave"); accessor.setProperty(keyBuilder.currentState(instance, sessionId, resourceGroupName), curState); } + + @Override + protected void setupInstances(String clusterName, int[] instances) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + Builder keyBuilder = accessor.keyBuilder(); + 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.setInstanceEnabled(true); + accessor.setProperty(keyBuilder.instanceConfig(instance), instanceConfig); + } + } } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java index dcb955c3e3..e320011eab 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java @@ -25,19 +25,21 @@ import java.util.UUID; import org.apache.helix.HelixDataAccessor; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.pipeline.StageContext; -import org.apache.helix.controller.stages.AttributeName; -import org.apache.helix.controller.stages.ClusterEvent; -import org.apache.helix.controller.stages.ReadClusterDataStage; -import org.apache.helix.controller.stages.ResourceComputationStage; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; import org.apache.helix.model.CurrentState; import org.apache.helix.model.IdealState; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.Resource; import org.apache.helix.model.IdealState.RebalanceMode; -import org.apache.helix.tools.DefaultIdealStateCalculator; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.LiveInstance; import org.testng.AssertJUnit; import org.testng.annotations.Test; @@ -57,10 +59,10 @@ public void testSimple() throws Exception { int replicas = 1; String resourceName = "testResource"; ZNRecord record = - DefaultIdealStateCalculator.calculateIdealState(instances, partitions, replicas, - resourceName, "MASTER", "SLAVE"); + DefaultTwoStateStrategy.calculateIdealState(instances, partitions, replicas, resourceName, + "MASTER", "SLAVE"); IdealState idealState = new IdealState(record); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); HelixDataAccessor accessor = manager.getHelixDataAccessor(); Builder keyBuilder = accessor.keyBuilder(); @@ -69,15 +71,18 @@ public void testSimple() throws Exception { runStage(event, new ReadClusterDataStage()); runStage(event, stage); - Map resource = event.getAttribute(AttributeName.RESOURCES.toString()); + Map resource = + event.getAttribute(AttributeName.RESOURCES.toString()); AssertJUnit.assertEquals(1, resource.size()); - AssertJUnit.assertEquals(resource.keySet().iterator().next(), resourceName); - AssertJUnit.assertEquals(resource.values().iterator().next().getResourceName(), resourceName); - AssertJUnit.assertEquals(resource.values().iterator().next().getStateModelDefRef(), - idealState.getStateModelDefRef()); + AssertJUnit.assertEquals(resource.keySet().iterator().next(), ResourceId.from(resourceName)); + AssertJUnit.assertEquals(resource.values().iterator().next().getId(), + ResourceId.from(resourceName)); + AssertJUnit.assertEquals(resource.values().iterator().next().getRebalancerConfig() + .getRebalancerContext(RebalancerContext.class).getStateModelDefId(), + idealState.getStateModelDefId()); AssertJUnit - .assertEquals(resource.values().iterator().next().getPartitions().size(), partitions); + .assertEquals(resource.values().iterator().next().getSubUnitSet().size(), partitions); } @Test @@ -91,17 +96,20 @@ public void testMultipleResources() throws Exception { runStage(event, new ReadClusterDataStage()); runStage(event, stage); - Map resourceMap = event.getAttribute(AttributeName.RESOURCES.toString()); + Map resourceMap = + event.getAttribute(AttributeName.RESOURCES.toString()); AssertJUnit.assertEquals(resources.length, resourceMap.size()); for (int i = 0; i < resources.length; i++) { String resourceName = resources[i]; + ResourceId resourceId = ResourceId.from(resourceName); IdealState idealState = idealStates.get(i); - AssertJUnit.assertTrue(resourceMap.containsKey(resourceName)); - AssertJUnit.assertEquals(resourceMap.get(resourceName).getResourceName(), resourceName); - AssertJUnit.assertEquals(resourceMap.get(resourceName).getStateModelDefRef(), - idealState.getStateModelDefRef()); - AssertJUnit.assertEquals(resourceMap.get(resourceName).getPartitions().size(), + AssertJUnit.assertTrue(resourceMap.containsKey(resourceId)); + AssertJUnit.assertEquals(resourceMap.get(resourceId).getId(), resourceId); + AssertJUnit.assertEquals(resourceMap.get(resourceId).getRebalancerConfig() + .getRebalancerContext(RebalancerContext.class).getStateModelDefId(), + idealState.getStateModelDefId()); + AssertJUnit.assertEquals(resourceMap.get(resourceId).getSubUnitSet().size(), idealState.getNumPartitions()); } } @@ -122,10 +130,10 @@ public void testMultipleResourcesWithSomeDropped() throws Exception { int replicas = 1; String resourceName = resources[i]; ZNRecord record = - DefaultIdealStateCalculator.calculateIdealState(instances, partitions, replicas, + DefaultTwoStateStrategy.calculateIdealState(instances, partitions, replicas, resourceName, "MASTER", "SLAVE"); IdealState idealState = new IdealState(record); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); HelixDataAccessor accessor = manager.getHelixDataAccessor(); Builder keyBuilder = accessor.keyBuilder(); @@ -140,15 +148,20 @@ public void testMultipleResourcesWithSomeDropped() throws Exception { String sessionId = UUID.randomUUID().toString(); liveInstance.setSessionId(sessionId); + InstanceConfig instanceConfig = new InstanceConfig(instanceName); + instanceConfig.setHostName("localhost"); + instanceConfig.setPort("3"); + HelixDataAccessor accessor = manager.getHelixDataAccessor(); Builder keyBuilder = accessor.keyBuilder(); accessor.setProperty(keyBuilder.liveInstance(instanceName), liveInstance); + accessor.setProperty(keyBuilder.instanceConfig(instanceName), instanceConfig); String oldResource = "testResourceOld"; CurrentState currentState = new CurrentState(oldResource); - currentState.setState("testResourceOld_0", "OFFLINE"); - currentState.setState("testResourceOld_1", "SLAVE"); - currentState.setState("testResourceOld_2", "MASTER"); + currentState.setState(PartitionId.from("testResourceOld_0"), State.from("OFFLINE")); + currentState.setState(PartitionId.from("testResourceOld_1"), State.from("SLAVE")); + currentState.setState(PartitionId.from("testResourceOld_2"), State.from("MASTER")); currentState.setStateModelDefRef("MasterSlave"); accessor.setProperty(keyBuilder.currentState(instanceName, sessionId, oldResource), currentState); @@ -157,30 +170,38 @@ public void testMultipleResourcesWithSomeDropped() throws Exception { runStage(event, new ReadClusterDataStage()); runStage(event, stage); - Map resourceMap = event.getAttribute(AttributeName.RESOURCES.toString()); + Map resourceMap = + event.getAttribute(AttributeName.RESOURCES.toString()); // +1 because it will have one for current state AssertJUnit.assertEquals(resources.length + 1, resourceMap.size()); for (int i = 0; i < resources.length; i++) { String resourceName = resources[i]; + ResourceId resourceId = ResourceId.from(resourceName); IdealState idealState = idealStates.get(i); - AssertJUnit.assertTrue(resourceMap.containsKey(resourceName)); - AssertJUnit.assertEquals(resourceMap.get(resourceName).getResourceName(), resourceName); - AssertJUnit.assertEquals(resourceMap.get(resourceName).getStateModelDefRef(), - idealState.getStateModelDefRef()); - AssertJUnit.assertEquals(resourceMap.get(resourceName).getPartitions().size(), + AssertJUnit.assertTrue(resourceMap.containsKey(resourceId)); + AssertJUnit.assertEquals(resourceMap.get(resourceId).getId(), resourceId); + AssertJUnit.assertEquals(resourceMap.get(resourceId).getRebalancerConfig() + .getRebalancerContext(RebalancerContext.class).getStateModelDefId(), + idealState.getStateModelDefId()); + AssertJUnit.assertEquals(resourceMap.get(resourceId).getSubUnitSet().size(), idealState.getNumPartitions()); } // Test the data derived from CurrentState - AssertJUnit.assertTrue(resourceMap.containsKey(oldResource)); - AssertJUnit.assertEquals(resourceMap.get(oldResource).getResourceName(), oldResource); - AssertJUnit.assertEquals(resourceMap.get(oldResource).getStateModelDefRef(), - currentState.getStateModelDefRef()); - AssertJUnit.assertEquals(resourceMap.get(oldResource).getPartitions().size(), currentState - .getPartitionStateMap().size()); - AssertJUnit.assertNotNull(resourceMap.get(oldResource).getPartition("testResourceOld_0")); - AssertJUnit.assertNotNull(resourceMap.get(oldResource).getPartition("testResourceOld_1")); - AssertJUnit.assertNotNull(resourceMap.get(oldResource).getPartition("testResourceOld_2")); + ResourceId oldResourceId = ResourceId.from(oldResource); + AssertJUnit.assertTrue(resourceMap.containsKey(oldResourceId)); + AssertJUnit.assertEquals(resourceMap.get(oldResourceId).getId(), oldResourceId); + AssertJUnit.assertEquals(resourceMap.get(oldResourceId).getRebalancerConfig() + .getRebalancerContext(RebalancerContext.class).getStateModelDefId(), + currentState.getStateModelDefId()); + AssertJUnit.assertEquals(resourceMap.get(oldResourceId).getSubUnitSet().size(), currentState + .getTypedPartitionStateMap().size()); + AssertJUnit.assertNotNull(resourceMap.get(oldResourceId).getSubUnit( + PartitionId.from("testResourceOld_0"))); + AssertJUnit.assertNotNull(resourceMap.get(oldResourceId).getSubUnit( + PartitionId.from("testResourceOld_1"))); + AssertJUnit.assertNotNull(resourceMap.get(oldResourceId).getSubUnit( + PartitionId.from("testResourceOld_2"))); } diff --git a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestAutoRebalanceStrategy.java b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestAutoRebalanceStrategy.java index 9fb7ba9a50..7c74035332 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestAutoRebalanceStrategy.java +++ b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestAutoRebalanceStrategy.java @@ -34,17 +34,28 @@ import java.util.TreeSet; import org.apache.helix.HelixDefinedState; -import org.apache.helix.Mocks.MockAccessor; -import org.apache.helix.PropertyKey.Builder; import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; -import org.apache.helix.controller.stages.ClusterDataCache; import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.IdealState; import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.tools.StateModelConfigGenerator; import org.apache.log4j.Logger; +import org.testng.Assert; import org.testng.annotations.Test; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + public class TestAutoRebalanceStrategy { private static Logger logger = Logger.getLogger(TestAutoRebalanceStrategy.class); @@ -122,13 +133,14 @@ private void runTest(String name, int numIterations, int numPartitions, int numL */ private StateModelDefinition getIncompleteStateModelDef(String modelName, String initialState, LinkedHashMap states) { - StateModelDefinition.Builder builder = new StateModelDefinition.Builder(modelName); - builder.initialState(initialState); - int i = states.size(); + StateModelDefinition.Builder builder = + new StateModelDefinition.Builder(StateModelDefId.from(modelName)); + builder.initialState(State.from(initialState)); + int i = 0; for (String state : states.keySet()) { - builder.addState(state, i); - builder.upperBound(state, states.get(state)); - i--; + builder.addState(State.from(state), i); + builder.upperBound(State.from(state), states.get(state)); + i++; } return builder.build(); } @@ -207,23 +219,28 @@ public void runRepeatedly(int numIterations) { private Map> getMapping(final Map> listResult) { final Map> mapResult = new HashMap>(); - ClusterDataCache cache = new ClusterDataCache(); - MockAccessor accessor = new MockAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - for (String node : _liveNodes) { - LiveInstance liveInstance = new LiveInstance(node); - liveInstance.setSessionId("testSession"); - accessor.setProperty(keyBuilder.liveInstance(node), liveInstance); - } - cache.refresh(accessor); for (String partition : _partitions) { - List preferenceList = listResult.get(partition); - Map currentStateMap = _currentMapping.get(partition); - Set disabled = Collections.emptySet(); - Map assignment = - ConstraintBasedAssignment.computeAutoBestStateForPartition(cache, _stateModelDef, - preferenceList, currentStateMap, disabled); - mapResult.put(partition, assignment); + Map rawCurStateMap = _currentMapping.get(partition); + ClusterConfig cluster = + new ClusterConfig.Builder(ClusterId.from("cluster")).addStateModelDefinition( + _stateModelDef).build(); + Set liveParticipantSet = Sets.newHashSet(); + for (String node : _liveNodes) { + liveParticipantSet.add(ParticipantId.from(node)); + } + List preferenceList = + IdealState.preferenceListFromStringList(listResult.get(partition)); + Set disabledParticipantsForPartition = Collections.emptySet(); + Map currentStateMap = + IdealState.participantStateMapFromStringMap(rawCurStateMap); + Map upperBounds = + ConstraintBasedAssignment.stateConstraints(_stateModelDef, + ResourceId.from(RESOURCE_NAME), cluster); + Map assignment = + ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds, + liveParticipantSet, _stateModelDef, preferenceList, currentStateMap, + disabledParticipantsForPartition); + mapResult.put(partition, IdealState.stringMapFromParticipantStateMap(assignment)); } return mapResult; } @@ -568,4 +585,207 @@ private T getRandomSetElement(Set source) { return null; } } + + /** + * Tests the following scenario: nodes come up one by one, then one node is taken down. Preference + * lists should prefer nodes in the current mapping at all times, but when all nodes are in the + * current mapping, then it should distribute states as evenly as possible. + */ + @Test + public void testOrphansNotPreferred() { + final ResourceId RESOURCE = ResourceId.from("resource"); + final PartitionId[] PARTITIONS = + { + PartitionId.from("resource_0"), PartitionId.from("resource_1"), + PartitionId.from("resource_2") + }; + final StateModelDefinition STATE_MODEL = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + final int REPLICA_COUNT = 2; + final ParticipantId[] NODES = { + ParticipantId.from("n0"), ParticipantId.from("n1"), ParticipantId.from("n2") + }; + + ReplicaPlacementScheme scheme = new AutoRebalanceStrategy.DefaultPlacementScheme(); + // initial state, one node, no mapping + List allNodes = Lists.newArrayList(NODES[0]); + List liveNodes = Lists.newArrayList(NODES[0]); + Map> currentMapping = Maps.newHashMap(); + for (PartitionId partition : PARTITIONS) { + currentMapping.put(partition, new HashMap()); + } + + // make sure that when the first node joins, a single replica is assigned fairly + List partitions = ImmutableList.copyOf(PARTITIONS); + Map upperBounds = Maps.newHashMap(); + for (State state : STATE_MODEL.getTypedStatesPriorityList()) { + upperBounds.put(state, STATE_MODEL.getNumParticipantsPerState(state)); + } + LinkedHashMap stateCount = + ConstraintBasedAssignment.stateCount(upperBounds, STATE_MODEL, liveNodes.size(), + REPLICA_COUNT); + ZNRecord znRecord = + new AutoRebalanceStrategy(RESOURCE, partitions, stateCount, Integer.MAX_VALUE, scheme) + .typedComputePartitionAssignment(liveNodes, currentMapping, allNodes); + Map> preferenceLists = znRecord.getListFields(); + for (PartitionId partition : currentMapping.keySet()) { + // make sure these are all MASTER + List preferenceList = preferenceLists.get(partition.toString()); + Assert.assertNotNull(preferenceList, "invalid preference list for " + partition); + Assert.assertEquals(preferenceList.size(), 1, "invalid preference list for " + partition); + } + + // now assign a replica to the first node in the current mapping, and add a second node + allNodes.add(NODES[1]); + liveNodes.add(NODES[1]); + stateCount = + ConstraintBasedAssignment.stateCount(upperBounds, STATE_MODEL, liveNodes.size(), + REPLICA_COUNT); + for (PartitionId partition : PARTITIONS) { + currentMapping.get(partition).put(NODES[0], State.from("MASTER")); + } + znRecord = + new AutoRebalanceStrategy(RESOURCE, partitions, stateCount, Integer.MAX_VALUE, scheme) + .typedComputePartitionAssignment(liveNodes, currentMapping, allNodes); + preferenceLists = znRecord.getListFields(); + for (PartitionId partition : currentMapping.keySet()) { + List preferenceList = preferenceLists.get(partition.toString()); + Assert.assertNotNull(preferenceList, "invalid preference list for " + partition); + Assert.assertEquals(preferenceList.size(), 2, "invalid preference list for " + partition); + Assert.assertEquals(preferenceList.get(0), NODES[0].toString(), + "invalid preference list for " + partition); + Assert.assertEquals(preferenceList.get(1), NODES[1].toString(), + "invalid preference list for " + partition); + } + + // now set the current mapping to reflect this update and make sure that it distributes masters + for (PartitionId partition : PARTITIONS) { + currentMapping.get(partition).put(NODES[1], State.from("SLAVE")); + } + znRecord = + new AutoRebalanceStrategy(RESOURCE, partitions, stateCount, Integer.MAX_VALUE, scheme) + .typedComputePartitionAssignment(liveNodes, currentMapping, allNodes); + preferenceLists = znRecord.getListFields(); + Set firstNodes = Sets.newHashSet(); + for (PartitionId partition : currentMapping.keySet()) { + List preferenceList = preferenceLists.get(partition.toString()); + Assert.assertNotNull(preferenceList, "invalid preference list for " + partition); + Assert.assertEquals(preferenceList.size(), 2, "invalid preference list for " + partition); + firstNodes.add(preferenceList.get(0)); + } + Assert.assertEquals(firstNodes.size(), 2, "masters not evenly distributed"); + + // set a mapping corresponding to a valid mapping for 2 nodes, add a third node, check that the + // new node is never the most preferred + allNodes.add(NODES[2]); + liveNodes.add(NODES[2]); + stateCount = + ConstraintBasedAssignment.stateCount(upperBounds, STATE_MODEL, liveNodes.size(), + REPLICA_COUNT); + + // recall that the other two partitions are [MASTER, SLAVE], which is fine, just reorder one + currentMapping.get(PARTITIONS[1]).put(NODES[0], State.from("SLAVE")); + currentMapping.get(PARTITIONS[1]).put(NODES[1], State.from("MASTER")); + znRecord = + new AutoRebalanceStrategy(RESOURCE, partitions, stateCount, Integer.MAX_VALUE, scheme) + .typedComputePartitionAssignment(liveNodes, currentMapping, allNodes); + preferenceLists = znRecord.getListFields(); + boolean newNodeUsed = false; + for (PartitionId partition : currentMapping.keySet()) { + List preferenceList = preferenceLists.get(partition.toString()); + Assert.assertNotNull(preferenceList, "invalid preference list for " + partition); + Assert.assertEquals(preferenceList.size(), 2, "invalid preference list for " + partition); + if (preferenceList.contains(NODES[2].toString())) { + newNodeUsed = true; + Assert.assertEquals(preferenceList.get(1), NODES[2].toString(), + "newly added node not at preference list tail for " + partition); + } + } + Assert.assertTrue(newNodeUsed, "not using " + NODES[2]); + + // now remap this to take the new node into account, should go back to balancing masters, slaves + // evenly across all nodes + for (PartitionId partition : PARTITIONS) { + currentMapping.get(partition).clear(); + } + currentMapping.get(PARTITIONS[0]).put(NODES[0], State.from("MASTER")); + currentMapping.get(PARTITIONS[0]).put(NODES[1], State.from("SLAVE")); + currentMapping.get(PARTITIONS[1]).put(NODES[1], State.from("MASTER")); + currentMapping.get(PARTITIONS[1]).put(NODES[2], State.from("SLAVE")); + currentMapping.get(PARTITIONS[2]).put(NODES[0], State.from("MASTER")); + currentMapping.get(PARTITIONS[2]).put(NODES[2], State.from("SLAVE")); + znRecord = + new AutoRebalanceStrategy(RESOURCE, partitions, stateCount, Integer.MAX_VALUE, scheme) + .typedComputePartitionAssignment(liveNodes, currentMapping, allNodes); + preferenceLists = znRecord.getListFields(); + firstNodes.clear(); + Set secondNodes = Sets.newHashSet(); + for (PartitionId partition : currentMapping.keySet()) { + List preferenceList = preferenceLists.get(partition.toString()); + Assert.assertNotNull(preferenceList, "invalid preference list for " + partition); + Assert.assertEquals(preferenceList.size(), 2, "invalid preference list for " + partition); + firstNodes.add(preferenceList.get(0)); + secondNodes.add(preferenceList.get(1)); + } + Assert.assertEquals(firstNodes.size(), 3, "masters not distributed evenly"); + Assert.assertEquals(secondNodes.size(), 3, "slaves not distributed evenly"); + + // remove a node now, but use the current mapping with everything balanced just prior + liveNodes.remove(0); + stateCount = + ConstraintBasedAssignment.stateCount(upperBounds, STATE_MODEL, liveNodes.size(), + REPLICA_COUNT); + + // remove all references of n0 from the mapping, keep everything else in a legal state + for (PartitionId partition : PARTITIONS) { + currentMapping.get(partition).clear(); + } + currentMapping.get(PARTITIONS[0]).put(NODES[1], State.from("MASTER")); + currentMapping.get(PARTITIONS[1]).put(NODES[1], State.from("MASTER")); + currentMapping.get(PARTITIONS[1]).put(NODES[2], State.from("SLAVE")); + currentMapping.get(PARTITIONS[2]).put(NODES[2], State.from("MASTER")); + znRecord = + new AutoRebalanceStrategy(RESOURCE, partitions, stateCount, Integer.MAX_VALUE, scheme) + .typedComputePartitionAssignment(liveNodes, currentMapping, allNodes); + preferenceLists = znRecord.getListFields(); + for (PartitionId partition : currentMapping.keySet()) { + List preferenceList = preferenceLists.get(partition.toString()); + Assert.assertNotNull(preferenceList, "invalid preference list for " + partition); + Assert.assertEquals(preferenceList.size(), 2, "invalid preference list for " + partition); + Map stateMap = currentMapping.get(partition); + for (ParticipantId participant : stateMap.keySet()) { + Assert.assertTrue(preferenceList.contains(participant.toString()), + "minimal movement violated for " + partition); + } + for (String participant : preferenceList) { + if (!stateMap.containsKey(ParticipantId.from(participant))) { + Assert.assertNotSame(preferenceList.get(0), participant, + "newly moved replica should not be master for " + partition); + } + } + } + + // finally, adjust the current mapping to reflect 2 nodes and make sure everything's even again + for (PartitionId partition : PARTITIONS) { + currentMapping.get(partition).clear(); + } + currentMapping.get(PARTITIONS[0]).put(NODES[1], State.from("MASTER")); + currentMapping.get(PARTITIONS[0]).put(NODES[2], State.from("SLAVE")); + currentMapping.get(PARTITIONS[1]).put(NODES[1], State.from("SLAVE")); + currentMapping.get(PARTITIONS[1]).put(NODES[2], State.from("MASTER")); + currentMapping.get(PARTITIONS[2]).put(NODES[1], State.from("SLAVE")); + currentMapping.get(PARTITIONS[2]).put(NODES[2], State.from("MASTER")); + znRecord = + new AutoRebalanceStrategy(RESOURCE, partitions, stateCount, Integer.MAX_VALUE, scheme) + .typedComputePartitionAssignment(liveNodes, currentMapping, allNodes); + preferenceLists = znRecord.getListFields(); + firstNodes.clear(); + for (PartitionId partition : currentMapping.keySet()) { + List preferenceList = preferenceLists.get(partition.toString()); + Assert.assertNotNull(preferenceList, "invalid preference list for " + partition); + Assert.assertEquals(preferenceList.size(), 2, "invalid preference list for " + partition); + firstNodes.add(preferenceList.get(0)); + } + Assert.assertEquals(firstNodes.size(), 2, "masters not evenly distributed"); + } } diff --git a/helix-core/src/test/java/org/apache/helix/TestRelayIdealStateCalculator.java b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestEspressoRelayStrategy.java similarity index 89% rename from helix-core/src/test/java/org/apache/helix/TestRelayIdealStateCalculator.java rename to helix-core/src/test/java/org/apache/helix/controller/strategy/TestEspressoRelayStrategy.java index ca58659ed6..ee0d14351f 100644 --- a/helix-core/src/test/java/org/apache/helix/TestRelayIdealStateCalculator.java +++ b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestEspressoRelayStrategy.java @@ -1,4 +1,4 @@ -package org.apache.helix; +package org.apache.helix.controller.strategy; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -20,17 +20,15 @@ */ import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; import org.apache.helix.model.IdealState; -import org.apache.helix.tools.IdealStateCalculatorForEspressoRelay; import org.testng.Assert; import org.testng.annotations.Test; -public class TestRelayIdealStateCalculator { +public class TestEspressoRelayStrategy { @Test() public void testEspressoStorageClusterIdealState() throws Exception { testEspressoStorageClusterIdealState(15, 9, 3); @@ -52,8 +50,8 @@ public void testEspressoStorageClusterIdealState(int partitions, int nodes, int } IdealState idealstate = - IdealStateCalculatorForEspressoRelay.calculateRelayIdealState(storageNodes, relays, "TEST", - replica, "Leader", "Standby", "LeaderStandby"); + EspressoRelayStrategy.calculateRelayIdealState(storageNodes, relays, "TEST", replica, + "Leader", "Standby", "LeaderStandby"); Assert.assertEquals(idealstate.getRecord().getListFields().size(), idealstate.getRecord() .getMapFields().size()); diff --git a/helix-core/src/test/java/org/apache/helix/TestEspressoStorageClusterIdealState.java b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestEspressoStorageClusterIdealState.java similarity index 90% rename from helix-core/src/test/java/org/apache/helix/TestEspressoStorageClusterIdealState.java rename to helix-core/src/test/java/org/apache/helix/controller/strategy/TestEspressoStorageClusterIdealState.java index 9b249ece51..dbb870ec6b 100644 --- a/helix-core/src/test/java/org/apache/helix/TestEspressoStorageClusterIdealState.java +++ b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestEspressoStorageClusterIdealState.java @@ -1,4 +1,4 @@ -package org.apache.helix; +package org.apache.helix.controller.strategy; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -29,15 +29,13 @@ import org.apache.helix.ZNRecord; import org.apache.helix.model.IdealState; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.apache.helix.util.RebalanceUtil; import org.testng.Assert; import org.testng.AssertJUnit; import org.testng.annotations.Test; public class TestEspressoStorageClusterIdealState { - @Test() + @Test public void testEspressoStorageClusterIdealState() throws Exception { List instanceNames = new ArrayList(); for (int i = 0; i < 5; i++) { @@ -45,7 +43,7 @@ public void testEspressoStorageClusterIdealState() throws Exception { } int partitions = 8, replicas = 0; Map result0 = - DefaultIdealStateCalculator.calculateInitialIdealState(instanceNames, partitions, replicas); + DefaultTwoStateStrategy.calculateInitialIdealState(instanceNames, partitions, replicas); Verify(result0, partitions, replicas); partitions = 8192; @@ -56,27 +54,27 @@ public void testEspressoStorageClusterIdealState() throws Exception { instanceNames.add("localhost:123" + i); } Map resultOriginal = - DefaultIdealStateCalculator.calculateInitialIdealState(instanceNames, partitions, replicas); + DefaultTwoStateStrategy.calculateInitialIdealState(instanceNames, partitions, replicas); Verify(resultOriginal, partitions, replicas); printStat(resultOriginal); Map result1 = - DefaultIdealStateCalculator.calculateInitialIdealState(instanceNames, partitions, replicas); + DefaultTwoStateStrategy.calculateInitialIdealState(instanceNames, partitions, replicas); List instanceNames2 = new ArrayList(); for (int i = 30; i < 35; i++) { instanceNames2.add("localhost:123" + i); } - DefaultIdealStateCalculator.calculateNextIdealState(instanceNames2, result1); + DefaultTwoStateStrategy.calculateNextIdealState(instanceNames2, result1); List instanceNames3 = new ArrayList(); for (int i = 35; i < 40; i++) { instanceNames3.add("localhost:123" + i); } - DefaultIdealStateCalculator.calculateNextIdealState(instanceNames3, result1); + DefaultTwoStateStrategy.calculateNextIdealState(instanceNames3, result1); Double masterKeepRatio = 0.0, slaveKeepRatio = 0.0; Verify(result1, partitions, replicas); double[] result = compareResult(resultOriginal, result1); @@ -97,21 +95,20 @@ public void testRebalance2() { } Map resultOriginal = - DefaultIdealStateCalculator.calculateInitialIdealState(instanceNames, partitions, replicas); + DefaultTwoStateStrategy.calculateInitialIdealState(instanceNames, partitions, replicas); ZNRecord idealState1 = - DefaultIdealStateCalculator.convertToZNRecord(resultOriginal, "TestDB", "MASTER", "SLAVE"); + DefaultTwoStateStrategy.convertToZNRecord(resultOriginal, "TestDB", "MASTER", "SLAVE"); Map result1 = RebalanceUtil.buildInternalIdealState(new IdealState(idealState1)); - List instanceNames2 = new ArrayList(); for (int i = 30; i < 35; i++) { instanceNames2.add("localhost:123" + i); } Map result2 = - DefaultIdealStateCalculator.calculateNextIdealState(instanceNames2, result1); + DefaultTwoStateStrategy.calculateNextIdealState(instanceNames2, result1); Verify(resultOriginal, partitions, replicas); Verify(result2, partitions, replicas); @@ -125,9 +122,9 @@ public void testRebalance2() { public static void Verify(Map result, int partitions, int replicas) { Map> masterAssignmentMap = - (Map>) (result.get("MasterAssignmentMap")); + (Map>) (result.get("PrimaryAssignmentMap")); Map>> nodeSlaveAssignmentMap = - (Map>>) (result.get("SlaveAssignmentMap")); + (Map>>) (result.get("SecondaryAssignmentMap")); AssertJUnit.assertTrue(partitions == (Integer) (result.get("partitions"))); @@ -224,14 +221,14 @@ public void printStat(Map result) { public static double[] compareResult(Map result1, Map result2) { double[] result = new double[2]; Map> masterAssignmentMap1 = - (Map>) (result1.get("MasterAssignmentMap")); + (Map>) (result1.get("PrimaryAssignmentMap")); Map>> nodeSlaveAssignmentMap1 = - (Map>>) (result1.get("SlaveAssignmentMap")); + (Map>>) (result1.get("SecondaryAssignmentMap")); Map> masterAssignmentMap2 = - (Map>) (result2.get("MasterAssignmentMap")); + (Map>) (result2.get("PrimaryAssignmentMap")); Map>> nodeSlaveAssignmentMap2 = - (Map>>) (result2.get("SlaveAssignmentMap")); + (Map>>) (result2.get("SecondaryAssignmentMap")); int commonMasters = 0; int commonSlaves = 0; diff --git a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java new file mode 100644 index 0000000000..9f52866ce8 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java @@ -0,0 +1,615 @@ +package org.apache.helix.controller.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.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; + +import org.apache.helix.HelixDefinedState; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.Participant; +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment; +import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.Message; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; +import org.testng.annotations.Test; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; + +public class TestNewAutoRebalanceStrategy { + private static Logger logger = Logger.getLogger(TestNewAutoRebalanceStrategy.class); + + /** + * Sanity test for a basic Master-Slave model + */ + @Test + public void simpleMasterSlaveTest() { + final int NUM_ITERATIONS = 10; + final int NUM_PARTITIONS = 10; + final int NUM_LIVE_NODES = 12; + final int NUM_TOTAL_NODES = 20; + final int MAX_PER_NODE = 5; + + final String[] STATE_NAMES = { + "MASTER", "SLAVE" + }; + final int[] STATE_COUNTS = { + 1, 2 + }; + + runTest("BasicMasterSlave", NUM_ITERATIONS, NUM_PARTITIONS, NUM_LIVE_NODES, NUM_TOTAL_NODES, + MAX_PER_NODE, STATE_NAMES, STATE_COUNTS); + } + + /** + * Run a test for an arbitrary state model. + * @param name Name of the test state model + * @param numIterations Number of rebalance tasks to run + * @param numPartitions Number of partitions for the resource + * @param numLiveNodes Number of live nodes in the cluster + * @param numTotalNodes Number of nodes in the cluster, must be greater than or equal to + * numLiveNodes + * @param maxPerNode Maximum number of replicas a node can serve + * @param stateNames States ordered by preference + * @param stateCounts Number of replicas that should be in each state + */ + private void runTest(String name, int numIterations, int numPartitions, int numLiveNodes, + int numTotalNodes, int maxPerNode, String[] stateNames, int[] stateCounts) { + List partitions = new ArrayList(); + for (int i = 0; i < numPartitions; i++) { + partitions.add("p_" + i); + } + + List liveNodes = new ArrayList(); + List allNodes = new ArrayList(); + for (int i = 0; i < numTotalNodes; i++) { + allNodes.add("n_" + i); + if (i < numLiveNodes) { + liveNodes.add("n_" + i); + } + } + + Map> currentMapping = new TreeMap>(); + + LinkedHashMap states = new LinkedHashMap(); + for (int i = 0; i < Math.min(stateNames.length, stateCounts.length); i++) { + states.put(stateNames[i], stateCounts[i]); + } + + StateModelDefinition stateModelDef = getIncompleteStateModelDef(name, stateNames[0], states); + + new AutoRebalanceTester(partitions, states, liveNodes, currentMapping, allNodes, maxPerNode, + stateModelDef, new AutoRebalanceStrategy.DefaultPlacementScheme()) + .runRepeatedly(numIterations); + } + + /** + * Get a StateModelDefinition without transitions. The auto rebalancer doesn't take transitions + * into account when computing mappings, so this is acceptable. + * @param modelName name to give the model + * @param initialState initial state for all nodes + * @param states ordered map of state to count + * @return incomplete StateModelDefinition for rebalancing + */ + private StateModelDefinition getIncompleteStateModelDef(String modelName, String initialState, + LinkedHashMap states) { + StateModelDefinition.Builder builder = + new StateModelDefinition.Builder(StateModelDefId.from(modelName)); + builder.initialState(State.from(initialState)); + int i = 0; + for (String state : states.keySet()) { + builder.addState(State.from(state), i); + builder.upperBound(State.from(state), states.get(state)); + i++; + } + return builder.build(); + } + + class AutoRebalanceTester { + private static final double P_KILL = 0.45; + private static final double P_ADD = 0.1; + private static final double P_RESURRECT = 0.45; + private static final String RESOURCE_NAME = "resource"; + + private List _partitions; + private LinkedHashMap _states; + private List _liveNodes; + private Set _liveSet; + private Set _removedSet; + private Set _nonLiveSet; + private Map> _currentMapping; + private List _allNodes; + private int _maxPerNode; + private StateModelDefinition _stateModelDef; + private ReplicaPlacementScheme _placementScheme; + private Random _random; + + public AutoRebalanceTester(List partitions, LinkedHashMap states, + List liveNodes, Map> currentMapping, + List allNodes, int maxPerNode, StateModelDefinition stateModelDef, + ReplicaPlacementScheme placementScheme) { + _partitions = partitions; + _states = states; + _liveNodes = liveNodes; + _liveSet = new TreeSet(); + for (String node : _liveNodes) { + _liveSet.add(node); + } + _removedSet = new TreeSet(); + _nonLiveSet = new TreeSet(); + _currentMapping = currentMapping; + _allNodes = allNodes; + for (String node : allNodes) { + if (!_liveSet.contains(node)) { + _nonLiveSet.add(node); + } + } + _maxPerNode = maxPerNode; + _stateModelDef = stateModelDef; + _placementScheme = placementScheme; + _random = new Random(); + } + + /** + * Repeatedly randomly select a task to run and report the result + * @param numIterations + * Number of random tasks to run in sequence + */ + public void runRepeatedly(int numIterations) { + logger.info("~~~~ Initial State ~~~~~"); + ZNRecord initialResult = + new AutoRebalanceStrategy(RESOURCE_NAME, _partitions, _states, _maxPerNode, + _placementScheme).computePartitionAssignment(_liveNodes, _currentMapping, _allNodes); + _currentMapping = getMapping(initialResult.getListFields()); + logger.info(_currentMapping); + getRunResult(_currentMapping, initialResult.getListFields()); + for (int i = 0; i < numIterations; i++) { + logger.info("~~~~ Iteration " + i + " ~~~~~"); + ZNRecord znRecord = runOnceRandomly(); + if (znRecord != null) { + final Map> listResult = znRecord.getListFields(); + final Map> mapResult = getMapping(listResult); + logger.info(mapResult); + logger.info(listResult); + getRunResult(mapResult, listResult); + _currentMapping = mapResult; + } + } + } + + private Map> getMapping(final Map> listResult) { + final Map> mapResult = + new HashMap>(); + ClusterId clusterId = ClusterId.from("clusterId"); + ClusterConfig.Builder clusterConfigBuilder = + new ClusterConfig.Builder(clusterId).addStateModelDefinition(_stateModelDef); + for (State state : _stateModelDef.getTypedStatesPriorityList()) { + clusterConfigBuilder.addStateUpperBoundConstraint(Scope.cluster(clusterId), + _stateModelDef.getStateModelDefId(), state, + _stateModelDef.getNumParticipantsPerState(state)); + } + ClusterConfig clusterConfig = clusterConfigBuilder.build(); + for (String partition : _partitions) { + PartitionId partitionId = PartitionId.from(partition); + Set disabledParticipantsForPartition = Collections.emptySet(); + Set disabledPartitionIdSet = Collections.emptySet(); + Set tags = Collections.emptySet(); + Map messageMap = Collections.emptyMap(); + Map currentStateMap = Collections.emptyMap(); + Map liveParticipantMap = + new HashMap(); + // set up some participants + for (String nodeName : _liveNodes) { + ParticipantId participantId = ParticipantId.from(nodeName); + Participant participant = + new Participant(participantId, "hostname", 0, true, disabledPartitionIdSet, tags, + null, currentStateMap, messageMap, new UserConfig( + Scope.participant(participantId))); + liveParticipantMap.put(participantId, participant); + } + List participantPreferenceList = + Lists.transform(listResult.get(partition), new Function() { + @Override + public ParticipantId apply(String participantId) { + return ParticipantId.from(participantId); + } + }); + // compute the mapping + Map replicaMap = + ResourceAssignment.replicaMapFromStringMap(_currentMapping.get(partition)); + Map upperBounds = + ConstraintBasedAssignment.stateConstraints(_stateModelDef, + ResourceId.from(RESOURCE_NAME), clusterConfig); + Map assignment = + ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds, + liveParticipantMap.keySet(), _stateModelDef, participantPreferenceList, replicaMap, + disabledParticipantsForPartition); + mapResult.put(partitionId, assignment); + } + + return ResourceAssignment.stringMapsFromReplicaMaps(mapResult); + } + + /** + * Output various statistics and correctness check results + * @param mapFields + * The map-map assignment generated by the rebalancer + * @param listFields + * The map-list assignment generated by the rebalancer + */ + public void getRunResult(final Map> mapFields, + final Map> listFields) { + logger.info("***** Statistics *****"); + dumpStatistics(mapFields); + verifyCorrectness(mapFields, listFields); + } + + /** + * Output statistics about the assignment + * @param mapFields + * The map-map assignment generated by the rebalancer + */ + public void dumpStatistics(final Map> mapFields) { + Map partitionsPerNode = getPartitionBucketsForNode(mapFields); + int nodeCount = _liveNodes.size(); + logger.info("Total number of nodes: " + nodeCount); + logger.info("Nodes: " + _liveNodes); + int sumPartitions = getSum(partitionsPerNode.values()); + logger.info("Total number of partitions: " + sumPartitions); + double averagePartitions = getAverage(partitionsPerNode.values()); + logger.info("Average number of partitions per node: " + averagePartitions); + double stdevPartitions = getStdev(partitionsPerNode.values(), averagePartitions); + logger.info("Standard deviation of partitions: " + stdevPartitions); + + // Statistics about each state + Map> statesPerNode = getStateBucketsForNode(mapFields); + for (String state : _states.keySet()) { + Map nodeStateCounts = new TreeMap(); + for (Entry> nodeStates : statesPerNode.entrySet()) { + Map stateCounts = nodeStates.getValue(); + if (stateCounts.containsKey(state)) { + nodeStateCounts.put(nodeStates.getKey(), stateCounts.get(state)); + } else { + nodeStateCounts.put(nodeStates.getKey(), 0); + } + } + int sumStates = getSum(nodeStateCounts.values()); + logger.info("Total number of state " + state + ": " + sumStates); + double averageStates = getAverage(nodeStateCounts.values()); + logger.info("Average number of state " + state + " per node: " + averageStates); + double stdevStates = getStdev(nodeStateCounts.values(), averageStates); + logger.info("Standard deviation of state " + state + " per node: " + stdevStates); + } + } + + /** + * Run a set of correctness tests, reporting success or failure + * @param mapFields + * The map-map assignment generated by the rebalancer + * @param listFields + * The map-list assignment generated by the rebalancer + */ + public void verifyCorrectness(final Map> mapFields, + final Map> listFields) { + final Map partitionsPerNode = getPartitionBucketsForNode(mapFields); + boolean maxConstraintMet = maxNotExceeded(partitionsPerNode); + assert maxConstraintMet : "Max per node constraint: FAIL"; + logger.info("Max per node constraint: PASS"); + + boolean liveConstraintMet = onlyLiveAssigned(partitionsPerNode); + assert liveConstraintMet : "Only live nodes have partitions constraint: FAIL"; + logger.info("Only live nodes have partitions constraint: PASS"); + + boolean stateAssignmentPossible = correctStateAssignmentCount(mapFields); + assert stateAssignmentPossible : "State replica constraint: FAIL"; + logger.info("State replica constraint: PASS"); + + boolean nodesUniqueForPartitions = atMostOnePartitionReplicaPerNode(listFields); + assert nodesUniqueForPartitions : "Node uniqueness per partition constraint: FAIL"; + logger.info("Node uniqueness per partition constraint: PASS"); + } + + private boolean maxNotExceeded(final Map partitionsPerNode) { + for (String node : partitionsPerNode.keySet()) { + Integer value = partitionsPerNode.get(node); + if (value > _maxPerNode) { + logger.error("ERROR: Node " + node + " has " + value + + " partitions despite a maximum of " + _maxPerNode); + return false; + } + } + return true; + } + + private boolean onlyLiveAssigned(final Map partitionsPerNode) { + for (final Entry nodeState : partitionsPerNode.entrySet()) { + boolean isLive = _liveSet.contains(nodeState.getKey()); + boolean isEmpty = nodeState.getValue() == 0; + if (!isLive && !isEmpty) { + logger.error("ERROR: Node " + nodeState.getKey() + " is not live, but has " + + nodeState.getValue() + " replicas!"); + return false; + } + } + return true; + } + + private boolean correctStateAssignmentCount(final Map> assignment) { + for (final Entry> partitionEntry : assignment.entrySet()) { + final Map nodeMap = partitionEntry.getValue(); + final Map stateCounts = new TreeMap(); + for (String state : nodeMap.values()) { + if (!stateCounts.containsKey(state)) { + stateCounts.put(state, 1); + } else { + stateCounts.put(state, stateCounts.get(state) + 1); + } + } + for (String state : stateCounts.keySet()) { + if (state.equals(HelixDefinedState.DROPPED.toString())) { + continue; + } + int count = stateCounts.get(state); + int maximumCount = _states.get(state); + if (count > maximumCount) { + logger.error("ERROR: State " + state + " for partition " + partitionEntry.getKey() + + " has " + count + " replicas when " + maximumCount + " is allowed!"); + return false; + } + } + } + return true; + } + + private boolean atMostOnePartitionReplicaPerNode(final Map> listFields) { + for (final Entry> partitionEntry : listFields.entrySet()) { + Set nodeSet = new HashSet(partitionEntry.getValue()); + int numUniques = nodeSet.size(); + int total = partitionEntry.getValue().size(); + if (numUniques < total) { + logger.error("ERROR: Partition " + partitionEntry.getKey() + " is assigned to " + total + + " nodes, but only " + numUniques + " are unique!"); + return false; + } + } + return true; + } + + private double getAverage(final Collection values) { + double sum = 0.0; + for (Integer value : values) { + sum += value; + } + if (values.size() != 0) { + return sum / values.size(); + } else { + return -1.0; + } + } + + private int getSum(final Collection values) { + int sum = 0; + for (Integer value : values) { + sum += value; + } + return sum; + } + + private double getStdev(final Collection values, double mean) { + double sum = 0.0; + for (Integer value : values) { + double deviation = mean - value; + sum += Math.pow(deviation, 2.0); + } + if (values.size() != 0) { + sum /= values.size(); + return Math.pow(sum, 0.5); + } else { + return -1.0; + } + } + + private Map getPartitionBucketsForNode( + final Map> assignment) { + Map partitionsPerNode = new TreeMap(); + for (String node : _liveNodes) { + partitionsPerNode.put(node, 0); + } + for (Entry> partitionEntry : assignment.entrySet()) { + final Map nodeMap = partitionEntry.getValue(); + for (String node : nodeMap.keySet()) { + String state = nodeMap.get(node); + if (state.equals(HelixDefinedState.DROPPED.toString())) { + continue; + } + // add 1 for every occurrence of a node + if (!partitionsPerNode.containsKey(node)) { + partitionsPerNode.put(node, 1); + } else { + partitionsPerNode.put(node, partitionsPerNode.get(node) + 1); + } + } + } + return partitionsPerNode; + } + + private Map> getStateBucketsForNode( + final Map> assignment) { + Map> result = new TreeMap>(); + for (String n : _liveNodes) { + result.put(n, new TreeMap()); + } + for (Map nodeStateMap : assignment.values()) { + for (Entry nodeState : nodeStateMap.entrySet()) { + if (!result.containsKey(nodeState.getKey())) { + result.put(nodeState.getKey(), new TreeMap()); + } + Map stateMap = result.get(nodeState.getKey()); + if (!stateMap.containsKey(nodeState.getValue())) { + stateMap.put(nodeState.getValue(), 1); + } else { + stateMap.put(nodeState.getValue(), stateMap.get(nodeState.getValue()) + 1); + } + } + } + return result; + } + + /** + * Randomly choose between killing, adding, or resurrecting a single node + * @return (Partition -> (Node -> State)) ZNRecord + */ + public ZNRecord runOnceRandomly() { + double choose = _random.nextDouble(); + ZNRecord result = null; + if (choose < P_KILL) { + result = removeSingleNode(null); + } else if (choose < P_KILL + P_ADD) { + result = addSingleNode(null); + } else if (choose < P_KILL + P_ADD + P_RESURRECT) { + result = resurrectSingleNode(null); + } + return result; + } + + /** + * Run rebalancer trying to add a never-live node + * @param node + * Optional String to add + * @return ZNRecord result returned by the rebalancer + */ + public ZNRecord addSingleNode(String node) { + logger.info("=================== add node ================="); + if (_nonLiveSet.size() == 0) { + logger.warn("Cannot add node because there are no nodes left to add."); + return null; + } + + // Get a random never-live node + if (node == null || !_nonLiveSet.contains(node)) { + node = getRandomSetElement(_nonLiveSet); + } + logger.info("Adding " + node); + _liveNodes.add(node); + _liveSet.add(node); + _nonLiveSet.remove(node); + + return new AutoRebalanceStrategy(RESOURCE_NAME, _partitions, _states, _maxPerNode, + _placementScheme).computePartitionAssignment(_liveNodes, _currentMapping, _allNodes); + } + + /** + * Run rebalancer trying to remove a live node + * @param node + * Optional String to remove + * @return ZNRecord result returned by the rebalancer + */ + public ZNRecord removeSingleNode(String node) { + logger.info("=================== remove node ================="); + if (_liveSet.size() == 0) { + logger.warn("Cannot remove node because there are no nodes left to remove."); + return null; + } + + // Get a random never-live node + if (node == null || !_liveSet.contains(node)) { + node = getRandomSetElement(_liveSet); + } + logger.info("Removing " + node); + _removedSet.add(node); + _liveNodes.remove(node); + _liveSet.remove(node); + + // the rebalancer expects that the current mapping doesn't contain deleted + // nodes + for (Map nodeMap : _currentMapping.values()) { + if (nodeMap.containsKey(node)) { + nodeMap.remove(node); + } + } + + return new AutoRebalanceStrategy(RESOURCE_NAME, _partitions, _states, _maxPerNode, + _placementScheme).computePartitionAssignment(_liveNodes, _currentMapping, _allNodes); + } + + /** + * Run rebalancer trying to add back a removed node + * @param node + * Optional String to resurrect + * @return ZNRecord result returned by the rebalancer + */ + public ZNRecord resurrectSingleNode(String node) { + logger.info("=================== resurrect node ================="); + if (_removedSet.size() == 0) { + logger.warn("Cannot remove node because there are no nodes left to resurrect."); + return null; + } + + // Get a random never-live node + if (node == null || !_removedSet.contains(node)) { + node = getRandomSetElement(_removedSet); + } + logger.info("Resurrecting " + node); + _removedSet.remove(node); + _liveNodes.add(node); + _liveSet.add(node); + + return new AutoRebalanceStrategy(RESOURCE_NAME, _partitions, _states, _maxPerNode, + _placementScheme).computePartitionAssignment(_liveNodes, _currentMapping, _allNodes); + } + + private T getRandomSetElement(Set source) { + int element = _random.nextInt(source.size()); + int i = 0; + for (T node : source) { + if (i == element) { + return node; + } + i++; + } + return null; + } + } +} diff --git a/helix-core/src/test/java/org/apache/helix/TestShuffledIdealState.java b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestShufflingTwoStateStrategy.java similarity index 54% rename from helix-core/src/test/java/org/apache/helix/TestShuffledIdealState.java rename to helix-core/src/test/java/org/apache/helix/controller/strategy/TestShufflingTwoStateStrategy.java index 5c158f8d10..aea9b70e75 100644 --- a/helix-core/src/test/java/org/apache/helix/TestShuffledIdealState.java +++ b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestShufflingTwoStateStrategy.java @@ -1,4 +1,4 @@ -package org.apache.helix; +package org.apache.helix.controller.strategy; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,7 +19,6 @@ * under the License. */ -import java.io.IOException; import java.io.StringReader; import java.io.StringWriter; import java.util.ArrayList; @@ -29,17 +28,12 @@ import java.util.Map; import org.apache.helix.ZNRecord; -import org.apache.helix.tools.IdealCalculatorByConsistentHashing; -import org.apache.helix.tools.IdealStateCalculatorByRush; -import org.apache.helix.tools.IdealStateCalculatorByShuffling; -import org.codehaus.jackson.JsonGenerationException; -import org.codehaus.jackson.map.JsonMappingException; import org.codehaus.jackson.map.ObjectMapper; import org.testng.Assert; import org.testng.AssertJUnit; import org.testng.annotations.Test; -public class TestShuffledIdealState { +public class TestShufflingTwoStateStrategy { @Test() public void testInvocation() throws Exception { int partitions = 6, replicas = 2; @@ -51,66 +45,53 @@ public void testInvocation() throws Exception { instanceNames.add("localhost_1234"); ZNRecord result = - IdealStateCalculatorByShuffling.calculateIdealState(instanceNames, partitions, replicas, - dbName); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "MASTER"); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "SLAVE"); + ShufflingTwoStateStrategy.calculateIdealState(instanceNames, partitions, replicas, dbName); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "MASTER"); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "SLAVE"); ZNRecord result2 = - IdealStateCalculatorByRush.calculateIdealState(instanceNames, 1, partitions, replicas, - dbName); + RUSHMasterSlaveStrategy.calculateIdealState(instanceNames, 1, partitions, replicas, dbName); ZNRecord result3 = - IdealCalculatorByConsistentHashing.calculateIdealState(instanceNames, partitions, replicas, - dbName, new IdealCalculatorByConsistentHashing.FnvHash()); - IdealCalculatorByConsistentHashing.printIdealStateStats(result3, "MASTER"); - IdealCalculatorByConsistentHashing.printIdealStateStats(result3, "SLAVE"); - IdealCalculatorByConsistentHashing.printIdealStateStats(result3, ""); - IdealCalculatorByConsistentHashing.printNodeOfflineOverhead(result3); + ConsistentHashingMasterSlaveStrategy.calculateIdealState(instanceNames, partitions, + replicas, dbName, new ConsistentHashingMasterSlaveStrategy.FnvHash()); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result3, "MASTER"); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result3, "SLAVE"); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result3, ""); + ConsistentHashingMasterSlaveStrategy.printNodeOfflineOverhead(result3); // System.out.println(result); ObjectMapper mapper = new ObjectMapper(); // ByteArrayOutputStream baos = new ByteArrayOutputStream(); StringWriter sw = new StringWriter(); - try { - mapper.writeValue(sw, result); - // System.out.println(sw.toString()); - - ZNRecord zn = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class); - System.out.println(result.toString()); - System.out.println(zn.toString()); - AssertJUnit.assertTrue(zn.toString().equalsIgnoreCase(result.toString())); - System.out.println(); - - sw = new StringWriter(); - mapper.writeValue(sw, result2); - - ZNRecord zn2 = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class); - System.out.println(result2.toString()); - System.out.println(zn2.toString()); - AssertJUnit.assertTrue(zn2.toString().equalsIgnoreCase(result2.toString())); - - sw = new StringWriter(); - mapper.writeValue(sw, result3); - System.out.println(); - - ZNRecord zn3 = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class); - System.out.println(result3.toString()); - System.out.println(zn3.toString()); - AssertJUnit.assertTrue(zn3.toString().equalsIgnoreCase(result3.toString())); - System.out.println(); - - } catch (JsonGenerationException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } catch (JsonMappingException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } catch (IOException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } + mapper.writeValue(sw, result); + // System.out.println(sw.toString()); + + ZNRecord zn = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class); + System.out.println(result.toString()); + System.out.println(zn.toString()); + AssertJUnit.assertTrue(zn.toString().equalsIgnoreCase(result.toString())); + System.out.println(); + + sw = new StringWriter(); + mapper.writeValue(sw, result2); + + ZNRecord zn2 = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class); + System.out.println(result2.toString()); + System.out.println(zn2.toString()); + AssertJUnit.assertTrue(zn2.toString().equalsIgnoreCase(result2.toString())); + + sw = new StringWriter(); + mapper.writeValue(sw, result3); + System.out.println(); + + ZNRecord zn3 = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class); + System.out.println(result3.toString()); + System.out.println(zn3.toString()); + AssertJUnit.assertTrue(zn3.toString().equalsIgnoreCase(result3.toString())); + System.out.println(); + } @Test @@ -125,10 +106,9 @@ public void testShuffledIdealState() { instanceNames.add("localhost_1234"); ZNRecord result = - IdealStateCalculatorByShuffling.calculateIdealState(instanceNames, partitions, replicas, - dbName); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "MASTER"); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "SLAVE"); + ShufflingTwoStateStrategy.calculateIdealState(instanceNames, partitions, replicas, dbName); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "MASTER"); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "SLAVE"); Assert.assertTrue(verify(result)); // partition is less than nodes @@ -141,10 +121,9 @@ public void testShuffledIdealState() { instanceNames.add("localhost_" + (1231 + i)); } result = - IdealStateCalculatorByShuffling.calculateIdealState(instanceNames, partitions, replicas, - dbName); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "MASTER"); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "SLAVE"); + ShufflingTwoStateStrategy.calculateIdealState(instanceNames, partitions, replicas, dbName); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "MASTER"); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "SLAVE"); Assert.assertTrue(verify(result)); // partitions is multiple of nodes @@ -157,10 +136,9 @@ public void testShuffledIdealState() { instanceNames.add("localhost_" + (1231 + i)); } result = - IdealStateCalculatorByShuffling.calculateIdealState(instanceNames, partitions, replicas, - dbName); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "MASTER"); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "SLAVE"); + ShufflingTwoStateStrategy.calculateIdealState(instanceNames, partitions, replicas, dbName); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "MASTER"); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "SLAVE"); Assert.assertTrue(verify(result)); // nodes are multiple of partitions @@ -173,10 +151,9 @@ public void testShuffledIdealState() { instanceNames.add("localhost_" + (1231 + i)); } result = - IdealStateCalculatorByShuffling.calculateIdealState(instanceNames, partitions, replicas, - dbName); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "MASTER"); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "SLAVE"); + ShufflingTwoStateStrategy.calculateIdealState(instanceNames, partitions, replicas, dbName); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "MASTER"); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "SLAVE"); Assert.assertTrue(verify(result)); // nodes are multiple of partitions @@ -189,10 +166,9 @@ public void testShuffledIdealState() { instanceNames.add("localhost_" + (1231 + i)); } result = - IdealStateCalculatorByShuffling.calculateIdealState(instanceNames, partitions, replicas, - dbName); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "MASTER"); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "SLAVE"); + ShufflingTwoStateStrategy.calculateIdealState(instanceNames, partitions, replicas, dbName); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "MASTER"); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "SLAVE"); Assert.assertTrue(verify(result)); // Just fits @@ -205,10 +181,9 @@ public void testShuffledIdealState() { instanceNames.add("localhost_" + (1231 + i)); } result = - IdealStateCalculatorByShuffling.calculateIdealState(instanceNames, partitions, replicas, - dbName); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "MASTER"); - IdealCalculatorByConsistentHashing.printIdealStateStats(result, "SLAVE"); + ShufflingTwoStateStrategy.calculateIdealState(instanceNames, partitions, replicas, dbName); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "MASTER"); + ConsistentHashingMasterSlaveStrategy.printIdealStateStats(result, "SLAVE"); Assert.assertTrue(verify(result)); } diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java deleted file mode 100644 index d5a1b08120..0000000000 --- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java +++ /dev/null @@ -1,180 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.Map; -import java.util.Set; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.NotificationContext; -import org.apache.helix.TestHelper; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.healthcheck.HealthStatsAggregationTask; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.integration.ZkIntegrationTestBase; -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.manager.zk.ZkClient; -import org.apache.helix.mock.participant.MockEspressoHealthReportProvider; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.mock.participant.MockTransition; -import org.apache.helix.model.Message; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.tools.ClusterStateVerifier; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class TestAddDropAlert extends ZkIntegrationTestBase { - ZkClient _zkClient; - protected ClusterSetup _setupTool = null; - protected final String _alertStr = - "EXP(accumulate()(localhost_12918.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)"; - protected final String _alertStatusStr = _alertStr; // +" : (*)"; - protected final String _dbName = "TestDB0"; - - @BeforeClass() - public void beforeClass() throws Exception { - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); - - _setupTool = new ClusterSetup(ZK_ADDR); - } - - @AfterClass - public void afterClass() { - _zkClient.close(); - } - - public class AddDropAlertTransition extends MockTransition { - @Override - public void doTransition(Message message, NotificationContext context) { - HelixManager manager = context.getManager(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - String fromState = message.getFromState(); - String toState = message.getToState(); - String instance = message.getTgtName(); - String partition = message.getPartitionName(); - - if (fromState.equalsIgnoreCase("SLAVE") && toState.equalsIgnoreCase("MASTER")) { - - // add a stat and report to ZK - // perhaps should keep reporter per instance... - ParticipantHealthReportCollectorImpl reporter = - new ParticipantHealthReportCollectorImpl(manager, instance); - MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider(); - reporter.addHealthReportProvider(provider); - String statName = "latency"; - provider.setStat(_dbName, statName, "15"); - reporter.transmitHealthReports(); - - // sleep long enough for first set of alerts to report and alert to get deleted - // then change reported data - try { - Thread.sleep(10000); - } catch (InterruptedException e) { - System.err.println("Error sleeping"); - } - provider.setStat(_dbName, statName, "1"); - reporter.transmitHealthReports(); - - /* - * for (int i = 0; i < 5; i++) { accessor.setProperty(PropertyType.HEALTHREPORT, - * new ZNRecord("mockAlerts" + i), instance, "mockAlerts"); try { - * Thread.sleep(1000); } catch (InterruptedException e) { // TODO Auto-generated - * catch block e.printStackTrace(); } } - */ - } - } - } - - @Test() - public void testAddDropAlert() throws Exception { - String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; - - System.out.println("START TestAddDropAlert at " + new Date(System.currentTimeMillis())); - - TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start port - "localhost", // participant name prefix - "TestDB", // resource name prefix - 1, // resources - 10, // partitions per resource group - 5, // number of nodes //change back to 5!!! - 1, // replicas //change back to 3!!! - "MasterSlave", true); // do rebalance - // enableHealthCheck(clusterName); - - _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr); - - StartCMResult cmResult = - TestHelper.startController(clusterName, "controller_0", ZK_ADDR, - HelixControllerMain.STANDALONE); - // start participants - for (int i = 0; i < 5; i++) // !!!change back to 5 - { - String instanceName = "localhost_" + (12918 + i); - - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new AddDropAlertTransition()); - participants[i].syncStart(); - // new Thread(participants[i]).start(); - } - - boolean result = - ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( - ZK_ADDR, clusterName)); - Assert.assertTrue(result); - - // drop alert soon after adding, but leave enough time for alert to fire once - // Thread.sleep(3000); - ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); - Builder keyBuilder = accessor.keyBuilder(); - - new HealthStatsAggregator(cmResult._manager).aggregate(); - String instance = "localhost_12918"; - ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord(); - Map> recMap = record.getMapFields(); - Set keySet = recMap.keySet(); - Assert.assertTrue(keySet.size() > 0); - - _setupTool.getClusterManagementTool().dropAlert(clusterName, _alertStr); - new HealthStatsAggregator(cmResult._manager).aggregate(); - // other verifications go here - // for (int i = 0; i < 1; i++) //change 1 back to 5 - // { - // String instance = "localhost_" + (12918 + i); - record = accessor.getProperty(keyBuilder.alertStatus()).getRecord(); - recMap = record.getMapFields(); - keySet = recMap.keySet(); - Assert.assertEquals(keySet.size(), 0); - // } - - System.out.println("END TestAddDropAlert at " + new Date(System.currentTimeMillis())); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java deleted file mode 100644 index 7f004d3994..0000000000 --- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java +++ /dev/null @@ -1,229 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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; - -import org.apache.helix.ConfigAccessor; -import org.apache.helix.model.ConfigScope; -import org.apache.helix.model.builder.ConfigScopeBuilder; -import org.apache.helix.model.builder.HelixConfigScopeBuilder; -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck; -import org.apache.helix.model.ExternalView; -import org.apache.helix.model.HealthStat; -import org.apache.helix.model.HelixConfigScope; -import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; -import org.apache.helix.model.InstanceConfig; -import org.apache.helix.model.InstanceConfig.InstanceConfigProperty; -import org.apache.helix.tools.ClusterStateVerifier; -import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; -import org.testng.Assert; -import org.testng.annotations.Test; - -public class TestAlertActionTriggering extends ZkStandAloneCMTestBaseWithPropertyServerCheck { - String _statName = "TestStat@DB=db1"; - String _stat = "TestStat"; - String metricName1 = "TestMetric1"; - String metricName2 = "TestMetric2"; - - void setHealthData(int[] val1, int[] val2) { - for (int i = 0; i < NODE_NR; i++) { - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - HelixManager manager = _startCMResultMap.get(instanceName)._manager; - ZNRecord record = new ZNRecord(_stat); - Map valMap = new HashMap(); - valMap.put(metricName1, val1[i] + ""); - valMap.put(metricName2, val2[i] + ""); - record.setSimpleField("TimeStamp", new Date().getTime() + ""); - record.setMapField(_statName, valMap); - HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor(); - Builder keyBuilder = helixDataAccessor.keyBuilder(); - helixDataAccessor.setProperty( - keyBuilder.healthReport(manager.getInstanceName(), record.getId()), - new HealthStat(record)); - } - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - - void setHealthData2(int[] val1) { - for (int i = 0; i < NODE_NR; i++) { - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - HelixManager manager = _startCMResultMap.get(instanceName)._manager; - ZNRecord record = new ZNRecord(_stat); - Map valMap = new HashMap(); - valMap.put(metricName2, val1[i] + ""); - record.setSimpleField("TimeStamp", new Date().getTime() + ""); - record.setMapField("TestStat@DB=TestDB;Partition=TestDB_3", valMap); - HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor(); - Builder keyBuilder = helixDataAccessor.keyBuilder(); - helixDataAccessor.setProperty( - keyBuilder.healthReport(manager.getInstanceName(), record.getId()), - new HealthStat(record)); - } - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - - @Test - public void testAlertActionDisableNode() throws InterruptedException { - // ConfigScope scope = new ConfigScopeBuilder().forCluster(CLUSTER_NAME).build(); - HelixConfigScope scope = - new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(CLUSTER_NAME).build(); - Map properties = new HashMap(); - properties.put("healthChange.enabled", "true"); - _setupTool.getClusterManagementTool().setConfig(scope, properties); - - String alertStr1 = - "EXP(decay(1.0)(localhost_*.TestStat@DB=db1.TestMetric1))CMP(GREATER)CON(20)ACTION(DISABLE_INSTANCE)"; - String alertStr2 = - "EXP(decay(1.0)(localhost_*.TestStat@DB=db1.TestMetric2))CMP(GREATER)CON(120)ACTION(DISABLE_INSTANCE)"; - String alertStr3 = - "EXP(decay(1.0)(localhost_*.TestStat@DB=TestDB;Partition=*.TestMetric2))CMP(GREATER)CON(160)ACTION(DISABLE_PARTITION)"; - - _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, alertStr1); - _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, alertStr2); - _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, alertStr3); - - int[] metrics1 = { - 10, 15, 22, 12, 16 - }; - int[] metrics2 = { - 22, 115, 22, 163, 16 - }; - int[] metrics3 = { - 0, 0, 0, 0, 0 - }; - setHealthData(metrics1, metrics2); - - String controllerName = CONTROLLER_PREFIX + "_0"; - HelixManager manager = _startCMResultMap.get(controllerName)._manager; - - HealthStatsAggregator task = - new HealthStatsAggregator(_startCMResultMap.get(controllerName)._manager); - task.aggregate(); - Thread.sleep(4000); - HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor(); - Builder keyBuilder = helixDataAccessor.keyBuilder(); - - boolean result = - ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - CLUSTER_NAME)); - Assert.assertTrue(result); - - Builder kb = manager.getHelixDataAccessor().keyBuilder(); - ExternalView externalView = - manager.getHelixDataAccessor().getProperty(kb.externalView("TestDB")); - // Test the DISABLE_INSTANCE alerts - String participant1 = "localhost_" + (START_PORT + 3); - String participant2 = "localhost_" + (START_PORT + 2); - ConfigAccessor configAccessor = manager.getConfigAccessor(); - // scope = new - // ConfigScopeBuilder().forCluster(manager.getClusterName()).forParticipant(participant1).build(); - scope = - new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT) - .forCluster(manager.getClusterName()).forParticipant(participant1).build(); - String isEnabled = configAccessor.get(scope, "HELIX_ENABLED"); - Assert.assertFalse(Boolean.parseBoolean(isEnabled)); - - // scope = new - // ConfigScopeBuilder().forCluster(manager.getClusterName()).forParticipant(participant2).build(); - scope = - new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT) - .forCluster(manager.getClusterName()).forParticipant(participant2).build(); - isEnabled = configAccessor.get(scope, "HELIX_ENABLED"); - Assert.assertFalse(Boolean.parseBoolean(isEnabled)); - - for (String partitionName : externalView.getRecord().getMapFields().keySet()) { - for (String hostName : externalView.getRecord().getMapField(partitionName).keySet()) { - if (hostName.equals(participant1) || hostName.equals(participant2)) { - Assert.assertEquals(externalView.getRecord().getMapField(partitionName).get(hostName), - "OFFLINE"); - } - } - } - - // enable the disabled instances - setHealthData(metrics3, metrics3); - task.aggregate(); - Thread.sleep(1000); - - manager.getClusterManagmentTool().enableInstance(manager.getClusterName(), participant2, true); - manager.getClusterManagmentTool().enableInstance(manager.getClusterName(), participant1, true); - - result = - ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - CLUSTER_NAME)); - Assert.assertTrue(result); - - // Test the DISABLE_PARTITION case - int[] metrics4 = { - 22, 115, 22, 16, 163 - }; - setHealthData2(metrics4); - task.aggregate(); - - // scope = new - // ConfigScopeBuilder().forCluster(manager.getClusterName()).forParticipant(participant1).build(); - scope = - new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT) - .forCluster(manager.getClusterName()).forParticipant(participant1).build(); - isEnabled = configAccessor.get(scope, "HELIX_ENABLED"); - Assert.assertTrue(Boolean.parseBoolean(isEnabled)); - - // scope = new - // ConfigScopeBuilder().forCluster(manager.getClusterName()).forParticipant(participant2).build(); - scope = - new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT) - .forCluster(manager.getClusterName()).forParticipant(participant2).build(); - isEnabled = configAccessor.get(scope, "HELIX_ENABLED"); - Assert.assertTrue(Boolean.parseBoolean(isEnabled)); - - result = - ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - CLUSTER_NAME)); - Assert.assertTrue(result); - String participant3 = "localhost_" + (START_PORT + 4); - externalView = manager.getHelixDataAccessor().getProperty(kb.externalView("TestDB")); - Assert.assertTrue(externalView.getRecord().getMapField("TestDB_3").get(participant3) - .equalsIgnoreCase("OFFLINE")); - - InstanceConfig nodeConfig = - helixDataAccessor.getProperty(keyBuilder.instanceConfig(participant3)); - Assert.assertTrue(nodeConfig.getRecord() - .getListField(InstanceConfigProperty.HELIX_DISABLED_PARTITION.toString()) - .contains("TestDB_3")); - - } -} diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java deleted file mode 100644 index 125f61fcd9..0000000000 --- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java +++ /dev/null @@ -1,423 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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; -import java.util.TreeMap; - -import org.apache.helix.model.ConfigScope; -import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; -import org.apache.helix.model.builder.ConfigScopeBuilder; -import org.apache.helix.model.builder.HelixConfigScopeBuilder; -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.HelixProperty; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck; -import org.apache.helix.model.AlertHistory; -import org.apache.helix.model.HealthStat; -import org.apache.helix.model.HelixConfigScope; -import org.testng.Assert; -import org.testng.annotations.Test; - -/** - * setup a storage cluster and start a zk-based cluster controller in stand-alone mode - * start 5 dummy participants verify the current states at end - */ - -public class TestAlertFireHistory extends ZkStandAloneCMTestBaseWithPropertyServerCheck { - String _statName = "TestStat@DB=db1"; - String _stat = "TestStat"; - String metricName1 = "TestMetric1"; - String metricName2 = "TestMetric2"; - - String _alertStr1 = "EXP(decay(1.0)(localhost_*.TestStat@DB=db1.TestMetric1))CMP(GREATER)CON(20)"; - String _alertStr2 = - "EXP(decay(1.0)(localhost_*.TestStat@DB=db1.TestMetric2))CMP(GREATER)CON(100)"; - - void setHealthData(int[] val1, int[] val2) { - for (int i = 0; i < NODE_NR; i++) { - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - HelixManager manager = _startCMResultMap.get(instanceName)._manager; - ZNRecord record = new ZNRecord(_stat); - Map valMap = new HashMap(); - valMap.put(metricName1, val1[i] + ""); - valMap.put(metricName2, val2[i] + ""); - record.setSimpleField("TimeStamp", new Date().getTime() + ""); - record.setMapField(_statName, valMap); - HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor(); - Builder keyBuilder = helixDataAccessor.keyBuilder(); - helixDataAccessor.setProperty( - keyBuilder.healthReport(manager.getInstanceName(), record.getId()), - new HealthStat(record)); - } - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - - @Test - public void TestAlertDisable() throws InterruptedException { - - int[] metrics1 = { - 10, 15, 22, 24, 16 - }; - int[] metrics2 = { - 22, 115, 22, 141, 16 - }; - setHealthData(metrics1, metrics2); - - String controllerName = CONTROLLER_PREFIX + "_0"; - HelixManager manager = _startCMResultMap.get(controllerName)._manager; - manager.startTimerTasks(); - - _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, _alertStr1); - _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, _alertStr2); - - // ConfigScope scope = new ConfigScopeBuilder().forCluster(CLUSTER_NAME).build(); - HelixConfigScope scope = - new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(CLUSTER_NAME).build(); - Map properties = new HashMap(); - properties.put("healthChange.enabled", "false"); - _setupTool.getClusterManagementTool().setConfig(scope, properties); - - HealthStatsAggregator task = - new HealthStatsAggregator(_startCMResultMap.get(controllerName)._manager); - task.aggregate(); - Thread.sleep(100); - HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor(); - Builder keyBuilder = helixDataAccessor.keyBuilder(); - - AlertHistory history = manager.getHelixDataAccessor().getProperty(keyBuilder.alertHistory()); - // - Assert.assertEquals(history, null); - - properties.put("healthChange.enabled", "true"); - _setupTool.getClusterManagementTool().setConfig(scope, properties); - - task.aggregate(); - Thread.sleep(100); - - history = manager.getHelixDataAccessor().getProperty(keyBuilder.alertHistory()); - // - Assert.assertNotNull(history); - Assert.assertEquals(history.getRecord().getMapFields().size(), 1); - } - - @Test - public void TestAlertHistory() throws InterruptedException { - int[] metrics1 = { - 10, 15, 22, 24, 16 - }; - int[] metrics2 = { - 22, 115, 22, 141, 16 - }; - setHealthData(metrics1, metrics2); - - String controllerName = CONTROLLER_PREFIX + "_0"; - HelixManager manager = _startCMResultMap.get(controllerName)._manager; - manager.stopTimerTasks(); - - _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, _alertStr1); - _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, _alertStr2); - - int historySize = 0; - HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor(); - Builder keyBuilder = helixDataAccessor.keyBuilder(); - HelixProperty property = helixDataAccessor.getProperty(keyBuilder.alertHistory()); - ZNRecord history = null; - if (property != null) { - history = property.getRecord(); - historySize = property.getRecord().getMapFields().size(); - } - - HealthStatsAggregator task = - new HealthStatsAggregator(_startCMResultMap.get(controllerName)._manager); - task.aggregate(); - Thread.sleep(100); - - history = helixDataAccessor.getProperty(keyBuilder.alertHistory()).getRecord(); - // - Assert.assertEquals(history.getMapFields().size(), 1 + historySize); - TreeMap> recordMap = new TreeMap>(); - recordMap.putAll(history.getMapFields()); - Map lastRecord = recordMap.firstEntry().getValue(); - Assert.assertTrue(lastRecord.size() == 4); - Assert.assertTrue(lastRecord.get("(localhost_12920.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("ON")); - Assert.assertTrue(lastRecord.get("(localhost_12919.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("ON")); - Assert.assertTrue(lastRecord.get("(localhost_12921.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("ON")); - Assert.assertTrue(lastRecord.get("(localhost_12921.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("ON")); - - setHealthData(metrics1, metrics2); - task.aggregate(); - Thread.sleep(100); - history = helixDataAccessor.getProperty(keyBuilder.alertHistory()).getRecord(); - // no change - Assert.assertEquals(history.getMapFields().size(), 1 + historySize); - recordMap = new TreeMap>(); - recordMap.putAll(history.getMapFields()); - lastRecord = recordMap.firstEntry().getValue(); - Assert.assertTrue(lastRecord.size() == 4); - Assert.assertTrue(lastRecord.get("(localhost_12920.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("ON")); - Assert.assertTrue(lastRecord.get("(localhost_12919.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("ON")); - Assert.assertTrue(lastRecord.get("(localhost_12921.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("ON")); - Assert.assertTrue(lastRecord.get("(localhost_12921.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("ON")); - - int[] metrics3 = { - 21, 44, 22, 14, 16 - }; - int[] metrics4 = { - 122, 115, 222, 41, 16 - }; - setHealthData(metrics3, metrics4); - task.aggregate(); - Thread.sleep(100); - history = helixDataAccessor.getProperty(keyBuilder.alertHistory()).getRecord(); - // new delta should be recorded - Assert.assertEquals(history.getMapFields().size(), 2 + historySize); - recordMap = new TreeMap>(); - recordMap.putAll(history.getMapFields()); - lastRecord = recordMap.lastEntry().getValue(); - Assert.assertEquals(lastRecord.size(), 6); - Assert.assertTrue(lastRecord.get("(localhost_12918.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("ON")); - Assert.assertTrue(lastRecord.get("(localhost_12921.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("OFF")); - Assert.assertTrue(lastRecord.get("(localhost_12920.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("ON")); - Assert.assertTrue(lastRecord.get("(localhost_12921.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("OFF")); - Assert.assertTrue(lastRecord.get("(localhost_12918.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("ON")); - Assert.assertTrue(lastRecord.get("(localhost_12919.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("ON")); - - int[] metrics5 = { - 0, 0, 0, 0, 0 - }; - int[] metrics6 = { - 0, 0, 0, 0, 0 - }; - setHealthData(metrics5, metrics6); - task.aggregate(); - - for (int i = 0; i < 10; i++) { - Thread.sleep(500); - history = helixDataAccessor.getProperty(keyBuilder.alertHistory()).getRecord(); - recordMap = new TreeMap>(); - recordMap.putAll(history.getMapFields()); - lastRecord = recordMap.lastEntry().getValue(); - - if (history.getMapFields().size() == 3 + historySize && lastRecord.size() == 6) { - break; - } - } - - // reset everything - Assert.assertEquals(history.getMapFields().size(), 3 + historySize, - "expect history-map-field size is " + (3 + historySize) + ", but was " + history); - Assert - .assertTrue(lastRecord.size() == 6, "expect last-record size is 6, but was " + lastRecord); - - Assert.assertTrue(lastRecord.get("(localhost_12918.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("OFF")); - Assert.assertTrue(lastRecord.get("(localhost_12920.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("OFF")); - Assert.assertTrue(lastRecord.get("(localhost_12920.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("OFF")); - Assert.assertTrue(lastRecord.get("(localhost_12919.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("OFF")); - Assert.assertTrue(lastRecord.get("(localhost_12918.TestStat@DB#db1.TestMetric2)GREATER(100)") - .equals("OFF")); - Assert.assertTrue(lastRecord.get("(localhost_12919.TestStat@DB#db1.TestMetric1)GREATER(20)") - .equals("OFF")); - - // Size of the history should be 30 - for (int i = 0; i < 27; i++) { - int x = i % 2; - int y = (i + 1) % 2; - int[] metricsx = { - 19 + 3 * x, 19 + 3 * y, 19 + 4 * x, 18 + 4 * y, 17 + 5 * y - }; - int[] metricsy = { - 99 + 3 * x, 99 + 3 * y, 98 + 4 * x, 98 + 4 * y, 97 + 5 * y - }; - - setHealthData(metricsx, metricsy); - task.aggregate(); - Thread.sleep(100); - history = helixDataAccessor.getProperty(keyBuilder.alertHistory()).getRecord(); - - Assert.assertEquals(history.getMapFields().size(), Math.min(3 + i + 1 + historySize, 30)); - recordMap = new TreeMap>(); - recordMap.putAll(history.getMapFields()); - lastRecord = recordMap.lastEntry().getValue(); - if (i == 0) { - Assert.assertTrue(lastRecord.size() == 6); - Assert.assertTrue(lastRecord - .get("(localhost_12922.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12922.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12919.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord - .get("(localhost_12921.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12921.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord - .get("(localhost_12919.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - } else { - System.out.println(lastRecord.size()); - Assert.assertEquals(lastRecord.size(), 10); - if (x == 0) { - Assert.assertTrue(lastRecord.get( - "(localhost_12922.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12922.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12920.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12918.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12919.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12921.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12920.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12921.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12918.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12919.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - } else { - Assert.assertTrue(lastRecord.get( - "(localhost_12922.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12922.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12920.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12918.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12919.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12921.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12920.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12921.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12918.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12919.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - } - } - } - // size limit is 30 - for (int i = 0; i < 10; i++) { - int x = i % 2; - int y = (i + 1) % 2; - int[] metricsx = { - 19 + 3 * x, 19 + 3 * y, 19 + 4 * x, 18 + 4 * y, 17 + 5 * y - }; - int[] metricsy = { - 99 + 3 * x, 99 + 3 * y, 98 + 4 * x, 98 + 4 * y, 97 + 5 * y - }; - - setHealthData(metricsx, metricsy); - task.aggregate(); - for (int j = 0; j < 10; j++) { - Thread.sleep(100); - history = helixDataAccessor.getProperty(keyBuilder.alertHistory()).getRecord(); - recordMap = new TreeMap>(); - recordMap.putAll(history.getMapFields()); - lastRecord = recordMap.lastEntry().getValue(); - - if (history.getMapFields().size() == 30 && lastRecord.size() == 10) - break; - } - Assert.assertEquals(history.getMapFields().size(), 30, - "expect history.map-field size is 30, but was " + history); - Assert.assertEquals(lastRecord.size(), 10, "expect last-record size is 10, but was " - + lastRecord); - - if (x == 0) { - Assert.assertTrue(lastRecord.get( - "(localhost_12922.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord - .get("(localhost_12922.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord - .get("(localhost_12920.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - Assert.assertTrue(lastRecord - .get("(localhost_12918.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12919.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord - .get("(localhost_12921.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12920.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12921.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12918.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord - .get("(localhost_12919.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - } else { - Assert.assertTrue(lastRecord.get( - "(localhost_12922.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord - .get("(localhost_12922.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - Assert.assertTrue(lastRecord - .get("(localhost_12920.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord - .get("(localhost_12918.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12919.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord - .get("(localhost_12921.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12920.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord.get( - "(localhost_12921.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("OFF")); - Assert.assertTrue(lastRecord.get( - "(localhost_12918.TestStat@DB#db1.TestMetric2)GREATER(100)").equals("ON")); - Assert.assertTrue(lastRecord - .get("(localhost_12919.TestStat@DB#db1.TestMetric1)GREATER(20)").equals("OFF")); - } - } - - } - -} diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java deleted file mode 100644 index c5f373cc5c..0000000000 --- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java +++ /dev/null @@ -1,148 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.concurrent.atomic.AtomicBoolean; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.NotificationContext; -import org.apache.helix.TestHelper; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.integration.ZkIntegrationTestBase; -import org.apache.helix.manager.zk.ZKHelixDataAccessor; -import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.mock.participant.MockTransition; -import org.apache.helix.model.HealthStat; -import org.apache.helix.model.Message; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.tools.ClusterStateVerifier; -import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; -import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier; -import org.testng.Assert; -import org.testng.annotations.Test; - -public class TestDummyAlerts extends ZkIntegrationTestBase { - public class DummyAlertsTransition extends MockTransition { - private final AtomicBoolean _done = new AtomicBoolean(false); - - @Override - public void doTransition(Message message, NotificationContext context) { - HelixManager manager = context.getManager(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - Builder keyBuilder = accessor.keyBuilder(); - - String instance = message.getTgtName(); - if (_done.getAndSet(true) == false) { - for (int i = 0; i < 5; i++) { - // System.out.println(instance + " sets healthReport: " + "mockAlerts" + i); - accessor.setProperty(keyBuilder.healthReport(instance, "mockAlerts"), new HealthStat( - new ZNRecord("mockAlerts" + i))); - } - } - } - - } - - @Test() - public void testDummyAlerts() throws Exception { - // Logger.getRootLogger().setLevel(Level.INFO); - String className = TestHelper.getTestClassName(); - String methodName = TestHelper.getTestMethodName(); - String clusterName = className + "_" + methodName; - final int n = 5; - - MockParticipant[] participants = new MockParticipant[n]; - - System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - - TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start - // port - "localhost", // participant name prefix - "TestDB", // resource name prefix - 1, // resources - 10, // partitions per resource - n, // number of nodes - 3, // replicas - "MasterSlave", true); // do rebalance - - ClusterSetup setupTool = new ClusterSetup(ZK_ADDR); - enableHealthCheck(clusterName); - setupTool - .getClusterManagementTool() - .addAlert(clusterName, - "EXP(decay(1.0)(*.defaultPerfCounters@defaultPerfCounters.availableCPUs))CMP(GREATER)CON(2)"); - - // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); - controller.syncStart(); - - // start participants - for (int i = 0; i < n; i++) { - String instanceName = "localhost_" + (12918 + i); - - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new DummyAlertsTransition()); - participants[i].syncStart(); - } - - boolean result = - ClusterStateVerifier - .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, clusterName)); - Assert.assertTrue(result); - - result = - ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, - clusterName)); - Assert.assertTrue(result); - - // other verifications go here - ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); - Builder keyBuilder = accessor.keyBuilder(); - - for (int i = 0; i < n; i++) { - String instance = "localhost_" + (12918 + i); - ZNRecord record = null; - for (int j = 0; j < 10; j++) { - record = accessor.getProperty(keyBuilder.healthReport(instance, "mockAlerts")).getRecord(); - if (record.getId().equals("mockAlerts4")) { - break; - } else { - Thread.sleep(500); - } - } - Assert.assertEquals(record.getId(), "mockAlerts4"); - } - - // clean up - Thread.sleep(1000); - controller.syncStop(); - for (int i = 0; i < 5; i++) { - participants[i].syncStop(); - } - - System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java deleted file mode 100644 index 69d1062de1..0000000000 --- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java +++ /dev/null @@ -1,186 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.Map; -import java.util.Set; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.NotificationContext; -import org.apache.helix.TestHelper; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.alerts.AlertValueAndStatus; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.healthcheck.HealthStatsAggregationTask; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.integration.ZkIntegrationTestBase; -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.manager.zk.ZkClient; -import org.apache.helix.mock.participant.MockEspressoHealthReportProvider; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.mock.participant.MockTransition; -import org.apache.helix.model.Message; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.tools.ClusterStateVerifier; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class TestExpandAlert extends ZkIntegrationTestBase { - ZkClient _zkClient; - protected ClusterSetup _setupTool = null; - protected final String _alertStr = - "EXP(decay(1.0)(localhost_*.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(16)"; - protected final String _alertStatusStr = _alertStr - + " : (localhost_12918.RestQueryStats@DBName=TestDB0.latency)"; - protected final String _dbName = "TestDB0"; - - @BeforeClass() - public void beforeClass() throws Exception { - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); - - _setupTool = new ClusterSetup(ZK_ADDR); - } - - @AfterClass - public void afterClass() { - _zkClient.close(); - } - - public class ExpandAlertTransition extends MockTransition { - @Override - public void doTransition(Message message, NotificationContext context) { - HelixManager manager = context.getManager(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - String fromState = message.getFromState(); - String toState = message.getToState(); - String instance = message.getTgtName(); - String partition = message.getPartitionName(); - - if (fromState.equalsIgnoreCase("SLAVE") && toState.equalsIgnoreCase("MASTER")) { - - // add a stat and report to ZK - // perhaps should keep reporter per instance... - ParticipantHealthReportCollectorImpl reporter = - new ParticipantHealthReportCollectorImpl(manager, instance); - MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider(); - reporter.addHealthReportProvider(provider); - String statName = "latency"; - provider.setStat(_dbName, statName, "15"); - reporter.transmitHealthReports(); - - /* - * for (int i = 0; i < 5; i++) - * { - * accessor.setProperty(PropertyType.HEALTHREPORT, - * new ZNRecord("mockAlerts" + i), - * instance, - * "mockAlerts"); - * try - * { - * Thread.sleep(1000); - * } - * catch (InterruptedException e) - * { - * // TODO Auto-generated catch block - * e.printStackTrace(); - * } - * } - */ - } - } - - } - - @Test() - public void testExpandAlert() throws Exception { - String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; - - System.out.println("START TestExpandAlert at " + new Date(System.currentTimeMillis())); - - TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start port - "localhost", // participant name prefix - "TestDB", // resource name prefix - 1, // resources - 10, // partitions per resource - 5, // number of nodes //change back to 5!!! - 3, // replicas //change back to 3!!! - "MasterSlave", true); // do rebalance - // enableHealthCheck(clusterName); - - _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr); - - StartCMResult cmResult = - TestHelper.startController(clusterName, "controller_0", ZK_ADDR, - HelixControllerMain.STANDALONE); - // start participants - for (int i = 0; i < 5; i++) // !!!change back to 5 - { - String instanceName = "localhost_" + (12918 + i); - - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new ExpandAlertTransition()); - participants[i].start(); - // new Thread(participants[i]).start(); - } - - boolean result = - ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( - ZK_ADDR, clusterName)); - Assert.assertTrue(result); - - Thread.sleep(1000); - // HealthAggregationTask is supposed to run by a timer every 30s - // To make sure HealthAggregationTask is run, we invoke it explicitly for this test - new HealthStatsAggregator(cmResult._manager).aggregate(); - // sleep for a few seconds to give stats stage time to trigger - Thread.sleep(3000); - - // other verifications go here - ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); - Builder keyBuilder = accessor.keyBuilder(); - - // for (int i = 0; i < 1; i++) //change 1 back to 5 - // { - // String instance = "localhost_" + (12918 + i); - // String instance = "localhost_12918"; - ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord(); - Map> recMap = record.getMapFields(); - Set keySet = recMap.keySet(); - Map alertStatusMap = recMap.get(_alertStatusStr); - String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME); - boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME)); - Assert.assertEquals(Double.parseDouble(val), Double.parseDouble("15.0")); - Assert.assertFalse(fired); - // } - - System.out.println("END TestExpandAlert at " + new Date(System.currentTimeMillis())); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java deleted file mode 100644 index 1db5ddd41d..0000000000 --- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java +++ /dev/null @@ -1,202 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.Map; -import java.util.Set; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.NotificationContext; -import org.apache.helix.TestHelper; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.alerts.AlertValueAndStatus; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.healthcheck.HealthStatsAggregationTask; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.integration.ZkIntegrationTestBase; -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.manager.zk.ZkClient; -import org.apache.helix.mock.participant.MockEspressoHealthReportProvider; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.mock.participant.MockTransition; -import org.apache.helix.model.Message; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.tools.ClusterStateVerifier; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class TestSimpleAlert extends ZkIntegrationTestBase { - ZkClient _zkClient; - protected ClusterSetup _setupTool = null; - protected final String _alertStr = - "EXP(decay(1.0)(localhost_12918.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)"; - protected final String _alertStatusStr = _alertStr; // +" : (*)"; - protected final String _dbName = "TestDB0"; - - @BeforeClass() - public void beforeClass() throws Exception { - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); - - _setupTool = new ClusterSetup(ZK_ADDR); - } - - @AfterClass - public void afterClass() { - _zkClient.close(); - } - - public class SimpleAlertTransition extends MockTransition { - int _alertValue; - - public SimpleAlertTransition(int value) { - _alertValue = value; - } - - @Override - public void doTransition(Message message, NotificationContext context) { - HelixManager manager = context.getManager(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - String fromState = message.getFromState(); - String toState = message.getToState(); - String instance = message.getTgtName(); - String partition = message.getPartitionName(); - - if (fromState.equalsIgnoreCase("SLAVE") && toState.equalsIgnoreCase("MASTER")) { - - // add a stat and report to ZK - // perhaps should keep reporter per instance... - ParticipantHealthReportCollectorImpl reporter = - new ParticipantHealthReportCollectorImpl(manager, instance); - MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider(); - reporter.addHealthReportProvider(provider); - String statName = "latency"; - provider.setStat(_dbName, statName, "" + (0.1 + _alertValue)); - reporter.transmitHealthReports(); - - /* - * for (int i = 0; i < 5; i++) - * { - * accessor.setProperty(PropertyType.HEALTHREPORT, - * new ZNRecord("mockAlerts" + i), - * instance, - * "mockAlerts"); - * try - * { - * Thread.sleep(1000); - * } - * catch (InterruptedException e) - * { - * // TODO Auto-generated catch block - * e.printStackTrace(); - * } - * } - */ - } - } - - } - - @Test() - public void testSimpleAlert() throws Exception { - String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; - - System.out.println("START TestSimpleAlert at " + new Date(System.currentTimeMillis())); - - TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start port - "localhost", // participant name prefix - "TestDB", // resource name prefix - 1, // resources - 10, // partitions per resource - 5, // number of nodes //change back to 5!!! - 3, // replicas //change back to 3!!! - "MasterSlave", true); // do rebalance - - // enableHealthCheck(clusterName); - - StartCMResult cmResult = - TestHelper.startController(clusterName, "controller_0", ZK_ADDR, - HelixControllerMain.STANDALONE); - cmResult._manager.startTimerTasks(); - _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr); - // start participants - for (int i = 0; i < 5; i++) // !!!change back to 5 - { - String instanceName = "localhost_" + (12918 + i); - - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new SimpleAlertTransition(15)); - participants[i].syncStart(); - // new Thread(participants[i]).start(); - } - - boolean result = - ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( - ZK_ADDR, clusterName)); - Assert.assertTrue(result); - - // HealthAggregationTask is supposed to run by a timer every 30s - // To make sure HealthAggregationTask is run, we invoke it explicitly for this test - new HealthStatsAggregator(cmResult._manager).aggregate(); - // sleep for a few seconds to give stats stage time to trigger - Thread.sleep(3000); - - // other verifications go here - ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); - Builder keyBuilder = accessor.keyBuilder(); - // for (int i = 0; i < 1; i++) //change 1 back to 5 - // { - // String instance = "localhost_" + (12918 + i); - String instance = "localhost_12918"; - ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord(); - Map> recMap = record.getMapFields(); - Set keySet = recMap.keySet(); - Map alertStatusMap = recMap.get(_alertStatusStr); - String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME); - boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME)); - Assert.assertEquals(Double.parseDouble(val), Double.parseDouble("15.1")); - Assert.assertTrue(fired); - - // Verify Alert history from ZK - ZNRecord alertHistory = accessor.getProperty(keyBuilder.alertHistory()).getRecord(); - - String deltakey = (String) (alertHistory.getMapFields().keySet().toArray()[0]); - Map delta = alertHistory.getMapField(deltakey); - Assert.assertTrue(delta.size() == 1); - Assert - .assertTrue(delta - .get( - "EXP(decay(1.0)(localhost_12918.RestQueryStats@DBName#TestDB0.latency))CMP(GREATER)CON(10)--(%)") - .equals("ON")); - // } - - System.out.println("END TestSimpleAlert at " + new Date(System.currentTimeMillis())); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java deleted file mode 100644 index c5b55dabea..0000000000 --- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java +++ /dev/null @@ -1,246 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.Map; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.NotificationContext; -import org.apache.helix.TestHelper; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.alerts.AlertValueAndStatus; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.healthcheck.HealthStatsAggregationTask; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.integration.ZkIntegrationTestBase; -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.manager.zk.ZkClient; -import org.apache.helix.mock.participant.MockEspressoHealthReportProvider; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.mock.participant.MockTransition; -import org.apache.helix.model.Message; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.tools.ClusterStateVerifier; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class TestSimpleWildcardAlert extends ZkIntegrationTestBase { - ZkClient _zkClient; - protected ClusterSetup _setupTool = null; - protected final String _alertStr = - "EXP(decay(1.0)(localhost_12918.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)"; - protected final String _alertStatusStr = _alertStr; // +" : (*)"; - protected final String _dbName = "TestDB0"; - - @BeforeClass() - public void beforeClass() throws Exception { - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); - - _setupTool = new ClusterSetup(ZK_ADDR); - } - - @AfterClass - public void afterClass() { - _zkClient.close(); - } - - public class SimpleAlertTransition extends MockTransition { - int _alertValue; - - public SimpleAlertTransition(int value) { - _alertValue = value; - } - - @Override - public void doTransition(Message message, NotificationContext context) { - HelixManager manager = context.getManager(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - String fromState = message.getFromState(); - String toState = message.getToState(); - String instance = message.getTgtName(); - String partition = message.getPartitionName(); - - if (fromState.equalsIgnoreCase("SLAVE") && toState.equalsIgnoreCase("MASTER")) { - - // add a stat and report to ZK - // perhaps should keep reporter per instance... - ParticipantHealthReportCollectorImpl reporter = - new ParticipantHealthReportCollectorImpl(manager, instance); - MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider(); - reporter.addHealthReportProvider(provider); - String statName = "latency"; - provider.setStat(_dbName, statName, "" + (0.1 + _alertValue)); - reporter.transmitHealthReports(); - - /* - * for (int i = 0; i < 5; i++) - * { - * accessor.setProperty(PropertyType.HEALTHREPORT, - * new ZNRecord("mockAlerts" + i), - * instance, - * "mockAlerts"); - * try - * { - * Thread.sleep(1000); - * } - * catch (InterruptedException e) - * { - * // TODO Auto-generated catch block - * e.printStackTrace(); - * } - * } - */ - } - } - - } - - @Test() - public void testSimpleWildcardAlert() throws Exception { - String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; - - System.out.println("START testSimpleWildcardAlert at " + new Date(System.currentTimeMillis())); - - TestHelper.setupCluster(clusterName, ZK_ADDR, 12944, // participant start port - "localhost", // participant name prefix - "TestDB", // resource name prefix - 1, // resources - 10, // partitions per resource - 5, // number of nodes //change back to 5!!! - 3, // replicas //change back to 3!!! - "MasterSlave", true); // do rebalance - - // enableHealthCheck(clusterName); - - StartCMResult cmResult = - TestHelper.startController(clusterName, "controller_0", ZK_ADDR, - HelixControllerMain.STANDALONE); - cmResult._manager.stopTimerTasks(); - - String alertwildcard = - "EXP(decay(1.0)(localhost*.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)"; - - _setupTool.getClusterManagementTool().addAlert(clusterName, alertwildcard); - // start participants - for (int i = 0; i < 5; i++) // !!!change back to 5 - { - String instanceName = "localhost_" + (12944 + i); - - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new SimpleAlertTransition(i * 5)); - participants[i].syncStart(); - // new Thread(participants[i]).start(); - } - - boolean result = - ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( - ZK_ADDR, clusterName)); - Assert.assertTrue(result); - - Thread.sleep(1000); - // HealthAggregationTask is supposed to run by a timer every 30s - // To make sure HealthAggregationTask is run, we invoke it explicitly for this test - new HealthStatsAggregator(cmResult._manager).aggregate(); - // sleep for a few seconds to give stats stage time to trigger - Thread.sleep(1000); - - // other verifications go here - ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); - Builder keyBuilder = accessor.keyBuilder(); - ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord(); - Map> recMap = record.getMapFields(); - for (int i = 0; i < 2; i++) { - String alertString = "(localhost_" + (12944 + i) + ".RestQueryStats@DBName=TestDB0.latency)"; - Map alertStatusMap = recMap.get(alertwildcard + " : " + alertString); - String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME); - boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME)); - Assert.assertEquals(Double.parseDouble(val), (double) i * 5 + 0.1); - Assert.assertFalse(fired); - } - for (int i = 2; i < 5; i++) { - String alertString = "(localhost_" + (12944 + i) + ".RestQueryStats@DBName=TestDB0.latency)"; - Map alertStatusMap = recMap.get(alertwildcard + " : " + alertString); - String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME); - boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME)); - Assert.assertEquals(Double.parseDouble(val), (double) i * 5 + 0.1); - Assert.assertTrue(fired); - } - ZNRecord alertHistory = accessor.getProperty(keyBuilder.alertHistory()).getRecord(); - - String deltakey = (String) (alertHistory.getMapFields().keySet().toArray()[0]); - Map delta = alertHistory.getMapField(deltakey); - Assert.assertEquals(delta.size(), 3); - for (int i = 2; i < 5; i++) { - String alertString = - "(localhost_" + (12944 + i) + ".RestQueryStats@DBName#TestDB0.latency)GREATER(10)"; - Assert.assertTrue(delta.get(alertString).equals("ON")); - } - - // Drop and add another alert - _setupTool.getClusterManagementTool().dropAlert(clusterName, alertwildcard); - alertwildcard = - "EXP(decay(1.0)(localhost*.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(15)"; - _setupTool.getClusterManagementTool().addAlert(clusterName, alertwildcard); - new HealthStatsAggregator(cmResult._manager).aggregate(); - Thread.sleep(1000); - - record = accessor.getProperty(keyBuilder.alertStatus()).getRecord(); - recMap = record.getMapFields(); - for (int i = 0; i < 3; i++) { - String alertString = "(localhost_" + (12944 + i) + ".RestQueryStats@DBName=TestDB0.latency)"; - Map alertStatusMap = recMap.get(alertwildcard + " : " + alertString); - String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME); - boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME)); - Assert.assertEquals(Double.parseDouble(val), (double) i * 5 + 0.1); - Assert.assertFalse(fired); - } - for (int i = 3; i < 5; i++) { - String alertString = "(localhost_" + (12944 + i) + ".RestQueryStats@DBName=TestDB0.latency)"; - Map alertStatusMap = recMap.get(alertwildcard + " : " + alertString); - String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME); - boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME)); - Assert.assertEquals(Double.parseDouble(val), (double) i * 5 + 0.1); - Assert.assertTrue(fired); - } - alertHistory = accessor.getProperty(keyBuilder.alertHistory()).getRecord(); - - deltakey = (String) (alertHistory.getMapFields().keySet().toArray()[1]); - delta = alertHistory.getMapField(deltakey); - Assert.assertTrue(delta.size() == 2); - for (int i = 3; i < 5; i++) { - String alertString = - "(localhost_" + (12944 + i) + ".RestQueryStats@DBName#TestDB0.latency)GREATER(15)"; - Assert.assertTrue(delta.get(alertString).equals("ON")); - } - - System.out.println("END testSimpleWildcardAlert at " + new Date(System.currentTimeMillis())); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java deleted file mode 100644 index 2304b417de..0000000000 --- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java +++ /dev/null @@ -1,182 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.Map; -import java.util.Set; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.NotificationContext; -import org.apache.helix.TestHelper; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.alerts.AlertValueAndStatus; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.healthcheck.HealthStatsAggregationTask; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.integration.ZkIntegrationTestBase; -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.manager.zk.ZkClient; -import org.apache.helix.mock.participant.MockEspressoHealthReportProvider; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.mock.participant.MockTransition; -import org.apache.helix.model.Message; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.tools.ClusterStateVerifier; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class TestStalenessAlert extends ZkIntegrationTestBase { - ZkClient _zkClient; - protected ClusterSetup _setupTool = null; - protected final String _alertStr = "EXP(decay(1)(localhost_*.reportingage))CMP(GREATER)CON(600)"; - protected final String _alertStatusStr = _alertStr + " : (localhost_12918.reportingage)"; - protected final String _dbName = "TestDB0"; - - @BeforeClass() - public void beforeClass() throws Exception { - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); - - _setupTool = new ClusterSetup(ZK_ADDR); - } - - @AfterClass - public void afterClass() { - _zkClient.close(); - } - - public class StalenessAlertTransition extends MockTransition { - @Override - public void doTransition(Message message, NotificationContext context) { - HelixManager manager = context.getManager(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - String fromState = message.getFromState(); - String toState = message.getToState(); - String instance = message.getTgtName(); - String partition = message.getPartitionName(); - - if (fromState.equalsIgnoreCase("SLAVE") && toState.equalsIgnoreCase("MASTER")) { - - // add a stat and report to ZK - // perhaps should keep reporter per instance... - ParticipantHealthReportCollectorImpl reporter = - new ParticipantHealthReportCollectorImpl(manager, instance); - MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider(); - reporter.addHealthReportProvider(provider); - String statName = "latency"; - provider.setStat(_dbName, statName, "15"); - reporter.transmitHealthReports(); - - /* - * for (int i = 0; i < 5; i++) - * { - * accessor.setProperty(PropertyType.HEALTHREPORT, - * new ZNRecord("mockAlerts" + i), - * instance, - * "mockAlerts"); - * try - * { - * Thread.sleep(1000); - * } - * catch (InterruptedException e) - * { - * // TODO Auto-generated catch block - * e.printStackTrace(); - * } - * } - */ - } - } - - } - - @Test() - public void testStalenessAlert() throws Exception { - String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; - - System.out.println("START TestStalenessAlert at " + new Date(System.currentTimeMillis())); - - TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start port - "localhost", // participant name prefix - "TestDB", // resource name prefix - 1, // resources - 10, // partitions per resource - 5, // number of nodes //change back to 5!!! - 3, // replicas //change back to 3!!! - "MasterSlave", true); // do rebalance - // enableHealthCheck(clusterName); - - _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr); - - StartCMResult cmResult = - TestHelper.startController(clusterName, "controller_0", ZK_ADDR, - HelixControllerMain.STANDALONE); - // start participants - for (int i = 0; i < 5; i++) // !!!change back to 5 - { - String instanceName = "localhost_" + (12918 + i); - - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new StalenessAlertTransition()); - participants[i].syncStart(); - // new Thread(participants[i]).start(); - } - - boolean result = - ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( - ZK_ADDR, clusterName)); - Assert.assertTrue(result); - - // HealthAggregationTask is supposed to run by a timer every 30s - // To make sure HealthAggregationTask is run, we invoke it explicitly for this test - new HealthStatsAggregator(cmResult._manager).aggregate(); - // sleep for a few seconds to give stats stage time to trigger - Thread.sleep(3000); - - // other verifications go here - ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); - Builder keyBuilder = accessor.keyBuilder(); - // for (int i = 0; i < 1; i++) //change 1 back to 5 - // { - // String instance = "localhost_" + (12918 + i); - // String instance = "localhost_12918"; - ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord(); - Map> recMap = record.getMapFields(); - Set keySet = recMap.keySet(); - Map alertStatusMap = recMap.get(_alertStatusStr); - String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME); - boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME)); - // Assert.assertEquals(Double.parseDouble(val), Double.parseDouble("75.0")); - // Assert.assertFalse(fired); - // } - - System.out.println("END TestStalenessAlert at " + new Date(System.currentTimeMillis())); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java deleted file mode 100644 index a0456a7e7f..0000000000 --- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java +++ /dev/null @@ -1,298 +0,0 @@ -package org.apache.helix.healthcheck; - -/* - * 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.Date; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -import javax.management.AttributeNotFoundException; -import javax.management.InstanceNotFoundException; -import javax.management.IntrospectionException; -import javax.management.MBeanAttributeInfo; -import javax.management.MBeanException; -import javax.management.MBeanInfo; -import javax.management.MBeanServerConnection; -import javax.management.MBeanServerNotification; -import javax.management.MalformedObjectNameException; -import javax.management.ObjectName; -import javax.management.ReflectionException; - -import org.apache.helix.HelixDataAccessor; -import org.apache.helix.HelixManager; -import org.apache.helix.NotificationContext; -import org.apache.helix.TestHelper; -import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.alerts.AlertValueAndStatus; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.healthcheck.HealthStatsAggregationTask; -import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl; -import org.apache.helix.integration.ZkIntegrationTestBase; -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.manager.zk.ZkClient; -import org.apache.helix.mock.participant.MockEspressoHealthReportProvider; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.mock.participant.MockTransition; -import org.apache.helix.model.Message; -import org.apache.helix.monitoring.mbeans.ClusterAlertMBeanCollection; -import org.apache.helix.monitoring.mbeans.ClusterMBeanObserver; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.tools.ClusterStateVerifier; -import org.apache.log4j.Logger; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class TestWildcardAlert extends ZkIntegrationTestBase { - public static class TestClusterMBeanObserver extends ClusterMBeanObserver { - public Map> _beanValueMap = - new ConcurrentHashMap>(); - - public TestClusterMBeanObserver(String domain) throws InstanceNotFoundException, IOException, - MalformedObjectNameException, NullPointerException { - super(domain); - } - - @Override - public void onMBeanRegistered(MBeanServerConnection server, - MBeanServerNotification mbsNotification) { - try { - MBeanInfo info = _server.getMBeanInfo(mbsNotification.getMBeanName()); - MBeanAttributeInfo[] infos = info.getAttributes(); - _beanValueMap.put(mbsNotification.getMBeanName().toString(), - new ConcurrentHashMap()); - for (MBeanAttributeInfo infoItem : infos) { - Object val = _server.getAttribute(mbsNotification.getMBeanName(), infoItem.getName()); - System.out.println(" " + infoItem.getName() + " : " - + _server.getAttribute(mbsNotification.getMBeanName(), infoItem.getName()) - + " type : " + infoItem.getType()); - _beanValueMap.get(mbsNotification.getMBeanName().toString()).put(infoItem.getName(), val); - } - } catch (Exception e) { - _logger.error("Error getting bean info, domain=" + _domain, e); - } - } - - @Override - public void onMBeanUnRegistered(MBeanServerConnection server, - MBeanServerNotification mbsNotification) { - _beanValueMap.remove(mbsNotification.getMBeanName().toString()); - } - - public void refresh() throws MalformedObjectNameException, NullPointerException, - InstanceNotFoundException, IntrospectionException, ReflectionException, IOException, - AttributeNotFoundException, MBeanException { - for (String beanName : _beanValueMap.keySet()) { - ObjectName objName = new ObjectName(beanName); - MBeanInfo info = _server.getMBeanInfo(objName); - MBeanAttributeInfo[] infos = info.getAttributes(); - _beanValueMap.put(objName.toString(), new HashMap()); - for (MBeanAttributeInfo infoItem : infos) { - Object val = _server.getAttribute(objName, infoItem.getName()); - System.out - .println(" " + infoItem.getName() + " : " - + _server.getAttribute(objName, infoItem.getName()) + " type : " - + infoItem.getType()); - _beanValueMap.get(objName.toString()).put(infoItem.getName(), val); - } - } - } - - } - - private static final Logger _logger = Logger.getLogger(TestWildcardAlert.class); - ZkClient _zkClient; - protected ClusterSetup _setupTool = null; - protected final String _alertStr = - "EXP(decay(1)(localhost_*.RestQueryStats@DBName=TestDB0.latency)|EXPAND|SUMEACH)CMP(GREATER)CON(10)"; - protected final String _alertStatusStr = _alertStr; // +" : (*)"; - protected final String _dbName = "TestDB0"; - - @BeforeClass() - public void beforeClass() throws Exception { - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); - - _setupTool = new ClusterSetup(ZK_ADDR); - } - - @AfterClass - public void afterClass() { - _zkClient.close(); - } - - public class WildcardAlertTransition extends MockTransition { - @Override - public void doTransition(Message message, NotificationContext context) { - HelixManager manager = context.getManager(); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); - String fromState = message.getFromState(); - String toState = message.getToState(); - String instance = message.getTgtName(); - String partition = message.getPartitionName(); - - if (fromState.equalsIgnoreCase("SLAVE") && toState.equalsIgnoreCase("MASTER")) { - // add a stat and report to ZK - // perhaps should keep reporter per instance... - ParticipantHealthReportCollectorImpl reporter = - new ParticipantHealthReportCollectorImpl(manager, instance); - MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider(); - reporter.addHealthReportProvider(provider); - String statName = "latency"; - // using constant as timestamp so that when each partition does this transition, - // they do not advance timestamp, and no stats double-counted - String timestamp = "12345"; - provider.setStat(_dbName, statName, "15", timestamp); - - // sleep for random time and see about errors. - /* - * Random r = new Random(); - * int x = r.nextInt(30000); - * try { - * Thread.sleep(x); - * } catch (InterruptedException e) { - * // TODO Auto-generated catch block - * e.printStackTrace(); - * } - */ - - reporter.transmitHealthReports(); - - /* - * for (int i = 0; i < 5; i++) - * { - * accessor.setProperty(PropertyType.HEALTHREPORT, - * new ZNRecord("mockAlerts" + i), - * instance, - * "mockAlerts"); - * try - * { - * Thread.sleep(1000); - * } - * catch (InterruptedException e) - * { - * // TODO Auto-generated catch block - * e.printStackTrace(); - * } - * } - */ - } - } - - } - - @Test() - public void testWildcardAlert() throws Exception { - String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; - - System.out.println("START TestWildcardAlert at " + new Date(System.currentTimeMillis())); - - TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start - // port - "localhost", // participant name prefix - "TestDB", // resource name prefix - 1, // resources - 10, // partitions per resource - 5, // number of nodes //change back to 5!!! - 3, // replicas //change back to 3!!! - "MasterSlave", true); // do rebalance - - // enableHealthCheck(clusterName); - - _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr); - // _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr2); - - StartCMResult cmResult = - TestHelper.startController(clusterName, "controller_0", ZK_ADDR, - HelixControllerMain.STANDALONE); - // start participants - for (int i = 0; i < 5; i++) // !!!change back to 5 - { - String instanceName = "localhost_" + (12918 + i); - - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new WildcardAlertTransition()); - participants[i].syncStart(); - // new Thread(participants[i]).start(); - } - - TestClusterMBeanObserver jmxMBeanObserver = - new TestClusterMBeanObserver(ClusterAlertMBeanCollection.DOMAIN_ALERT); - - boolean result = - ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( - ZK_ADDR, clusterName)); - Assert.assertTrue(result); - Thread.sleep(3000); - // HealthAggregationTask is supposed to run by a timer every 30s - // To make sure HealthAggregationTask is run, we invoke it explicitly for this test - new HealthStatsAggregator(cmResult._manager).aggregate(); - - // sleep for a few seconds to give stats stage time to trigger and for bean to trigger - Thread.sleep(3000); - - ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient)); - Builder keyBuilder = accessor.keyBuilder(); - - // for (int i = 0; i < 1; i++) //change 1 back to 5 - // { - // String instance = "localhost_" + (12918 + i); - // String instance = "localhost_12918"; - ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord(); - Map> recMap = record.getMapFields(); - Set keySet = recMap.keySet(); - Map alertStatusMap = recMap.get(_alertStatusStr); - String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME); - boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME)); - Assert.assertEquals(Double.parseDouble(val), Double.parseDouble("75.0")); - Assert.assertTrue(fired); - - // Make sure that the jmxObserver has received all the jmx bean value that is corresponding - // to the alerts. - jmxMBeanObserver.refresh(); - Assert.assertTrue(jmxMBeanObserver._beanValueMap.size() >= 1); - - String beanName = - "HelixAlerts:alert=EXP(decay(1)(localhost_%.RestQueryStats@DBName#TestDB0.latency)|EXPAND|SUMEACH)CMP(GREATER)CON(10)--(%)"; - Assert.assertTrue(jmxMBeanObserver._beanValueMap.containsKey(beanName)); - - Map beanValueMap = jmxMBeanObserver._beanValueMap.get(beanName); - Assert.assertEquals(beanValueMap.size(), 4); - Assert.assertEquals((beanValueMap.get("AlertFired")), new Integer(1)); - Assert.assertEquals((beanValueMap.get("AlertValue")), new Double(75.0)); - Assert - .assertEquals( - (String) (beanValueMap.get("SensorName")), - "EXP(decay(1)(localhost_%.RestQueryStats@DBName#TestDB0.latency)|EXPAND|SUMEACH)CMP(GREATER)CON(10)--(%)"); - // } - - System.out.println("END TestWildcardAlert at " + new Date(System.currentTimeMillis())); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java index 8547666a7a..32fdcff001 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java @@ -20,16 +20,9 @@ */ import java.util.Date; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.helix.PropertyType; -import org.apache.helix.TestHelper; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.model.PauseSignal; + +import org.apache.helix.integration.manager.ClusterDistributedController; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.log4j.Logger; @@ -46,13 +39,15 @@ public class TestAddClusterV2 extends ZkIntegrationTestBase { protected static final int START_PORT = 12918; protected static final String STATE_MODEL = "MasterSlave"; protected ClusterSetup _setupTool = null; - protected Map _startCMResultMap = new HashMap(); protected final String CLASS_NAME = getShortClassName(); protected final String CONTROLLER_CLUSTER = CONTROLLER_CLUSTER_PREFIX + "_" + CLASS_NAME; protected static final String TEST_DB = "TestDB"; + MockParticipantManager[] _participants = new MockParticipantManager[NODE_NR]; + ClusterDistributedController[] _distControllers = new ClusterDistributedController[NODE_NR]; + @BeforeClass public void beforeClass() throws Exception { System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); @@ -90,29 +85,18 @@ public void beforeClass() throws Exception { "MasterSlave", 3, true); // start dummy participants for the first cluster - for (int i = 0; i < 5; i++) { + for (int i = 0; i < NODE_NR; i++) { String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - if (_startCMResultMap.get(instanceName) != null) { - LOG.error("fail to start participant:" + instanceName - + "(participant with the same name already running"); - } else { - StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, firstCluster, instanceName); - _startCMResultMap.put(instanceName, result); - } + _participants[i] = new MockParticipantManager(ZK_ADDR, firstCluster, instanceName); + _participants[i].syncStart(); } // start distributed cluster controllers - for (int i = 0; i < 5; i++) { + for (int i = 0; i < NODE_NR; i++) { String controllerName = CONTROLLER_PREFIX + "_" + i; - if (_startCMResultMap.get(controllerName) != null) { - LOG.error("fail to start controller:" + controllerName - + "(controller with the same name already running"); - } else { - StartCMResult result = - TestHelper.startController(CONTROLLER_CLUSTER, controllerName, ZK_ADDR, - HelixControllerMain.DISTRIBUTED); - _startCMResultMap.put(controllerName, result); - } + _distControllers[i] = + new ClusterDistributedController(ZK_ADDR, CONTROLLER_CLUSTER, controllerName); + _distControllers[i].syncStart(); } verifyClusters(); @@ -134,36 +118,22 @@ public void afterClass() throws Exception { * 3) disconnect leader/disconnect participant */ String leader = getCurrentLeader(_gZkClient, CONTROLLER_CLUSTER); - // pauseController(_startCMResultMap.get(leader)._manager.getDataAccessor()); - - StartCMResult result; - - Iterator> it = _startCMResultMap.entrySet().iterator(); - - while (it.hasNext()) { - String instanceName = it.next().getKey(); - if (!instanceName.equals(leader) && instanceName.startsWith(CONTROLLER_PREFIX)) { - result = _startCMResultMap.get(instanceName); - result._manager.disconnect(); - result._thread.interrupt(); - it.remove(); + int leaderIdx = -1; + for (int i = 0; i < NODE_NR; i++) { + if (!_distControllers[i].getInstanceName().equals(leader)) { + _distControllers[i].syncStop(); + verifyClusters(); + } else { + leaderIdx = i; } - verifyClusters(); } + Assert.assertNotSame(leaderIdx, -1); - result = _startCMResultMap.remove(leader); - result._manager.disconnect(); - result._thread.interrupt(); - - it = _startCMResultMap.entrySet().iterator(); - while (it.hasNext()) { - String instanceName = it.next().getKey(); - result = _startCMResultMap.get(instanceName); - result._manager.disconnect(); - result._thread.interrupt(); - it.remove(); - } + _distControllers[leaderIdx].syncStop(); + for (int i = 0; i < NODE_NR; i++) { + _participants[i].syncStop(); + } System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java index b135d9263c..79d8b89c7e 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java @@ -21,18 +21,15 @@ import java.util.Date; import java.util.List; -import java.util.Set; -import org.apache.helix.InstanceType; import org.apache.helix.PropertyPathConfig; import org.apache.helix.PropertyType; import org.apache.helix.TestHelper; -import org.apache.helix.ZkHelixTestManager; import org.apache.helix.ZkTestHelper; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.ClusterDistributedController; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.CallbackHandler; -import org.apache.helix.manager.zk.ZKHelixManager; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.log4j.Logger; import org.testng.Assert; @@ -52,16 +49,17 @@ public void testStandalone() throws Exception { TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, "localhost", "TestDB", 1, 20, nodeNr - 1, 3, "MasterSlave", true); - MockParticipant[] participants = new MockParticipant[nodeNr]; + MockParticipantManager[] participants = new MockParticipantManager[nodeNr]; for (int i = 0; i < nodeNr - 1; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); - new Thread(participants[i]).start(); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].syncStart(); } - ZkHelixTestManager controller = - new ZkHelixTestManager(clusterName, "controller_0", InstanceType.CONTROLLER, ZK_ADDR); - controller.connect(); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + boolean result; result = ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( @@ -75,7 +73,7 @@ public void testStandalone() throws Exception { _gSetupTool.addInstanceToCluster(clusterName, "localhost_12922"); _gSetupTool.rebalanceStorageCluster(clusterName, "TestDB0", 3); - participants[nodeNr - 1] = new MockParticipant(clusterName, "localhost_12922", ZK_ADDR); + participants[nodeNr - 1] = new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12922"); new Thread(participants[nodeNr - 1]).start(); result = ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( @@ -86,11 +84,10 @@ public void testStandalone() throws Exception { Assert.assertTrue(result); // clean up - // controller.disconnect(); - // for (int i = 0; i < nodeNr; i++) - // { - // participants[i].syncStop(); - // } + controller.syncStop(); + for (int i = 0; i < nodeNr; i++) { + participants[i].syncStop(); + } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } @@ -101,11 +98,13 @@ public void testDistributed() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); // setup grand cluster - TestHelper.setupCluster("GRAND_" + clusterName, ZK_ADDR, 0, "controller", null, 0, 0, 1, 0, + final String grandClusterName = "GRAND_" + clusterName; + TestHelper.setupCluster(grandClusterName, ZK_ADDR, 0, "controller", null, 0, 0, 1, 0, null, true); - TestHelper.startController("GRAND_" + clusterName, "controller_0", ZK_ADDR, - HelixControllerMain.DISTRIBUTED); + ClusterDistributedController distController = + new ClusterDistributedController(ZK_ADDR, grandClusterName, "controller_0"); + distController.syncStart(); // setup cluster _gSetupTool.addCluster(clusterName, true); @@ -127,12 +126,11 @@ public void testDistributed() throws Exception { _gSetupTool.addResourceToCluster(clusterName, "TestDB0", 1, "LeaderStandby"); _gSetupTool.rebalanceStorageCluster(clusterName, "TestDB0", 1); - MockParticipant[] participants = new MockParticipant[nodeNr]; + MockParticipantManager[] participants = new MockParticipantManager[nodeNr]; for (int i = 0; i < nodeNr - 1; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); - // new Thread(participants[i]).start(); } result = @@ -150,10 +148,8 @@ public void testDistributed() throws Exception { _gSetupTool.addInstanceToCluster(clusterName, "localhost_12919"); _gSetupTool.rebalanceStorageCluster(clusterName, "TestDB0", 2); - participants[nodeNr - 1] = new MockParticipant(clusterName, "localhost_12919", ZK_ADDR); + participants[nodeNr - 1] = new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12919"); participants[nodeNr - 1].syncStart(); - // new Thread(participants[nodeNr - 1]).start(); - result = ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( ZK_ADDR, clusterName)); @@ -165,10 +161,10 @@ public void testDistributed() throws Exception { Assert.assertEquals(numberOfListeners, 2); // 1 of participant, and 1 of controller // clean up - // for (int i = 0; i < nodeNr; i++) - // { - // participants[i].syncStop(); - // } + distController.syncStop(); + for (int i = 0; i < nodeNr; i++) { + participants[i].syncStop(); + } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java index 33938ad027..123ce6e6bb 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java @@ -19,20 +19,18 @@ * under the License. */ -import java.util.ArrayList; import java.util.Date; import java.util.List; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.id.StateModelFactoryId; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.mock.participant.MockMSModelFactory; -import org.apache.helix.model.CurrentState; -import org.apache.helix.model.ExternalView; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.model.IdealState; import org.apache.helix.model.Message; import org.apache.helix.tools.ClusterSetup; @@ -52,7 +50,7 @@ public void testBasic() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -63,14 +61,15 @@ public void testBasic() throws Exception { 3, // replicas "MasterSlave", true); // do rebalance - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -87,7 +86,7 @@ public void testBasic() throws Exception { ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, baseAccessor); Builder keyBuilder = accessor.keyBuilder(); IdealState idealState = accessor.getProperty(keyBuilder.idealStates("TestDB1")); - idealState.setStateModelFactoryName("TestDB1_Factory"); + idealState.setStateModelFactoryId(StateModelFactoryId.from("TestDB1_Factory")); accessor.setProperty(keyBuilder.idealStates("TestDB1"), idealState); setupTool.rebalanceStorageCluster(clusterName, "TestDB1", 3); @@ -116,7 +115,7 @@ public void testBasic() throws Exception { // register "TestDB1_Factory" state model factory // Logger.getRootLogger().setLevel(Level.INFO); for (int i = 0; i < n; i++) { - participants[i].getManager().getStateMachineEngine() + participants[i].getStateMachineEngine() .registerStateModelFactory("MasterSlave", new MockMSModelFactory(), "TestDB1_Factory"); } @@ -127,7 +126,6 @@ public void testBasic() throws Exception { // clean up // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAutoIsWithEmptyMap.java b/helix-core/src/test/java/org/apache/helix/integration/TestAutoIsWithEmptyMap.java index 1ffb86fda3..fe23cb990a 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestAutoIsWithEmptyMap.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestAutoIsWithEmptyMap.java @@ -27,11 +27,11 @@ import org.apache.helix.PropertyType; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.model.IdealState; import org.apache.helix.tools.ClusterStateVerifier; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; import org.testng.Assert; import org.testng.annotations.Test; @@ -63,7 +63,7 @@ public void testAutoIsWithEmptyMap() throws Exception { instanceNames.add("localhost_" + port); } ZNRecord idealState = - DefaultIdealStateCalculator.calculateIdealState(instanceNames, 10, 2, "TestDB0", "LEADER", + DefaultTwoStateStrategy.calculateIdealState(instanceNames, 10, 2, "TestDB0", "LEADER", "STANDBY"); // System.out.println(idealState); // curIdealState.setSimpleField(IdealState.IdealStateProperty.IDEAL_STATE_MODE.toString(), @@ -74,15 +74,16 @@ public void testAutoIsWithEmptyMap() throws Exception { _gZkClient.writeData(idealPath, curIdealState); // start controller - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); // start participants - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -92,6 +93,7 @@ public void testAutoIsWithEmptyMap() throws Exception { Assert.assertTrue(result); // clean up + controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java index 1943364767..2aec114557 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java @@ -26,18 +26,23 @@ import java.util.Set; import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; +import org.apache.helix.api.State; import org.apache.helix.controller.HelixControllerMain; import org.apache.helix.controller.stages.ClusterDataCache; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; 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.manager.zk.ZkClient; import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.StateModelDefinition; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.ZkVerifier; @@ -51,18 +56,17 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC String db2 = TEST_DB + "2"; String _tag = "SSDSSD"; + @Override @BeforeClass public void beforeClass() throws Exception { // Logger.getRootLogger().setLevel(Level.INFO); System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); String namespace = "/" + CLUSTER_NAME; - if (_zkClient.exists(namespace)) { - _zkClient.deleteRecursive(namespace); + if (_gZkClient.exists(namespace)) { + _gZkClient.deleteRecursive(namespace); } - _setupTool = new ClusterSetup(ZK_ADDR); + _setupTool = new ClusterSetup(_gZkClient); // setup storage cluster _setupTool.addCluster(CLUSTER_NAME, true); @@ -89,24 +93,21 @@ public void beforeClass() throws Exception { // start dummy participants for (int i = 0; i < NODE_NR; i++) { String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - if (_startCMResultMap.get(instanceName) != null) { - LOG.error("fail to start particpant:" + instanceName - + "(participant with same name already exists)"); - } else { - StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName); - _startCMResultMap.put(instanceName, result); - } + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName); + participant.syncStart(); + _participants[i] = participant; + } // start controller String controllerName = CONTROLLER_PREFIX + "_0"; - StartCMResult startResult = - TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR, - HelixControllerMain.STANDALONE); - _startCMResultMap.put(controllerName, startResult); + _controller = + new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName); + _controller.syncStart(); boolean result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, TEST_DB)); Assert.assertTrue(result); @@ -122,7 +123,7 @@ public void testDropResourceAutoRebalance() throws Exception { _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", 1); boolean result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, "MyDB")); Assert.assertTrue(result); @@ -140,7 +141,7 @@ TestHelper. setOf("localhost_12918", "localhost_12919", "localhost_12920 _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB2", 3); result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, "MyDB2")); Assert.assertTrue(result); @@ -154,16 +155,11 @@ TestHelper. setOf("localhost_12918", "localhost_12919", "localhost_12920 @Test() public void testAutoRebalance() throws Exception { - // kill 1 node - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0); - _startCMResultMap.get(instanceName)._manager.disconnect(); - Thread.currentThread().sleep(1000); - _startCMResultMap.get(instanceName)._thread.interrupt(); + _participants[0].syncStop(); - // verifyBalanceExternalView(); boolean result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, TEST_DB)); Assert.assertTrue(result); @@ -172,22 +168,22 @@ public void testAutoRebalance() throws Exception { String storageNodeName = PARTICIPANT_PREFIX + "_" + (1000 + i); _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName); - StartCMResult resultx = - TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, storageNodeName.replace(':', '_')); - _startCMResultMap.put(storageNodeName, resultx); + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, storageNodeName.replace(':', '_')); + participant.syncStart(); } - Thread.sleep(1000); + Thread.sleep(5000); result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, TEST_DB)); Assert.assertTrue(result); result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, db2)); Assert.assertTrue(result); HelixDataAccessor accessor = - new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor(_zkClient)); + new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); ExternalView ev = accessor.getProperty(keyBuilder.externalView(db2)); Set instancesSet = new HashSet(); @@ -225,16 +221,14 @@ static boolean verifyBalanceExternalView(ZNRecord externalView, int partitionCou for (String instanceName : masterPartitionsCountMap.keySet()) { int instancePartitionCount = masterPartitionsCountMap.get(instanceName); totalCount += instancePartitionCount; - if (!(instancePartitionCount == perInstancePartition || instancePartitionCount == perInstancePartition + 1)) { + if (Math.abs(instancePartitionCount - perInstancePartition) > 1) { + // System.out.println("instanceName: " + instanceName + ", instancePartitionCnt: " + // + instancePartitionCount + ", perInstancePartition: " + perInstancePartition); return false; } - if (instancePartitionCount == perInstancePartition + 1) { - if (partitionCount % instances == 0) { - return false; - } - } } if (partitionCount != totalCount) { + // System.out.println("partitionCnt: " + partitionCount + ", totalCnt: " + totalCount); return false; } return true; @@ -242,12 +236,10 @@ static boolean verifyBalanceExternalView(ZNRecord externalView, int partitionCou } public static class ExternalViewBalancedVerifier implements ZkVerifier { - ZkClient _client; String _clusterName; String _resourceName; public ExternalViewBalancedVerifier(ZkClient client, String clusterName, String resourceName) { - _client = client; _clusterName = clusterName; _resourceName = resourceName; } @@ -255,38 +247,46 @@ public ExternalViewBalancedVerifier(ZkClient client, String clusterName, String @Override public boolean verify() { HelixDataAccessor accessor = - new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor(_client)); + new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); - int numberOfPartitions = - accessor.getProperty(keyBuilder.idealStates(_resourceName)).getRecord().getListFields() - .size(); - ClusterDataCache cache = new ClusterDataCache(); - cache.refresh(accessor); - String masterValue = - cache.getStateModelDef(cache.getIdealState(_resourceName).getStateModelDefRef()) - .getStatesPriorityList().get(0); - int replicas = Integer.parseInt(cache.getIdealState(_resourceName).getReplicas()); - String instanceGroupTag = cache.getIdealState(_resourceName).getInstanceGroupTag(); + IdealState idealState = accessor.getProperty(keyBuilder.idealStates(_resourceName)); + if (idealState == null) { + return false; + } + + int numberOfPartitions = idealState.getRecord().getListFields().size(); + String stateModelDefName = idealState.getStateModelDefId().stringify(); + StateModelDefinition stateModelDef = + accessor.getProperty(keyBuilder.stateModelDef(stateModelDefName)); + State masterValue = stateModelDef.getTypedStatesPriorityList().get(0); + int replicas = Integer.parseInt(idealState.getReplicas()); + + String instanceGroupTag = idealState.getInstanceGroupTag(); + int instances = 0; - for (String liveInstanceName : cache.getLiveInstances().keySet()) { - if (cache.getInstanceConfigMap().get(liveInstanceName).containsTag(instanceGroupTag)) { + Map liveInstanceMap = + accessor.getChildValuesMap(keyBuilder.liveInstances()); + Map instanceConfigMap = + accessor.getChildValuesMap(keyBuilder.instanceConfigs()); + for (String liveInstanceName : liveInstanceMap.keySet()) { + if (instanceConfigMap.get(liveInstanceName).containsTag(instanceGroupTag)) { instances++; } } if (instances == 0) { - instances = cache.getLiveInstances().size(); + instances = liveInstanceMap.size(); } ExternalView ev = accessor.getProperty(keyBuilder.externalView(_resourceName)); if (ev == null) { return false; } - return verifyBalanceExternalView(ev.getRecord(), numberOfPartitions, masterValue, replicas, - instances); + return verifyBalanceExternalView(ev.getRecord(), numberOfPartitions, masterValue.toString(), + replicas, instances); } @Override public ZkClient getZkClient() { - return _client; + return _gZkClient; } @Override diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java index 32cafcfa72..5f9f48c4f5 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java @@ -25,18 +25,21 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.api.State; import org.apache.helix.controller.stages.ClusterDataCache; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; 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.manager.zk.ZkClient; import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.StateModelDefinition; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.ZkVerifier; @@ -49,18 +52,17 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP private static final Logger LOG = Logger.getLogger(TestAutoRebalancePartitionLimit.class .getName()); + @Override @BeforeClass public void beforeClass() throws Exception { // Logger.getRootLogger().setLevel(Level.INFO); System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); String namespace = "/" + CLUSTER_NAME; - if (_zkClient.exists(namespace)) { - _zkClient.deleteRecursive(namespace); + if (_gZkClient.exists(namespace)) { + _gZkClient.deleteRecursive(namespace); } - _setupTool = new ClusterSetup(ZK_ADDR); + _setupTool = new ClusterSetup(_gZkClient); // setup storage cluster _setupTool.addCluster(CLUSTER_NAME, true); @@ -75,40 +77,33 @@ public void beforeClass() throws Exception { // start controller String controllerName = CONTROLLER_PREFIX + "_0"; - StartCMResult startResult = - TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR, - HelixControllerMain.STANDALONE); - _startCMResultMap.put(controllerName, startResult); + _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName); + _controller.syncStart(); - HelixManager manager = _startCMResultMap.get(controllerName)._manager; + HelixManager manager = _controller; // _startCMResultMap.get(controllerName)._manager; HelixDataAccessor accessor = manager.getHelixDataAccessor(); // start dummy participants for (int i = 0; i < NODE_NR; i++) { String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - if (_startCMResultMap.get(instanceName) != null) { - LOG.error("fail to start particpant:" + instanceName - + "(participant with same name already exists)"); + _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName); + _participants[i].syncStart(); + Thread.sleep(2000); + boolean result = + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, + CLUSTER_NAME, TEST_DB)); + Assert.assertTrue(result); + ExternalView ev = + manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB)); + System.out.println(ev.getPartitionSet().size()); + if (i < 3) { + Assert.assertEquals(ev.getPartitionSet().size(), 25 * (i + 1)); } else { - startResult = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName); - _startCMResultMap.put(instanceName, startResult); - Thread.sleep(2000); - boolean result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, - CLUSTER_NAME, TEST_DB)); - Assert.assertTrue(result); - ExternalView ev = - manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB)); - System.out.println(ev.getPartitionSet().size()); - if (i < 3) { - Assert.assertEquals(ev.getPartitionSet().size(), 25 * (i + 1)); - } else { - Assert.assertEquals(ev.getPartitionSet().size(), 100); - } + Assert.assertEquals(ev.getPartitionSet().size(), 100); } } boolean result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, TEST_DB)); Assert.assertTrue(result); @@ -116,49 +111,50 @@ public void beforeClass() throws Exception { @Test() public void testAutoRebalanceWithMaxPartitionPerNode() throws Exception { - String controllerName = CONTROLLER_PREFIX + "_0"; - HelixManager manager = _startCMResultMap.get(controllerName)._manager; + HelixManager manager = _controller; // kill 1 node - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0); - _startCMResultMap.get(instanceName)._manager.disconnect(); - Thread.currentThread().sleep(1000); - _startCMResultMap.get(instanceName)._thread.interrupt(); + _participants[0].syncStop(); // verifyBalanceExternalView(); boolean result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, TEST_DB)); Assert.assertTrue(result); - HelixDataAccessor accessor = manager.getHelixDataAccessor(); + final HelixDataAccessor accessor = manager.getHelixDataAccessor(); ExternalView ev = manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB)); Assert.assertEquals(ev.getPartitionSet().size(), 100); - instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 1); - _startCMResultMap.get(instanceName)._manager.disconnect(); - Thread.currentThread().sleep(1000); - _startCMResultMap.get(instanceName)._thread.interrupt(); + _participants[1].syncStop(); // verifyBalanceExternalView(); result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, TEST_DB)); Assert.assertTrue(result); - ev = manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB)); - Assert.assertEquals(ev.getPartitionSet().size(), 75); + result = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + ExternalView ev = accessor.getProperty(accessor.keyBuilder().externalView(TEST_DB)); + return ev.getPartitionSet().size() == 75; + } + }, 3 * 1000); // add 2 nodes for (int i = 0; i < 2; i++) { String storageNodeName = PARTICIPANT_PREFIX + "_" + (1000 + i); _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName); - StartCMResult resultx = - TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, storageNodeName.replace(':', '_')); - _startCMResultMap.put(storageNodeName, resultx); + String newInstanceName = storageNodeName.replace(':', '_'); + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, newInstanceName); + participant.syncStart(); } - Thread.sleep(1000); + + // Thread.sleep(1000); result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, TEST_DB)); Assert.assertTrue(result); } @@ -209,12 +205,10 @@ static boolean verifyBalanceExternalView(ZNRecord externalView, int partitionCou } public static class ExternalViewBalancedVerifier implements ZkVerifier { - ZkClient _client; String _clusterName; String _resourceName; public ExternalViewBalancedVerifier(ZkClient client, String clusterName, String resourceName) { - _client = client; _clusterName = clusterName; _resourceName = resourceName; } @@ -222,25 +216,25 @@ public ExternalViewBalancedVerifier(ZkClient client, String clusterName, String @Override public boolean verify() { HelixDataAccessor accessor = - new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor(_client)); + new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); - int numberOfPartitions = - accessor.getProperty(keyBuilder.idealStates(_resourceName)).getRecord().getListFields() - .size(); - ClusterDataCache cache = new ClusterDataCache(); - cache.refresh(accessor); - String masterValue = - cache.getStateModelDef(cache.getIdealState(_resourceName).getStateModelDefRef()) - .getStatesPriorityList().get(0); - int replicas = Integer.parseInt(cache.getIdealState(_resourceName).getReplicas()); + IdealState idealState = accessor.getProperty(keyBuilder.idealStates(_resourceName)); + int numberOfPartitions = idealState.getRecord().getListFields().size(); + String stateModelDefName = idealState.getStateModelDefId().stringify(); + StateModelDefinition stateModelDef = + accessor.getProperty(keyBuilder.stateModelDef(stateModelDefName)); + State masterValue = stateModelDef.getTypedStatesPriorityList().get(0); + Map liveInstanceMap = + accessor.getChildValuesMap(keyBuilder.liveInstances()); + int replicas = Integer.parseInt(idealState.getReplicas()); return verifyBalanceExternalView(accessor.getProperty(keyBuilder.externalView(_resourceName)) - .getRecord(), numberOfPartitions, masterValue, replicas, cache.getLiveInstances().size(), - cache.getIdealState(_resourceName).getMaxPartitionsPerInstance()); + .getRecord(), numberOfPartitions, masterValue.toString(), replicas, + liveInstanceMap.size(), idealState.getMaxPartitionsPerInstance()); } @Override public ZkClient getZkClient() { - return _client; + return _gZkClient; } @Override diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java index bf2de1e3f3..af079c1429 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java @@ -26,17 +26,14 @@ import java.util.Set; import org.I0Itec.zkclient.IZkChildListener; -import org.apache.helix.TestHelper; -import org.apache.helix.ZNRecord; import org.apache.helix.HelixProperty.HelixPropertyAttribute; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.ZkTestHelper; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.manager.zk.ZKHelixAdmin; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.ErrTransition; import org.apache.helix.model.IdealState; import org.apache.helix.model.LiveInstance; @@ -91,15 +88,16 @@ public void testBasic() throws Exception { TestZkChildListener listener = new TestZkChildListener(); _gZkClient.subscribeChildChanges(keyBuilder.messages("localhost_12918").getPath(), listener); - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -112,7 +110,6 @@ public void testBasic() throws Exception { // clean up // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < n; i++) { participants[i].syncStop(); @@ -141,15 +138,16 @@ public void testChangeBatchMessageMode() throws Exception { 2, // replicas "MasterSlave", true); // do rebalance - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -159,7 +157,6 @@ public void testChangeBatchMessageMode() throws Exception { Assert.assertTrue(result); // stop all participants - Thread.sleep(1000); for (int i = 0; i < n; i++) { participants[i].syncStop(); } @@ -180,7 +177,7 @@ public void testChangeBatchMessageMode() throws Exception { for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -193,7 +190,6 @@ public void testChangeBatchMessageMode() throws Exception { // clean up // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < n; i++) { participants[i].syncStop(); @@ -209,10 +205,9 @@ public void testSubMsgExecutionFail() throws Exception { String clusterName = className + "_" + methodName; final int n = 5; - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - // ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient); TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, "localhost", "TestDB", 1, // resource# 6, // partition# @@ -228,33 +223,37 @@ public void testSubMsgExecutionFail() throws Exception { idealState.setBatchMessageMode(true); accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState); - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + final String hostToFail = "localhost_12921"; + final String partitionToFail = "TestDB0_4"; + + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - if (i == 1) { + if (instanceName.equals(hostToFail)) { Map> errPartitions = new HashMap>(); errPartitions.put("SLAVE-MASTER", TestHelper.setOf("TestDB0_4")); - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, - new ErrTransition(errPartitions)); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errPartitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } Map> errStates = new HashMap>(); errStates.put("TestDB0", new HashMap()); - errStates.get("TestDB0").put("TestDB0_4", "localhost_12919"); + errStates.get("TestDB0").put(partitionToFail, hostToFail); boolean result = ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( ZK_ADDR, clusterName, errStates)); Assert.assertTrue(result); Map> errorStateMap = new HashMap>(); - errorStateMap.put("TestDB0_4", TestHelper.setOf("localhost_12919")); + errorStateMap.put(partitionToFail, TestHelper.setOf(hostToFail)); // verify "TestDB0_4", "localhost_12919" is in ERROR state TestHelper.verifyState(clusterName, ZK_ADDR, errorStateMap, "ERROR"); @@ -297,7 +296,8 @@ public void testParticipantIncompatibleWithBatchMsg() throws Exception { TestZkChildListener listener = new TestZkChildListener(); _gZkClient.subscribeChildChanges(keyBuilder.messages("localhost_12918").getPath(), listener); - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // pause controller @@ -307,11 +307,11 @@ public void testParticipantIncompatibleWithBatchMsg() throws Exception { }); // start participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -335,7 +335,6 @@ public void testParticipantIncompatibleWithBatchMsg() throws Exception { // clean up // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < n; i++) { participants[i].syncStop(); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageWrapper.java b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageWrapper.java index 2ae8bf3ef4..9bfdfb5768 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageWrapper.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageWrapper.java @@ -26,12 +26,12 @@ import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.messaging.handling.BatchMessageWrapper; -import org.apache.helix.mock.controller.ClusterController; import org.apache.helix.mock.participant.MockMSModelFactory; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.IdealState; import org.apache.helix.model.Message; import org.apache.helix.tools.ClusterStateVerifier; @@ -90,17 +90,19 @@ public void testBasic() throws Exception { idealState.setBatchMessageMode(true); accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState); - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestMockMSModelFactory[] ftys = new TestMockMSModelFactory[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); ftys[i] = new TestMockMSModelFactory(); - participants[i] = new MockParticipant(ftys[i], clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].getStateMachineEngine().registerStateModelFactory("MasterSlave", ftys[i]); participants[i].syncStart(); // wait for each participant to complete state transitions, so we have deterministic results @@ -133,6 +135,12 @@ public void testBasic() throws Exception { Assert.assertEquals(wrapper._startCount, 2, "Expect 2 batch.end: O->S and S->M for 2nd participant"); + // clean up + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java index 8e75537f52..51b0d3cbae 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java @@ -21,13 +21,13 @@ import java.util.Date; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.IdealState; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; @@ -45,7 +45,7 @@ public void testBucketizedResource() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; // ClusterSetup setupTool = new ClusterSetup(ZK_ADDR); TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port @@ -66,13 +66,15 @@ public void testBucketizedResource() throws Exception { idealState.setBucketSize(1); accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState); - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + // start participants for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -87,6 +89,7 @@ public void testBucketizedResource() throws Exception { Assert.assertTrue(result); // clean up + controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCarryOverBadCurState.java b/helix-core/src/test/java/org/apache/helix/integration/TestCarryOverBadCurState.java index 9d63a65b59..97b6ebb31e 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestCarryOverBadCurState.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestCarryOverBadCurState.java @@ -25,8 +25,8 @@ import org.apache.helix.PropertyType; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier; @@ -39,7 +39,7 @@ public void testCarryOverBadCurState() throws Exception { System.out.println("START testCarryOverBadCurState at " + new Date(System.currentTimeMillis())); String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -58,13 +58,15 @@ public void testCarryOverBadCurState() throws Exception { _gZkClient.createPersistent(path, true); _gZkClient.writeData(path, badCurState); - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + // start participants for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -78,6 +80,11 @@ public void testCarryOverBadCurState() throws Exception { clusterName)); Assert.assertTrue(result); + // clean up + controller.syncStop(); + for (int i = 0; i < 5; i++) { + participants[i].syncStop(); + } System.out.println("END testCarryOverBadCurState at " + new Date(System.currentTimeMillis())); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java b/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java index 781aa89a5e..93c58eb318 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java @@ -19,20 +19,24 @@ * under the License. */ -import org.apache.helix.*; +import java.util.Date; + +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkTestHelper; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.ExternalView; import org.apache.helix.model.LiveInstance; import org.apache.helix.tools.ClusterStateVerifier; import org.testng.Assert; import org.testng.annotations.Test; -import java.util.Date; - /** * Test clean external-view - if current-state is remove externally, controller should remove the * orphan external-view @@ -57,15 +61,16 @@ public void test() throws Exception { 2, // replicas "MasterSlave", true); // do rebalance - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -80,7 +85,7 @@ public void test() throws Exception { admin.enableCluster(clusterName, false); // wait all pending zk-events being processed, otherwise remove current-state will cause // controller send O->S message - ZkTestHelper.tryWaitZkEventsCleaned(controller.getManager().getZkClient()); + ZkTestHelper.tryWaitZkEventsCleaned(controller.getZkClient()); // System.out.println("paused controller"); // drop resource @@ -94,11 +99,11 @@ public void test() throws Exception { // System.out.println("remove current-state"); LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance("localhost_12918")); - accessor.removeProperty(keyBuilder.currentState("localhost_12918", liveInstance.getSessionId(), - "TestDB0")); + accessor.removeProperty(keyBuilder.currentState("localhost_12918", liveInstance.getTypedSessionId() + .stringify(), "TestDB0")); liveInstance = accessor.getProperty(keyBuilder.liveInstance("localhost_12919")); - accessor.removeProperty(keyBuilder.currentState("localhost_12919", liveInstance.getSessionId(), - "TestDB0")); + accessor.removeProperty(keyBuilder.currentState("localhost_12919", liveInstance.getTypedSessionId() + .stringify(), "TestDB0")); // re-enable controller shall remove orphan external-view // System.out.println("re-enabling controller"); @@ -117,8 +122,13 @@ public void test() throws Exception { Assert.assertNull(externalView, "external-view for TestDB0 should be removed, but was: " + externalView); - System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + // clean up + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestClusterStartsup.java b/helix-core/src/test/java/org/apache/helix/integration/TestClusterStartsup.java index c0ced727ef..e844a271cb 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestClusterStartsup.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestClusterStartsup.java @@ -26,7 +26,6 @@ import org.apache.helix.HelixManagerFactory; import org.apache.helix.InstanceType; import org.apache.helix.PropertyType; -import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.util.HelixUtil; import org.testng.Assert; @@ -40,8 +39,8 @@ void setupCluster() throws HelixException { System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); String namespace = "/" + CLUSTER_NAME; - if (_zkClient.exists(namespace)) { - _zkClient.deleteRecursive(namespace); + if (_gZkClient.exists(namespace)) { + _gZkClient.deleteRecursive(namespace); } _setupTool = new ClusterSetup(ZK_ADDR); @@ -58,13 +57,12 @@ void setupCluster() throws HelixException { @Override @BeforeClass() public void beforeClass() throws Exception { - _zkClient = new ZkClient(ZK_ADDR); + } @Override @AfterClass() public void afterClass() { - _zkClient.close(); } @Test() @@ -72,9 +70,8 @@ public void testParticipantStartUp() throws Exception { setupCluster(); String controllerMsgPath = HelixUtil.getControllerPropertyPath(CLUSTER_NAME, PropertyType.MESSAGES_CONTROLLER); - _zkClient.deleteRecursive(controllerMsgPath); + _gZkClient.deleteRecursive(controllerMsgPath); HelixManager manager = null; - ; try { manager = @@ -106,7 +103,7 @@ public void testParticipantStartUp() throws Exception { setupCluster(); String stateModelPath = HelixUtil.getStateModelDefinitionPath(CLUSTER_NAME); - _zkClient.deleteRecursive(stateModelPath); + _gZkClient.deleteRecursive(stateModelPath); try { manager = @@ -125,7 +122,7 @@ public void testParticipantStartUp() throws Exception { String instanceStatusUpdatePath = HelixUtil.getInstancePropertyPath(CLUSTER_NAME, "localhost_" + (START_PORT + 1), PropertyType.STATUSUPDATES); - _zkClient.deleteRecursive(instanceStatusUpdatePath); + _gZkClient.deleteRecursive(instanceStatusUpdatePath); try { manager = diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCustomIdealState.java b/helix-core/src/test/java/org/apache/helix/integration/TestCustomIdealState.java index 43cfa5a36d..5467932e48 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestCustomIdealState.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestCustomIdealState.java @@ -32,18 +32,6 @@ public class TestCustomIdealState extends ZkIntegrationTestBase { private static Logger LOG = Logger.getLogger(TestCustomIdealState.class); - ZkClient _zkClient; - - @BeforeClass - public void beforeClass() throws Exception { - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); - } - - @AfterClass - public void afterClass() { - _zkClient.close(); - } @Test public void testBasic() throws Exception { diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java index 7811c0d3b9..ce26e2e359 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java @@ -19,16 +19,23 @@ * under the License. */ +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.rebalancer.SemiAutoRebalancer; -import org.apache.helix.controller.stages.ClusterDataCache; -import org.apache.helix.controller.stages.CurrentStateOutput; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.controller.rebalancer.HelixRebalancer; +import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.stages.ResourceCurrentState; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.manager.zk.ZkClient; @@ -36,6 +43,10 @@ import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.IdealStateProperty; import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.ZkVerifier; import org.testng.Assert; @@ -45,27 +56,40 @@ public class TestCustomizedIdealStateRebalancer extends ZkStandAloneCMTestBaseWithPropertyServerCheck { String db2 = TEST_DB + "2"; static boolean testRebalancerCreated = false; + static boolean testRebalancerInvoked = false; - public static class TestRebalancer extends SemiAutoRebalancer { + public static class TestRebalancer implements HelixRebalancer { + /** + * Very basic mapping that evenly assigns one replica of each partition to live nodes, each of + * which is in the highest-priority state. + */ @Override - public void init(HelixManager manager) { - testRebalancerCreated = true; + public ResourceAssignment computeResourceMapping(RebalancerConfig config, Cluster cluster, + ResourceCurrentState currentState) { + PartitionedRebalancerContext context = + config.getRebalancerContext(PartitionedRebalancerContext.class); + StateModelDefinition stateModelDef = + cluster.getStateModelMap().get(context.getStateModelDefId()); + List liveParticipants = + new ArrayList(cluster.getLiveParticipantMap().keySet()); + ResourceAssignment resourceMapping = new ResourceAssignment(context.getResourceId()); + int i = 0; + for (PartitionId partitionId : context.getPartitionSet()) { + int nodeIndex = i % liveParticipants.size(); + Map replicaMap = new HashMap(); + replicaMap.put(liveParticipants.get(nodeIndex), stateModelDef.getTypedStatesPriorityList() + .get(0)); + resourceMapping.addReplicaMap(partitionId, replicaMap); + i++; + } + testRebalancerInvoked = true; + return resourceMapping; } @Override - public IdealState computeNewIdealState(String resourceName, IdealState currentIdealState, - CurrentStateOutput currentStateOutput, ClusterDataCache clusterData) { - for (String partition : currentIdealState.getPartitionSet()) { - String instance = currentIdealState.getPreferenceList(partition).get(0); - currentIdealState.getPreferenceList(partition).clear(); - currentIdealState.getPreferenceList(partition).add(instance); - - currentIdealState.getInstanceStateMap(partition).clear(); - currentIdealState.getInstanceStateMap(partition).put(instance, "MASTER"); - } - currentIdealState.setReplicas("1"); - return currentIdealState; + public void init(HelixManager helixManager) { + testRebalancerCreated = true; } } @@ -81,12 +105,12 @@ public void testCustomizedIdealStateRebalancer() throws InterruptedException { _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 3); boolean result = - ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient, + ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient, CLUSTER_NAME, db2)); Assert.assertTrue(result); Thread.sleep(1000); HelixDataAccessor accessor = - new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor(_zkClient)); + new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); ExternalView ev = accessor.getProperty(keyBuilder.externalView(db2)); Assert.assertEquals(ev.getPartitionSet().size(), 60); @@ -94,11 +118,12 @@ public void testCustomizedIdealStateRebalancer() throws InterruptedException { Assert.assertEquals(ev.getStateMap(partition).size(), 1); } IdealState is = accessor.getProperty(keyBuilder.idealStates(db2)); - for (String partition : is.getPartitionSet()) { - Assert.assertEquals(is.getPreferenceList(partition).size(), 3); - Assert.assertEquals(is.getInstanceStateMap(partition).size(), 3); + for (PartitionId partition : is.getPartitionIdSet()) { + Assert.assertEquals(is.getPreferenceList(partition).size(), 0); + Assert.assertEquals(is.getParticipantStateMap(partition).size(), 0); } Assert.assertTrue(testRebalancerCreated); + Assert.assertTrue(testRebalancerInvoked); } public static class ExternalViewBalancedVerifier implements ZkVerifier { @@ -116,30 +141,32 @@ public ExternalViewBalancedVerifier(ZkClient client, String clusterName, String public boolean verify() { try { HelixDataAccessor accessor = - new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor(_client)); + new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor(_client)); Builder keyBuilder = accessor.keyBuilder(); - int numberOfPartitions = - accessor.getProperty(keyBuilder.idealStates(_resourceName)).getRecord().getListFields() - .size(); - ClusterDataCache cache = new ClusterDataCache(); - cache.refresh(accessor); - String masterValue = - cache.getStateModelDef(cache.getIdealState(_resourceName).getStateModelDefRef()) - .getStatesPriorityList().get(0); - int replicas = Integer.parseInt(cache.getIdealState(_resourceName).getReplicas()); - String instanceGroupTag = cache.getIdealState(_resourceName).getInstanceGroupTag(); + IdealState idealState = accessor.getProperty(keyBuilder.idealStates(_resourceName)); + int numberOfPartitions = idealState.getRecord().getListFields().size(); + String stateModelDefName = idealState.getStateModelDefId().stringify(); + StateModelDefinition stateModelDef = + accessor.getProperty(keyBuilder.stateModelDef(stateModelDefName)); + State masterValue = stateModelDef.getTypedStatesPriorityList().get(0); + int replicas = Integer.parseInt(idealState.getReplicas()); + String instanceGroupTag = idealState.getInstanceGroupTag(); int instances = 0; - for (String liveInstanceName : cache.getLiveInstances().keySet()) { - if (cache.getInstanceConfigMap().get(liveInstanceName).containsTag(instanceGroupTag)) { + Map liveInstanceMap = + accessor.getChildValuesMap(keyBuilder.liveInstances()); + Map instanceCfgMap = + accessor.getChildValuesMap(keyBuilder.instanceConfigs()); + for (String liveInstanceName : liveInstanceMap.keySet()) { + if (instanceCfgMap.get(liveInstanceName).containsTag(instanceGroupTag)) { instances++; } } if (instances == 0) { - instances = cache.getLiveInstances().size(); + instances = liveInstanceMap.size(); } - return verifyBalanceExternalView( - accessor.getProperty(keyBuilder.externalView(_resourceName)).getRecord(), - numberOfPartitions, masterValue, replicas, instances); + ExternalView externalView = accessor.getProperty(keyBuilder.externalView(_resourceName)); + return verifyBalanceExternalView(externalView.getRecord(), numberOfPartitions, + masterValue.toString(), replicas, instances); } catch (Exception e) { return false; } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java index 3341e6b225..cc7fd5f6af 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java @@ -23,14 +23,14 @@ import java.util.HashMap; import java.util.Map; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.ZkTestHelper; -import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.RebalanceMode; import org.apache.helix.tools.ClusterSetup; @@ -52,7 +52,7 @@ public void testDisableNodeCustomIS() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -72,14 +72,15 @@ public void testDisableNodeCustomIS() throws Exception { accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState); // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -120,7 +121,6 @@ public void testDisableNodeCustomIS() throws Exception { // clean up // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); @@ -140,7 +140,7 @@ public void testDisableNodeAutoIS() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -152,14 +152,15 @@ public void testDisableNodeAutoIS() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -200,7 +201,6 @@ public void testDisableNodeAutoIS() throws Exception { // clean up // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); @@ -219,7 +219,7 @@ public void testDisablePartitionCustomIS() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -239,14 +239,15 @@ public void testDisablePartitionCustomIS() throws Exception { accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState); // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -292,7 +293,6 @@ public void testDisablePartitionCustomIS() throws Exception { // clean up // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); @@ -311,7 +311,7 @@ public void testDisablePartitionAutoIS() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -323,14 +323,15 @@ public void testDisablePartitionAutoIS() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -376,7 +377,6 @@ public void testDisablePartitionAutoIS() throws Exception { // clean up // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisableNode.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisableNode.java index 93c765ced8..be523d0fe9 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestDisableNode.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisableNode.java @@ -38,7 +38,7 @@ public void testDisableNode() throws Exception { ZK_ADDR, CLUSTER_NAME)); Assert.assertTrue(result); - ZKHelixAdmin tool = new ZKHelixAdmin(_zkClient); + ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient); tool.enableInstance(CLUSTER_NAME, PARTICIPANT_PREFIX + "_12918", true); result = diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisablePartition.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisablePartition.java index 3067a0bb18..ba7e8e4997 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestDisablePartition.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisablePartition.java @@ -56,7 +56,7 @@ public void testDisablePartition() throws Exception { TestHelper.verifyState(CLUSTER_NAME, ZK_ADDR, map, "OFFLINE"); - ZKHelixAdmin tool = new ZKHelixAdmin(_zkClient); + ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient); tool.enablePartition(true, CLUSTER_NAME, "localhost_12919", "TestDB", Arrays.asList("TestDB_9")); result = diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java index 29627a25e6..e29e73f90b 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java @@ -21,15 +21,14 @@ import java.util.Date; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.integration.manager.ClusterDistributedController; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Leader; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; @@ -77,11 +76,10 @@ public void testDistributedCMMain() throws Exception { "LeaderStandby", true); // do rebalance // start distributed cluster controllers - ClusterController[] controllers = new ClusterController[n + n]; + ClusterDistributedController[] controllers = new ClusterDistributedController[n + n]; for (int i = 0; i < n; i++) { controllers[i] = - new ClusterController(controllerClusterName, "controller_" + i, ZK_ADDR, - HelixControllerMain.DISTRIBUTED.toString()); + new ClusterDistributedController(ZK_ADDR, controllerClusterName, "controller_" + i); controllers[i].syncStart(); } @@ -92,11 +90,11 @@ public void testDistributedCMMain() throws Exception { Assert.assertTrue(result, "Controller cluster NOT in ideal state"); // start first cluster - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; final String firstClusterName = clusterNamePrefix + "0_0"; for (int i = 0; i < n; i++) { String instanceName = "localhost0_" + (12918 + i); - participants[i] = new MockParticipant(firstClusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, firstClusterName, instanceName); participants[i].syncStart(); } @@ -114,8 +112,7 @@ public void testDistributedCMMain() throws Exception { setupTool.rebalanceStorageCluster(controllerClusterName, clusterNamePrefix + "0", 6); for (int i = n; i < 2 * n; i++) { controllers[i] = - new ClusterController(controllerClusterName, "controller_" + i, ZK_ADDR, - HelixControllerMain.DISTRIBUTED.toString()); + new ClusterDistributedController(ZK_ADDR, controllerClusterName, "controller_" + i); controllers[i].syncStart(); } @@ -137,7 +134,7 @@ public void testDistributedCMMain() throws Exception { ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(controllerClusterName, baseAccessor); Builder keyBuilder = accessor.keyBuilder(); for (int i = 0; i < n; i++) { - LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); String leaderName = leader.getId(); int j = Integer.parseInt(leaderName.substring(leaderName.lastIndexOf('_') + 1)); controllers[j].syncStop(); @@ -157,7 +154,6 @@ public void testDistributedCMMain() throws Exception { // clean up // wait for all zk callbacks done System.out.println("Cleaning up..."); - Thread.sleep(2000); for (int i = 0; i < 5; i++) { result = ClusterStateVerifier @@ -166,7 +162,6 @@ public void testDistributedCMMain() throws Exception { controllers[i].syncStop(); } - // Thread.sleep(2000); for (int i = 0; i < 5; i++) { participants[i].syncStop(); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java index ed609713da..c073c440eb 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java @@ -21,15 +21,14 @@ import java.util.Date; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.integration.manager.ClusterDistributedController; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Leader; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; import org.testng.Assert; @@ -76,11 +75,10 @@ public void testDistributedClusterController() throws Exception { "LeaderStandby", true); // do rebalance // start distributed cluster controllers - ClusterController[] controllers = new ClusterController[n]; + ClusterDistributedController[] controllers = new ClusterDistributedController[n]; for (int i = 0; i < n; i++) { controllers[i] = - new ClusterController(controllerClusterName, "controller_" + i, ZK_ADDR, - HelixControllerMain.DISTRIBUTED.toString()); + new ClusterDistributedController(ZK_ADDR, controllerClusterName, "controller_" + i); controllers[i].syncStart(); } @@ -91,11 +89,11 @@ public void testDistributedClusterController() throws Exception { Assert.assertTrue(result, "Controller cluster NOT in ideal state"); // start first cluster - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; final String firstClusterName = clusterNamePrefix + "0_0"; for (int i = 0; i < n; i++) { String instanceName = "localhost0_" + (12918 + i); - participants[i] = new MockParticipant(firstClusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, firstClusterName, instanceName); participants[i].syncStart(); } @@ -108,17 +106,17 @@ public void testDistributedClusterController() throws Exception { ZkBaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(controllerClusterName, baseAccessor); Builder keyBuilder = accessor.keyBuilder(); - LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); String leaderName = leader.getId(); int j = Integer.parseInt(leaderName.substring(leaderName.lastIndexOf('_') + 1)); controllers[j].syncStop(); // setup the second cluster - MockParticipant[] participants2 = new MockParticipant[n]; + MockParticipantManager[] participants2 = new MockParticipantManager[n]; final String secondClusterName = clusterNamePrefix + "0_1"; for (int i = 0; i < n; i++) { String instanceName = "localhost1_" + (12918 + i); - participants2[i] = new MockParticipant(secondClusterName, instanceName, ZK_ADDR, null); + participants2[i] = new MockParticipantManager(ZK_ADDR, secondClusterName, instanceName); participants2[i].syncStart(); } @@ -130,7 +128,6 @@ public void testDistributedClusterController() throws Exception { // clean up // wait for all zk callbacks done System.out.println("Cleaning up..."); - Thread.sleep(1000); for (int i = 0; i < 5; i++) { result = ClusterStateVerifier diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDriver.java b/helix-core/src/test/java/org/apache/helix/integration/TestDriver.java index 951607bc8e..2f2431b17f 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestDriver.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestDriver.java @@ -21,20 +21,18 @@ import java.util.ArrayList; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import org.apache.helix.HelixManager; import org.apache.helix.PropertyPathConfig; import org.apache.helix.PropertyType; -import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.model.IdealState.IdealStateProperty; @@ -43,14 +41,12 @@ import org.apache.helix.store.PropertyStoreException; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.apache.helix.tools.TestCommand; +import org.apache.helix.tools.TestCommand.CommandType; import org.apache.helix.tools.TestExecutor; +import org.apache.helix.tools.TestExecutor.ZnodePropertyType; import org.apache.helix.tools.TestTrigger; import org.apache.helix.tools.ZnodeOpArg; -import org.apache.helix.tools.TestCommand.CommandType; -import org.apache.helix.tools.TestCommand.NodeOpArg; -import org.apache.helix.tools.TestExecutor.ZnodePropertyType; import org.apache.log4j.Logger; import org.testng.Assert; @@ -79,10 +75,8 @@ public static class TestInfo { public final int _numNode; public final int _replica; - // public final Map _idealStateMap = new - // ConcurrentHashMap(); - public final Map _startCMResultMap = - new ConcurrentHashMap(); + public final Map _managers = + new ConcurrentHashMap(); public TestInfo(String clusterName, ZkClient zkClient, int numDb, int numPartitionsPerDb, int numNode, int replica) { @@ -118,10 +112,6 @@ public static void setupCluster(String uniqClusterName, String zkAddr, int numRe replica, true); } - // public static void setupCluster(String uniqTestName, ZkClient zkClient, int - // numDb, - // int numPartitionPerDb, int numNodes, int replica, boolean doRebalance) - // throws Exception public static void setupCluster(String uniqClusterName, String zkAddr, int numResources, int numPartitionsPerResource, int numInstances, int replica, boolean doRebalance) throws Exception { @@ -193,11 +183,15 @@ public static void startDummyParticipants(String uniqClusterName, int[] instance for (int id : instanceIds) { String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + id); - if (testInfo._startCMResultMap.containsKey(instanceName)) { + // if (testInfo._startCMResultMap.containsKey(instanceName)) { + if (testInfo._managers.containsKey(instanceName)) { LOG.warn("Dummy participant:" + instanceName + " has already started; skip starting it"); } else { - StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, clusterName, instanceName); - testInfo._startCMResultMap.put(instanceName, result); + // StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, clusterName, instanceName); + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participant.syncStart(); + testInfo._managers.put(instanceName, participant); // testInfo._instanceStarted.countDown(); } } @@ -220,13 +214,13 @@ public static void startController(String uniqClusterName, int[] nodeIds) throws for (int id : nodeIds) { String controllerName = CONTROLLER_PREFIX + "_" + id; - if (testInfo._startCMResultMap.containsKey(controllerName)) { + if (testInfo._managers.containsKey(controllerName)) { LOG.warn("Controller:" + controllerName + " has already started; skip starting it"); } else { - StartCMResult result = - TestHelper.startController(clusterName, controllerName, ZK_ADDR, - HelixControllerMain.STANDALONE); - testInfo._startCMResultMap.put(controllerName, result); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, controllerName); + controller.syncStart(); + testInfo._managers.put(controllerName, controller); } } } @@ -257,27 +251,22 @@ public static void stopCluster(String uniqClusterName) throws Exception { TestInfo testInfo = _testInfoMap.remove(uniqClusterName); // stop controller first - for (Iterator> it = - testInfo._startCMResultMap.entrySet().iterator(); it.hasNext();) { - Map.Entry entry = it.next(); - String instanceName = entry.getKey(); + for (String instanceName : testInfo._managers.keySet()) { if (instanceName.startsWith(CONTROLLER_PREFIX)) { - it.remove(); - HelixManager manager = entry.getValue()._manager; - manager.disconnect(); - Thread thread = entry.getValue()._thread; - thread.interrupt(); + ClusterControllerManager controller = + (ClusterControllerManager) testInfo._managers.get(instanceName); + controller.syncStop(); } } Thread.sleep(1000); - // stop the rest - for (Map.Entry entry : testInfo._startCMResultMap.entrySet()) { - HelixManager manager = entry.getValue()._manager; - manager.disconnect(); - Thread thread = entry.getValue()._thread; - thread.interrupt(); + for (String instanceName : testInfo._managers.keySet()) { + if (!instanceName.startsWith(CONTROLLER_PREFIX)) { + MockParticipantManager participant = + (MockParticipantManager) testInfo._managers.get(instanceName); + participant.syncStop(); + } } testInfo._zkClient.close(); @@ -292,23 +281,24 @@ public static void stopDummyParticipant(String uniqClusterName, long beginTime, } TestInfo testInfo = _testInfoMap.get(uniqClusterName); - // String clusterName = testInfo._clusterName; String failHost = PARTICIPANT_PREFIX + "_" + (START_PORT + instanceId); - StartCMResult result = testInfo._startCMResultMap.remove(failHost); + MockParticipantManager participant = + (MockParticipantManager) testInfo._managers.remove(failHost); // TODO need sync - if (result == null || result._manager == null || result._thread == null) { + if (participant == null) { String errMsg = "Dummy participant:" + failHost + " seems not running"; LOG.error(errMsg); } else { // System.err.println("try to stop participant: " + // result._manager.getInstanceName()); - NodeOpArg arg = new NodeOpArg(result._manager, result._thread); - TestCommand command = new TestCommand(CommandType.STOP, new TestTrigger(beginTime), arg); - List commandList = new ArrayList(); - commandList.add(command); - TestExecutor.executeTestAsync(commandList, ZK_ADDR); + // NodeOpArg arg = new NodeOpArg(result._manager, result._thread); + // TestCommand command = new TestCommand(CommandType.STOP, new TestTrigger(beginTime), arg); + // List commandList = new ArrayList(); + // commandList.add(command); + // TestExecutor.executeTestAsync(commandList, ZK_ADDR); + participant.syncStop(); } } @@ -331,8 +321,8 @@ public static void setIdealState(String uniqClusterName, long beginTime, int per for (int i = 0; i < testInfo._numDb; i++) { String dbName = TEST_DB_PREFIX + i; ZNRecord destIS = - DefaultIdealStateCalculator.calculateIdealState(instanceNames, - testInfo._numPartitionsPerDb, testInfo._replica - 1, dbName, "MASTER", "SLAVE"); + DefaultTwoStateStrategy.calculateIdealState(instanceNames, testInfo._numPartitionsPerDb, + testInfo._replica - 1, dbName, "MASTER", "SLAVE"); // destIS.setId(dbName); destIS.setSimpleField(IdealStateProperty.REBALANCE_MODE.toString(), RebalanceMode.CUSTOMIZED.toString()); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java b/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java index b1fcc6077f..a81e35bd86 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java @@ -25,33 +25,29 @@ import java.util.Map; import java.util.Set; -import org.I0Itec.zkclient.DataUpdater; -import org.I0Itec.zkclient.IZkChildListener; -import org.I0Itec.zkclient.IZkDataListener; -import org.apache.helix.BaseDataAccessor; import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.ErrTransition; import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.builder.CustomModeISBuilder; -import org.apache.helix.model.builder.IdealStateBuilder; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; -import org.apache.zookeeper.data.Stat; import org.testng.Assert; import org.testng.annotations.Test; public class TestDrop extends ZkIntegrationTestBase { @Test - public void testDropErrorPartitionAutoIS() throws Exception { + public void testDropResourceWithErrorPartitionSemiAuto() throws Exception { // Logger.getRootLogger().setLevel(Level.INFO); String className = TestHelper.getTestClassName(); String methodName = TestHelper.getTestMethodName(); @@ -60,7 +56,7 @@ public void testDropErrorPartitionAutoIS() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -72,7 +68,8 @@ public void testDropErrorPartitionAutoIS() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants @@ -84,11 +81,10 @@ public void testDropErrorPartitionAutoIS() throws Exception { String instanceName = "localhost_" + (12918 + i); if (i == 0) { - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new ErrTransition( - errTransitions)); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errTransitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -114,19 +110,16 @@ public void testDropErrorPartitionAutoIS() throws Exception { Assert.assertTrue(result); // clean up - // wait for all zk callbacks done - // Thread.sleep(1000); - // controller.syncStop(); - // for (int i = 0; i < 5; i++) - // { - // participants[i].syncStop(); - // } + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } @Test - public void testDropErrorPartitionFailedAutoIS() throws Exception { + public void testFailToDropResourceWithErrorPartitionSemiAuto() throws Exception { // Logger.getRootLogger().setLevel(Level.INFO); String className = TestHelper.getTestClassName(); String methodName = TestHelper.getTestMethodName(); @@ -135,7 +128,7 @@ public void testDropErrorPartitionFailedAutoIS() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -147,7 +140,8 @@ public void testDropErrorPartitionFailedAutoIS() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants @@ -159,11 +153,10 @@ public void testDropErrorPartitionFailedAutoIS() throws Exception { String instanceName = "localhost_" + (12918 + i); if (i == 0) { - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new ErrTransition( - errTransitions)); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errTransitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -198,19 +191,16 @@ public void testDropErrorPartitionFailedAutoIS() throws Exception { Assert.assertEquals(disabledPartitions.get(0), "TestDB0_4"); // clean up - // wait for all zk callbacks done - // Thread.sleep(1000); - // controller.syncStop(); - // for (int i = 0; i < 5; i++) - // { - // participants[i].syncStop(); - // } + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } @Test - public void testDropErrorPartitionCustomIS() throws Exception { + public void testDropResourceWithErrorPartitionCustom() throws Exception { // Logger.getRootLogger().setLevel(Level.INFO); String className = TestHelper.getTestClassName(); String methodName = TestHelper.getTestMethodName(); @@ -219,7 +209,7 @@ public void testDropErrorPartitionCustomIS() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -246,7 +236,8 @@ public void testDropErrorPartitionCustomIS() throws Exception { accessor.setProperty(keyBuiler.idealStates("TestDB0"), isBuilder.build()); // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants @@ -257,11 +248,10 @@ public void testDropErrorPartitionCustomIS() throws Exception { String instanceName = "localhost_" + (12918 + i); if (i == 0) { - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new ErrTransition( - errTransitions)); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errTransitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -285,6 +275,12 @@ public void testDropErrorPartitionCustomIS() throws Exception { clusterName)); Assert.assertTrue(result, "Should be empty exeternal-view"); + // clean up + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } @@ -298,7 +294,7 @@ public void testDropSchemataResource() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -310,14 +306,15 @@ public void testDropSchemataResource() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -357,13 +354,74 @@ public void testDropSchemataResource() throws Exception { "schemata externalView should be empty but was \"" + extView + "\""); // clean up - // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); - for (int i = 0; i < 5; i++) { + for (int i = 0; i < n; i++) { participants[i].syncStop(); } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } + + /** + * Drop a single partition in a resource of semi-auto mode + */ + @Test + public void testDropSinglePartitionSemiAuto() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + int n = 2; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 4, // partitions per resource + n, // number of nodes + 2, // replicas + "MasterSlave", true); // do rebalance + + ZkBaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); + ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, baseAccessor); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + + // start participants + MockParticipantManager[] participants = new MockParticipantManager[n]; + for (int i = 0; i < n; i++) { + String instanceName = "localhost_" + (12918 + i); + + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].syncStart(); + } + + boolean result = + ClusterStateVerifier + .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(result); + + // remove one partition from ideal-state should drop that partition + String partitionToDrop = "TestDB0_1"; + IdealState idealState = accessor.getProperty(keyBuilder.idealStates("TestDB0")); + idealState.getRecord().getListFields().remove(partitionToDrop); + idealState.getRecord().getMapFields().remove(partitionToDrop); + accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState); + + result = + ClusterStateVerifier + .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(result); + + ExternalView externalView = accessor.getProperty(keyBuilder.externalView("TestDB0")); + Assert.assertFalse(externalView.getPartitionSet().contains(partitionToDrop), + "TestDB0_0 should be dropped since it's not in ideal-state"); + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDropResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestDropResource.java index ba88370472..0d02d12e5b 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestDropResource.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestDropResource.java @@ -20,7 +20,7 @@ */ import org.apache.helix.TestHelper; -import org.apache.helix.TestHelper.StartCMResult; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.testng.Assert; @@ -59,9 +59,8 @@ public void testDropResourceWhileNodeDead() throws Exception { String hostToKill = "localhost_12920"; - _startCMResultMap.get(hostToKill)._manager.disconnect(); + _participants[2].syncStop(); Thread.sleep(1000); - _startCMResultMap.get(hostToKill)._thread.interrupt(); String command = "-zkSvr " + ZK_ADDR + " -dropResource " + CLUSTER_NAME + " " + "MyDB2"; ClusterSetup.processCommandLineArgs(command.split(" ")); @@ -70,8 +69,8 @@ public void testDropResourceWhileNodeDead() throws Exception { TestHelper. setOf("localhost_12918", "localhost_12919", /* "localhost_12920", */"localhost_12921", "localhost_12922"), ZK_ADDR); - StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, hostToKill); - _startCMResultMap.put(hostToKill, result); + _participants[2] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, hostToKill); + _participants[2].syncStart(); TestHelper.verifyWithTimeout("verifyEmptyCurStateAndExtView", 30 * 1000, CLUSTER_NAME, "MyDB2", TestHelper. setOf("localhost_12918", "localhost_12919", "localhost_12920", diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java b/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java index 48cabbd13f..029184421d 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java @@ -24,17 +24,22 @@ import org.apache.helix.HelixManager; import org.apache.helix.NotificationContext; import org.apache.helix.TestHelper; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.api.State; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.MockTransition; import org.apache.helix.model.Message; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; +import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.annotations.Test; public class TestEnablePartitionDuringDisable extends ZkIntegrationTestBase { + private static Logger LOG = Logger.getLogger(TestEnablePartitionDuringDisable.class); + static { // Logger.getRootLogger().setLevel(Level.INFO); } @@ -49,10 +54,10 @@ public void doTransition(Message message, NotificationContext context) { String clusterName = manager.getClusterName(); String instance = message.getTgtName(); - String partitionName = message.getPartitionName(); - String fromState = message.getFromState(); - String toState = message.getToState(); - if (instance.equals("localhost_12919") && partitionName.equals("TestDB0_0")) { + PartitionId partitionId = message.getPartitionId(); + State fromState = message.getTypedFromState(); + State toState = message.getTypedToState(); + if (instance.equals("localhost_12919") && partitionId.equals(PartitionId.from("TestDB0_0"))) { if (fromState.equals("SLAVE") && toState.equals("OFFLINE")) { slaveToOfflineCnt++; @@ -63,11 +68,11 @@ public void doTransition(Message message, NotificationContext context) { ClusterSetup.processCommandLineArgs(command.split("\\s+")); } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Exception in cluster setup", e); } - } else if (slaveToOfflineCnt > 0 && fromState.equals("OFFLINE") && toState.equals("SLAVE")) { + } else if (slaveToOfflineCnt > 0 && fromState.equals(State.from("OFFLINE")) + && toState.equals(State.from("SLAVE"))) { offlineToSlave++; } } @@ -93,19 +98,21 @@ public void testEnablePartitionDuringDisable() throws Exception { 3, // replicas "MasterSlave", true); // do rebalance - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants EnablePartitionTransition transition = new EnablePartitionTransition(); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); if (instanceName.equals("localhost_12919")) { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, transition); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(transition); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -138,8 +145,6 @@ public void testEnablePartitionDuringDisable() throws Exception { Assert.assertEquals(transition.offlineToSlave, 1, "should get 1 offlineToSlave transition"); // clean up - // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java b/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java index bebefd117d..71c43390aa 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java @@ -26,9 +26,9 @@ import java.util.Set; import org.apache.helix.TestHelper; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixAdmin; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.mock.participant.ErrTransition; import org.apache.helix.tools.ClusterStateVerifier; import org.testng.Assert; @@ -38,7 +38,7 @@ public class TestErrorPartition extends ZkIntegrationTestBase { @Test() public void testErrorPartition() throws Exception { String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; System.out.println("START testErrorPartition() at " + new Date(System.currentTimeMillis())); ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient); @@ -46,8 +46,10 @@ public void testErrorPartition() throws Exception { TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, "localhost", "TestDB", 1, 10, 5, 3, "MasterSlave", true); - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); @@ -57,14 +59,12 @@ public void testErrorPartition() throws Exception { put("SLAVE-MASTER", TestHelper.setOf("TestDB0_4")); } }; - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, - new ErrTransition(errPartitions)); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errPartitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); - // new Thread(participants[i]).start(); } Map> errStates = new HashMap>(); @@ -113,7 +113,7 @@ public void testErrorPartition() throws Exception { ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( ZK_ADDR, clusterName)); Assert.assertTrue(result); - participants[0] = new MockParticipant(clusterName, "localhost_12918", ZK_ADDR); + participants[0] = new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12918"); new Thread(participants[0]).start(); result = @@ -121,6 +121,12 @@ public void testErrorPartition() throws Exception { ZK_ADDR, clusterName)); Assert.assertTrue(result); + // clean up + controller.syncStop(); + for (int i = 0; i < 5; i++) { + participants[i].syncStop(); + } + System.out.println("END testErrorPartition() at " + new Date(System.currentTimeMillis())); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestExpandCluster.java b/helix-core/src/test/java/org/apache/helix/integration/TestExpandCluster.java index a9aa3b9334..8268679638 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestExpandCluster.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestExpandCluster.java @@ -21,7 +21,7 @@ import java.util.Map; -import org.apache.helix.TestEspressoStorageClusterIdealState; +import org.apache.helix.controller.strategy.TestEspressoStorageClusterIdealState; import org.apache.helix.model.IdealState; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.util.RebalanceUtil; diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java b/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java index 26da639f0b..3e31f177b9 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java @@ -23,12 +23,12 @@ import java.util.Date; import java.util.List; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier; @@ -42,7 +42,7 @@ public void testExternalViewUpdates() throws Exception { System.out.println("START testExternalViewUpdates at " + new Date(System.currentTimeMillis())); String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; int resourceNb = 10; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -53,13 +53,15 @@ public void testExternalViewUpdates() throws Exception { 1, // replicas "MasterSlave", true); // do rebalance - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + // start participants for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -90,8 +92,11 @@ public void testExternalViewUpdates() throws Exception { Assert.assertTrue(stat.getVersion() <= 2, "ExternalView should be updated at most 2 times"); } - // TODO: need stop controller and participants - + // clean up + controller.syncStop(); + for (int i = 0; i < 5; i++) { + participants[i].syncStop(); + } System.out.println("END testExternalViewUpdates at " + new Date(System.currentTimeMillis())); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestFailOverPerf1kp.java b/helix-core/src/test/java/org/apache/helix/integration/TestFailOverPerf1kp.java index 464d827137..962cecf134 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestFailOverPerf1kp.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestFailOverPerf1kp.java @@ -25,7 +25,6 @@ import org.apache.helix.ScriptTestHelper; import org.apache.helix.TestHelper; import org.testng.Assert; -import org.testng.annotations.Test; public class TestFailOverPerf1kp { // TODO: renable this test. disable it because the script is not running properly on apache diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java b/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java new file mode 100644 index 0000000000..e0c8b6f177 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java @@ -0,0 +1,438 @@ +package org.apache.helix.integration; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixAdmin; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; +import org.apache.helix.manager.zk.ZKHelixAdmin; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.manager.zk.ZkClient; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.tools.ClusterStateVerifier; +import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; +import org.apache.helix.tools.ClusterStateVerifier.ZkVerifier; +import org.apache.helix.util.ZKClientPool; +import org.apache.log4j.Logger; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +/** + * Test that node tagging behaves correctly in FULL_AUTO mode + */ +public class TestFullAutoNodeTagging extends ZkUnitTestBase { + private static final Logger LOG = Logger.getLogger(TestFullAutoNodeTagging.class); + + /** + * Ensure that no assignments happen when there are no tagged nodes, but the resource is tagged + */ + @Test + public void testResourceTaggedFirst() throws Exception { + final int NUM_PARTICIPANTS = 10; + final int NUM_PARTITIONS = 4; + final int NUM_REPLICAS = 2; + final String RESOURCE_NAME = "TestDB0"; + final String TAG = "ASSIGNABLE"; + + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + // Set up cluster + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + NUM_PARTITIONS, // partitions per resource + NUM_PARTICIPANTS, // number of nodes + NUM_REPLICAS, // replicas + "MasterSlave", RebalanceMode.FULL_AUTO, // use FULL_AUTO mode to test node tagging + true); // do rebalance + + // tag the resource + HelixAdmin helixAdmin = new ZKHelixAdmin(ZK_ADDR); + IdealState idealState = helixAdmin.getResourceIdealState(clusterName, RESOURCE_NAME); + idealState.setInstanceGroupTag(TAG); + helixAdmin.setResourceIdealState(clusterName, RESOURCE_NAME, idealState); + + // start controller + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller"); + controller.syncStart(); + + // start participants + MockParticipantManager[] participants = new MockParticipantManager[NUM_PARTICIPANTS]; + for (int i = 0; i < NUM_PARTICIPANTS; i++) { + final String instanceName = "localhost_" + (12918 + i); + + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].syncStart(); + } + + Thread.sleep(1000); + boolean result = + ClusterStateVerifier.verifyByZkCallback(new EmptyZkVerifier(clusterName, RESOURCE_NAME)); + Assert.assertTrue(result, "External view and current state must be empty"); + + // cleanup + for (int i = 0; i < NUM_PARTICIPANTS; i++) { + participants[i].syncStop(); + } + controller.syncStop(); + } + + /** + * Basic test for tagging behavior. 10 participants, of which 4 are tagged. Launch all 10, + * checking external view every time a tagged node is started. Then shut down all 10, checking + * external view every time a tagged node is killed. + */ + @Test + public void testSafeAssignment() throws Exception { + final int NUM_PARTICIPANTS = 10; + final int NUM_PARTITIONS = 4; + final int NUM_REPLICAS = 2; + final String RESOURCE_NAME = "TestDB0"; + final String TAG = "ASSIGNABLE"; + + final String[] TAGGED_NODES = { + "localhost_12920", "localhost_12922", "localhost_12924", "localhost_12925" + }; + Set taggedNodes = Sets.newHashSet(TAGGED_NODES); + + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + // Set up cluster + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + NUM_PARTITIONS, // partitions per resource + NUM_PARTICIPANTS, // number of nodes + NUM_REPLICAS, // replicas + "MasterSlave", RebalanceMode.FULL_AUTO, // use FULL_AUTO mode to test node tagging + true); // do rebalance + + // tag the resource and participants + HelixAdmin helixAdmin = new ZKHelixAdmin(ZK_ADDR); + for (String taggedNode : TAGGED_NODES) { + helixAdmin.addInstanceTag(clusterName, taggedNode, TAG); + } + IdealState idealState = helixAdmin.getResourceIdealState(clusterName, RESOURCE_NAME); + idealState.setInstanceGroupTag(TAG); + helixAdmin.setResourceIdealState(clusterName, RESOURCE_NAME, idealState); + + // start controller + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller"); + controller.syncStart(); + + // start participants + MockParticipantManager[] participants = new MockParticipantManager[NUM_PARTICIPANTS]; + for (int i = 0; i < NUM_PARTICIPANTS; i++) { + final String instanceName = "localhost_" + (12918 + i); + + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].syncStart(); + + // ensure that everything is valid if this is a tagged node that is starting + if (taggedNodes.contains(instanceName)) { + // make sure that the best possible matches the external view + Thread.sleep(500); + boolean result = + ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(result); + + // make sure that the tagged state of the nodes is still balanced + result = + ClusterStateVerifier.verifyByZkCallback(new TaggedZkVerifier(clusterName, + RESOURCE_NAME, TAGGED_NODES, false)); + Assert.assertTrue(result, "initial assignment with all tagged nodes live is invalid"); + } + } + + // cleanup + for (int i = 0; i < NUM_PARTICIPANTS; i++) { + String participantName = participants[i].getInstanceName(); + participants[i].syncStop(); + if (taggedNodes.contains(participantName)) { + // check that the external view is still correct even after removing tagged nodes + taggedNodes.remove(participantName); + Thread.sleep(500); + boolean result = + ClusterStateVerifier.verifyByZkCallback(new TaggedZkVerifier(clusterName, + RESOURCE_NAME, TAGGED_NODES, taggedNodes.isEmpty())); + Assert.assertTrue(result, "incorrect state after removing " + participantName + ", " + + taggedNodes + " remain"); + } + } + controller.syncStop(); + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } + + /** + * Checker for basic validity of the external view given node tagging requirements + */ + private static class TaggedZkVerifier implements ZkVerifier { + private final String _clusterName; + private final String _resourceName; + private final String[] _taggedNodes; + private final boolean _isEmptyAllowed; + private final ZkClient _zkClient; + + /** + * Create a verifier for a specific cluster and resource + * @param clusterName the cluster to verify + * @param resourceName the resource within the cluster to verify + * @param taggedNodes nodes tagged with the resource tag + * @param isEmptyAllowed true if empty assignments are legal + */ + public TaggedZkVerifier(String clusterName, String resourceName, String[] taggedNodes, + boolean isEmptyAllowed) { + _clusterName = clusterName; + _resourceName = resourceName; + _taggedNodes = taggedNodes; + _isEmptyAllowed = isEmptyAllowed; + _zkClient = ZKClientPool.getZkClient(ZK_ADDR); + } + + @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)); + + Set taggedNodeSet = ImmutableSet.copyOf(_taggedNodes); + + // set up counts of partitions, masters, and slaves per node + Map partitionCount = Maps.newHashMap(); + int partitionSum = 0; + Map masterCount = Maps.newHashMap(); + int masterSum = 0; + Map slaveCount = Maps.newHashMap(); + int slaveSum = 0; + + for (String partitionName : externalView.getPartitionSet()) { + Map stateMap = externalView.getStateMap(partitionName); + for (String participantName : stateMap.keySet()) { + String state = stateMap.get(participantName); + if (state.equalsIgnoreCase("MASTER") || state.equalsIgnoreCase("SLAVE")) { + partitionSum++; + incrementCount(partitionCount, participantName); + if (!taggedNodeSet.contains(participantName)) { + // not allowed to have a non-tagged node assigned + LOG.error("Participant " + participantName + " is not tag, but has an assigned node"); + return false; + } else if (state.equalsIgnoreCase("MASTER")) { + masterSum++; + incrementCount(masterCount, participantName); + } else if (state.equalsIgnoreCase("SLAVE")) { + slaveSum++; + incrementCount(slaveCount, participantName); + } + } + } + } + + // check balance in partitions per node + if (partitionCount.size() > 0) { + boolean partitionMapDividesEvenly = partitionSum % partitionCount.size() == 0; + boolean withinAverage = + withinAverage(partitionCount, _isEmptyAllowed, partitionMapDividesEvenly); + if (!withinAverage) { + LOG.error("partition counts deviate from average"); + return false; + } + } else { + if (!_isEmptyAllowed) { + LOG.error("partition assignments are empty"); + return false; + } + } + + // check balance in masters per node + if (masterCount.size() > 0) { + boolean masterMapDividesEvenly = masterSum % masterCount.size() == 0; + boolean withinAverage = withinAverage(masterCount, _isEmptyAllowed, masterMapDividesEvenly); + if (!withinAverage) { + LOG.error("master counts deviate from average"); + return false; + } + } else { + if (!_isEmptyAllowed) { + LOG.error("master assignments are empty"); + return false; + } + } + + // check balance in slaves per node + if (slaveCount.size() > 0) { + boolean slaveMapDividesEvenly = slaveSum % slaveCount.size() == 0; + boolean withinAverage = withinAverage(slaveCount, true, slaveMapDividesEvenly); + if (!withinAverage) { + LOG.error("slave counts deviate from average"); + return false; + } + } + return true; + } + + private void incrementCount(Map countMap, String key) { + if (!countMap.containsKey(key)) { + countMap.put(key, 0); + } + countMap.put(key, countMap.get(key) + 1); + } + + private boolean withinAverage(Map countMap, boolean isEmptyAllowed, + boolean dividesEvenly) { + if (countMap.size() == 0) { + if (!isEmptyAllowed) { + LOG.error("Map not allowed to be empty"); + return false; + } + return true; + } + int upperBound = 1; + if (!dividesEvenly) { + upperBound = 2; + } + int average = computeAverage(countMap); + for (String participantName : countMap.keySet()) { + int count = countMap.get(participantName); + if (count < average - 1 || count > average + upperBound) { + LOG.error("Count " + count + " for " + participantName + " too far from average of " + + average); + return false; + } + } + return true; + } + + private int computeAverage(Map countMap) { + if (countMap.size() == 0) { + return -1; + } + int total = 0; + for (int value : countMap.values()) { + total += value; + } + return total / countMap.size(); + } + + @Override + public ZkClient getZkClient() { + return _zkClient; + } + + @Override + public String getClusterName() { + return _clusterName; + } + } + + /** + * Ensures that external view and current state are empty + */ + private static class EmptyZkVerifier implements ZkVerifier { + private final String _clusterName; + private final String _resourceName; + private final ZkClient _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 = ZKClientPool.getZkClient(ZK_ADDR); + } + + @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 + 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; + } + } + } + return true; + } + + @Override + public ZkClient getZkClient() { + return _zkClient; + } + + @Override + public String getClusterName() { + return _clusterName; + } + } +} diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixConnection.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixConnection.java new file mode 100644 index 0000000000..29e0a44d13 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixConnection.java @@ -0,0 +1,173 @@ +package org.apache.helix.integration; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import java.util.Arrays; +import java.util.Date; +import java.util.Map; + +import org.apache.helix.HelixConnection; +import org.apache.helix.HelixController; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixParticipant; +import org.apache.helix.NotificationContext; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.State; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.rebalancer.context.RebalancerContext; +import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext; +import org.apache.helix.manager.zk.ZkHelixConnection; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.Message; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.participant.statemachine.HelixStateModelFactory; +import org.apache.helix.participant.statemachine.StateModel; +import org.apache.helix.participant.statemachine.StateModelInfo; +import org.apache.helix.participant.statemachine.Transition; +import org.apache.log4j.Logger; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestHelixConnection extends ZkUnitTestBase { + private static final Logger LOG = Logger.getLogger(TestHelixConnection.class.getName()); + + @StateModelInfo(initialState = "OFFLINE", states = { + "MASTER", "SLAVE", "OFFLINE", "ERROR" + }) + public static class MockStateModel extends StateModel { + public MockStateModel() { + + } + + @Transition(to = "*", from = "*") + public void onBecomeAnyFromAny(Message message, NotificationContext context) { + String from = message.getFromState(); + String to = message.getToState(); + LOG.info("Become " + to + " from " + from); + } + } + + public static class MockStateModelFactory extends HelixStateModelFactory { + + public MockStateModelFactory() { + } + + @Override + public MockStateModel createNewStateModel(PartitionId partitionId) { + MockStateModel model = new MockStateModel(); + + return model; + } + } + + @Test + public void test() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + String zkAddr = ZK_ADDR; + ClusterId clusterId = ClusterId.from(clusterName); + ControllerId controllerId = ControllerId.from("controller"); + final ParticipantId participantId = ParticipantId.from("participant1"); + + ResourceId resourceId = ResourceId.from("testDB"); + State master = State.from("MASTER"); + State slave = State.from("SLAVE"); + State offline = State.from("OFFLINE"); + State dropped = State.from("DROPPED"); + StateModelDefId stateModelDefId = StateModelDefId.from("MasterSlave"); + + // create connection + HelixConnection connection = new ZkHelixConnection(zkAddr); + connection.connect(); + + // setup cluster + ClusterAccessor clusterAccessor = connection.createClusterAccessor(clusterId); + clusterAccessor.dropCluster(); + + StateModelDefinition stateModelDef = + new StateModelDefinition.Builder(stateModelDefId).addState(master, 1).addState(slave, 2) + .addState(offline, 3).addState(dropped).addTransition(offline, slave, 3) + .addTransition(slave, offline, 4).addTransition(slave, master, 2) + .addTransition(master, slave, 1).addTransition(offline, dropped).initialState(offline) + .upperBound(master, 1).dynamicUpperBound(slave, "R").build(); + RebalancerContext rebalancerCtx = + new SemiAutoRebalancerContext.Builder(resourceId).addPartitions(1).replicaCount(1) + .stateModelDefId(stateModelDefId) + .preferenceList(PartitionId.from("testDB_0"), Arrays.asList(participantId)).build(); + clusterAccessor.createCluster(new ClusterConfig.Builder(clusterId).addStateModelDefinition( + stateModelDef).build()); + clusterAccessor.addResourceToCluster(new ResourceConfig.Builder(resourceId).rebalancerContext( + rebalancerCtx).build()); + clusterAccessor.addParticipantToCluster(new ParticipantConfig.Builder(participantId).build()); + + // start controller + HelixController controller = connection.createController(clusterId, controllerId); + controller.startAsync(); + + // start participant + HelixParticipant participant = connection.createParticipant(clusterId, participantId); + participant.getStateMachineEngine().registerStateModelFactory( + StateModelDefId.from("MasterSlave"), new MockStateModelFactory()); + + participant.startAsync(); + + // verify + final HelixDataAccessor accessor = connection.createDataAccessor(clusterId); + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + boolean success = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + ExternalView externalView = accessor.getProperty(keyBuilder.externalView("testDB")); + Map stateMap = externalView.getStateMap(PartitionId.from("testDB_0")); + + if (stateMap == null || !stateMap.containsKey(participantId)) { + return false; + } + + return stateMap.get(participantId).equals(State.from("MASTER")); + } + }, 10 * 1000); + + Assert.assertTrue(success); + + // clean up + controller.stopAsync(); + participant.stopAsync(); + connection.disconnect(); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java index b2bb561b5c..c7a1700bea 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java @@ -21,27 +21,29 @@ import java.util.Date; +import org.apache.helix.HelixConstants.ChangeType; import org.apache.helix.HelixManager; import org.apache.helix.NotificationContext; -import org.apache.helix.TestHelper; -import org.apache.helix.HelixConstants.ChangeType; import org.apache.helix.NotificationContext.Type; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; 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.manager.zk.ZkClient; import org.apache.helix.mock.participant.MockJobIntf; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.LiveInstance; import org.apache.helix.participant.CustomCodeCallbackHandler; import org.apache.helix.participant.HelixCustomCodeRunner; import org.apache.helix.tools.ClusterStateVerifier; +import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.annotations.Test; public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase { + private static Logger LOG = Logger.getLogger(TestHelixCustomCodeRunner.class); + private final String _clusterName = "CLUSTER_" + getShortClassName(); private final int _nodeNb = 5; private final int _startPort = 12918; @@ -74,8 +76,7 @@ public void doPreConnectJob(HelixManager manager) { customCodeRunner.invoke(_callback).on(ChangeType.LIVE_INSTANCE) .usingLeaderStandbyModel("TestParticLeader").start(); } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Exception do pre-connect job", e); } } @@ -100,16 +101,19 @@ public void testCustomCodeRunner() throws Exception { _nodeNb, // replica "MasterSlave", true); - TestHelper.startController(_clusterName, "controller_0", ZK_ADDR, - HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, _clusterName, "controller_0"); + controller.syncStart(); - MockParticipant[] partics = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; for (int i = 0; i < _nodeNb; i++) { String instanceName = "localhost_" + (_startPort + i); - partics[i] = new MockParticipant(_clusterName, instanceName, ZK_ADDR, null, new MockJob()); - partics[i].syncStart(); - // new Thread(partics[i]).start(); + MockJob job = new MockJob(); + participants[i] = new MockParticipantManager(ZK_ADDR, _clusterName, instanceName); + + job.doPreConnectJob(participants[i]); + participants[i].syncStart(); } boolean result = ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( @@ -121,20 +125,26 @@ public void testCustomCodeRunner() throws Exception { _callback._isCallbackInvoked = false; // add a new live instance - ZkClient zkClient = new ZkClient(ZK_ADDR); - zkClient.setZkSerializer(new ZNRecordSerializer()); + // ZkClient zkClient = new ZkClient(ZK_ADDR); + // zkClient.setZkSerializer(new ZNRecordSerializer()); ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor(zkClient)); + new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); LiveInstance newLiveIns = new LiveInstance("newLiveInstance"); - newLiveIns.setHelixVersion("0.0.0"); + newLiveIns.setHelixVersion("0.6.0"); newLiveIns.setSessionId("randomSessionId"); accessor.setProperty(keyBuilder.liveInstance("newLiveInstance"), newLiveIns); Thread.sleep(1000); // wait for the CALLBACK type callback to finish Assert.assertTrue(_callback._isCallbackInvoked); + // clean up + controller.syncStop(); + for (int i = 0; i < _nodeNb; i++) { + participants[i].syncStop(); + } + System.out.println("END " + _clusterName + " at " + new Date(System.currentTimeMillis())); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java index 448438635c..1f906d006b 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java @@ -33,8 +33,7 @@ public class TestHelixInstanceTag extends ZkStandAloneCMTestBase { @Test public void testInstanceTag() throws Exception { - String controllerName = CONTROLLER_PREFIX + "_0"; - HelixManager manager = _startCMResultMap.get(controllerName)._manager; + HelixManager manager = _controller; HelixDataAccessor accessor = manager.getHelixDataAccessor(); String DB2 = "TestDB2"; diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestInstanceAutoJoin.java b/helix-core/src/test/java/org/apache/helix/integration/TestInstanceAutoJoin.java index f69aeec643..276163448d 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestInstanceAutoJoin.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestInstanceAutoJoin.java @@ -2,8 +2,7 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; -import org.apache.helix.TestHelper; -import org.apache.helix.TestHelper.StartCMResult; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.model.ConfigScope; import org.apache.helix.model.IdealState.RebalanceMode; @@ -35,8 +34,7 @@ public class TestInstanceAutoJoin extends ZkStandAloneCMTestBase { @Test public void testInstanceAutoJoin() throws Exception { - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0); - HelixManager manager = _startCMResultMap.get(instanceName)._manager; + HelixManager manager = _participants[0]; HelixDataAccessor accessor = manager.getHelixDataAccessor(); _setupTool.addResourceToCluster(CLUSTER_NAME, db2, 60, "OnlineOffline", RebalanceMode.FULL_AUTO @@ -44,10 +42,13 @@ public void testInstanceAutoJoin() throws Exception { _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 1); String instance2 = "localhost_279699"; - StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instance2); + // StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instance2); + MockParticipantManager newParticipant = + new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instance2); + newParticipant.syncStart(); Thread.sleep(500); - Assert.assertFalse(result._thread.isAlive()); + // Assert.assertFalse(result._thread.isAlive()); Assert.assertTrue(null == manager.getHelixDataAccessor().getProperty( accessor.keyBuilder().liveInstance(instance2))); @@ -55,12 +56,11 @@ public void testInstanceAutoJoin() throws Exception { manager.getConfigAccessor().set(scope, ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, "true"); - result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instance2); - - StartCMResult result2 = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instance2); + newParticipant = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instance2); + newParticipant.syncStart(); Thread.sleep(500); - Assert.assertTrue(result._thread.isAlive() || result2._thread.isAlive()); + // Assert.assertTrue(result._thread.isAlive() || result2._thread.isAlive()); for (int i = 0; i < 20; i++) { if (null == manager.getHelixDataAccessor().getProperty( accessor.keyBuilder().liveInstance(instance2))) { @@ -71,9 +71,6 @@ public void testInstanceAutoJoin() throws Exception { Assert.assertTrue(null != manager.getHelixDataAccessor().getProperty( accessor.keyBuilder().liveInstance(instance2))); - result._manager.disconnect(); - result2._manager.disconnect(); - result._thread.interrupt(); - result2._thread.interrupt(); + newParticipant.syncStop(); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java index 837d0a1bda..f2bf7f6ae4 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java @@ -25,21 +25,23 @@ import org.apache.helix.HelixAdmin; import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixAdmin; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.RebalanceMode; import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.StateModelDefinition; import org.apache.helix.tools.ClusterStateVerifier; -import org.apache.helix.tools.StateModelConfigGenerator; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; +import org.apache.helix.tools.StateModelConfigGenerator; import org.testng.Assert; +import org.testng.annotations.Test; // Helix-50: integration test for generate message based on state priority public class TestInvalidAutoIdealState extends ZkUnitTestBase { @@ -81,7 +83,7 @@ void testInvalidReplica2() throws Exception { idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setNumPartitions(2); idealState.setReplicas("" + 2); // should be 3 - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.getRecord().setListField("TestDB_0", Arrays.asList("localhost_12918", "localhost_12919", "localhost_12920")); idealState.getRecord().setListField("TestDB_1", @@ -90,15 +92,16 @@ void testInvalidReplica2() throws Exception { admin.setResourceIdealState(clusterName, "TestDB", idealState); // start participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); boolean result = @@ -107,7 +110,7 @@ void testInvalidReplica2() throws Exception { Assert.assertTrue(result); // make sure localhost_12919 is master on TestDB_1 - HelixDataAccessor accessor = controller.getManager().getHelixDataAccessor(); + HelixDataAccessor accessor = controller.getHelixDataAccessor(); Builder keyBuilder = accessor.keyBuilder(); ExternalView extView = accessor.getProperty(keyBuilder.externalView(db)); Map stateMap = extView.getStateMap(db + "_1"); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java index 487e68947e..bc7e972f1d 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java @@ -26,6 +26,12 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.model.ExternalView; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; @@ -37,9 +43,9 @@ public class TestMessagePartitionStateMismatch extends ZkStandAloneCMTestBase { @Test public void testStateMismatch() throws InterruptedException { - String controllerName = CONTROLLER_PREFIX + "_0"; + // String controllerName = CONTROLLER_PREFIX + "_0"; - HelixManager manager = _startCMResultMap.get(controllerName)._manager; + HelixManager manager = _controller; // _startCMResultMap.get(controllerName)._manager; HelixDataAccessor accessor = manager.getHelixDataAccessor(); Builder kb = accessor.keyBuilder(); ExternalView ev = accessor.getProperty(kb.externalView(TEST_DB)); @@ -47,39 +53,39 @@ public void testStateMismatch() throws InterruptedException { accessor.getChildValuesMap(accessor.keyBuilder().liveInstances()); for (String instanceName : liveinstanceMap.keySet()) { - String sessionid = liveinstanceMap.get(instanceName).getSessionId(); + String sessionid = liveinstanceMap.get(instanceName).getTypedSessionId().stringify(); for (String partition : ev.getPartitionSet()) { if (ev.getStateMap(partition).containsKey(instanceName)) { - String uuid = UUID.randomUUID().toString(); + MessageId uuid = MessageId.from(UUID.randomUUID().toString()); Message message = new Message(MessageType.STATE_TRANSITION, uuid); boolean rand = new Random().nextInt(10) > 5; if (ev.getStateMap(partition).get(instanceName).equals("MASTER")) { message.setSrcName(manager.getInstanceName()); message.setTgtName(instanceName); message.setMsgState(MessageState.NEW); - message.setPartitionName(partition); - message.setResourceName(TEST_DB); - message.setFromState(rand ? "SLAVE" : "OFFLINE"); - message.setToState(rand ? "MASTER" : "SLAVE"); - message.setTgtSessionId(sessionid); - message.setSrcSessionId(manager.getSessionId()); - message.setStateModelDef("MasterSlave"); + message.setPartitionId(PartitionId.from(partition)); + message.setResourceId(ResourceId.from(TEST_DB)); + message.setFromState(State.from(rand ? "SLAVE" : "OFFLINE")); + message.setToState(State.from(rand ? "MASTER" : "SLAVE")); + message.setTgtSessionId(SessionId.from(sessionid)); + message.setSrcSessionId(SessionId.from(manager.getSessionId())); + message.setStateModelDef(StateModelDefId.from("MasterSlave")); message.setStateModelFactoryName("DEFAULT"); } else if (ev.getStateMap(partition).get(instanceName).equals("SLAVE")) { message.setSrcName(manager.getInstanceName()); message.setTgtName(instanceName); message.setMsgState(MessageState.NEW); - message.setPartitionName(partition); - message.setResourceName(TEST_DB); - message.setFromState(rand ? "MASTER" : "OFFLINE"); - message.setToState(rand ? "SLAVE" : "SLAVE"); - message.setTgtSessionId(sessionid); - message.setSrcSessionId(manager.getSessionId()); - message.setStateModelDef("MasterSlave"); + message.setPartitionId(PartitionId.from(partition)); + message.setResourceId(ResourceId.from(TEST_DB)); + message.setFromState(State.from(rand ? "MASTER" : "OFFLINE")); + message.setToState(State.from(rand ? "SLAVE" : "SLAVE")); + message.setTgtSessionId(SessionId.from(sessionid)); + message.setSrcSessionId(SessionId.from(manager.getSessionId())); + message.setStateModelDef(StateModelDefId.from("MasterSlave")); message.setStateModelFactoryName("DEFAULT"); } - accessor.setProperty(accessor.keyBuilder().message(instanceName, message.getMsgId()), - message); + accessor.setProperty( + accessor.keyBuilder().message(instanceName, message.getMessageId().stringify()), message); } } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java index fbff4e151b..0c7c1318d3 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java @@ -21,8 +21,6 @@ import java.util.Date; import java.util.List; -import java.util.Map; -import java.util.TreeMap; import java.util.concurrent.atomic.AtomicBoolean; import org.I0Itec.zkclient.IZkChildListener; @@ -31,17 +29,13 @@ import org.apache.helix.PropertyType; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.HelixControllerMain; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.participant.MockParticipant; -import org.apache.helix.model.ClusterConstraints; -import org.apache.helix.model.ConstraintItem; -import org.apache.helix.model.Message; import org.apache.helix.model.ClusterConstraints.ConstraintType; -import org.apache.helix.model.builder.ClusterConstraintsBuilder; +import org.apache.helix.model.Message; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.model.builder.ConstraintItemBuilder; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; @@ -55,8 +49,7 @@ public void testMessageThrottle() throws Exception { // Logger.getRootLogger().setLevel(Level.INFO); String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; - // ClusterSetup setupTool = new ClusterSetup(ZK_ADDR); + MockParticipantManager[] participants = new MockParticipantManager[5]; System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); @@ -120,13 +113,15 @@ public void handleChildChange(String parentPath, List currentChilds) }); } - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + // start participants for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -143,6 +138,7 @@ public void handleChildChange(String parentPath, List currentChilds) Assert.assertTrue(success.get()); // clean up + controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java index cdd644b4e0..a182753b5e 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java @@ -32,7 +32,6 @@ import org.apache.helix.HelixAdmin; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; -import org.apache.helix.HelixManagerFactory; import org.apache.helix.IdealStateChangeListener; import org.apache.helix.InstanceConfigChangeListener; import org.apache.helix.InstanceType; @@ -41,6 +40,7 @@ import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.controller.HelixControllerMain; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.helix.manager.zk.ZKHelixDataAccessor; @@ -245,8 +245,7 @@ public MyProcess(String instanceName) { public void start() throws Exception { helixManager = - HelixManagerFactory.getZKHelixManager(clusterName, instanceName, - InstanceType.PARTICIPANT, ZK_ADDR); + new ZKHelixManager(clusterName, instanceName, InstanceType.PARTICIPANT, ZK_ADDR); { // hack to set sessionTimeout Field sessionTimeout = ZKHelixManager.class.getDeclaredField("_sessionTimeout"); @@ -281,44 +280,44 @@ public MyStateModel(HelixManager helixManager) { @Transition(to = "SLAVE", from = "OFFLINE") public void onBecomeSlaveFromOffline(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - LOGGER.info(instanceName + " becomes SLAVE from OFFLINE for " + partitionName); + LOGGER.info(instanceName + " becomes SLAVE from OFFLINE for " + partitionId); } @Transition(to = "SLAVE", from = "MASTER") public void onBecomeSlaveFromMaster(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - LOGGER.info(instanceName + " becomes SLAVE from MASTER for " + partitionName); + LOGGER.info(instanceName + " becomes SLAVE from MASTER for " + partitionId); } @Transition(to = "MASTER", from = "SLAVE") public void onBecomeMasterFromSlave(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - LOGGER.info(instanceName + " becomes MASTER from SLAVE for " + partitionName); + LOGGER.info(instanceName + " becomes MASTER from SLAVE for " + partitionId); } @Transition(to = "OFFLINE", from = "SLAVE") public void onBecomeOfflineFromSlave(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - LOGGER.info(instanceName + " becomes OFFLINE from SLAVE for " + partitionName); + LOGGER.info(instanceName + " becomes OFFLINE from SLAVE for " + partitionId); } @Transition(to = "DROPPED", from = "OFFLINE") public void onBecomeDroppedFromOffline(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - LOGGER.info(instanceName + " becomes DROPPED from OFFLINE for " + partitionName); + LOGGER.info(instanceName + " becomes DROPPED from OFFLINE for " + partitionId); } @Transition(to = "OFFLINE", from = "ERROR") public void onBecomeOfflineFromError(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - LOGGER.info(instanceName + " becomes OFFLINE from ERROR for " + partitionName); + LOGGER.info(instanceName + " becomes OFFLINE from ERROR for " + partitionId); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java index 2354ebd2bf..86f1ce49e0 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java @@ -23,9 +23,11 @@ import java.util.UUID; import org.apache.helix.Criteria; +import org.apache.helix.Criteria.DataSource; import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; -import org.apache.helix.Criteria.DataSource; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.messaging.AsyncCallback; import org.apache.helix.messaging.handling.HelixTaskResult; import org.apache.helix.messaging.handling.MessageHandler; @@ -67,7 +69,7 @@ public HelixTaskResult handleMessage() throws InterruptedException { HelixTaskResult result = new HelixTaskResult(); result.setSuccess(true); Thread.sleep(1000); - System.out.println("TestMessagingHandler " + _message.getMsgId()); + System.out.println("TestMessagingHandler " + _message.getMessageId()); _processedMsgIds.add(_message.getRecord().getSimpleField("TestMessagingPara")); result.getTaskResultMap().put("ReplyMessage", "TestReplyMessage"); return result; @@ -87,14 +89,14 @@ public void TestMessageSimpleSend() throws Exception { String hostDest = "localhost_" + (START_PORT + 1); TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory(); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( - factory.getMessageType(), factory); + _participants[1].getMessagingService().registerMessageHandlerFactory(factory.getMessageType(), + factory); - String msgId = new UUID(123, 456).toString(); + MessageId msgId = MessageId.from(new UUID(123, 456).toString()); Message msg = new Message(factory.getMessageType(), msgId); - msg.setMsgId(msgId); + msg.setMessageId(msgId); msg.setSrcName(hostSrc); - msg.setTgtSessionId("*"); + msg.setTgtSessionId(SessionId.from("*")); msg.setMsgState(MessageState.NEW); String para = "Testing messaging para"; msg.getRecord().setSimpleField("TestMessagingPara", para); @@ -104,7 +106,8 @@ public void TestMessageSimpleSend() throws Exception { cr.setRecipientInstanceType(InstanceType.PARTICIPANT); cr.setSessionSpecific(false); - int nMsgs = _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg); + // int nMsgs = _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg); + int nMsgs = _participants[0].getMessagingService().send(cr, msg); AssertJUnit.assertTrue(nMsgs == 1); Thread.sleep(2500); // Thread.currentThread().join(); @@ -116,7 +119,8 @@ public void TestMessageSimpleSend() throws Exception { cr.setSessionSpecific(false); cr.setDataSource(DataSource.IDEALSTATES); - nMsgs = _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg); + // nMsgs = _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg); + nMsgs = _participants[0].getMessagingService().send(cr, msg); AssertJUnit.assertTrue(nMsgs == 1); Thread.sleep(2500); // Thread.currentThread().join(); @@ -179,18 +183,18 @@ public void TestMessageSimpleSendReceiveAsync() throws Exception { String hostDest = "localhost_" + (START_PORT + 1); TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory(); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( - factory.getMessageType(), factory); + _participants[1].getMessagingService().registerMessageHandlerFactory(factory.getMessageType(), + factory); - _startCMResultMap.get(hostSrc)._manager.getMessagingService().registerMessageHandlerFactory( - factory.getMessageType(), factory); + _participants[0].getMessagingService().registerMessageHandlerFactory(factory.getMessageType(), + factory); - String msgId = new UUID(123, 456).toString(); + MessageId msgId = MessageId.from(new UUID(123, 456).toString()); Message msg = new Message(factory.getMessageType(), msgId); - msg.setMsgId(msgId); + msg.setMessageId(msgId); msg.setSrcName(hostSrc); - msg.setTgtSessionId("*"); + msg.setTgtSessionId(SessionId.from("*")); msg.setMsgState(MessageState.NEW); String para = "Testing messaging para"; msg.getRecord().setSimpleField("TestMessagingPara", para); @@ -202,7 +206,7 @@ public void TestMessageSimpleSendReceiveAsync() throws Exception { TestAsyncCallback callback = new TestAsyncCallback(60000); - _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg, callback, 60000); + _participants[0].getMessagingService().send(cr, msg, callback, 60000); Thread.sleep(2000); // Thread.currentThread().join(); @@ -210,7 +214,7 @@ public void TestMessageSimpleSendReceiveAsync() throws Exception { AssertJUnit.assertTrue(callback.getMessageReplied().size() == 1); TestAsyncCallback callback2 = new TestAsyncCallback(500); - _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg, callback2, 500); + _participants[0].getMessagingService().send(cr, msg, callback2, 500); Thread.sleep(3000); // Thread.currentThread().join(); @@ -224,7 +228,7 @@ public void TestMessageSimpleSendReceiveAsync() throws Exception { callback = new TestAsyncCallback(60000); - _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg, callback, 60000); + _participants[0].getMessagingService().send(cr, msg, callback, 60000); Thread.sleep(2000); // Thread.currentThread().join(); @@ -232,7 +236,7 @@ public void TestMessageSimpleSendReceiveAsync() throws Exception { AssertJUnit.assertTrue(callback.getMessageReplied().size() == 1); callback2 = new TestAsyncCallback(500); - _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg, callback2, 500); + _participants[0].getMessagingService().send(cr, msg, callback2, 500); Thread.sleep(3000); // Thread.currentThread().join(); @@ -246,15 +250,15 @@ public void TestBlockingSendReceive() throws Exception { String hostDest = "localhost_" + (START_PORT + 1); TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory(); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( - factory.getMessageType(), factory); + _participants[1].getMessagingService().registerMessageHandlerFactory(factory.getMessageType(), + factory); - String msgId = new UUID(123, 456).toString(); + MessageId msgId = MessageId.from(new UUID(123, 456).toString()); Message msg = new Message(factory.getMessageType(), msgId); - msg.setMsgId(msgId); + msg.setMessageId(msgId); msg.setSrcName(hostSrc); - msg.setTgtSessionId("*"); + msg.setTgtSessionId(SessionId.from("*")); msg.setMsgState(MessageState.NEW); String para = "Testing messaging para"; msg.getRecord().setSimpleField("TestMessagingPara", para); @@ -266,8 +270,7 @@ public void TestBlockingSendReceive() throws Exception { AsyncCallback asyncCallback = new MockAsyncCallback(); int messagesSent = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - asyncCallback, 60000); + _participants[0].getMessagingService().sendAndWait(cr, msg, asyncCallback, 60000); AssertJUnit.assertTrue(asyncCallback.getMessageReplied().get(0).getRecord() .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ReplyMessage") @@ -275,9 +278,7 @@ public void TestBlockingSendReceive() throws Exception { AssertJUnit.assertTrue(asyncCallback.getMessageReplied().size() == 1); AsyncCallback asyncCallback2 = new MockAsyncCallback(); - messagesSent = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - asyncCallback2, 500); + messagesSent = _participants[0].getMessagingService().sendAndWait(cr, msg, asyncCallback2, 500); AssertJUnit.assertTrue(asyncCallback2.isTimedOut()); } @@ -289,15 +290,16 @@ public void TestMultiMessageCriteria() throws Exception { for (int i = 0; i < NODE_NR; i++) { TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory(); String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[0].getMessagingService().registerMessageHandlerFactory( factory.getMessageType(), factory); + } - String msgId = new UUID(123, 456).toString(); + MessageId msgId = MessageId.from(new UUID(123, 456).toString()); Message msg = new Message(new TestMessagingHandlerFactory().getMessageType(), msgId); - msg.setMsgId(msgId); + msg.setMessageId(msgId); msg.setSrcName(hostSrc); - msg.setTgtSessionId("*"); + msg.setTgtSessionId(SessionId.from("*")); msg.setMsgState(MessageState.NEW); String para = "Testing messaging para"; msg.getRecord().setSimpleField("TestMessagingPara", para); @@ -308,8 +310,7 @@ public void TestMultiMessageCriteria() throws Exception { cr.setSessionSpecific(false); AsyncCallback callback1 = new MockAsyncCallback(); int messageSent1 = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback1, 10000); + _participants[0].getMessagingService().sendAndWait(cr, msg, callback1, 10000); AssertJUnit.assertTrue(callback1.getMessageReplied().get(0).getRecord() .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ReplyMessage") @@ -317,37 +318,32 @@ public void TestMultiMessageCriteria() throws Exception { AssertJUnit.assertTrue(callback1.getMessageReplied().size() == NODE_NR - 1); AsyncCallback callback2 = new MockAsyncCallback(); - int messageSent2 = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback2, 500); + int messageSent2 = _participants[0].getMessagingService().sendAndWait(cr, msg, callback2, 500); + AssertJUnit.assertTrue(callback2.isTimedOut()); cr.setPartition("TestDB_17"); AsyncCallback callback3 = new MockAsyncCallback(); int messageSent3 = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback3, 10000); + _participants[0].getMessagingService().sendAndWait(cr, msg, callback3, 10000); AssertJUnit.assertTrue(callback3.getMessageReplied().size() == _replica - 1); cr.setPartition("TestDB_15"); AsyncCallback callback4 = new MockAsyncCallback(); int messageSent4 = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback4, 10000); + _participants[0].getMessagingService().sendAndWait(cr, msg, callback4, 10000); AssertJUnit.assertTrue(callback4.getMessageReplied().size() == _replica); cr.setPartitionState("SLAVE"); AsyncCallback callback5 = new MockAsyncCallback(); int messageSent5 = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback5, 10000); + _participants[0].getMessagingService().sendAndWait(cr, msg, callback5, 10000); AssertJUnit.assertTrue(callback5.getMessageReplied().size() == _replica - 1); cr.setDataSource(DataSource.IDEALSTATES); AsyncCallback callback6 = new MockAsyncCallback(); int messageSent6 = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback6, 10000); + _participants[0].getMessagingService().sendAndWait(cr, msg, callback6, 10000); AssertJUnit.assertTrue(callback6.getMessageReplied().size() == _replica - 1); } @@ -358,15 +354,16 @@ public void sendSelfMsg() { for (int i = 0; i < NODE_NR; i++) { TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory(); String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[i].getMessagingService().registerMessageHandlerFactory( factory.getMessageType(), factory); + } - String msgId = new UUID(123, 456).toString(); + MessageId msgId = MessageId.from(new UUID(123, 456).toString()); Message msg = new Message(new TestMessagingHandlerFactory().getMessageType(), msgId); - msg.setMsgId(msgId); + msg.setMessageId(msgId); msg.setSrcName(hostSrc); - msg.setTgtSessionId("*"); + msg.setTgtSessionId(SessionId.from("*")); msg.setMsgState(MessageState.NEW); String para = "Testing messaging para"; msg.getRecord().setSimpleField("TestMessagingPara", para); @@ -378,8 +375,7 @@ public void sendSelfMsg() { cr.setSelfExcluded(false); AsyncCallback callback1 = new MockAsyncCallback(); int messageSent1 = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback1, 10000); + _participants[0].getMessagingService().sendAndWait(cr, msg, callback1, 10000); AssertJUnit.assertTrue(callback1.getMessageReplied().size() == NODE_NR); AssertJUnit.assertTrue(callback1.getMessageReplied().get(0).getRecord() @@ -394,15 +390,16 @@ public void TestControllerMessage() throws Exception { for (int i = 0; i < NODE_NR; i++) { TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory(); String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[i].getMessagingService().registerMessageHandlerFactory( factory.getMessageType(), factory); + } - String msgId = new UUID(123, 456).toString(); + MessageId msgId = MessageId.from(new UUID(123, 456).toString()); Message msg = new Message(MessageType.CONTROLLER_MSG, msgId); - msg.setMsgId(msgId); + msg.setMessageId(msgId); msg.setSrcName(hostSrc); - msg.setTgtSessionId("*"); + msg.setTgtSessionId(SessionId.from("*")); msg.setMsgState(MessageState.NEW); String para = "Testing messaging para"; msg.getRecord().setSimpleField("TestMessagingPara", para); @@ -414,34 +411,30 @@ public void TestControllerMessage() throws Exception { AsyncCallback callback1 = new MockAsyncCallback(); int messagesSent = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback1, 10000); + _participants[0].getMessagingService().sendAndWait(cr, msg, callback1, 10000); AssertJUnit.assertTrue(callback1.getMessageReplied().get(0).getRecord() .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ControllerResult") .indexOf(hostSrc) != -1); AssertJUnit.assertTrue(callback1.getMessageReplied().size() == 1); - msgId = UUID.randomUUID().toString(); - msg.setMsgId(msgId); + msgId = MessageId.from(UUID.randomUUID().toString()); + msg.setMessageId(msgId); cr.setPartition("TestDB_17"); AsyncCallback callback2 = new MockAsyncCallback(); - messagesSent = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback2, 10000); + messagesSent = _participants[0].getMessagingService().sendAndWait(cr, msg, callback2, 10000); + AssertJUnit.assertTrue(callback2.getMessageReplied().get(0).getRecord() .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ControllerResult") .indexOf(hostSrc) != -1); AssertJUnit.assertTrue(callback2.getMessageReplied().size() == 1); - msgId = UUID.randomUUID().toString(); - msg.setMsgId(msgId); + msgId = MessageId.from(UUID.randomUUID().toString()); + msg.setMessageId(msgId); cr.setPartitionState("SLAVE"); AsyncCallback callback3 = new MockAsyncCallback(); - messagesSent = - _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg, - callback3, 10000); + messagesSent = _participants[0].getMessagingService().sendAndWait(cr, msg, callback3, 10000); AssertJUnit.assertTrue(callback3.getMessageReplied().get(0).getRecord() .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ControllerResult") .indexOf(hostSrc) != -1); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java b/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java index 5a6e5e6508..aa48c90b97 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java @@ -22,9 +22,8 @@ import java.util.Date; import org.apache.helix.TestHelper; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.manager.zk.ZkClient; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.MockBootstrapModelFactory; import org.apache.helix.participant.StateMachineEngine; import org.apache.helix.tools.ClusterSetup; @@ -51,18 +50,19 @@ public void testNonOfflineInitState() throws Exception { 1, // replicas "Bootstrap", true); // do rebalance - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); // start participants - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); // add a state model with non-OFFLINE initial state - StateMachineEngine stateMach = participants[i].getManager().getStateMachineEngine(); + StateMachineEngine stateMach = participants[i].getStateMachineEngine(); MockBootstrapModelFactory bootstrapFactory = new MockBootstrapModelFactory(); stateMach.registerStateModelFactory("Bootstrap", bootstrapFactory); @@ -74,16 +74,21 @@ public void testNonOfflineInitState() throws Exception { clusterName)); Assert.assertTrue(result); + // clean up + controller.syncStop(); + for (int i = 0; i < 5; i++) { + participants[i].syncStop(); + } + System.out.println("END testNonOfflineInitState at " + new Date(System.currentTimeMillis())); } private 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 { - ZkClient zkClient = new ZkClient(ZkAddr); - if (zkClient.exists("/" + clusterName)) { + if (_gZkClient.exists("/" + clusterName)) { LOG.warn("Cluster already exists:" + clusterName + ". Deleting it"); - zkClient.deleteRecursive("/" + clusterName); + _gZkClient.deleteRecursive("/" + clusterName); } ClusterSetup setupTool = new ClusterSetup(ZkAddr); @@ -103,7 +108,6 @@ private static void setupCluster(String clusterName, String ZkAddr, int startPor setupTool.rebalanceStorageCluster(clusterName, dbName, replica); } } - zkClient.close(); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestNullReplica.java b/helix-core/src/test/java/org/apache/helix/integration/TestNullReplica.java index 496d1a65dc..0f5cc72513 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestNullReplica.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestNullReplica.java @@ -25,8 +25,8 @@ import org.apache.helix.PropertyType; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.model.IdealState; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; @@ -44,7 +44,7 @@ public void testNullReplica() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -61,14 +61,15 @@ public void testNullReplica() throws Exception { idealState.getSimpleFields().remove(IdealState.IdealStateProperty.REPLICAS.toString()); _gZkClient.writeData(idealStatePath, idealState); - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -78,13 +79,11 @@ public void testNullReplica() throws Exception { Assert.assertTrue(result); // clean up + controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); } - Thread.sleep(2000); - controller.syncStop(); - System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java index 833693907c..750e2b7554 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java @@ -24,6 +24,10 @@ import org.apache.helix.Criteria; import org.apache.helix.InstanceType; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory; import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory.ActionOnError; import org.apache.helix.model.ExternalView; @@ -31,55 +35,59 @@ import org.apache.helix.model.Message.MessageType; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; +import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.annotations.Test; public class TestParticipantErrorMessage extends ZkStandAloneCMTestBase { + private static Logger LOG = Logger.getLogger(TestParticipantErrorMessage.class); + @Test() public void TestParticipantErrorMessageSend() { String participant1 = "localhost_" + START_PORT; String participant2 = "localhost_" + (START_PORT + 1); Message errorMessage1 = - new Message(MessageType.PARTICIPANT_ERROR_REPORT, UUID.randomUUID().toString()); - errorMessage1.setTgtSessionId("*"); + new Message(MessageType.PARTICIPANT_ERROR_REPORT, MessageId.from(UUID.randomUUID() + .toString())); + errorMessage1.setTgtSessionId(SessionId.from("*")); errorMessage1.getRecord().setSimpleField( DefaultParticipantErrorMessageHandlerFactory.ACTIONKEY, ActionOnError.DISABLE_INSTANCE.toString()); Criteria recipientCriteria = new Criteria(); recipientCriteria.setRecipientInstanceType(InstanceType.CONTROLLER); recipientCriteria.setSessionSpecific(false); - _startCMResultMap.get(participant1)._manager.getMessagingService().send(recipientCriteria, + _participants[0].getMessagingService().send(recipientCriteria, errorMessage1); Message errorMessage2 = - new Message(MessageType.PARTICIPANT_ERROR_REPORT, UUID.randomUUID().toString()); - errorMessage2.setTgtSessionId("*"); - errorMessage2.setResourceName("TestDB"); - errorMessage2.setPartitionName("TestDB_14"); + new Message(MessageType.PARTICIPANT_ERROR_REPORT, MessageId.from(UUID.randomUUID() + .toString())); + errorMessage2.setTgtSessionId(SessionId.from("*")); + errorMessage2.setResourceId(ResourceId.from("TestDB")); + errorMessage2.setPartitionId(PartitionId.from("TestDB_14")); errorMessage2.getRecord().setSimpleField( DefaultParticipantErrorMessageHandlerFactory.ACTIONKEY, ActionOnError.DISABLE_PARTITION.toString()); Criteria recipientCriteria2 = new Criteria(); recipientCriteria2.setRecipientInstanceType(InstanceType.CONTROLLER); recipientCriteria2.setSessionSpecific(false); - _startCMResultMap.get(participant2)._manager.getMessagingService().send(recipientCriteria2, + _participants[1].getMessagingService().send(recipientCriteria2, errorMessage2); try { Thread.sleep(1500); } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Interrupted sleep", e); } boolean result = ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME)); Assert.assertTrue(result); - Builder kb = _startCMResultMap.get(participant2)._manager.getHelixDataAccessor().keyBuilder(); + Builder kb = _participants[1].getHelixDataAccessor().keyBuilder(); ExternalView externalView = - _startCMResultMap.get(participant2)._manager.getHelixDataAccessor().getProperty( + _participants[1].getHelixDataAccessor().getProperty( kb.externalView("TestDB")); for (String partitionName : externalView.getRecord().getMapFields().keySet()) { diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantNameCollision.java b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantNameCollision.java index 4227688dd5..aa67ac9275 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantNameCollision.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantNameCollision.java @@ -22,7 +22,7 @@ import java.util.Date; import org.apache.helix.TestHelper; -import org.apache.helix.TestHelper.StartCMResult; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.log4j.Logger; import org.testng.annotations.Test; @@ -33,20 +33,21 @@ public class TestParticipantNameCollision extends ZkStandAloneCMTestBase { public void testParticiptantNameCollision() throws Exception { logger.info("RUN TestParticipantNameCollision() at " + new Date(System.currentTimeMillis())); - StartCMResult result = null; + MockParticipantManager newParticipant = null; for (int i = 0; i < 1; i++) { String instanceName = "localhost_" + (START_PORT + i); try { // the call fails on getClusterManagerForParticipant() // no threads start - result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName); + newParticipant = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName); + newParticipant.syncStart(); } catch (Exception e) { e.printStackTrace(); } } Thread.sleep(30000); - TestHelper.verifyWithTimeout("verifyNotConnected", 30 * 1000, result._manager); + TestHelper.verifyWithTimeout("verifyNotConnected", 30 * 1000, newParticipant); logger.info("STOP TestParticipantNameCollision() at " + new Date(System.currentTimeMillis())); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestPauseSignal.java b/helix-core/src/test/java/org/apache/helix/integration/TestPauseSignal.java index d900d98286..a1c413b12c 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestPauseSignal.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestPauseSignal.java @@ -24,12 +24,12 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; 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.manager.zk.ZkClient; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.PauseSignal; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; @@ -47,7 +47,7 @@ public void testPauseSignal() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -59,14 +59,15 @@ public void testPauseSignal() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -112,13 +113,11 @@ TestHelper. setOf("localhost_12918", "localhost_12919", "localhost_12920 Assert.assertTrue(result); // clean up + controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); } - Thread.sleep(2000); - controller.syncStop(); - System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestRedefineStateModelDef.java b/helix-core/src/test/java/org/apache/helix/integration/TestRedefineStateModelDef.java new file mode 100644 index 0000000000..3057ee0e38 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/integration/TestRedefineStateModelDef.java @@ -0,0 +1,161 @@ +package org.apache.helix.integration; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.TreeMap; + +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.tools.ClusterStateVerifier; +import org.testng.Assert; +import org.testng.annotations.Test; + +/** + * test case for redefining state model definition + * the new state model definition should be compatible with the old state model definition (i.e. + * states and transitions of old state model definition should be a subset of the new state model + * definition) + */ +public class TestRedefineStateModelDef extends ZkUnitTestBase { + + @Test + public void test() throws Exception { + // Logger.getRootLogger().setLevel(Level.INFO); + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + int n = 2; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 8, // partitions per resource + n, // number of nodes + 2, // replicas + "MasterSlave", false); + autoRebalance(clusterName); + + // start controller + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + + // start participants + MockParticipantManager[] participants = new MockParticipantManager[n]; + for (int i = 0; i < n; i++) { + String instanceName = "localhost_" + (12918 + i); + + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].syncStart(); + } + + boolean result = + ClusterStateVerifier + .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(result); + + // stop controller, redefine state model definition, and re-start controller + controller.syncStop(); + redefineStateModelDef(clusterName); + controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + + result = + ClusterStateVerifier + .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(result); + + // clean up + // wait for all zk callbacks done + Thread.sleep(1000); + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } + + // auto-rebalance + private void autoRebalance(String clusterName) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + IdealState idealState = accessor.getProperty(keyBuilder.idealStates("TestDB0")); + + idealState.setReplicas("" + 2); + idealState.setRebalanceMode(RebalanceMode.FULL_AUTO); + for (int i = 0; i < idealState.getNumPartitions(); i++) { + String partitionName = "TestDB0_" + i; + idealState.getRecord().setMapField(partitionName, new HashMap()); + idealState.getRecord().setListField(partitionName, new ArrayList()); + } + + accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState); + } + + // redefine a new master-slave state machine + // the new state machine adds a new LEADER state which transfers to/from MASTER + private void redefineStateModelDef(String clusterName) { + ZKHelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + StateModelDefinition masterSlave = + accessor.getProperty(keyBuilder.stateModelDef("MasterSlave")); + masterSlave.getRecord().getListField("STATE_PRIORITY_LIST").add(0, "LEADER"); + masterSlave.getRecord().getListField("STATE_TRANSITION_PRIORITYLIST").add(0, "LEADER-MASTER"); + masterSlave.getRecord().getListField("STATE_TRANSITION_PRIORITYLIST").add(0, "MASTER-LEADER"); + masterSlave.getRecord().getMapFields().put("LEADER.meta", new TreeMap()); + masterSlave.getRecord().getMapField("LEADER.meta").put("count", "1"); + masterSlave.getRecord().getMapFields().put("LEADER.next", new TreeMap()); + masterSlave.getRecord().getMapField("LEADER.next").put("MASTER", "MASTER"); + masterSlave.getRecord().getMapField("LEADER.next").put("SLAVE", "MASTER"); + masterSlave.getRecord().getMapField("LEADER.next").put("OFFLINE", "MASTER"); + masterSlave.getRecord().getMapField("LEADER.next").put("OFFLINE", "MASTER"); + masterSlave.getRecord().getMapField("LEADER.next").put("DROPPED", "MASTER"); + + masterSlave.getRecord().getMapField("MASTER.meta").put("count", "R"); + masterSlave.getRecord().getMapField("MASTER.next").put("LEADER", "LEADER"); + masterSlave.getRecord().getMapField("SLAVE.next").put("LEADER", "MASTER"); + masterSlave.getRecord().getMapField("OFFLINE.next").put("LEADER", "MASTER"); + + StateModelDefinition newMasterSlave = new StateModelDefinition(masterSlave.getRecord()); + accessor.setProperty(keyBuilder.stateModelDef("MasterSlave"), newMasterSlave); + } + +} diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java b/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java index ed056ab6c6..d3a370d2c7 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java @@ -23,22 +23,32 @@ import java.util.Date; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.RebalanceMode; import org.apache.helix.tools.ClusterStateVerifier; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.testng.Assert; import org.testng.annotations.Test; public class TestRenamePartition extends ZkIntegrationTestBase { + // map from clusterName to participants + final Map _participantMap = + new ConcurrentHashMap(); + + // map from clusterName to controllers + final Map _controllerMap = + new ConcurrentHashMap(); + @Test() public void testRenamePartitionAutoIS() throws Exception { String clusterName = "CLUSTER_" + getShortClassName() + "_auto"; @@ -57,7 +67,7 @@ public void testRenamePartitionAutoIS() throws Exception { // rename partition name TestDB0_0 tp TestDB0_100 ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); IdealState idealState = accessor.getProperty(keyBuilder.idealStates("TestDB0")); @@ -71,8 +81,8 @@ public void testRenamePartitionAutoIS() throws Exception { ZK_ADDR, clusterName)); Assert.assertTrue(result); + stop(clusterName); System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); - } @Test() @@ -95,15 +105,15 @@ public void testRenamePartitionCustomIS() throws Exception { Arrays.asList("localhost_12918", "localhost_12919", "localhost_12920", "localhost_12921", "localhost_12922"); ZNRecord destIS = - DefaultIdealStateCalculator.calculateIdealState(instanceNames, 10, 3 - 1, "TestDB0", - "MASTER", "SLAVE"); + DefaultTwoStateStrategy.calculateIdealState(instanceNames, 10, 3 - 1, "TestDB0", "MASTER", + "SLAVE"); IdealState idealState = new IdealState(destIS); idealState.setRebalanceMode(RebalanceMode.CUSTOMIZED); idealState.setReplicas("3"); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState); @@ -118,23 +128,25 @@ public void testRenamePartitionCustomIS() throws Exception { ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( ZK_ADDR, clusterName)); Assert.assertTrue(result); + + stop(clusterName); System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } private void startAndVerify(String clusterName) throws Exception { - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); // start participants for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); - // new Thread(participants[i]).start(); } boolean result = @@ -142,5 +154,21 @@ private void startAndVerify(String clusterName) throws Exception { ZK_ADDR, clusterName)); Assert.assertTrue(result); + _participantMap.put(clusterName, participants); + _controllerMap.put(clusterName, controller); + } + + private void stop(String clusterName) { + ClusterControllerManager controller = _controllerMap.get(clusterName); + if (controller != null) { + controller.syncStop(); + } + + MockParticipantManager[] participants = _participantMap.get(clusterName); + if (participants != null) { + for (MockParticipantManager participant : participants) { + participant.syncStop(); + } + } } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java b/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java index 2715932bd4..7159a1774f 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java @@ -25,8 +25,8 @@ import java.util.Set; import org.apache.helix.TestHelper; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.ErrTransition; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; @@ -54,7 +54,8 @@ public void testResetInstance() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); Map> errPartitions = new HashMap>() { @@ -65,16 +66,15 @@ public void testResetInstance() throws Exception { }; // start mock participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); if (i == 0) { - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, - new ErrTransition(errPartitions)); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errPartitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -102,8 +102,6 @@ public void testResetInstance() throws Exception { Assert.assertTrue(result, "Cluster verification fails"); // clean up - // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java b/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java index 09e57c601c..af1ef13b0a 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java @@ -25,13 +25,14 @@ import java.util.Set; import org.apache.helix.NotificationContext; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.State; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.ErrTransition; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; @@ -52,9 +53,9 @@ public ErrTransitionWithResetCnt(Map> errPartitions) { public void doTransition(Message message, NotificationContext context) { // System.err.println("doReset() invoked"); super.doTransition(message, context); - String fromState = message.getFromState(); - String toState = message.getToState(); - if (fromState.equals("ERROR") && toState.equals("OFFLINE")) { + State fromState = message.getTypedFromState(); + State toState = message.getTypedToState(); + if (fromState.toString().equals("ERROR") && toState.toString().equals("OFFLINE")) { _errToOfflineInvoked++; } } @@ -80,7 +81,8 @@ public void testResetPartitionState() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); Map> errPartitions = new HashMap>() { @@ -91,16 +93,16 @@ public void testResetPartitionState() throws Exception { }; // start mock participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); if (i == 0) { participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, - new ErrTransition(errPartitions)); + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errPartitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -169,8 +171,6 @@ public void testResetPartitionState() throws Exception { Assert.assertEquals(_errToOfflineInvoked, 2, "Should reset 2 partitions"); // clean up - // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); @@ -189,8 +189,8 @@ private void clearStatusUpdate(String clusterName, String instance, String resou Builder keyBuilder = accessor.keyBuilder(); LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance(instance)); - accessor.removeProperty(keyBuilder.stateTransitionStatus(instance, liveInstance.getSessionId(), - resource, partition)); + accessor.removeProperty(keyBuilder.stateTransitionStatus(instance, liveInstance.getTypedSessionId() + .stringify(), resource, partition)); } // TODO: throw exception in reset() diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java index de4ad1a7ce..46a05d83b5 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java @@ -25,8 +25,8 @@ import java.util.Set; import org.apache.helix.TestHelper; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.ErrTransition; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; @@ -53,7 +53,8 @@ public void testResetNode() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); Map> errPartitions = new HashMap>() { @@ -64,16 +65,15 @@ public void testResetNode() throws Exception { }; // start mock participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); if (i == 0) { - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, - new ErrTransition(errPartitions)); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new ErrTransition(errPartitions)); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); } @@ -101,8 +101,6 @@ public void testResetNode() throws Exception { Assert.assertTrue(result, "Cluster verification fails"); // clean up - // wait for all zk callbacks done - Thread.sleep(1000); controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestRestartParticipant.java b/helix-core/src/test/java/org/apache/helix/integration/TestRestartParticipant.java index 008782c246..64043ed360 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestRestartParticipant.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestRestartParticipant.java @@ -24,8 +24,8 @@ import org.apache.helix.NotificationContext; import org.apache.helix.TestHelper; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.MockTransition; import org.apache.helix.model.Message; import org.apache.helix.tools.ClusterStateVerifier; @@ -35,15 +35,15 @@ public class TestRestartParticipant extends ZkIntegrationTestBase { public class KillOtherTransition extends MockTransition { - final AtomicReference _other; + final AtomicReference _other; - public KillOtherTransition(MockParticipant other) { - _other = new AtomicReference(other); + public KillOtherTransition(MockParticipantManager other) { + _other = new AtomicReference(other); } @Override public void doTransition(Message message, NotificationContext context) { - MockParticipant other = _other.getAndSet(null); + MockParticipantManager other = _other.getAndSet(null); if (other != null) { System.err.println("Kill " + other.getInstanceName() + ". Interrupted exceptions are IGNORABLE"); @@ -58,7 +58,7 @@ public void testRestartParticipant() throws Exception { System.out.println("START testRestartParticipant at " + new Date(System.currentTimeMillis())); String clusterName = getShortClassName(); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -69,19 +69,19 @@ public void testRestartParticipant() throws Exception { 3, // replicas "MasterSlave", true); // do rebalance - TestHelper - .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); + // start participants for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); if (i == 4) { - participants[i] = - new MockParticipant(clusterName, instanceName, ZK_ADDR, new KillOtherTransition( - participants[0])); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new KillOtherTransition(participants[0])); } else { - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); - // Thread.sleep(100); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); } participants[i].syncStart(); @@ -94,9 +94,9 @@ public void testRestartParticipant() throws Exception { // restart Thread.sleep(500); - MockParticipant participant = - new MockParticipant(participants[0].getClusterName(), participants[0].getInstanceName(), - ZK_ADDR, null); + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, participants[0].getClusterName(), + participants[0].getInstanceName()); System.err.println("Restart " + participant.getInstanceName()); participant.syncStart(); result = @@ -104,6 +104,13 @@ public void testRestartParticipant() throws Exception { clusterName)); Assert.assertTrue(result); + // clean up + controller.syncStop(); + for (int i = 0; i < 5; i++) { + participants[i].syncStop(); + } + participant.syncStop(); + System.out.println("START testRestartParticipant at " + new Date(System.currentTimeMillis())); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java index 2c174c4f77..30f5807668 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.StringWriter; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -38,6 +39,7 @@ import org.apache.helix.NotificationContext; import org.apache.helix.PropertyKey; import org.apache.helix.PropertyType; +import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory; @@ -82,7 +84,7 @@ public void onReplyMessage(Message message) { } } - TestMessagingHandlerFactory _factory = new TestMessagingHandlerFactory(); + final TestMessagingHandlerFactory _factory = new TestMessagingHandlerFactory(); public static class TestMessagingHandlerFactory implements MessageHandlerFactory { public Map> _results = new ConcurrentHashMap>(); @@ -114,14 +116,16 @@ public HelixTaskResult handleMessage() throws InterruptedException { HelixTaskResult result = new HelixTaskResult(); result.setSuccess(true); String destName = _message.getTgtName(); + String partitionName = _message.getPartitionName(); result.getTaskResultMap().put("Message", _message.getMsgId()); synchronized (_results) { - if (!_results.containsKey(_message.getPartitionName())) { - _results.put(_message.getPartitionName(), new ConcurrentSkipListSet()); + if (!_results.containsKey(partitionName)) { + _results.put(partitionName, new HashSet()); } + _results.get(partitionName).add(_message.getMsgId()); } - _results.get(_message.getPartitionName()).add(_message.getMsgId()); - // System.err.println("Message " + _message.getMsgId() + " executed"); + // System.err.println("handle msg: " + _message.getPartitionName() + ", from: " + // + _message.getFromState() + ", to: " + _message.getToState()); return result; } @@ -189,15 +193,15 @@ public void onError(Exception e, ErrorCode code, ErrorType type) { } @Test() - public void TestSchedulerMsgUsingQueue() throws Exception { + public void testSchedulerMsgUsingQueue() throws Exception { Logger.getRootLogger().setLevel(Level.INFO); _factory._results.clear(); HelixManager manager = null; for (int i = 0; i < NODE_NR; i++) { - String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[i].getMessagingService().registerMessageHandlerFactory( _factory.getMessageType(), _factory); - manager = _startCMResultMap.get(hostDest)._manager; + + manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager; } Message schedulerMessage = @@ -277,15 +281,15 @@ public void TestSchedulerMsgUsingQueue() throws Exception { } @Test() - public void TestSchedulerMsg() throws Exception { + public void testSchedulerMsg() throws Exception { Logger.getRootLogger().setLevel(Level.INFO); _factory._results.clear(); HelixManager manager = null; for (int i = 0; i < NODE_NR; i++) { - String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[i].getMessagingService().registerMessageHandlerFactory( _factory.getMessageType(), _factory); - manager = _startCMResultMap.get(hostDest)._manager; + + manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager; } Message schedulerMessage = @@ -367,11 +371,11 @@ public void TestSchedulerMsg() throws Exception { String controllerStatusPath = HelixUtil.getControllerPropertyPath(manager.getClusterName(), PropertyType.STATUSUPDATES_CONTROLLER); - List subPaths = _zkClient.getChildren(controllerStatusPath); + List subPaths = _gZkClient.getChildren(controllerStatusPath); Assert.assertTrue(subPaths.size() > 0); for (String subPath : subPaths) { String nextPath = controllerStatusPath + "/" + subPath; - List subsubPaths = _zkClient.getChildren(nextPath); + List subsubPaths = _gZkClient.getChildren(nextPath); Assert.assertTrue(subsubPaths.size() > 0); } @@ -379,51 +383,51 @@ public void TestSchedulerMsg() throws Exception { HelixUtil.getInstancePropertyPath(manager.getClusterName(), "localhost_" + (START_PORT), PropertyType.STATUSUPDATES); - subPaths = _zkClient.getChildren(instanceStatusPath); + subPaths = _gZkClient.getChildren(instanceStatusPath); Assert.assertTrue(subPaths.size() > 0); for (String subPath : subPaths) { String nextPath = instanceStatusPath + "/" + subPath; - List subsubPaths = _zkClient.getChildren(nextPath); + List subsubPaths = _gZkClient.getChildren(nextPath); Assert.assertTrue(subsubPaths.size() > 0); for (String subsubPath : subsubPaths) { String nextnextPath = nextPath + "/" + subsubPath; - Assert.assertTrue(_zkClient.getChildren(nextnextPath).size() > 0); + Assert.assertTrue(_gZkClient.getChildren(nextnextPath).size() > 0); } } Thread.sleep(3000); - ZKPathDataDumpTask dumpTask = new ZKPathDataDumpTask(manager, _zkClient, 0); + ZKPathDataDumpTask dumpTask = new ZKPathDataDumpTask(manager, _gZkClient, 0); dumpTask.run(); - subPaths = _zkClient.getChildren(controllerStatusPath); + subPaths = _gZkClient.getChildren(controllerStatusPath); Assert.assertTrue(subPaths.size() > 0); for (String subPath : subPaths) { String nextPath = controllerStatusPath + "/" + subPath; - List subsubPaths = _zkClient.getChildren(nextPath); + List subsubPaths = _gZkClient.getChildren(nextPath); Assert.assertTrue(subsubPaths.size() == 0); } - subPaths = _zkClient.getChildren(instanceStatusPath); + subPaths = _gZkClient.getChildren(instanceStatusPath); Assert.assertTrue(subPaths.size() > 0); for (String subPath : subPaths) { String nextPath = instanceStatusPath + "/" + subPath; - List subsubPaths = _zkClient.getChildren(nextPath); + List subsubPaths = _gZkClient.getChildren(nextPath); Assert.assertTrue(subsubPaths.size() > 0); for (String subsubPath : subsubPaths) { String nextnextPath = nextPath + "/" + subsubPath; - Assert.assertTrue(_zkClient.getChildren(nextnextPath).size() == 0); + Assert.assertTrue(_gZkClient.getChildren(nextnextPath).size() == 0); } } } @Test() - public void TestSchedulerMsg2() throws Exception { + public void testSchedulerMsg2() throws Exception { _factory._results.clear(); HelixManager manager = null; for (int i = 0; i < NODE_NR; i++) { - String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[i].getMessagingService().registerMessageHandlerFactory( _factory.getMessageType(), _factory); - manager = _startCMResultMap.get(hostDest)._manager; + + manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager; } Message schedulerMessage = @@ -507,14 +511,14 @@ public void TestSchedulerMsg2() throws Exception { } @Test() - public void TestSchedulerZeroMsg() throws Exception { + public void testSchedulerZeroMsg() throws Exception { TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory(); HelixManager manager = null; for (int i = 0; i < NODE_NR; i++) { - String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[i].getMessagingService().registerMessageHandlerFactory( factory.getMessageType(), factory); - manager = _startCMResultMap.get(hostDest)._manager; + + manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager; } Message schedulerMessage = @@ -577,17 +581,17 @@ public void TestSchedulerZeroMsg() throws Exception { } @Test() - public void TestSchedulerMsg3() throws Exception { + public void testSchedulerMsg3() throws Exception { _factory._results.clear(); HelixManager manager = null; for (int i = 0; i < NODE_NR; i++) { - String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[i].getMessagingService().registerMessageHandlerFactory( _factory.getMessageType(), _factory); - // - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( - _factory.getMessageType(), _factory); - manager = _startCMResultMap.get(hostDest)._manager; + + // _participants[i].getMessagingService().registerMessageHandlerFactory( + // _factory.getMessageType(), _factory); + + manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager; } Message schedulerMessage = @@ -686,29 +690,47 @@ public void TestSchedulerMsg3() throws Exception { } Assert.assertEquals(messageResultCount, _PARTITIONS * 3 / 5); - int count = 0; - // System.out.println(i); - for (Set val : _factory._results.values()) { - // System.out.println(val); - count += val.size(); + + boolean success = false; + for (int j = 0; j < 6; j++) { + int count = 0; + // System.out.println(i); + for (Set val : _factory._results.values()) { + // System.out.println(val); + count += val.size(); + } + // System.out.println(count); + // Assert.assertEquals(count, _PARTITIONS * 3 / 5 * (i + 1)); + success = count == _PARTITIONS * 3 / 5 * (i + 1); + if (success) { + break; + } + Thread.sleep(500); } - // System.out.println(count); - Assert.assertEquals(count, _PARTITIONS * 3 / 5 * (i + 1)); + Assert.assertTrue(success); + } + } + + private int count(TestMessagingHandlerFactory factory) { + int cnt = 0; + for (Set val : factory._results.values()) { + cnt += val.size(); } + return cnt; } @Test() - public void TestSchedulerMsg4() throws Exception { + public void testSchedulerMsg4() throws Exception { _factory._results.clear(); HelixManager manager = null; for (int i = 0; i < NODE_NR; i++) { - String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[i].getMessagingService().registerMessageHandlerFactory( _factory.getMessageType(), _factory); - // - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( - _factory.getMessageType(), _factory); - manager = _startCMResultMap.get(hostDest)._manager; + + // _participants[i].getMessagingService().registerMessageHandlerFactory( + // _factory.getMessageType(), _factory); + + manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager; } Message schedulerMessage = @@ -719,9 +741,9 @@ public void TestSchedulerMsg4() throws Exception { schedulerMessage.setSrcName("CONTROLLER"); // Template for the individual message sent to each participant - Message msg = new Message(_factory.getMessageType(), "Template"); - msg.setTgtSessionId("*"); - msg.setMsgState(MessageState.NEW); + Message msgTemplate = new Message(_factory.getMessageType(), "Template"); + msgTemplate.setTgtSessionId("*"); + msgTemplate.setMsgState(MessageState.NEW); // Criteria to send individual messages Criteria cr = new Criteria(); @@ -741,7 +763,7 @@ public void TestSchedulerMsg4() throws Exception { String crString = sw.toString(); schedulerMessage.getRecord().setSimpleField("Criteria", crString); - schedulerMessage.getRecord().setMapField("MessageTemplate", msg.getRecord().getSimpleFields()); + schedulerMessage.getRecord().setMapField("MessageTemplate", msgTemplate.getRecord().getSimpleFields()); schedulerMessage.getRecord().setSimpleField("TIMEOUT", "-1"); schedulerMessage.getRecord().setSimpleField("WAIT_ALL", "true"); @@ -776,6 +798,15 @@ public void TestSchedulerMsg4() throws Exception { callback._message.getResultMap() .get(DefaultSchedulerMessageHandlerFactory.SCHEDULER_MSG_ID); + boolean success = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + return count(_factory) == 60; // TestDB number_of_partitions x replicas + } + }, 10 * 1000); + Assert.assertTrue(success, "If not specifying participant, controller will send 60 messages"); + HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor(); Builder keyBuilder = helixDataAccessor.keyBuilder(); ArrayList msgIds = new ArrayList(); @@ -792,11 +823,18 @@ public void TestSchedulerMsg4() throws Exception { crString = sw.toString(); schedulerMessage.getRecord().setSimpleField("Criteria", crString); manager.getMessagingService().sendAndWait(cr2, schedulerMessage, callback, -1); + + Thread.sleep(5000); + System.err.println("count: " + count(_factory)); + String msgId = callback._message.getResultMap().get( DefaultSchedulerMessageHandlerFactory.SCHEDULER_MSG_ID); msgIds.add(msgId); } + + // System.err.println("count: " + count(_factory)); + for (int i = 0; i < NODE_NR; i++) { String msgId = msgIds.get(i); for (int j = 0; j < 100; j++) { @@ -837,6 +875,8 @@ public void TestSchedulerMsg4() throws Exception { break; } } + + // Thread.sleep(5000); int count = 0; for (Set val : _factory._results.values()) { // System.out.println(val); @@ -847,18 +887,18 @@ public void TestSchedulerMsg4() throws Exception { } @Test - public void TestSchedulerMsgContraints() throws JsonGenerationException, JsonMappingException, + public void testSchedulerMsgContraints() throws JsonGenerationException, JsonMappingException, IOException, InterruptedException { TestMessagingHandlerFactoryLatch factory = new TestMessagingHandlerFactoryLatch(); HelixManager manager = null; for (int i = 0; i < NODE_NR; i++) { - String hostDest = "localhost_" + (START_PORT + i); - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + _participants[i].getMessagingService().registerMessageHandlerFactory( factory.getMessageType(), factory); - // - _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory( + + _participants[i].getMessagingService().registerMessageHandlerFactory( factory.getMessageType(), factory); - manager = _startCMResultMap.get(hostDest)._manager; + + manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager; } Message schedulerMessage = @@ -989,4 +1029,4 @@ public void TestSchedulerMsgContraints() throws JsonGenerationException, JsonMap ConstraintType.MESSAGE_CONSTRAINT, "constraint1"); } -} +} \ No newline at end of file diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchemataSM.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchemataSM.java index 3024f45757..a927520177 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestSchemataSM.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestSchemataSM.java @@ -27,10 +27,10 @@ import org.apache.helix.PropertyKey; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.tools.ClusterStateVerifier; @@ -46,7 +46,7 @@ public void testSchemataSM() throws Exception { String clusterName = className + "_" + methodName; int n = 5; - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); @@ -70,14 +70,15 @@ public void testSchemataSM() throws Exception { Arrays.asList(HelixConstants.StateModelToken.ANY_LIVEINSTANCE.toString())); accessor.setProperty(key, idealState); - ClusterController controller = new ClusterController(clusterName, "controller", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller"); controller.syncStart(); // start n-1 participants for (int i = 1; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -87,7 +88,7 @@ public void testSchemataSM() throws Exception { Assert.assertTrue(result); // start the remaining 1 participant - participants[0] = new MockParticipant(clusterName, "localhost_12918", ZK_ADDR, null); + participants[0] = new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12918"); participants[0].syncStart(); // make sure we have all participants in MASTER state @@ -107,6 +108,7 @@ public void testSchemataSM() throws Exception { } // clean up + controller.syncStop(); for (int i = 0; i < n; i++) { participants[i].syncStop(); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java b/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java index f38c6de6dd..4abb519669 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java @@ -22,41 +22,40 @@ import java.util.Date; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; import org.apache.helix.TestHelper; -import org.apache.helix.ZkHelixTestManager; import org.apache.helix.ZkTestHelper; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.mock.participant.MockTransition; import org.apache.helix.model.Message; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; -import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.annotations.Test; public class TestSessionExpiryInTransition extends ZkIntegrationTestBase { + private static Logger LOG = Logger.getLogger(TestSessionExpiryInTransition.class); public class SessionExpiryTransition extends MockTransition { private final AtomicBoolean _done = new AtomicBoolean(); @Override public void doTransition(Message message, NotificationContext context) { - ZkHelixTestManager manager = (ZkHelixTestManager) context.getManager(); + MockParticipantManager manager = (MockParticipantManager) context.getManager(); String instance = message.getTgtName(); - String partition = message.getPartitionName(); - if (instance.equals("localhost_12918") && partition.equals("TestDB0_1") // TestDB0_1 is SLAVE - // on localhost_12918 + PartitionId partition = message.getPartitionId(); + if (instance.equals("localhost_12918") && partition.toString().equals("TestDB0_1") // TestDB0_1 + // is SLAVE + // on localhost_12918 && _done.getAndSet(true) == false) { try { ZkTestHelper.expireSession(manager.getZkClient()); } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Exception expire zk-session", e); } } } @@ -64,7 +63,7 @@ public void doTransition(Message message, NotificationContext context) { @Test public void testSessionExpiryInTransition() throws Exception { - Logger.getRootLogger().setLevel(Level.WARN); + // Logger.getRootLogger().setLevel(Level.WARN); String className = TestHelper.getTestClassName(); String methodName = TestHelper.getTestMethodName(); @@ -72,7 +71,7 @@ public void testSessionExpiryInTransition() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port "localhost", // participant name prefix @@ -84,15 +83,15 @@ public void testSessionExpiryInTransition() throws Exception { "MasterSlave", true); // do rebalance // start controller - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - ZkHelixTestManager manager = - new ZkHelixTestManager(clusterName, instanceName, InstanceType.PARTICIPANT, ZK_ADDR); - participants[i] = new MockParticipant(manager, new SessionExpiryTransition()); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].setTransition(new SessionExpiryTransition()); participants[i].syncStart(); } @@ -102,13 +101,11 @@ public void testSessionExpiryInTransition() throws Exception { Assert.assertTrue(result); // clean up + controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); } - Thread.sleep(2000); - controller.syncStop(); - System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMMain.java b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMMain.java index 02a34d8af6..5ea912e895 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMMain.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMMain.java @@ -21,9 +21,15 @@ import java.util.Date; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; import org.apache.helix.TestHelper; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.TestHelper.Verifier; +import org.apache.helix.ZNRecord; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.model.Leader; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.log4j.Logger; import org.testng.Assert; @@ -35,16 +41,33 @@ public class TestStandAloneCMMain extends ZkStandAloneCMTestBase { @Test() public void testStandAloneCMMain() throws Exception { logger.info("RUN testStandAloneCMMain() at " + new Date(System.currentTimeMillis())); - + ClusterControllerManager newController = null; for (int i = 1; i <= 2; i++) { String controllerName = "controller_" + i; - StartCMResult startResult = - TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR, - HelixControllerMain.STANDALONE); - _startCMResultMap.put(controllerName, startResult); + newController = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName); + newController.syncStart(); } - stopCurrentLeader(_zkClient, CLUSTER_NAME, _startCMResultMap); + // stopCurrentLeader(_zkClient, CLUSTER_NAME, _startCMResultMap); + _controller.syncStop(); + + final HelixDataAccessor accessor = + new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor(_gZkClient)); + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + final String newControllerName = newController.getInstanceName(); + TestHelper.verify(new Verifier() { + + @Override + public boolean verify() throws Exception { + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); + if (leader == null) { + return false; + } + return leader.getInstanceName().equals(newControllerName); + + } + }, 30 * 1000); + boolean result = ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( ZK_ADDR, CLUSTER_NAME)); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java index 347ff7e512..dad998de3c 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java @@ -21,11 +21,10 @@ import java.util.Date; -import org.apache.helix.InstanceType; import org.apache.helix.TestHelper; -import org.apache.helix.ZkHelixTestManager; import org.apache.helix.ZkTestHelper; -import org.apache.helix.mock.participant.MockParticipant; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.log4j.Logger; @@ -47,18 +46,16 @@ public void testStandAloneCMSessionExpiry() throws Exception { TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, PARTICIPANT_PREFIX, "TestDB", 1, 20, 5, 3, "MasterSlave", true); - MockParticipant[] participants = new MockParticipant[5]; + MockParticipantManager[] participants = new MockParticipantManager[5]; for (int i = 0; i < 5; i++) { String instanceName = "localhost_" + (12918 + i); - ZkHelixTestManager manager = - new ZkHelixTestManager(clusterName, instanceName, InstanceType.PARTICIPANT, ZK_ADDR); - participants[i] = new MockParticipant(manager, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } - ZkHelixTestManager controller = - new ZkHelixTestManager(clusterName, "controller_0", InstanceType.CONTROLLER, ZK_ADDR); - controller.connect(); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); boolean result; result = @@ -67,7 +64,7 @@ public void testStandAloneCMSessionExpiry() throws Exception { Assert.assertTrue(result); // participant session expiry - ZkHelixTestManager participantToExpire = (ZkHelixTestManager) participants[1].getManager(); + MockParticipantManager participantToExpire = participants[1]; System.out.println("Expire participant session"); String oldSessionId = participantToExpire.getSessionId(); @@ -107,8 +104,7 @@ public void testStandAloneCMSessionExpiry() throws Exception { // clean up System.out.println("Clean up ..."); // Logger.getRootLogger().setLevel(Level.DEBUG); - controller.disconnect(); - Thread.sleep(100); + controller.syncStop(); for (int i = 0; i < 5; i++) { participants[i].syncStop(); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java b/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java index dce3fd40db..d191c189d6 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java @@ -25,7 +25,7 @@ import org.apache.helix.PropertyType; import org.apache.helix.TestHelper; import org.apache.helix.ZkTestHelper; -import org.apache.helix.mock.controller.ClusterController; +import org.apache.helix.integration.manager.ClusterControllerManager; import org.apache.helix.model.IdealState.RebalanceMode; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -54,10 +54,10 @@ public void test() throws Exception { // rebalance // start controller - ClusterController[] controllers = new ClusterController[4]; + ClusterControllerManager[] controllers = new ClusterControllerManager[4]; for (int i = 0; i < 4; i++) { - controllers[i] = new ClusterController(clusterName, "controller_0", ZK_ADDR); - controllers[i].start(); + controllers[i] = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controllers[i].syncStart(); } Thread.sleep(500); // wait leader election finishes @@ -69,7 +69,6 @@ public void test() throws Exception { // clean up for (int i = 0; i < 4; i++) { controllers[i].syncStop(); - Thread.sleep(1000); // wait for all zk callbacks done } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java index edc10c6dfd..318177a005 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java @@ -28,22 +28,19 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.NotificationContext; -import org.apache.helix.TestHelper; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.manager.zk.ZNRecordSerializer; -import org.apache.helix.manager.zk.ZkClient; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.messaging.handling.MessageHandler.ErrorCode; -import org.apache.helix.mock.participant.MockJobIntf; import org.apache.helix.mock.participant.MockMSStateModel; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.mock.participant.MockTransition; import org.apache.helix.mock.participant.SleepTransition; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.model.Message; -import org.apache.helix.participant.statemachine.StateModel; import org.apache.helix.participant.statemachine.StateModelFactory; import org.apache.helix.participant.statemachine.StateModelInfo; import org.apache.helix.participant.statemachine.StateTransitionError; @@ -59,15 +56,14 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase { private static Logger LOG = Logger.getLogger(TestStateTransitionTimeout.class); + @Override @BeforeClass public void beforeClass() throws Exception { System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); String namespace = "/" + CLUSTER_NAME; - if (_zkClient.exists(namespace)) { - _zkClient.deleteRecursive(namespace); + if (_gZkClient.exists(namespace)) { + _gZkClient.deleteRecursive(namespace); } _setupTool = new ClusterSetup(ZK_ADDR); @@ -106,12 +102,14 @@ public TimeOutStateModel(MockTransition transition, boolean sleep) { _sleep = sleep; } + @Override @Transition(to = "SLAVE", from = "OFFLINE") public void onBecomeSlaveFromOffline(Message message, NotificationContext context) { LOG.info("Become SLAVE from OFFLINE"); } + @Override @Transition(to = "MASTER", from = "SLAVE") public void onBecomeMasterFromSlave(Message message, NotificationContext context) throws InterruptedException { @@ -121,23 +119,27 @@ public void onBecomeMasterFromSlave(Message message, NotificationContext context } } + @Override @Transition(to = "SLAVE", from = "MASTER") public void onBecomeSlaveFromMaster(Message message, NotificationContext context) { LOG.info("Become SLAVE from MASTER"); } + @Override @Transition(to = "OFFLINE", from = "SLAVE") public void onBecomeOfflineFromSlave(Message message, NotificationContext context) { LOG.info("Become OFFLINE from SLAVE"); } + @Override @Transition(to = "DROPPED", from = "OFFLINE") public void onBecomeDroppedFromOffline(Message message, NotificationContext context) { LOG.info("Become DROPPED from OFFLINE"); } + @Override public void rollbackOnError(Message message, NotificationContext context, StateTransitionError error) { _error = error; @@ -171,40 +173,41 @@ public TimeOutStateModel createNewStateModel(String stateUnitKey) { @Test public void testStateTransitionTimeOut() throws Exception { Map factories = new HashMap(); - MockParticipant[] participants = new MockParticipant[NODE_NR]; + // MockParticipantManager[] participants = new MockParticipantManager[NODE_NR]; IdealState idealState = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB); for (int i = 0; i < NODE_NR; i++) { String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); SleepStateModelFactory factory = new SleepStateModelFactory(1000); factories.put(instanceName, factory); - for (String p : idealState.getPartitionSet()) { - if (idealState.getPreferenceList(p).get(0).equals(instanceName)) { - factory.addPartition(p); + for (PartitionId p : idealState.getPartitionIdSet()) { + if (idealState.getPreferenceList(p).get(0).equals(ParticipantId.from(instanceName))) { + factory.addPartition(p.stringify()); } } - participants[i] = new MockParticipant(factory, CLUSTER_NAME, instanceName, ZK_ADDR, null); - participants[i].syncStart(); + _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName); + _participants[i].getStateMachineEngine().registerStateModelFactory("MasterSlave", factory); + _participants[i].syncStart(); } String controllerName = CONTROLLER_PREFIX + "_0"; - StartCMResult startResult = - TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR, - HelixControllerMain.STANDALONE); - _startCMResultMap.put(controllerName, startResult); + _controller = + new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName); + _controller.syncStart(); + boolean result = ClusterStateVerifier .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME)); Assert.assertTrue(result); - HelixDataAccessor accessor = participants[0].getManager().getHelixDataAccessor(); + HelixDataAccessor accessor = _participants[0].getHelixDataAccessor(); Builder kb = accessor.keyBuilder(); ExternalView ev = accessor.getProperty(kb.externalView(TEST_DB)); - for (String p : idealState.getPartitionSet()) { - String idealMaster = idealState.getPreferenceList(p).get(0); - Assert.assertTrue(ev.getStateMap(p).get(idealMaster).equals("ERROR")); + for (PartitionId p : idealState.getPartitionIdSet()) { + ParticipantId idealMaster = idealState.getPreferenceList(p).get(0); + Assert.assertTrue(ev.getStateMap(p).get(idealMaster).equals(State.from("ERROR"))); - TimeOutStateModel model = factories.get(idealMaster).getStateModel(p); + TimeOutStateModel model = factories.get(idealMaster.stringify()).getStateModel(p.stringify()); Assert.assertEquals(model._errorCallcount, 1); Assert.assertEquals(model._error.getCode(), ErrorCode.TIMEOUT); } diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSwapInstance.java b/helix-core/src/test/java/org/apache/helix/integration/TestSwapInstance.java index a1f63aa9b3..6cce71649b 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestSwapInstance.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestSwapInstance.java @@ -21,9 +21,8 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; -import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.TestHelper.StartCMResult; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.helix.model.IdealState; import org.apache.helix.tools.ClusterStateVerifier; @@ -33,8 +32,7 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase { @Test public void TestSwap() throws Exception { - String controllerName = CONTROLLER_PREFIX + "_0"; - HelixManager manager = _startCMResultMap.get(controllerName)._manager; + HelixManager manager = _controller; HelixDataAccessor helixAccessor = manager.getHelixDataAccessor(); _setupTool.addResourceToCluster(CLUSTER_NAME, "MyDB", 64, STATE_MODEL); _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", _replica); @@ -49,7 +47,7 @@ public void TestSwap() throws Exception { idealStateOld2.merge(is2.getRecord()); String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0); - ZKHelixAdmin tool = new ZKHelixAdmin(_zkClient); + ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient); _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, instanceName, false); boolean result = @@ -68,8 +66,7 @@ public void TestSwap() throws Exception { } Assert.assertTrue(exception); - _startCMResultMap.get(instanceName)._manager.disconnect(); - _startCMResultMap.get(instanceName)._thread.interrupt(); + _participants[0].syncStop(); Thread.sleep(1000); exception = false; @@ -80,8 +77,9 @@ public void TestSwap() throws Exception { exception = true; } Assert.assertFalse(exception); - StartCMResult result2 = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName2); - _startCMResultMap.put(instanceName2, result2); + MockParticipantManager newParticipant = + new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName2); + newParticipant.syncStart(); result = ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier( diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestUserDefRebalancerCompatibility.java b/helix-core/src/test/java/org/apache/helix/integration/TestUserDefRebalancerCompatibility.java new file mode 100644 index 0000000000..27004fe94f --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/integration/TestUserDefRebalancerCompatibility.java @@ -0,0 +1,104 @@ +package org.apache.helix.integration; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixManager; +import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.controller.rebalancer.Rebalancer; +import org.apache.helix.controller.stages.ClusterDataCache; +import org.apache.helix.controller.stages.CurrentStateOutput; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; +import org.apache.helix.model.IdealState.IdealStateProperty; +import org.apache.helix.tools.ClusterStateVerifier; +import org.testng.Assert; +import org.testng.annotations.Test; + +@SuppressWarnings("deprecation") +public class TestUserDefRebalancerCompatibility extends + ZkStandAloneCMTestBaseWithPropertyServerCheck { + String db2 = TEST_DB + "2"; + static boolean testRebalancerCreated = false; + static boolean testRebalancerInvoked = false; + + public static class TestRebalancer implements Rebalancer { + @Override + public void init(HelixManager helixManager) { + testRebalancerCreated = true; + } + + /** + * Very basic mapping that evenly assigns one replica of each partition to live nodes, each of + * which is in the highest-priority state. + */ + @Override + public IdealState computeResourceMapping(String resourceName, IdealState currentIdealState, + CurrentStateOutput currentStateOutput, ClusterDataCache clusterData) { + testRebalancerInvoked = true; + for (String partition : currentIdealState.getPartitionSet()) { + String instance = currentIdealState.getPreferenceList(partition).get(0); + currentIdealState.getPreferenceList(partition).clear(); + currentIdealState.getPreferenceList(partition).add(instance); + + currentIdealState.getInstanceStateMap(partition).clear(); + currentIdealState.getInstanceStateMap(partition).put(instance, "MASTER"); + } + currentIdealState.setReplicas("1"); + return currentIdealState; + } + } + + @Test + public void testCustomizedIdealStateRebalancer() throws InterruptedException { + _setupTool.addResourceToCluster(CLUSTER_NAME, db2, 60, "MasterSlave"); + _setupTool.addResourceProperty(CLUSTER_NAME, db2, + IdealStateProperty.REBALANCER_CLASS_NAME.toString(), + TestUserDefRebalancerCompatibility.TestRebalancer.class.getName()); + + _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 3); + + boolean result = + ClusterStateVerifier + .verifyByZkCallback(new TestCustomizedIdealStateRebalancer.ExternalViewBalancedVerifier( + _gZkClient, CLUSTER_NAME, db2)); + Assert.assertTrue(result); + Thread.sleep(1000); + HelixDataAccessor accessor = + new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor(_gZkClient)); + Builder keyBuilder = accessor.keyBuilder(); + ExternalView ev = accessor.getProperty(keyBuilder.externalView(db2)); + Assert.assertEquals(ev.getPartitionSet().size(), 60); + for (String partition : ev.getPartitionSet()) { + Assert.assertEquals(ev.getStateMap(partition).size(), 1); + } + IdealState is = accessor.getProperty(keyBuilder.idealStates(db2)); + for (PartitionId partition : is.getPartitionIdSet()) { + Assert.assertEquals(is.getPreferenceList(partition).size(), 3); + Assert.assertEquals(is.getParticipantStateMap(partition).size(), 3); + } + Assert.assertTrue(testRebalancerCreated); + Assert.assertTrue(testRebalancerInvoked); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java b/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java index 347fcb630f..00537a4b92 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java @@ -26,11 +26,17 @@ import org.I0Itec.zkclient.IZkChildListener; import org.I0Itec.zkclient.IZkDataListener; -import org.apache.helix.*; +import org.apache.helix.CurrentStateChangeListener; +import org.apache.helix.NotificationContext; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZkTestHelper; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; +import org.apache.helix.integration.manager.ZkTestManager; import org.apache.helix.manager.zk.CallbackHandler; import org.apache.helix.manager.zk.ZkClient; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.CurrentState; import org.apache.helix.tools.ClusterStateVerifier; import org.testng.Assert; @@ -57,15 +63,16 @@ public void testCbHandlerLeakOnParticipantSessionExpiry() throws Exception { 2, // replicas "MasterSlave", true); // do rebalance - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + final ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -74,9 +81,7 @@ public void testCbHandlerLeakOnParticipantSessionExpiry() throws Exception { .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, clusterName)); Assert.assertTrue(result); - final ZkHelixTestManager controllerManager = controller.getManager(); - final ZkHelixTestManager participantManagerToExpire = - (ZkHelixTestManager) participants[1].getManager(); + final MockParticipantManager participantManagerToExpire = participants[1]; // check controller zk-watchers result = TestHelper.verify(new TestHelper.Verifier() { @@ -84,7 +89,8 @@ public void testCbHandlerLeakOnParticipantSessionExpiry() throws Exception { @Override public boolean verify() throws Exception { Map> watchers = ZkTestHelper.getListenersBySession(ZK_ADDR); - Set watchPaths = watchers.get("0x" + controllerManager.getSessionId()); + // Set watchPaths = watchers.get("0x" + controllerManager.getSessionId()); + Set watchPaths = watchers.get("0x" + controller.getSessionId()); // System.out.println("controller watch paths: " + watchPaths); // controller should have 5 + 2n + m + (m+2)n zk-watchers @@ -112,7 +118,7 @@ public boolean verify() throws Exception { // check HelixManager#_handlers // printHandlers(controllerManager); // printHandlers(participantManagerToExpire); - int controllerHandlerNb = controllerManager.getHandlers().size(); + int controllerHandlerNb = controller.getHandlers().size(); int particHandlerNb = participantManagerToExpire.getHandlers().size(); Assert.assertEquals(controllerHandlerNb, 9, "HelixController should have 9 (5+2n) callback handlers for 2 (n) participant"); @@ -139,7 +145,7 @@ public boolean verify() throws Exception { @Override public boolean verify() throws Exception { Map> watchers = ZkTestHelper.getListenersBySession(ZK_ADDR); - Set watchPaths = watchers.get("0x" + controllerManager.getSessionId()); + Set watchPaths = watchers.get("0x" + controller.getSessionId()); // System.out.println("controller watch paths after session expiry: " + watchPaths); // controller should have 5 + 2n + m + (m+2)n zk-watchers @@ -167,13 +173,19 @@ public boolean verify() throws Exception { // check handlers // printHandlers(controllerManager); // printHandlers(participantManagerToExpire); - int handlerNb = controllerManager.getHandlers().size(); + int handlerNb = controller.getHandlers().size(); Assert.assertEquals(handlerNb, controllerHandlerNb, "controller callback handlers should not increase after participant session expiry"); handlerNb = participantManagerToExpire.getHandlers().size(); Assert.assertEquals(handlerNb, particHandlerNb, "participant callback handlers should not increase after participant session expiry"); + // clean up + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } @@ -196,15 +208,16 @@ public void testCbHandlerLeakOnControllerSessionExpiry() throws Exception { 2, // replicas "MasterSlave", true); // do rebalance - ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR); + final ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); controller.syncStart(); // start participants - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } @@ -213,15 +226,16 @@ public void testCbHandlerLeakOnControllerSessionExpiry() throws Exception { .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, clusterName)); Assert.assertTrue(result); - final ZkHelixTestManager controllerManager = controller.getManager(); - final ZkHelixTestManager participantManager = participants[0].getManager(); + // final ZkHelixTestManager controllerManager = controller.getManager(); + // final ZkHelixTestManager participantManager = participants[0].getManager(); + final MockParticipantManager participantManager = participants[0]; // wait until we get all the listeners registered result = TestHelper.verify(new TestHelper.Verifier() { @Override public boolean verify() throws Exception { - int controllerHandlerNb = controllerManager.getHandlers().size(); + int controllerHandlerNb = controller.getHandlers().size(); int particHandlerNb = participantManager.getHandlers().size(); if (controllerHandlerNb == 9 && particHandlerNb == 2) return true; @@ -230,21 +244,21 @@ public boolean verify() throws Exception { } }, 1000); - int controllerHandlerNb = controllerManager.getHandlers().size(); + int controllerHandlerNb = controller.getHandlers().size(); int particHandlerNb = participantManager.getHandlers().size(); Assert.assertEquals(controllerHandlerNb, 9, "HelixController should have 9 (5+2n) callback handlers for 2 participant, but was " - + controllerHandlerNb + ", " + printHandlers(controllerManager)); + + controllerHandlerNb + ", " + printHandlers(controller)); Assert.assertEquals(particHandlerNb, 2, "HelixParticipant should have 2 (msg+cur-state) callback handlers, but was " + particHandlerNb + ", " + printHandlers(participantManager)); // expire controller System.out.println("Expiring controller session..."); - String oldSessionId = controllerManager.getSessionId(); + String oldSessionId = controller.getSessionId(); - ZkTestHelper.expireSession(controllerManager.getZkClient()); - String newSessionId = controllerManager.getSessionId(); + ZkTestHelper.expireSession(controller.getZkClient()); + String newSessionId = controller.getSessionId(); System.out.println("Expired controller session. oldSessionId: " + oldSessionId + ", newSessionId: " + newSessionId); @@ -259,7 +273,7 @@ public boolean verify() throws Exception { @Override public boolean verify() throws Exception { Map> watchers = ZkTestHelper.getListenersBySession(ZK_ADDR); - Set watchPaths = watchers.get("0x" + controllerManager.getSessionId()); + Set watchPaths = watchers.get("0x" + controller.getSessionId()); // System.out.println("controller watch paths after session expiry: " + watchPaths); // controller should have 5 + 2n + m + (m+2)n zk-watchers @@ -286,15 +300,21 @@ public boolean verify() throws Exception { // check HelixManager#_handlers // printHandlers(controllerManager); - int handlerNb = controllerManager.getHandlers().size(); + int handlerNb = controller.getHandlers().size(); Assert.assertEquals(handlerNb, controllerHandlerNb, "controller callback handlers should not increase after participant session expiry, but was " - + printHandlers(controllerManager)); + + printHandlers(controller)); handlerNb = participantManager.getHandlers().size(); Assert.assertEquals(handlerNb, particHandlerNb, "participant callback handlers should not increase after participant session expiry, but was " + printHandlers(participantManager)); + // clean up + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } @@ -313,18 +333,20 @@ public void testRemoveUserCbHandlerOnPathRemoval() throws Exception { 2, // replicas "MasterSlave", true); - ClusterController controller = new ClusterController(clusterName, "controller_0", zkAddr); + final ClusterControllerManager controller = + new ClusterControllerManager(zkAddr, clusterName, "controller_0"); controller.syncStart(); - MockParticipant[] participants = new MockParticipant[n]; + MockParticipantManager[] participants = new MockParticipantManager[n]; for (int i = 0; i < n; i++) { String instanceName = "localhost_" + (12918 + i); - participants[i] = new MockParticipant(clusterName, instanceName, zkAddr, null); + participants[i] = new MockParticipantManager(zkAddr, clusterName, instanceName); participants[i].syncStart(); // register a controller listener on participant_0 if (i == 0) { - ZkHelixTestManager manager = participants[0].getManager(); + // ZkHelixTestManager manager = participants[0].getManager(); + MockParticipantManager manager = participants[0]; manager.addCurrentStateChangeListener(new CurrentStateChangeListener() { @Override public void onStateChange(String instanceName, List statesInfo, @@ -343,7 +365,7 @@ public void onStateChange(String instanceName, List statesInfo, clusterName)); Assert.assertTrue(result); - ZkHelixTestManager participantToExpire = participants[0].getManager(); + MockParticipantManager participantToExpire = participants[0]; String oldSessionId = participantToExpire.getSessionId(); PropertyKey.Builder keyBuilder = new PropertyKey.Builder(clusterName); @@ -468,11 +490,18 @@ public void onStateChange(String instanceName, List statesInfo, "Should have no exist-watches. exist-watches on CURRENTSTATE/{oldSessionId} and CURRENTSTATE/{oldSessionId}/TestDB0 should be cleared during handleNewSession"); // Thread.sleep(1000); + + // clean up + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } // debug code - static String printHandlers(ZkHelixTestManager manager) { + static String printHandlers(ZkTestManager manager) { StringBuilder sb = new StringBuilder(); List handlers = manager.getHandlers(); sb.append(manager.getInstanceName() + " has " + handlers.size() + " cb-handlers. ["); diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestZkReconnect.java b/helix-core/src/test/java/org/apache/helix/integration/TestZkReconnect.java index cc6c0b50f3..5ce6664068 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/TestZkReconnect.java +++ b/helix-core/src/test/java/org/apache/helix/integration/TestZkReconnect.java @@ -31,6 +31,11 @@ import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; import org.apache.helix.TestHelper; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelFactoryId; import org.apache.helix.model.IdealState; import org.apache.helix.model.Message; import org.apache.helix.participant.statemachine.StateModel; @@ -96,8 +101,9 @@ public StateModel createNewStateModel(String stateUnitKey) { IdealState idealState = helixAdmin.getResourceIdealState(clusterName, resourceName); idealState.setReplicas("1"); - idealState.setStateModelFactoryName("test"); - idealState.setPartitionState(resourceName + "_0", instanceId, "ONLINE"); + idealState.setStateModelFactoryId(StateModelFactoryId.from("test")); + idealState.setPartitionState(PartitionId.from(ResourceId.from(resourceName), "0"), + ParticipantId.from(instanceId), State.from("ONLINE")); LOG.info("Shutdown ZK server"); TestHelper.stopZkServer(zkServerRef.get()); diff --git a/helix-core/src/test/java/org/apache/helix/integration/ZkIntegrationTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/ZkIntegrationTestBase.java index 2ab0aafc5a..65d6597e9c 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/ZkIntegrationTestBase.java +++ b/helix-core/src/test/java/org/apache/helix/integration/ZkIntegrationTestBase.java @@ -19,25 +19,23 @@ * under the License. */ -import java.util.Map; import java.util.logging.Level; import org.I0Itec.zkclient.ZkServer; import org.apache.helix.ConfigAccessor; -import org.apache.helix.model.ConfigScope; -import org.apache.helix.model.builder.ConfigScopeBuilder; -import org.apache.helix.TestHelper; import org.apache.helix.PropertyKey.Builder; -import org.apache.helix.TestHelper.StartCMResult; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; 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.manager.zk.ZkClient; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.ConfigScope; +import org.apache.helix.model.Leader; +import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.util.ZKClientPool; import org.apache.log4j.Logger; -import org.testng.Assert; import org.testng.AssertJUnit; import org.testng.annotations.AfterSuite; import org.testng.annotations.BeforeSuite; @@ -85,59 +83,16 @@ protected String getShortClassName() { protected String getCurrentLeader(ZkClient zkClient, String clusterName) { ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient)); Builder keyBuilder = accessor.keyBuilder(); - LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); if (leader == null) { return null; } return leader.getInstanceName(); } - /** - * Stop current leader and returns the new leader - * @param zkClient - * @param clusterName - * @param startCMResultMap - * @return - */ - protected String stopCurrentLeader(ZkClient zkClient, String clusterName, - Map startCMResultMap) { - String leader = getCurrentLeader(zkClient, clusterName); - Assert.assertTrue(leader != null); - System.out.println("stop leader: " + leader + " in " + clusterName); - Assert.assertTrue(leader != null); - - StartCMResult result = startCMResultMap.remove(leader); - Assert.assertTrue(result._manager != null); - result._manager.disconnect(); - - Assert.assertTrue(result._thread != null); - result._thread.interrupt(); - - boolean isNewLeaderElected = false; - String newLeader = null; - try { - for (int i = 0; i < 5; i++) { - Thread.sleep(1000); - newLeader = getCurrentLeader(zkClient, clusterName); - if (!newLeader.equals(leader)) { - isNewLeaderElected = true; - System.out.println("new leader elected: " + newLeader + " in " + clusterName); - break; - } - } - } catch (InterruptedException e) { - e.printStackTrace(); - } - if (isNewLeaderElected == false) { - System.out.println("fail to elect a new leader in " + clusterName); - } - AssertJUnit.assertTrue(isNewLeaderElected); - return newLeader; - } - protected void enableHealthCheck(String clusterName) { ConfigScope scope = new ConfigScopeBuilder().forCluster(clusterName).build(); new ConfigAccessor(_gZkClient).set(scope, "healthChange" + ".enabled", "" + true); diff --git a/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBase.java index e759fc768a..5d169d5021 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBase.java +++ b/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBase.java @@ -20,16 +20,9 @@ */ import java.util.Date; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.helix.TestHelper; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.manager.zk.ZNRecordSerializer; -import org.apache.helix.manager.zk.ZkClient; + +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.tools.ClusterSetup; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; @@ -57,8 +50,8 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase { protected final String CLASS_NAME = getShortClassName(); protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME; - protected Map _startCMResultMap = new HashMap(); - protected ZkClient _zkClient; + protected MockParticipantManager[] _participants = new MockParticipantManager[NODE_NR]; + protected ClusterControllerManager _controller; int _replica = 3; @@ -67,11 +60,9 @@ public void beforeClass() throws Exception { // Logger.getRootLogger().setLevel(Level.INFO); System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); - _zkClient = new ZkClient(ZK_ADDR); - _zkClient.setZkSerializer(new ZNRecordSerializer()); String namespace = "/" + CLUSTER_NAME; - if (_zkClient.exists(namespace)) { - _zkClient.deleteRecursive(namespace); + if (_gZkClient.exists(namespace)) { + _gZkClient.deleteRecursive(namespace); } _setupTool = new ClusterSetup(ZK_ADDR); @@ -87,21 +78,14 @@ public void beforeClass() throws Exception { // start dummy participants for (int i = 0; i < NODE_NR; i++) { String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - if (_startCMResultMap.get(instanceName) != null) { - LOG.error("fail to start particpant:" + instanceName - + "(participant with same name already exists)"); - } else { - StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName); - _startCMResultMap.put(instanceName, result); - } + _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName); + _participants[i].syncStart(); } // start controller String controllerName = CONTROLLER_PREFIX + "_0"; - StartCMResult startResult = - TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR, - HelixControllerMain.STANDALONE); - _startCMResultMap.put(controllerName, startResult); + _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName); + _controller.syncStart(); boolean result = ClusterStateVerifier @@ -119,30 +103,11 @@ public void afterClass() throws Exception { * shutdown order: 1) disconnect the controller 2) disconnect participants */ - StartCMResult result; - Iterator> it = _startCMResultMap.entrySet().iterator(); - while (it.hasNext()) { - String instanceName = it.next().getKey(); - if (instanceName.startsWith(CONTROLLER_PREFIX)) { - result = _startCMResultMap.get(instanceName); - result._manager.disconnect(); - result._thread.interrupt(); - it.remove(); - } - } - - Thread.sleep(100); - it = _startCMResultMap.entrySet().iterator(); - while (it.hasNext()) { - String instanceName = it.next().getKey(); - result = _startCMResultMap.get(instanceName); - result._manager.disconnect(); - result._thread.interrupt(); - it.remove(); + _controller.syncStop(); + for (int i = 0; i < NODE_NR; i++) { + _participants[i].syncStop(); } - _zkClient.close(); - // logger.info("END at " + new Date(System.currentTimeMillis())); System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis())); } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBaseWithPropertyServerCheck.java b/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBaseWithPropertyServerCheck.java index f19e5dd7de..c6fbea6adf 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBaseWithPropertyServerCheck.java +++ b/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBaseWithPropertyServerCheck.java @@ -23,8 +23,11 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; import org.apache.helix.controller.restlet.ZKPropertyTransferServer; import org.apache.helix.controller.restlet.ZkPropertyTransferClient; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.model.StatusUpdate; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -36,6 +39,7 @@ */ public class ZkStandAloneCMTestBaseWithPropertyServerCheck extends ZkStandAloneCMTestBase { + @Override @BeforeClass public void beforeClass() throws Exception { ZKPropertyTransferServer.PERIOD = 500; @@ -44,19 +48,20 @@ public void beforeClass() throws Exception { super.beforeClass(); Thread.sleep(1000); + HelixDataAccessor accessor = + new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor(_gZkClient)); + Builder kb = accessor.keyBuilder(); + for (int i = 0; i < NODE_NR; i++) { - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - if (_startCMResultMap.get(instanceName) != null) { - HelixDataAccessor accessor = - _startCMResultMap.get(instanceName)._manager.getHelixDataAccessor(); - Builder kb = accessor.keyBuilder(); - List statusUpdates = - accessor.getChildValues(kb.stateTransitionStatus(instanceName, - _startCMResultMap.get(instanceName)._manager.getSessionId(), TEST_DB)); + String instanceName = _participants[i].getInstanceName(); + List statusUpdates = + accessor.getChildValues(kb.stateTransitionStatus(instanceName, + _participants[i].getSessionId(), TEST_DB)); + for (int j = 0; j < 10; j++) { statusUpdates = accessor.getChildValues(kb.stateTransitionStatus(instanceName, - _startCMResultMap.get(instanceName)._manager.getSessionId(), TEST_DB)); + _participants[i].getSessionId(), TEST_DB)); if (statusUpdates.size() == 0) { Thread.sleep(500); } else { @@ -70,10 +75,10 @@ public void beforeClass() throws Exception { Assert .assertTrue(update.getRecord().getSimpleField(ZKPropertyTransferServer.SERVER) != null); } - } } } + @Override @AfterClass public void afterClass() throws Exception { super.afterClass(); diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterControllerManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterControllerManager.java index e0da9fb8b6..b8f0f2bbb9 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterControllerManager.java +++ b/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterControllerManager.java @@ -22,12 +22,14 @@ import java.util.List; import java.util.concurrent.CountDownLatch; +import org.apache.helix.HelixTimerTask; +import org.apache.helix.InstanceType; import org.apache.helix.manager.zk.CallbackHandler; -import org.apache.helix.manager.zk.ControllerManager; +import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.manager.zk.ZkClient; import org.apache.log4j.Logger; -public class ClusterControllerManager extends ControllerManager implements Runnable, ZkTestManager { +public class ClusterControllerManager extends ZKHelixManager implements Runnable, ZkTestManager { private static Logger LOG = Logger.getLogger(ClusterControllerManager.class); private final CountDownLatch _startCountDown = new CountDownLatch(1); @@ -35,7 +37,7 @@ public class ClusterControllerManager extends ControllerManager implements Runna private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1); public ClusterControllerManager(String zkAddr, String clusterName, String controllerName) { - super(zkAddr, clusterName, controllerName); + super(clusterName, controllerName, InstanceType.CONTROLLER, zkAddr); } public void syncStop() { @@ -43,8 +45,7 @@ public void syncStop() { try { _waitStopFinishCountDown.await(); } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Interrupted waiting for finish", e); } } @@ -54,8 +55,7 @@ public void syncStart() { try { _startCountDown.await(); } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Interrupted waiting for start", e); } } @@ -84,4 +84,7 @@ public List getHandlers() { return _handlers; } + public List getControllerTimerTasks() { + return _controllerTimerTasks; + } } diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterDistributedController.java b/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterDistributedController.java index 751c3cb8e3..44d095793a 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterDistributedController.java +++ b/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterDistributedController.java @@ -22,12 +22,15 @@ import java.util.List; import java.util.concurrent.CountDownLatch; +import org.apache.helix.InstanceType; import org.apache.helix.manager.zk.CallbackHandler; -import org.apache.helix.manager.zk.DistributedControllerManager; +import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.manager.zk.ZkClient; +import org.apache.helix.participant.DistClusterControllerStateModelFactory; +import org.apache.helix.participant.StateMachineEngine; import org.apache.log4j.Logger; -public class ClusterDistributedController extends DistributedControllerManager implements Runnable, +public class ClusterDistributedController extends ZKHelixManager implements Runnable, ZkTestManager { private static Logger LOG = Logger.getLogger(ClusterDistributedController.class); @@ -36,7 +39,7 @@ public class ClusterDistributedController extends DistributedControllerManager i private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1); public ClusterDistributedController(String zkAddr, String clusterName, String controllerName) { - super(zkAddr, clusterName, controllerName); + super(clusterName, controllerName, InstanceType.CONTROLLER_PARTICIPANT, zkAddr); } public void syncStop() { @@ -44,8 +47,7 @@ public void syncStop() { try { _waitStopFinishCountDown.await(); } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Interrupted waiting for finish", e); } } @@ -55,14 +57,18 @@ public void syncStart() { try { _startCountDown.await(); } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Interrupted waiting for start", e); } } @Override public void run() { try { + StateMachineEngine stateMach = getStateMachineEngine(); + DistClusterControllerStateModelFactory lsModelFactory = + new DistClusterControllerStateModelFactory(_zkAddress); + stateMach.registerStateModelFactory("LeaderStandby", lsModelFactory); + connect(); _startCountDown.countDown(); _stopCountDown.await(); diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java index 8249f4af93..34efe3418d 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java +++ b/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java @@ -22,18 +22,20 @@ import java.util.List; import java.util.concurrent.CountDownLatch; +import org.apache.helix.InstanceType; import org.apache.helix.manager.zk.CallbackHandler; -import org.apache.helix.manager.zk.ParticipantManager; +import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.mock.participant.DummyProcess.DummyLeaderStandbyStateModelFactory; import org.apache.helix.mock.participant.DummyProcess.DummyOnlineOfflineStateModelFactory; +import org.apache.helix.mock.participant.MockJobIntf; import org.apache.helix.mock.participant.MockMSModelFactory; import org.apache.helix.mock.participant.MockSchemataModelFactory; import org.apache.helix.mock.participant.MockTransition; import org.apache.helix.participant.StateMachineEngine; import org.apache.log4j.Logger; -public class MockParticipantManager extends ParticipantManager implements Runnable, ZkTestManager { +public class MockParticipantManager extends ZKHelixManager implements Runnable, ZkTestManager { private static Logger LOG = Logger.getLogger(MockParticipantManager.class); private final CountDownLatch _startCountDown = new CountDownLatch(1); @@ -43,7 +45,7 @@ public class MockParticipantManager extends ParticipantManager implements Runnab private final MockMSModelFactory _msModelFactory = new MockMSModelFactory(null); public MockParticipantManager(String zkAddr, String clusterName, String instanceName) { - super(zkAddr, clusterName, instanceName); + super(clusterName, instanceName, InstanceType.PARTICIPANT, zkAddr); } public void setTransition(MockTransition transition) { diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java index 296181b41c..af427f77a1 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java +++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java @@ -30,11 +30,10 @@ import org.apache.helix.ZkTestHelper; import org.apache.helix.ZkUnitTestBase; import org.apache.helix.manager.zk.CallbackHandler; -import org.apache.helix.manager.zk.DistributedControllerManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.mock.participant.MockMSModelFactory; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Leader; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; import org.apache.log4j.Logger; @@ -231,7 +230,7 @@ public void testDistributedController() throws Exception { new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); PropertyKey.Builder keyBuilder = accessor.keyBuilder(); Assert.assertNotNull(accessor.getProperty(keyBuilder.liveInstance("localhost_12918"))); - LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); Assert.assertNotNull(leader); Assert.assertEquals(leader.getId(), "localhost_12919"); diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java index b5ef255814..6ddfac83f1 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java +++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java @@ -22,17 +22,19 @@ import java.util.Date; import java.util.List; +import org.apache.helix.HelixManager; +import org.apache.helix.InstanceType; import org.apache.helix.PropertyKey; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.ZkTestHelper; import org.apache.helix.integration.ZkIntegrationTestBase; import org.apache.helix.manager.zk.CallbackHandler; -import org.apache.helix.manager.zk.DistributedControllerManager; 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.mock.participant.MockMSModelFactory; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Leader; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; import org.apache.log4j.Logger; @@ -61,11 +63,12 @@ public void simpleIntegrationTest() throws Exception { 2, // replicas "MasterSlave", true); // do rebalance - DistributedControllerManager[] distributedControllers = new DistributedControllerManager[n]; + HelixManager[] distributedControllers = new HelixManager[n]; for (int i = 0; i < n; i++) { int port = 12918 + i; distributedControllers[i] = - new DistributedControllerManager(ZK_ADDR, clusterName, "localhost_" + port); + new ZKHelixManager(clusterName, "localhost_" + port, InstanceType.CONTROLLER_PARTICIPANT, + ZK_ADDR); distributedControllers[i].getStateMachineEngine().registerStateModelFactory("MasterSlave", new MockMSModelFactory()); distributedControllers[i].connect(); @@ -90,7 +93,7 @@ public void simpleIntegrationTest() throws Exception { new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); PropertyKey.Builder keyBuilder = accessor.keyBuilder(); Assert.assertNull(accessor.getProperty(keyBuilder.liveInstance("localhost_12918"))); - LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); Assert.assertNotNull(leader); Assert.assertEquals(leader.getId(), "localhost_12919"); @@ -131,7 +134,7 @@ void expireController(ClusterDistributedController expireController, PropertyKey.Builder keyBuilder = accessor.keyBuilder(); Assert.assertNotNull(accessor.getProperty(keyBuilder.liveInstance(expireController .getInstanceName()))); - LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); Assert.assertNotNull(leader); Assert.assertEquals(leader.getId(), newController.getInstanceName()); diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java index 85dc029022..82f583f528 100644 --- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java +++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java @@ -24,6 +24,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.helix.HelixManager; +import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; import org.apache.helix.PropertyKey; import org.apache.helix.PropertyPathConfig; @@ -31,10 +33,10 @@ import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.ZkTestHelper; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.integration.ZkIntegrationTestBase; -import org.apache.helix.manager.zk.ControllerManager; -import org.apache.helix.manager.zk.ParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.mock.participant.MockMSModelFactory; @@ -42,7 +44,6 @@ import org.apache.helix.model.Message; import org.apache.helix.tools.ClusterStateVerifier; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; -import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.annotations.Test; @@ -70,13 +71,14 @@ public void simpleIntegrationTest() throws Exception { 1, // replicas "MasterSlave", true); // do rebalance - ParticipantManager participant = - new ParticipantManager(ZK_ADDR, clusterName, "localhost_12918"); + HelixManager participant = + new ZKHelixManager(clusterName, "localhost_12918", InstanceType.PARTICIPANT, ZK_ADDR); participant.getStateMachineEngine().registerStateModelFactory("MasterSlave", new MockMSModelFactory()); participant.connect(); - ControllerManager controller = new ControllerManager(ZK_ADDR, clusterName, "controller_0"); + HelixManager controller = + new ZKHelixManager(clusterName, "controller_0", InstanceType.CONTROLLER, ZK_ADDR); controller.connect(); boolean result = @@ -121,8 +123,9 @@ public void simpleSessionExpiryTest() throws Exception { "MasterSlave", true); // do rebalance // start controller - ControllerManager controller = new ControllerManager(ZK_ADDR, clusterName, "controller_0"); - controller.connect(); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); // start participants for (int i = 0; i < n; i++) { @@ -151,7 +154,7 @@ public void simpleSessionExpiryTest() throws Exception { Assert.assertNotSame(newSessionId, oldSessionId); // cleanup - controller.disconnect(); + controller.syncStop(); for (int i = 0; i < n; i++) { participants[i].syncStop(); } @@ -174,7 +177,7 @@ public SessionExpiryTransition(CountDownLatch startCountdown, CountDownLatch end public void doTransition(Message message, NotificationContext context) throws InterruptedException { String instance = message.getTgtName(); - String partition = message.getPartitionName(); + PartitionId partition = message.getPartitionId(); if (instance.equals("localhost_12918") && partition.equals("TestDB0_0") && _done.getAndSet(true) == false) { _startCountdown.countDown(); @@ -207,8 +210,9 @@ public void testSessionExpiryInTransition() throws Exception { "MasterSlave", true); // do rebalance // start controller - ControllerManager controller = new ControllerManager(ZK_ADDR, clusterName, "controller_0"); - controller.connect(); + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + controller.syncStart(); // start participants for (int i = 0; i < n; i++) { @@ -245,7 +249,7 @@ public void testSessionExpiryInTransition() throws Exception { Assert.assertTrue(errString.indexOf("InterruptedException") != -1); // cleanup - controller.disconnect(); + controller.syncStop(); for (int i = 0; i < n; i++) { participants[i].syncStop(); } diff --git a/helix-core/src/test/java/org/apache/helix/josql/TestClusterJosqlQueryProcessor.java b/helix-core/src/test/java/org/apache/helix/josql/TestClusterJosqlQueryProcessor.java index 8090201a3e..30c23fb081 100644 --- a/helix-core/src/test/java/org/apache/helix/josql/TestClusterJosqlQueryProcessor.java +++ b/helix-core/src/test/java/org/apache/helix/josql/TestClusterJosqlQueryProcessor.java @@ -28,10 +28,8 @@ import org.apache.helix.Criteria; import org.apache.helix.InstanceType; import org.apache.helix.ZNRecord; -import org.apache.helix.josql.ZNRecordJosqlFunctionHandler; -import org.apache.helix.josql.ZNRecordRow; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; import org.apache.helix.model.LiveInstance.LiveInstanceProperty; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.josql.Query; import org.josql.QueryExecutionException; import org.josql.QueryParseException; @@ -59,9 +57,7 @@ public void queryClusterDataSample() { // liveInstances.remove(0); ZNRecord externalView = - DefaultIdealStateCalculator.calculateIdealState(instances, 21, 3, "TestDB", "MASTER", - "SLAVE"); - + DefaultTwoStateStrategy.calculateIdealState(instances, 21, 3, "TestDB", "MASTER", "SLAVE"); Criteria criteria = new Criteria(); criteria.setInstanceName("%"); criteria.setResource("TestDB"); diff --git a/helix-core/src/test/java/org/apache/helix/josql/TestJosqlProcessor.java b/helix-core/src/test/java/org/apache/helix/josql/TestJosqlProcessor.java index aabc2b9cbe..2aa4544d54 100644 --- a/helix-core/src/test/java/org/apache/helix/josql/TestJosqlProcessor.java +++ b/helix-core/src/test/java/org/apache/helix/josql/TestJosqlProcessor.java @@ -31,7 +31,6 @@ import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.integration.ZkStandAloneCMTestBase; -import org.apache.helix.josql.ClusterJosqlQueryProcessor; import org.apache.helix.model.LiveInstance.LiveInstanceProperty; import org.testng.Assert; import org.testng.annotations.Test; @@ -41,8 +40,8 @@ public class TestJosqlProcessor extends ZkStandAloneCMTestBase { "integrationTest" }) public void testJosqlQuery() throws Exception { - HelixManager manager = - ((TestHelper.StartCMResult) (_startCMResultMap.values().toArray()[0]))._manager; + HelixManager manager = _participants[0]; + // ((TestHelper.StartCMResult) (_startCMResultMap.values().toArray()[0]))._manager; // Find the instance name that contains partition TestDB_2 and state is 'MASTER' String SQL = @@ -184,8 +183,8 @@ public void parseFromTarget() { @Test(groups = ("unitTest")) public void testOrderby() throws Exception { - HelixManager manager = - ((TestHelper.StartCMResult) (_startCMResultMap.values().toArray()[0]))._manager; + HelixManager manager = _participants[0]; + // ((TestHelper.StartCMResult) (_startCMResultMap.values().toArray()[0]))._manager; Map scnMap = new HashMap(); for (int i = 0; i < NODE_NR; i++) { diff --git a/helix-core/src/test/java/org/apache/helix/lock/zk/TestZKHelixLock.java b/helix-core/src/test/java/org/apache/helix/lock/zk/TestZKHelixLock.java new file mode 100644 index 0000000000..f39ca11d73 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/lock/zk/TestZKHelixLock.java @@ -0,0 +1,127 @@ +package org.apache.helix.lock.zk; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.Scope; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.lock.HelixLock; +import org.testng.Assert; +import org.testng.annotations.Test; + +/* + * 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. + */ + +/** + * Tests that the Zookeeper-based Helix lock can acquire, block, and release as appropriate + */ +public class TestZKHelixLock extends ZkUnitTestBase { + @Test + public void basicTest() throws InterruptedException { + final long TIMEOUT = 30000; + final long RETRY_INTERVAL = 100; + _gZkClient.waitUntilConnected(TIMEOUT, TimeUnit.MILLISECONDS); + final AtomicBoolean t1Locked = new AtomicBoolean(false); + final AtomicBoolean t1Done = new AtomicBoolean(false); + final AtomicInteger field1 = new AtomicInteger(0); + final AtomicInteger field2 = new AtomicInteger(1); + final ClusterId clusterId = ClusterId.from("testCluster"); + final HelixLock lock1 = new ZKHelixLock(clusterId, Scope.cluster(clusterId), _gZkClient); + final HelixLock lock2 = new ZKHelixLock(clusterId, Scope.cluster(clusterId), _gZkClient); + + // thread 1: get a lock, set fields to 1 + Thread t1 = new Thread() { + @Override + public void run() { + lock1.lock(); + synchronized (t1Locked) { + t1Locked.set(true); + t1Locked.notify(); + } + yield(); // if locking doesn't work, t2 will set the fields first + field1.set(1); + field2.set(1); + synchronized (t1Done) { + t1Done.set(true); + t1Done.notify(); + } + } + }; + + // thread 2: wait for t1 to acquire the lock, get a lock, set fields to 2 + Thread t2 = new Thread() { + @Override + public void run() { + synchronized (t1Locked) { + while (!t1Locked.get()) { + try { + t1Locked.wait(); + } catch (InterruptedException e) { + } + } + } + lock2.lock(); + field1.set(2); + field2.set(2); + } + }; + + // start the threads + t1.setPriority(Thread.MIN_PRIORITY); + t2.setPriority(Thread.MAX_PRIORITY); + t1.start(); + t2.start(); + + // wait for t1 to finish setting fields + synchronized (t1Done) { + while (!t1Done.get()) { + try { + t1Done.wait(); + } catch (InterruptedException e) { + } + } + } + + // make sure both fields are 1 + Assert.assertEquals(field1.get(), 1); + Assert.assertEquals(field2.get(), 1); + + // unlock t1's lock after checking that t2 is blocked + long count = 0; + while (!lock2.isBlocked()) { + if (count > TIMEOUT) { + break; + } + Thread.sleep(RETRY_INTERVAL); + count += RETRY_INTERVAL; + } + Assert.assertTrue(lock2.isBlocked()); + lock1.unlock(); + + try { + // wait for t2, make sure both fields are 2 + t2.join(10000); + Assert.assertEquals(field1.get(), 2); + Assert.assertEquals(field2.get(), 2); + } catch (InterruptedException e) { + } + } +} diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultAlertMsgHandler.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultAlertMsgHandler.java new file mode 100644 index 0000000000..b5a7049e7c --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultAlertMsgHandler.java @@ -0,0 +1,182 @@ +package org.apache.helix.manager.zk; + +/* + * 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 org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; +import org.apache.helix.messaging.handling.MessageHandlerFactory; +import org.apache.helix.controller.alert.AlertAction; +import org.apache.helix.model.AlertConfig; +import org.apache.helix.controller.alert.AlertName; +import org.apache.helix.model.Error; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.Message; +import org.apache.helix.tools.ClusterStateVerifier; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestDefaultAlertMsgHandler extends ZkUnitTestBase { + + @Test + public void testBasic() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + int n = 2; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 32, // partitions per resource + n, // number of nodes + 2, // replicas + "MasterSlave", true); // do rebalance + + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0"); + MessageHandlerFactory fty = new DefaultAlertMsgHandlerFactory(); + controller.getMessagingService().registerMessageHandlerFactory(fty.getMessageType(), fty); + controller.syncStart(); + + // start participants + MockParticipantManager[] participants = new MockParticipantManager[n]; + for (int i = 0; i < n; i++) { + String instanceName = "localhost_" + (12918 + i); + + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participants[i].syncStart(); + } + + boolean result = + ClusterStateVerifier + .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, + clusterName)); + Assert.assertTrue(result); + + // add alert config + final HelixDataAccessor accessor = controller.getHelixDataAccessor(); + AlertConfig config = new AlertConfig("default"); + AlertName name = + new AlertName.Builder().cluster(ClusterId.from(clusterName)).metric("latency95") + .largerThan("1000").build(); + AlertAction action = + new AlertAction.Builder().cmd("enableInstance") + .args("{cluster}", "{node}", "false").build(); + config.putConfig(name, action); + final PropertyKey.Builder keyBuilder = new PropertyKey.Builder(clusterName); + accessor.setProperty(keyBuilder.alertConfig("default"), config); + + Message alertMsg; + + // test send an alert message that matches the config + alertMsg = new Message(MessageType.ALERT, MessageId.from("msg_1")); + alertMsg.setAttribute(Message.Attributes.ALERT_NAME, + String.format("(%s.%s.%s)(latency95)>(1000)", clusterName, "tenant1", "localhost_12918")); + alertMsg.setTgtSessionId(controller.getSessionId()); + alertMsg.setTgtName("controller"); + accessor.setProperty(keyBuilder.controllerMessage(alertMsg.getId()), alertMsg); + + // verify localhost_12918 is disabled + result = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + InstanceConfig config = accessor.getProperty(keyBuilder.instanceConfig("localhost_12918")); + return !config.getInstanceEnabled(); + } + }, 10 * 1000); + Assert.assertTrue(result); + + // test send an alert message that doesn't match any config + alertMsg = new Message(MessageType.ALERT, MessageId.from("msg_2")); + alertMsg.setAttribute(Message.Attributes.ALERT_NAME, + String.format("(%s.%s.%s)(errorRatio)>(0.1)", clusterName, "tenant1", "localhost_12918")); + alertMsg.setTgtSessionId(controller.getSessionId()); + alertMsg.setTgtName("controller"); + accessor.setProperty(keyBuilder.controllerMessage(alertMsg.getId()), alertMsg); + result = containsCtrlErrMsg(accessor, "msg_2"); + Assert.assertTrue(result); + + // test send an invalid alert message that doesn't have {node} scope + alertMsg = new Message(MessageType.ALERT, MessageId.from("msg_3")); + alertMsg.setAttribute(Message.Attributes.ALERT_NAME, + String.format("(%s.%s.%%.%s)(latency95)>(1000)", clusterName, "tenant1", "TestDB")); + alertMsg.setTgtSessionId(controller.getSessionId()); + alertMsg.setTgtName("controller"); + accessor.setProperty(keyBuilder.controllerMessage(alertMsg.getId()), alertMsg); + result = containsCtrlErrMsg(accessor, "msg_3"); + Assert.assertTrue(result); + + // clean up + controller.syncStop(); + for (int i = 0; i < n; i++) { + participants[i].syncStop(); + } + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + + } + + /** + * verify we have an error entry for msgId in CONTROLLER/ERRORS/ALERT + * @param msgId + * @return + */ + private boolean containsCtrlErrMsg(final HelixDataAccessor accessor, final String msgId) + throws Exception { + boolean result = TestHelper.verify(new TestHelper.Verifier() { + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + @Override + public boolean verify() throws Exception { + Error error = + accessor.getProperty(keyBuilder.controllerTaskError(MessageType.ALERT.name())); + if (error == null) { + return false; + } + + for (String key : error.getRecord().getMapFields().keySet()) { + if (!key.startsWith("HELIX_ERROR")) { + continue; + } + String tmpMsgId = error.getRecord().getMapField(key).get("MSG_ID"); + if (tmpMsgId != null && tmpMsgId.equals(msgId)) { + return true; + } + } + return false; + } + }, 10 * 1000); + + return result; + } +} \ No newline at end of file diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java index a6aeb03403..8b5b30c69a 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java @@ -25,15 +25,18 @@ import org.apache.helix.HelixException; import org.apache.helix.NotificationContext; -import org.apache.helix.manager.zk.DefaultControllerMessageHandlerFactory; +import org.apache.helix.api.id.MessageId; import org.apache.helix.manager.zk.DefaultControllerMessageHandlerFactory.DefaultControllerMessageHandler; import org.apache.helix.messaging.handling.MessageHandler; import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageType; +import org.apache.log4j.Logger; import org.testng.AssertJUnit; import org.testng.annotations.Test; public class TestDefaultControllerMsgHandlerFactory { + private static Logger LOG = Logger.getLogger(TestDefaultControllerMsgHandlerFactory.class); + @Test() public void testDefaultControllerMsgHandlerFactory() { System.out.println("START TestDefaultControllerMsgHandlerFactory at " @@ -41,7 +44,7 @@ public void testDefaultControllerMsgHandlerFactory() { DefaultControllerMessageHandlerFactory facotry = new DefaultControllerMessageHandlerFactory(); - Message message = new Message(MessageType.NO_OP, "0"); + Message message = new Message(MessageType.NO_OP, MessageId.from("0")); NotificationContext context = new NotificationContext(null); boolean exceptionCaught = false; @@ -52,7 +55,7 @@ public void testDefaultControllerMsgHandlerFactory() { } AssertJUnit.assertTrue(exceptionCaught); - message = new Message(MessageType.CONTROLLER_MSG, "1"); + message = new Message(MessageType.CONTROLLER_MSG, MessageId.from("1")); exceptionCaught = false; try { MessageHandler handler = facotry.createHandler(message, context); @@ -62,7 +65,7 @@ public void testDefaultControllerMsgHandlerFactory() { AssertJUnit.assertFalse(exceptionCaught); Map resultMap = new HashMap(); - message = new Message(MessageType.NO_OP, "3"); + message = new Message(MessageType.NO_OP, MessageId.from("3")); DefaultControllerMessageHandler defaultHandler = new DefaultControllerMessageHandler(message, context); try { @@ -70,12 +73,11 @@ public void testDefaultControllerMsgHandlerFactory() { } catch (HelixException e) { exceptionCaught = true; } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Interrupted handling message", e); } AssertJUnit.assertTrue(exceptionCaught); - message = new Message(MessageType.CONTROLLER_MSG, "4"); + message = new Message(MessageType.CONTROLLER_MSG, MessageId.from("4")); defaultHandler = new DefaultControllerMessageHandler(message, context); exceptionCaught = false; try { @@ -83,8 +85,7 @@ public void testDefaultControllerMsgHandlerFactory() { } catch (HelixException e) { exceptionCaught = true; } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Interrupted handling message", e); } AssertJUnit.assertFalse(exceptionCaught); System.out.println("END TestDefaultControllerMsgHandlerFactory at " diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestHandleNewSession.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestHandleNewSession.java index f9afe91887..352cdd5f61 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestHandleNewSession.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestHandleNewSession.java @@ -21,11 +21,10 @@ import java.util.Date; -import org.apache.helix.InstanceType; import org.apache.helix.TestHelper; import org.apache.helix.ZkTestHelper; import org.apache.helix.integration.ZkIntegrationTestBase; -import org.apache.helix.manager.zk.ZKHelixManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.testng.Assert; import org.testng.annotations.Test; @@ -48,17 +47,17 @@ public void testHandleNewSession() throws Exception { 3, // replicas "MasterSlave", true); // do rebalance - ZKHelixManager manager = - new ZKHelixManager(clusterName, "localhost_12918", InstanceType.PARTICIPANT, ZK_ADDR); - manager.connect(); + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12918"); + participant.syncStart(); // Logger.getRootLogger().setLevel(Level.INFO); - String lastSessionId = manager.getSessionId(); + String lastSessionId = participant.getSessionId(); for (int i = 0; i < 3; i++) { // System.err.println("curSessionId: " + lastSessionId); - ZkTestHelper.expireSession(manager._zkClient); + ZkTestHelper.expireSession(participant.getZkClient()); - String sessionId = manager.getSessionId(); + String sessionId = participant.getSessionId(); Assert.assertTrue(sessionId.compareTo(lastSessionId) > 0, "Session id should be increased after expiry"); lastSessionId = sessionId; @@ -72,7 +71,7 @@ public void testHandleNewSession() throws Exception { // Logger.getRootLogger().setLevel(Level.INFO); System.out.println("Disconnecting ..."); - manager.disconnect(); + participant.syncStop(); System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestLiveInstanceBounce.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestLiveInstanceBounce.java index 23fdbfaddd..547e8637e4 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestLiveInstanceBounce.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestLiveInstanceBounce.java @@ -19,12 +19,8 @@ * under the License. */ -import org.apache.helix.TestHelper; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.ZkHelixTestManager; -import org.apache.helix.integration.ZkStandAloneCMTestBase; import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck; -import org.apache.helix.manager.zk.ZKHelixManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.tools.ClusterStateVerifier; import org.testng.Assert; import org.testng.annotations.Test; @@ -32,24 +28,20 @@ public class TestLiveInstanceBounce extends ZkStandAloneCMTestBaseWithPropertyServerCheck { @Test public void testInstanceBounce() throws Exception { - String controllerName = CONTROLLER_PREFIX + "_0"; - StartCMResult controllerResult = _startCMResultMap.get(controllerName); - ZkHelixTestManager controller = (ZkHelixTestManager) controllerResult._manager; - int handlerSize = controller.getHandlers().size(); + int handlerSize = _controller.getHandlers().size(); for (int i = 0; i < 2; i++) { String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); // kill 2 participants - _startCMResultMap.get(instanceName)._manager.disconnect(); - _startCMResultMap.get(instanceName)._thread.interrupt(); + _participants[i].syncStop(); try { Thread.sleep(1000); } catch (InterruptedException e) { e.printStackTrace(); } // restart the participant - StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName); - _startCMResultMap.put(instanceName, result); + _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName); + _participants[i].syncStart(); Thread.sleep(100); } Thread.sleep(4000); @@ -63,11 +55,11 @@ public void testInstanceBounce() throws Exception { // and we will remove current-state listener on expired session // so the number of callback handlers is unchanged for (int j = 0; j < 10; j++) { - if (controller.getHandlers().size() == (handlerSize)) { + if (_controller.getHandlers().size() == (handlerSize)) { break; } Thread.sleep(400); } - Assert.assertEquals(controller.getHandlers().size(), handlerSize); + Assert.assertEquals(_controller.getHandlers().size(), handlerSize); } } diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheAsyncOpMultiThread.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheAsyncOpMultiThread.java index aa0bd2b098..c6875be40c 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheAsyncOpMultiThread.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheAsyncOpMultiThread.java @@ -33,8 +33,6 @@ import org.apache.helix.ZNRecord; import org.apache.helix.ZNRecordUpdater; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.manager.zk.ZkCacheBaseDataAccessor; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheAsyncOpSingleThread.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheAsyncOpSingleThread.java index 4d684d332c..ac7a0dba04 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheAsyncOpSingleThread.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheAsyncOpSingleThread.java @@ -32,8 +32,6 @@ import org.apache.helix.ZNRecord; import org.apache.helix.ZNRecordUpdater; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.manager.zk.ZkCacheBaseDataAccessor; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheSyncOpSingleThread.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheSyncOpSingleThread.java index edbcedc054..979a179fe4 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheSyncOpSingleThread.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestWtCacheSyncOpSingleThread.java @@ -30,8 +30,6 @@ import org.apache.helix.ZNRecord; import org.apache.helix.ZNRecordUpdater; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.manager.zk.ZkCacheBaseDataAccessor; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKLiveInstanceData.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKLiveInstanceData.java index 0044630d48..29207ca684 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKLiveInstanceData.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKLiveInstanceData.java @@ -35,8 +35,6 @@ import org.apache.helix.NotificationContext; import org.apache.helix.PropertyKey; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.manager.zk.ZNRecordSerializer; -import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.model.LiveInstance; import org.apache.helix.tools.ClusterSetup; import org.testng.Assert; diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKPropertyTransferServer.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKPropertyTransferServer.java index f40445e0bf..50a9a78d6e 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKPropertyTransferServer.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKPropertyTransferServer.java @@ -20,11 +20,9 @@ */ import org.apache.helix.HelixDataAccessor; -import org.apache.helix.TestHelper; -import org.apache.helix.TestHelper.StartCMResult; -import org.apache.helix.controller.HelixControllerMain; import org.apache.helix.controller.restlet.ZKPropertyTransferServer; import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck; +import org.apache.helix.integration.manager.ClusterControllerManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.log4j.Logger; import org.testng.Assert; @@ -36,34 +34,28 @@ public class TestZKPropertyTransferServer extends ZkStandAloneCMTestBaseWithProp @Test public void TestControllerChange() throws Exception { String controllerName = CONTROLLER_PREFIX + "_0"; - _startCMResultMap.get(controllerName)._manager.disconnect(); + _controller.syncStop(); Thread.sleep(1000); // kill controller, participant should not know about the svc url for (int i = 0; i < NODE_NR; i++) { - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); HelixDataAccessor accessor = - _startCMResultMap.get(instanceName)._manager.getHelixDataAccessor(); + _participants[i].getHelixDataAccessor(); ZKHelixDataAccessor zkAccessor = (ZKHelixDataAccessor) accessor; Assert.assertTrue(zkAccessor._zkPropertyTransferSvcUrl == null || zkAccessor._zkPropertyTransferSvcUrl.equals("")); } - _startCMResultMap.get(controllerName)._thread.interrupt(); - _startCMResultMap.remove(controllerName); - StartCMResult startResult = - TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR, - HelixControllerMain.STANDALONE); - _startCMResultMap.put(controllerName, startResult); + _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName); + _controller.syncStart(); Thread.sleep(1000); // create controller again, the svc url is notified to the participants for (int i = 0; i < NODE_NR; i++) { - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); HelixDataAccessor accessor = - _startCMResultMap.get(instanceName)._manager.getHelixDataAccessor(); + _participants[i].getHelixDataAccessor(); ZKHelixDataAccessor zkAccessor = (ZKHelixDataAccessor) accessor; Assert.assertTrue(zkAccessor._zkPropertyTransferSvcUrl.equals(ZKPropertyTransferServer .getInstance().getWebserviceUrl())); diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordSizeLimit.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordSizeLimit.java index 286593a4bd..c47ef6f405 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordSizeLimit.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordSizeLimit.java @@ -24,15 +24,10 @@ import org.apache.helix.HelixException; import org.apache.helix.HelixProperty; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.ZNRecord; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.PropertyKey.Builder; -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.ZNRecordStreamingSerializer; -import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.manager.zk.ZkClient; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.model.IdealState; import org.apache.helix.model.IdealState.RebalanceMode; import org.apache.helix.model.InstanceConfig; @@ -114,11 +109,11 @@ record = zkClient.readData(path1); // oversized data should not create any new data on zk ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(className, new ZkBaseDataAccessor(zkClient)); + new ZKHelixDataAccessor(className, new ZkBaseDataAccessor(zkClient)); Builder keyBuilder = accessor.keyBuilder(); IdealState idealState = new IdealState("currentState"); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setNumPartitions(10); @@ -134,7 +129,7 @@ record = zkClient.readData(path1); // legal sized data gets written to zk idealState.getRecord().getSimpleFields().clear(); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setNumPartitions(10); @@ -148,7 +143,7 @@ record = accessor.getProperty(keyBuilder.idealStates("TestDB1")).getRecord(); // oversized data should not update existing data on zk idealState.getRecord().getSimpleFields().clear(); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setNumPartitions(10); for (int i = 900; i < 1024; i++) { @@ -237,12 +232,12 @@ record = zkClient.readData(path1); // oversized data should not create any new data on zk ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(className, new ZkBaseDataAccessor(zkClient)); + new ZKHelixDataAccessor(className, new ZkBaseDataAccessor(zkClient)); Builder keyBuilder = accessor.keyBuilder(); // ZNRecord statusUpdates = new ZNRecord("statusUpdates"); IdealState idealState = new IdealState("currentState"); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setNumPartitions(10); @@ -256,7 +251,7 @@ record = zkClient.readData(path1); // legal sized data gets written to zk idealState.getRecord().getSimpleFields().clear(); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setNumPartitions(10); @@ -270,7 +265,7 @@ record = accessor.getProperty(keyBuilder.idealStates("TestDB_2")).getRecord(); // oversized data should not update existing data on zk idealState.getRecord().getSimpleFields().clear(); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setNumPartitions(10); diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordStreamingSerializer.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordStreamingSerializer.java new file mode 100644 index 0000000000..e4b0b258ed --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordStreamingSerializer.java @@ -0,0 +1,92 @@ +package org.apache.helix.manager.zk; + +import org.apache.helix.ZNRecord; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +/* + * 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 TestZNRecordStreamingSerializer { + /** + * Test the normal case of serialize/deserialize where ZNRecord is well-formed + */ + @Test + public void basicTest() { + ZNRecord record = new ZNRecord("testId"); + record.setMapField("k1", ImmutableMap.of("a", "b", "c", "d")); + record.setMapField("k2", ImmutableMap.of("e", "f", "g", "h")); + record.setListField("k3", ImmutableList.of("a", "b", "c", "d")); + record.setListField("k4", ImmutableList.of("d", "e", "f", "g")); + record.setSimpleField("k5", "a"); + record.setSimpleField("k5", "b"); + ZNRecordStreamingSerializer serializer = new ZNRecordStreamingSerializer(); + ZNRecord result = (ZNRecord) serializer.deserialize(serializer.serialize(record)); + Assert.assertEquals(result, record); + } + + /** + * Check that the ZNRecord is not constructed if there is no id in the json + */ + @Test + public void noIdTest() { + StringBuilder jsonString = + new StringBuilder("{\n").append(" \"simpleFields\": {},\n") + .append(" \"listFields\": {},\n").append(" \"mapFields\": {}\n").append("}\n"); + ZNRecordStreamingSerializer serializer = new ZNRecordStreamingSerializer(); + ZNRecord result = (ZNRecord) serializer.deserialize(jsonString.toString().getBytes()); + Assert.assertNull(result); + } + + /** + * Test that the json still deserizalizes correctly if id is not first + */ + @Test + public void idNotFirstTest() { + StringBuilder jsonString = + new StringBuilder("{\n").append(" \"simpleFields\": {},\n") + .append(" \"listFields\": {},\n").append(" \"mapFields\": {},\n") + .append("\"id\": \"myId\"\n").append("}"); + ZNRecordStreamingSerializer serializer = new ZNRecordStreamingSerializer(); + ZNRecord result = (ZNRecord) serializer.deserialize(jsonString.toString().getBytes()); + Assert.assertNotNull(result); + Assert.assertEquals(result.getId(), "myId"); + } + + /** + * Test that simple, list, and map fields are initialized as empty even when not in json + */ + @Test + public void fieldAutoInitTest() { + StringBuilder jsonString = new StringBuilder("{\n").append("\"id\": \"myId\"\n").append("}"); + ZNRecordStreamingSerializer serializer = new ZNRecordStreamingSerializer(); + ZNRecord result = (ZNRecord) serializer.deserialize(jsonString.toString().getBytes()); + Assert.assertNotNull(result); + Assert.assertEquals(result.getId(), "myId"); + Assert.assertNotNull(result.getSimpleFields()); + Assert.assertTrue(result.getSimpleFields().isEmpty()); + Assert.assertNotNull(result.getListFields()); + Assert.assertTrue(result.getListFields().isEmpty()); + Assert.assertNotNull(result.getMapFields()); + Assert.assertTrue(result.getMapFields().isEmpty()); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBaseDataAccessor.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBaseDataAccessor.java index 133fb4ede6..66c069f779 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBaseDataAccessor.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBaseDataAccessor.java @@ -32,11 +32,8 @@ import org.apache.helix.ZNRecord; import org.apache.helix.ZNRecordUpdater; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.manager.zk.ZNRecordSerializer; -import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor.AccessResult; import org.apache.helix.manager.zk.ZkBaseDataAccessor.RetCode; -import org.apache.helix.manager.zk.ZkClient; import org.apache.zookeeper.data.Stat; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java index e2c503b289..ed8471900d 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java @@ -32,10 +32,6 @@ import org.apache.helix.ZNRecord; import org.apache.helix.ZNRecordUpdater; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.manager.zk.ZNRecordSerializer; -import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.manager.zk.ZkCacheBaseDataAccessor; -import org.apache.helix.manager.zk.ZkClient; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheSyncOpSingleThread.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheSyncOpSingleThread.java index 251234f18e..985a82fde8 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheSyncOpSingleThread.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheSyncOpSingleThread.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Set; import java.util.TreeSet; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import org.apache.helix.AccessOption; @@ -36,10 +35,6 @@ import org.apache.helix.ZNRecord; import org.apache.helix.ZNRecordUpdater; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.manager.zk.ZNRecordSerializer; -import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.manager.zk.ZkCacheBaseDataAccessor; -import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.store.HelixPropertyListener; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java index 78097119a8..b3f87ad790 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java @@ -27,6 +27,7 @@ import java.util.Map; import org.apache.helix.AccessOption; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.model.ConfigScope; import org.apache.helix.model.HelixConfigScope; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; @@ -40,11 +41,9 @@ import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.ZkHelixTestManager; import org.apache.helix.ZkTestHelper; import org.apache.helix.ZkUnitTestBase; import org.apache.helix.manager.MockListener; -import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.model.LiveInstance; import org.apache.helix.store.zk.ZkHelixPropertyStore; import org.apache.zookeeper.data.Stat; @@ -68,6 +67,7 @@ public void testController() throws Exception { ZKHelixManager controller = new ZKHelixManager(clusterName, null, InstanceType.CONTROLLER, ZK_ADDR); + try { controller.connect(); Assert.fail("Should throw HelixException if initial cluster structure is not setup"); @@ -106,7 +106,6 @@ record = store.get("/node_1", stat, options); AssertJUnit.assertEquals("node_1", record.getId()); controller.getMessagingService(); - controller.getHealthReportCollector(); controller.getClusterManagmentTool(); controller.handleNewSession(); @@ -191,13 +190,14 @@ public ZNRecord getAdditionalLiveInstanceInfo() { Assert.assertTrue(liveInstance.getRecord().getListFields().size() == 1); Assert.assertTrue(liveInstance.getRecord().getMapFields().size() == 1); Assert.assertTrue(liveInstance.getRecord().getSimpleFields().size() == 5); - Assert.assertFalse(liveInstance.getSessionId().equals("value")); + Assert.assertFalse(liveInstance.getTypedSessionId().stringify().equals("value")); Assert.assertFalse(liveInstance.getLiveInstance().equals("value")); // ////////////////////////////////// - ZkHelixTestManager manager2 = - new ZkHelixTestManager(clusterName, "localhost_3", InstanceType.PARTICIPANT, ZK_ADDR); + MockParticipantManager manager2 = + new MockParticipantManager(ZK_ADDR, clusterName, "localhost_3"); + manager2.setLiveInstanceInfoProvider(new provider(true)); manager2.connect(); @@ -207,9 +207,9 @@ public ZNRecord getAdditionalLiveInstanceInfo() { Assert.assertTrue(liveInstance.getRecord().getListFields().size() == 1); Assert.assertTrue(liveInstance.getRecord().getMapFields().size() == 1); Assert.assertTrue(liveInstance.getRecord().getSimpleFields().size() == 5); - Assert.assertFalse(liveInstance.getSessionId().equals("value")); + Assert.assertFalse(liveInstance.getTypedSessionId().stringify().equals("value")); Assert.assertFalse(liveInstance.getLiveInstance().equals("value")); - String sessionId = liveInstance.getSessionId(); + String sessionId = liveInstance.getTypedSessionId().stringify(); ZkTestHelper.expireSession(manager2.getZkClient()); Thread.sleep(1000); @@ -218,9 +218,9 @@ public ZNRecord getAdditionalLiveInstanceInfo() { Assert.assertTrue(liveInstance.getRecord().getListFields().size() == 1); Assert.assertTrue(liveInstance.getRecord().getMapFields().size() == 1); Assert.assertTrue(liveInstance.getRecord().getSimpleFields().size() == 5); - Assert.assertFalse(liveInstance.getSessionId().equals("value")); + Assert.assertFalse(liveInstance.getTypedSessionId().stringify().equals("value")); Assert.assertFalse(liveInstance.getLiveInstance().equals("value")); - Assert.assertFalse(sessionId.equals(liveInstance.getSessionId())); + Assert.assertFalse(sessionId.equals(liveInstance.getTypedSessionId().stringify())); System.out.println("END " + className + ".testLiveInstanceInfoProvider() at " + new Date(System.currentTimeMillis())); diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkFlapping.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkFlapping.java new file mode 100644 index 0000000000..f84c372e5d --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkFlapping.java @@ -0,0 +1,294 @@ +package org.apache.helix.manager.zk; + +/* + * 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.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +import org.I0Itec.zkclient.IZkDataListener; +import org.I0Itec.zkclient.IZkStateListener; +import org.apache.helix.PropertyKey; +import org.apache.helix.ZNRecord; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.TestHelper; +import org.apache.helix.TestHelper.Verifier; +import org.apache.helix.ZkTestHelper; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; +import org.apache.helix.model.Leader; +import org.apache.helix.model.LiveInstance; +import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestZkFlapping extends ZkUnitTestBase { + + @Test + public void testZkSessionExpiry() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + ZkClient client = + new ZkClient(ZK_ADDR, ZkClient.DEFAULT_SESSION_TIMEOUT, + ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer()); + + String path = String.format("/%s", clusterName); + client.createEphemeral(path); + String oldSessionId = ZkTestHelper.getSessionId(client); + ZkTestHelper.expireSession(client); + String newSessionId = ZkTestHelper.getSessionId(client); + Assert.assertNotSame(newSessionId, oldSessionId); + Assert.assertFalse(client.exists(path), "Ephemeral znode should be gone after session expiry"); + client.close(); + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } + + @Test + public void testCloseZkClient() { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + ZkClient client = + new ZkClient(ZK_ADDR, ZkClient.DEFAULT_SESSION_TIMEOUT, + ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer()); + String path = String.format("/%s", clusterName); + client.createEphemeral(path); + + client.close(); + Assert.assertFalse(_gZkClient.exists(path), "Ephemeral node: " + path + + " should be removed after ZkClient#close()"); + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } + + @Test + public void testCloseZkClientInZkClientEventThread() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + final CountDownLatch waitCallback = new CountDownLatch(1); + final ZkClient client = + new ZkClient(ZK_ADDR, ZkClient.DEFAULT_SESSION_TIMEOUT, + ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer()); + String path = String.format("/%s", clusterName); + client.createEphemeral(path); + client.subscribeDataChanges(path, new IZkDataListener() { + + @Override + public void handleDataDeleted(String dataPath) throws Exception { + } + + @Override + public void handleDataChange(String dataPath, Object data) throws Exception { + client.close(); + waitCallback.countDown(); + } + }); + + client.writeData(path, new ZNRecord("test")); + waitCallback.await(); + Assert.assertFalse(_gZkClient.exists(path), "Ephemeral node: " + path + + " should be removed after ZkClient#close() in its own event-thread"); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + + } + + class ZkStateCountListener implements IZkStateListener { + int count = 0; + + @Override + public void handleStateChanged(KeeperState state) throws Exception { + if (state == KeeperState.Disconnected) { + count++; + } + } + + @Override + public void handleNewSession() throws Exception { + } + } + + @Test + public void testParticipantFlapping() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + final HelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 32, // partitions per resource + 1, // number of nodes + 1, // replicas + "MasterSlave", false); + + final String instanceName = "localhost_12918"; + MockParticipantManager participant = + new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + participant.syncStart(); + + final ZkClient client = participant.getZkClient(); + final ZkStateCountListener listener = new ZkStateCountListener(); + client.subscribeStateChanges(listener); + + final AtomicInteger expectDisconnectCnt = new AtomicInteger(0); + final int n = ZKHelixManager.MAX_DISCONNECT_THRESHOLD; + for (int i = 0; i < n; i++) { + String oldSessionId = ZkTestHelper.getSessionId(client); + ZkTestHelper.simulateZkStateDisconnected(client); + expectDisconnectCnt.incrementAndGet(); + // wait until we get invoked by zk state change to disconnected + TestHelper.verify(new Verifier() { + + @Override + public boolean verify() throws Exception { + return listener.count == expectDisconnectCnt.get(); + } + }, 30 * 1000); + + String newSessionId = ZkTestHelper.getSessionId(client); + Assert.assertEquals(newSessionId, oldSessionId); + } + client.unsubscribeStateChanges(listener); + // make sure participant is NOT disconnected + LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance(instanceName)); + Assert.assertNotNull(liveInstance, "Live-instance should exist after " + n + " disconnects"); + + // trigger flapping + ZkTestHelper.simulateZkStateDisconnected(client); + // wait until we get invoked by zk state change to disconnected + boolean success = TestHelper.verify(new Verifier() { + + @Override + public boolean verify() throws Exception { + return client.getShutdownTrigger(); + } + }, 30 * 1000); + + Assert.assertTrue(success, "The " + (n + 1) + + "th disconnect event should trigger ZkHelixManager#disonnect"); + + // make sure participant is disconnected + success = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance(instanceName)); + return liveInstance == null; + } + }, 3 * 1000); + Assert.assertTrue(success, "Live-instance should be gone after " + (n + 1) + " disconnects"); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } + + @Test + public void testControllerFlapping() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + final HelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 32, // partitions per resource + 1, // number of nodes + 1, // replicas + "MasterSlave", false); + + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller"); + controller.syncStart(); + + final ZkClient client = controller.getZkClient(); + final ZkStateCountListener listener = new ZkStateCountListener(); + client.subscribeStateChanges(listener); + + final AtomicInteger expectDisconnectCnt = new AtomicInteger(0); + final int n = ZKHelixManager.MAX_DISCONNECT_THRESHOLD; + for (int i = 0; i < n; i++) { + String oldSessionId = ZkTestHelper.getSessionId(client); + ZkTestHelper.simulateZkStateDisconnected(client); + expectDisconnectCnt.incrementAndGet(); + // wait until we get invoked by zk state change to disconnected + TestHelper.verify(new Verifier() { + + @Override + public boolean verify() throws Exception { + return listener.count == expectDisconnectCnt.get(); + } + }, 30 * 1000); + + String newSessionId = ZkTestHelper.getSessionId(client); + Assert.assertEquals(newSessionId, oldSessionId); + } + + // make sure controller is NOT disconnected + LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Assert.assertNotNull(leader, "Leader should exist after " + n + " disconnects"); + + // trigger flapping + ZkTestHelper.simulateZkStateDisconnected(client); + // wait until we get invoked by zk state change to disconnected + boolean success = TestHelper.verify(new Verifier() { + + @Override + public boolean verify() throws Exception { + return client.getShutdownTrigger(); + } + }, 30 * 1000); + + Assert.assertTrue(success, "The " + (n + 1) + + "th disconnect event should trigger ZkHelixManager#disonnect"); + + // make sure controller is disconnected + success = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); + return leader == null; + } + }, 5 * 1000); + Assert.assertTrue(success, "Leader should be gone after " + (n + 1) + " disconnects"); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAdmin.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAdmin.java index 4b3764fb51..ed2f0f72c2 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAdmin.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAdmin.java @@ -25,18 +25,24 @@ import java.util.List; import java.util.Map; -import org.apache.helix.*; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixException; +import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyPathConfig; +import org.apache.helix.PropertyType; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.id.ConstraintId; import org.apache.helix.model.ClusterConstraints; import org.apache.helix.model.ClusterConstraints.ConstraintAttribute; import org.apache.helix.model.ClusterConstraints.ConstraintType; -import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; -import org.apache.helix.model.ConfigScope; import org.apache.helix.model.ConstraintItem; import org.apache.helix.model.ExternalView; import org.apache.helix.model.HelixConfigScope; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.StateModelDefinition; -import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.helix.model.builder.ConstraintItemBuilder; import org.apache.helix.model.builder.HelixConfigScopeBuilder; import org.apache.helix.tools.StateModelConfigGenerator; @@ -94,6 +100,7 @@ public void testZkHelixAdmin() { config = tool.getInstanceConfig(clusterName, "host1_9999"); AssertJUnit.assertEquals(config.getId(), "host1_9999"); + tool.enableInstance(clusterName, "host1_9999", false); tool.dropInstance(clusterName, config); try { tool.getInstanceConfig(clusterName, "host1_9999"); @@ -265,7 +272,7 @@ public void testAddRemoveMsgConstraint() { constraints = accessor.getProperty(keyBuilder.constraint(ConstraintType.MESSAGE_CONSTRAINT.toString())); Assert.assertNotNull(constraints, "message-constraint should exist"); - ConstraintItem item = constraints.getConstraintItem("constraint1"); + ConstraintItem item = constraints.getConstraintItem(ConstraintId.from("constraint1")); Assert.assertNotNull(item, "message-constraint for constraint1 should exist"); Assert.assertEquals(item.getConstraintValue(), "1"); Assert.assertEquals(item.getAttributeValue(ConstraintAttribute.RESOURCE), "MyDB"); @@ -273,7 +280,7 @@ public void testAddRemoveMsgConstraint() { // test admin.getMessageConstraints() constraints = tool.getConstraints(clusterName, ConstraintType.MESSAGE_CONSTRAINT); Assert.assertNotNull(constraints, "message-constraint should exist"); - item = constraints.getConstraintItem("constraint1"); + item = constraints.getConstraintItem(ConstraintId.from("constraint1")); Assert.assertNotNull(item, "message-constraint for constraint1 should exist"); Assert.assertEquals(item.getConstraintValue(), "1"); Assert.assertEquals(item.getAttributeValue(ConstraintAttribute.RESOURCE), "MyDB"); @@ -283,7 +290,7 @@ public void testAddRemoveMsgConstraint() { constraints = accessor.getProperty(keyBuilder.constraint(ConstraintType.MESSAGE_CONSTRAINT.toString())); Assert.assertNotNull(constraints, "message-constraint should exist"); - item = constraints.getConstraintItem("constraint1"); + item = constraints.getConstraintItem(ConstraintId.from("constraint1")); Assert.assertNull(item, "message-constraint for constraint1 should NOT exist"); System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAutoController.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAutoController.java new file mode 100644 index 0000000000..438ebd4cca --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAutoController.java @@ -0,0 +1,121 @@ +package org.apache.helix.manager.zk; + +/* + * 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 org.apache.helix.HelixAutoController; +import org.apache.helix.HelixConnection; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.model.Leader; +import org.apache.helix.model.LiveInstance; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestZkHelixAutoController extends ZkUnitTestBase { + @Test + public void testOnConnectedAndDisconnecting() throws Exception { + // Logger.getRootLogger().setLevel(Level.INFO); + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + int n = 2; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 32, // partitions per resource + n, // number of nodes + 2, // replicas + "MasterSlave", true); // do rebalance + + // create connection + HelixConnection connection = new ZkHelixConnection(ZK_ADDR); + connection.connect(); + + // start auto-controller + ClusterId clusterId = ClusterId.from(clusterName); + final HelixAutoController[] controllers = new HelixAutoController[n]; + for (int i = 0; i < n; i++) { + int port = 12918 + i; + ControllerId controllerId = ControllerId.from("localhost_" + port); + controllers[i] = connection.createAutoController(clusterId, controllerId); + controllers[i].startAsync(); + } + + // check live-instance znode for localhost_12918/12919 exists + final HelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + for (int i = 0; i < n; i++) { + String instanceName = controllers[i].getControllerId().stringify(); + Assert.assertNotNull(accessor.getProperty(keyBuilder.liveInstance(instanceName))); + } + + // check leader znode exists + LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Assert.assertNotNull(leader); + Assert.assertEquals(leader.getInstanceName(), controllers[0].getControllerId().stringify()); + + // stop controller localhost_12918 + controllers[0].stopAsync(); + + // check live-instance znode for localhost_12918 is gone + String instanceName = controllers[0].getControllerId().stringify(); + Assert.assertNull(accessor.getProperty(keyBuilder.liveInstance(instanceName))); + + // check localhost_12919 becomes the new leader + boolean success = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); + if (leader == null) { + return false; + } + return leader.getInstanceName().equals(controllers[1].getControllerId().stringify()); + + } + }, 3 * 1000); + Assert.assertTrue(success, "fail to re-elect new leader"); + + // clean up + connection.disconnect(); + + // check live-instance znode for localhost_12919 is gone + instanceName = controllers[1].getControllerId().stringify(); + Assert.assertNull(accessor.getProperty(keyBuilder.liveInstance(instanceName))); + + // check leader znode is gone + Assert.assertNull(accessor.getProperty(keyBuilder.controllerLeader())); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixController.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixController.java new file mode 100644 index 0000000000..d823ae0554 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixController.java @@ -0,0 +1,161 @@ +package org.apache.helix.manager.zk; + +/* + * 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 org.apache.helix.HelixConnection; +import org.apache.helix.HelixController; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.model.Leader; +import org.apache.helix.model.LiveInstance; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestZkHelixController extends ZkUnitTestBase { + + @Test + public void testOnConnectedAndDisconnecting() throws Exception { + // Logger.getRootLogger().setLevel(Level.INFO); + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + int n = 2; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 32, // partitions per resource + n, // number of nodes + 2, // replicas + "MasterSlave", true); // do rebalance + + // create connection + HelixConnection connection = new ZkHelixConnection(ZK_ADDR); + connection.connect(); + + // start controller + ClusterId clusterId = ClusterId.from(clusterName); + ControllerId controllerId = ControllerId.from("controller"); + HelixController controller = connection.createController(clusterId, controllerId); + controller.startAsync(); + + // check leader znode exists + HelixDataAccessor accessor = connection.createDataAccessor(clusterId); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); + Assert.assertNotNull(leader); + Assert.assertEquals(leader.getInstanceName(), controllerId.stringify()); + + // stop participant + controller.stopAsync(); + + // check leader znode is gone + Assert.assertNull(accessor.getProperty(keyBuilder.controllerLeader())); + + // clean up + connection.disconnect(); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } + + /** + * Remove leader znode externally should invoke another round of leader-election + * this simulates the race condition in + * {@link ZkHelixLeaderElection#onControllerChange(org.apache.helix.NotificationContext)} + * @throws Exception + */ + @Test + public void testRemoveLeaderZnode() throws Exception { + + // Logger.getRootLogger().setLevel(Level.INFO); + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + int n = 2; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 32, // partitions per resource + n, // number of nodes + 2, // replicas + "MasterSlave", true); // do rebalance + + // create connection + HelixConnection connection = new ZkHelixConnection(ZK_ADDR); + connection.connect(); + + // start controller + ClusterId clusterId = ClusterId.from(clusterName); + final ControllerId controllerId = ControllerId.from("controller"); + + // start controller + HelixController controller = connection.createController(clusterId, controllerId); + controller.startAsync(); + + // check live-instance znode for localhost_12918 exists + final HelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader()); + Assert.assertNotNull(leader); + Assert.assertEquals(leader.getInstanceName(), controllerId.stringify()); + + // remove leader znode externally + accessor.removeProperty(keyBuilder.controllerLeader()); + + // verify leader is re-elected + boolean result = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); + if (leader == null) { + return false; + } + + return leader.getInstanceName().equals(controllerId.stringify()); + } + }, 3 * 1000); + + Assert.assertTrue(result, "Fail to re-elect a new leader"); + + // clean up + connection.disconnect(); + + // check leader znode is gone + Assert.assertNull(accessor.getProperty(keyBuilder.controllerLeader())); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixParticipant.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixParticipant.java new file mode 100644 index 0000000000..8658736e6d --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixParticipant.java @@ -0,0 +1,103 @@ +package org.apache.helix.manager.zk; + +/* + * 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 org.apache.helix.HelixConnection; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixParticipant; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.integration.TestHelixConnection; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestZkHelixParticipant extends ZkUnitTestBase { + + @Test + public void testOnConnectedAndDisconnecting() throws Exception { + // Logger.getRootLogger().setLevel(Level.INFO); + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + int n = 2; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 32, // partitions per resource + n, // number of nodes + 2, // replicas + "MasterSlave", true); // do rebalance + + // create connection + HelixConnection connection = new ZkHelixConnection(ZK_ADDR); + connection.connect(); + + // start participant + ClusterId clusterId = ClusterId.from(clusterName); + HelixParticipant[] participants = new HelixParticipant[n]; + for (int i = 0; i < n; i++) { + int port = 12918 + i; + ParticipantId participantId = ParticipantId.from("localhost_" + port); + + participants[i] = connection.createParticipant(clusterId, participantId); + participants[i].getStateMachineEngine().registerStateModelFactory( + StateModelDefId.from("MasterSlave"), new TestHelixConnection.MockStateModelFactory()); + + participants[i].startAsync(); + } + + // check live-instance znode for localhost_12918/12919 exist + HelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + for (int i = 0; i < n; i++) { + Assert.assertNotNull(accessor.getProperty(keyBuilder.liveInstance(participants[i].getParticipantId().stringify()))); + } + + // stop participant localhost_12918 + participants[0].stopAsync(); + + // check live-instance znode for localhost_12918 is gone + Assert.assertNull(accessor.getProperty(keyBuilder.liveInstance(participants[0] + .getParticipantId().stringify()))); + + // clean up + connection.disconnect(); + + // check live-instance znode for localhost_12919 is gone + Assert.assertNull(accessor.getProperty(keyBuilder.liveInstance(participants[1] + .getParticipantId().stringify()))); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } + +} diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkManagerFlappingDetection.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkManagerFlappingDetection.java index 249fcea1ee..a62e39d1cf 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkManagerFlappingDetection.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkManagerFlappingDetection.java @@ -19,13 +19,12 @@ * under the License. */ -import java.util.UUID; -import org.apache.helix.InstanceType; import org.apache.helix.TestHelper; -import org.apache.helix.ZkHelixTestManager; import org.apache.helix.ZkTestHelper; import org.apache.helix.integration.ZkIntegrationTestBase; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.integration.manager.MockParticipantManager; import org.testng.Assert; import org.testng.annotations.Test; @@ -46,8 +45,8 @@ public void testDisconnectHistory() throws Exception { "MasterSlave", true); // do rebalance String instanceName = "localhost_" + (12918 + 0); - ZkHelixTestManager manager = - new ZkHelixTestManager(clusterName, instanceName, InstanceType.PARTICIPANT, ZK_ADDR); + MockParticipantManager manager = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); + manager.connect(); ZkClient zkClient = manager.getZkClient(); ZkTestHelper.expireSession(zkClient); @@ -69,58 +68,59 @@ public void testDisconnectHistory() throws Exception { Assert.assertFalse(manager.isConnected()); } - @Test(enabled = false) - public void testDisconnectFlappingWindow() throws Exception { - String className = TestHelper.getTestClassName(); - String methodName = TestHelper.getTestMethodName(); - String instanceName = "localhost_" + (12918 + 1); - final String clusterName = className + "_" + methodName + UUID.randomUUID(); - - testDisconnectFlappingWindow2(instanceName, InstanceType.PARTICIPANT); - testDisconnectFlappingWindow2("admin", InstanceType.ADMINISTRATOR); - } - - public void testDisconnectFlappingWindow2(String instanceName, InstanceType type) - throws Exception { - String className = TestHelper.getTestClassName(); - String methodName = TestHelper.getTestMethodName(); - final String clusterName = className + "_" + methodName + UUID.randomUUID(); - - TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port - "localhost", // participant name prefix - "TestDB", // resource name prefix - 1, // resources - 10, // partitions per resource - 5, // number of nodes - 3, // replicas - "MasterSlave", true); // do rebalance - - // flapping time window to 5 sec - System.setProperty("helixmanager.flappingTimeWindow", "15000"); - System.setProperty("helixmanager.maxDisconnectThreshold", "7"); - ZkHelixTestManager manager2 = new ZkHelixTestManager(clusterName, instanceName, type, ZK_ADDR); - manager2.connect(); - ZkClient zkClient = manager2.getZkClient(); - for (int i = 0; i < 3; i++) { - ZkTestHelper.expireSession(zkClient); - Thread.sleep(500); - Assert.assertTrue(manager2.isConnected()); - } - Thread.sleep(15000); - // Old entries should be cleaned up - for (int i = 0; i < 7; i++) { - ZkTestHelper.expireSession(zkClient); - Thread.sleep(1000); - Assert.assertTrue(manager2.isConnected()); - } - ZkTestHelper.disconnectSession(zkClient); - for (int i = 0; i < 20; i++) { - Thread.sleep(500); - if (!manager2.isConnected()) - break; - } - Assert.assertFalse(manager2.isConnected()); - } + // TODO test was disabled. check if it is still needed + // @Test(enabled = false) + // public void testDisconnectFlappingWindow() throws Exception { + // String className = TestHelper.getTestClassName(); + // String methodName = TestHelper.getTestMethodName(); + // String instanceName = "localhost_" + (12918 + 1); + // final String clusterName = className + "_" + methodName + UUID.randomUUID(); + // + // testDisconnectFlappingWindow2(instanceName, InstanceType.PARTICIPANT); + // testDisconnectFlappingWindow2("admin", InstanceType.ADMINISTRATOR); + // } + // + // public void testDisconnectFlappingWindow2(String instanceName, InstanceType type) + // throws Exception { + // String className = TestHelper.getTestClassName(); + // String methodName = TestHelper.getTestMethodName(); + // final String clusterName = className + "_" + methodName + UUID.randomUUID(); + // + // TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + // "localhost", // participant name prefix + // "TestDB", // resource name prefix + // 1, // resources + // 10, // partitions per resource + // 5, // number of nodes + // 3, // replicas + // "MasterSlave", true); // do rebalance + // + // // flapping time window to 5 sec + // System.setProperty("helixmanager.flappingTimeWindow", "15000"); + // System.setProperty("helixmanager.maxDisconnectThreshold", "7"); + // ZkHelixTestManager manager2 = new ZkHelixTestManager(clusterName, instanceName, type, ZK_ADDR); + // manager2.connect(); + // ZkClient zkClient = manager2.getZkClient(); + // for (int i = 0; i < 3; i++) { + // ZkTestHelper.expireSession(zkClient); + // Thread.sleep(500); + // Assert.assertTrue(manager2.isConnected()); + // } + // Thread.sleep(15000); + // // Old entries should be cleaned up + // for (int i = 0; i < 7; i++) { + // ZkTestHelper.expireSession(zkClient); + // Thread.sleep(1000); + // Assert.assertTrue(manager2.isConnected()); + // } + // ZkTestHelper.disconnectSession(zkClient); + // for (int i = 0; i < 20; i++) { + // Thread.sleep(500); + // if (!manager2.isConnected()) + // break; + // } + // Assert.assertFalse(manager2.isConnected()); + // } // @Test public void testDisconnectFlappingWindowController() throws Exception { @@ -140,8 +140,7 @@ public void testDisconnectFlappingWindowController() throws Exception { // flapping time window to 5 sec System.setProperty("helixmanager.flappingTimeWindow", "5000"); System.setProperty("helixmanager.maxDisconnectThreshold", "3"); - ZkHelixTestManager manager2 = - new ZkHelixTestManager(clusterName, null, InstanceType.CONTROLLER, ZK_ADDR); + ClusterControllerManager manager2 = new ClusterControllerManager(ZK_ADDR, clusterName, null); manager2.connect(); Thread.sleep(100); ZkClient zkClient = manager2.getZkClient(); diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkStateChangeListener.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkStateChangeListener.java index c208cbf656..aeb32f9516 100644 --- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkStateChangeListener.java +++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkStateChangeListener.java @@ -19,68 +19,70 @@ * under the License. */ -import org.apache.helix.TestHelper.StartCMResult; import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck; +import org.apache.zookeeper.Watcher.Event.KeeperState; import org.testng.Assert; import org.testng.annotations.Test; -import org.apache.zookeeper.Watcher.Event.KeeperState; public class TestZkStateChangeListener extends ZkStandAloneCMTestBaseWithPropertyServerCheck { - @Test + // TODO this test has been covered by TestZkFlapping. check if still needed + // @Test public void testDisconnectHistory() throws Exception { - String controllerName = CONTROLLER_PREFIX + "_0"; - StartCMResult controllerResult = _startCMResultMap.get(controllerName); - ZKHelixManager controller = (ZKHelixManager) controllerResult._manager; - ZkStateChangeListener listener1 = new ZkStateChangeListener(controller, 5000, 10); + // String controllerName = CONTROLLER_PREFIX + "_0"; + // StartCMResult controllerResult = _startCMResultMap.get(controllerName); + // ZKHelixManager controller = (ZKHelixManager) controllerResult._manager; + // ZkStateChangeListener listener1 = new ZkStateChangeListener(controller, 5000, 10); + // ZkStateChangeListener listener1 = new ZkStateChangeListener(_controller, 5000, 10); + // 11 disconnects in 5 sec for (int i = 0; i < 11; i++) { Thread.sleep(200); - listener1.handleStateChanged(KeeperState.Disconnected); + _controller.handleStateChanged(KeeperState.Disconnected); if (i < 10) { - Assert.assertTrue(controller.isConnected()); + Assert.assertTrue(_controller.isConnected()); } else { - Assert.assertFalse(controller.isConnected()); + Assert.assertFalse(_controller.isConnected()); } } // If maxDisconnectThreshold is 0 it should be set to 1 - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0); - ZKHelixManager manager = (ZKHelixManager) _startCMResultMap.get(instanceName)._manager; + // String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0); + // ZKHelixManager manager = (ZKHelixManager) _startCMResultMap.get(instanceName)._manager; - ZkStateChangeListener listener2 = new ZkStateChangeListener(manager, 5000, 0); + // ZkStateChangeListener listener2 = new ZkStateChangeListener(_participants[0], 5000, 0); for (int i = 0; i < 2; i++) { Thread.sleep(200); - listener2.handleStateChanged(KeeperState.Disconnected); + _participants[0].handleStateChanged(KeeperState.Disconnected); if (i < 1) { - Assert.assertTrue(manager.isConnected()); + Assert.assertTrue(_participants[0].isConnected()); } else { - Assert.assertFalse(manager.isConnected()); + Assert.assertFalse(_participants[0].isConnected()); } } // If there are long time after disconnect, older history should be cleanup - instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 1); - manager = (ZKHelixManager) _startCMResultMap.get(instanceName)._manager; + // instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 1); + // manager = (ZKHelixManager) _startCMResultMap.get(instanceName)._manager; - ZkStateChangeListener listener3 = new ZkStateChangeListener(manager, 5000, 5); + // ZkStateChangeListener listener3 = new ZkStateChangeListener(_participants[1], 5000, 5); for (int i = 0; i < 3; i++) { Thread.sleep(200); - listener3.handleStateChanged(KeeperState.Disconnected); - Assert.assertTrue(manager.isConnected()); + _participants[1].handleStateChanged(KeeperState.Disconnected); + Assert.assertTrue(_participants[1].isConnected()); } Thread.sleep(5000); // Old entries should be cleaned up for (int i = 0; i < 3; i++) { Thread.sleep(200); - listener3.handleStateChanged(KeeperState.Disconnected); - Assert.assertTrue(manager.isConnected()); + _participants[1].handleStateChanged(KeeperState.Disconnected); + Assert.assertTrue(_participants[1].isConnected()); } for (int i = 0; i < 2; i++) { Thread.sleep(200); - listener3.handleStateChanged(KeeperState.Disconnected); - Assert.assertTrue(manager.isConnected()); + _participants[1].handleStateChanged(KeeperState.Disconnected); + Assert.assertTrue(_participants[1].isConnected()); } - listener3.handleStateChanged(KeeperState.Disconnected); - Assert.assertFalse(manager.isConnected()); + _participants[1].handleStateChanged(KeeperState.Disconnected); + Assert.assertFalse(_participants[1].isConnected()); } } diff --git a/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallback.java b/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallback.java index 0c268d3686..52d43ce0c5 100644 --- a/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallback.java +++ b/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallback.java @@ -24,7 +24,7 @@ import java.util.List; import java.util.UUID; -import org.apache.helix.messaging.AsyncCallback; +import org.apache.helix.api.id.MessageId; import org.apache.helix.model.Message; import org.testng.AssertJUnit; import org.testng.annotations.Test; @@ -58,14 +58,14 @@ public void testAsyncCallback() throws Exception { List messageSent = new ArrayList(); for (int i = 0; i < nMsgs; i++) { - messageSent.add(new Message("Test", UUID.randomUUID().toString())); + messageSent.add(new Message("Test", MessageId.from(UUID.randomUUID().toString()))); } callback.setMessagesSent(messageSent); for (int i = 0; i < nMsgs; i++) { AssertJUnit.assertFalse(callback.isDone()); - callback.onReply(new Message("TestReply", UUID.randomUUID().toString())); + callback.onReply(new Message("TestReply", MessageId.from(UUID.randomUUID().toString()))); } AssertJUnit.assertTrue(callback.isDone()); @@ -82,7 +82,7 @@ public void testAsyncCallback() throws Exception { sleep(50); AssertJUnit.assertFalse(callback.isDone()); AssertJUnit.assertTrue(callback._onReplyMessageCalled == i); - callback.onReply(new Message("TestReply", UUID.randomUUID().toString())); + callback.onReply(new Message("TestReply", MessageId.from(UUID.randomUUID().toString()))); } sleep(1000); AssertJUnit.assertTrue(callback.isTimedOut()); @@ -99,7 +99,7 @@ public void testAsyncCallback() throws Exception { AssertJUnit.assertFalse(callback.isDone()); sleep(50); AssertJUnit.assertTrue(callback._onReplyMessageCalled == i); - callback.onReply(new Message("TestReply", UUID.randomUUID().toString())); + callback.onReply(new Message("TestReply", MessageId.from(UUID.randomUUID().toString()))); } AssertJUnit.assertTrue(callback.isDone()); sleep(1300); diff --git a/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallbackSvc.java b/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallbackSvc.java index 2be955ffc7..da686fec0e 100644 --- a/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallbackSvc.java +++ b/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallbackSvc.java @@ -19,25 +19,23 @@ * under the License. */ +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.UUID; + import org.apache.helix.HelixException; import org.apache.helix.HelixManager; import org.apache.helix.Mocks; import org.apache.helix.NotificationContext; -import org.apache.helix.messaging.AsyncCallback; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.messaging.handling.AsyncCallbackService; import org.apache.helix.messaging.handling.MessageHandler; -import org.apache.helix.messaging.handling.TestHelixTaskExecutor.MockClusterManager; import org.apache.helix.model.Message; -import org.testng.annotations.Test; import org.testng.AssertJUnit; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.UUID; - -import org.testng.Assert; import org.testng.annotations.Test; public class TestAsyncCallbackSvc { @@ -48,7 +46,7 @@ public String getSessionId() { } class TestAsyncCallback extends AsyncCallback { - HashSet _repliedMessageId = new HashSet(); + HashSet _repliedMessageId = new HashSet(); @Override public void onTimeOut() { @@ -59,7 +57,7 @@ public void onTimeOut() { @Override public void onReplyMessage(Message message) { // TODO Auto-generated method stub - _repliedMessageId.add(message.getMsgId()); + _repliedMessageId.add(message.getMessageId()); } } @@ -72,27 +70,27 @@ public void testAsyncCallbackSvc() throws Exception { HelixManager manager = new MockHelixManager(); NotificationContext changeContext = new NotificationContext(manager); - Message msg = new Message(svc.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId(manager.getSessionId()); + Message msg = new Message(svc.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from(manager.getSessionId())); try { MessageHandler aHandler = svc.createHandler(msg, changeContext); } catch (HelixException e) { - AssertJUnit.assertTrue(e.getMessage().indexOf(msg.getMsgId()) != -1); + AssertJUnit.assertTrue(e.getMessage().indexOf(msg.getMessageId().stringify()) != -1); } - Message msg2 = new Message("RandomType", UUID.randomUUID().toString()); - msg2.setTgtSessionId(manager.getSessionId()); + Message msg2 = new Message("RandomType", MessageId.from(UUID.randomUUID().toString())); + msg2.setTgtSessionId(SessionId.from(manager.getSessionId())); try { MessageHandler aHandler = svc.createHandler(msg2, changeContext); } catch (HelixException e) { - AssertJUnit.assertTrue(e.getMessage().indexOf(msg2.getMsgId()) != -1); + AssertJUnit.assertTrue(e.getMessage().indexOf(msg2.getMessageId().stringify()) != -1); } - Message msg3 = new Message(svc.getMessageType(), UUID.randomUUID().toString()); - msg3.setTgtSessionId(manager.getSessionId()); + Message msg3 = new Message(svc.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg3.setTgtSessionId(SessionId.from(manager.getSessionId())); msg3.setCorrelationId("wfwegw"); try { MessageHandler aHandler = svc.createHandler(msg3, changeContext); } catch (HelixException e) { - AssertJUnit.assertTrue(e.getMessage().indexOf(msg3.getMsgId()) != -1); + AssertJUnit.assertTrue(e.getMessage().indexOf(msg3.getMessageId().stringify()) != -1); } TestAsyncCallback callback = new TestAsyncCallback(); @@ -101,11 +99,11 @@ public void testAsyncCallbackSvc() throws Exception { svc.registerAsyncCallback(corrId, callback); List msgSent = new ArrayList(); - msgSent.add(new Message("Test", UUID.randomUUID().toString())); + msgSent.add(new Message("Test", MessageId.from(UUID.randomUUID().toString()))); callback.setMessagesSent(msgSent); - msg = new Message(svc.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("*"); + msg = new Message(svc.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("*")); msg.setCorrelationId(corrId); MessageHandler aHandler = svc.createHandler(msg, changeContext); @@ -113,6 +111,6 @@ public void testAsyncCallbackSvc() throws Exception { aHandler.handleMessage(); AssertJUnit.assertTrue(callback.isDone()); - AssertJUnit.assertTrue(callback._repliedMessageId.contains(msg.getMsgId())); + AssertJUnit.assertTrue(callback._repliedMessageId.contains(msg.getMessageId())); } } diff --git a/helix-core/src/test/java/org/apache/helix/messaging/TestDefaultMessagingService.java b/helix-core/src/test/java/org/apache/helix/messaging/TestDefaultMessagingService.java index 9686e16d48..11bdeb5e33 100644 --- a/helix-core/src/test/java/org/apache/helix/messaging/TestDefaultMessagingService.java +++ b/helix-core/src/test/java/org/apache/helix/messaging/TestDefaultMessagingService.java @@ -33,13 +33,14 @@ import org.apache.helix.PropertyKey; import org.apache.helix.PropertyType; import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; import org.apache.helix.messaging.handling.HelixTaskResult; import org.apache.helix.messaging.handling.MessageHandler; import org.apache.helix.messaging.handling.MessageHandlerFactory; import org.apache.helix.model.ExternalView; import org.apache.helix.model.LiveInstance.LiveInstanceProperty; import org.apache.helix.model.Message; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.testng.AssertJUnit; import org.testng.annotations.Test; @@ -94,7 +95,7 @@ public MockHelixManager() { _liveInstances.add(metaData); } _externalView = - DefaultIdealStateCalculator.calculateIdealState(_instances, _partitions, _replicas, _db, + DefaultTwoStateStrategy.calculateIdealState(_instances, _partitions, _replicas, _db, "MASTER", "SLAVE"); } @@ -173,7 +174,8 @@ public void TestMessageSend() { recipientCriteria.setRecipientInstanceType(InstanceType.PARTICIPANT); recipientCriteria.setSelfExcluded(true); - Message template = new Message(factory.getMessageType(), UUID.randomUUID().toString()); + Message template = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); AssertJUnit.assertEquals(0, svc.send(recipientCriteria, template)); recipientCriteria.setSelfExcluded(false); diff --git a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestConfigThreadpoolSize.java b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestConfigThreadpoolSize.java index 374f30de85..c71a782f7a 100644 --- a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestConfigThreadpoolSize.java +++ b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestConfigThreadpoolSize.java @@ -23,13 +23,13 @@ import java.util.concurrent.ThreadPoolExecutor; import org.apache.helix.ConfigAccessor; -import org.apache.helix.model.ConfigScope; -import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.helix.HelixManager; import org.apache.helix.NotificationContext; import org.apache.helix.integration.ZkStandAloneCMTestBase; import org.apache.helix.messaging.DefaultMessagingService; +import org.apache.helix.model.ConfigScope; import org.apache.helix.model.Message; +import org.apache.helix.model.builder.ConfigScopeBuilder; import org.testng.Assert; import org.testng.annotations.Test; @@ -77,7 +77,7 @@ public void reset() { @Test public void TestThreadPoolSizeConfig() { String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0); - HelixManager manager = _startCMResultMap.get(instanceName)._manager; + HelixManager manager = _participants[0]; ConfigAccessor accessor = manager.getConfigAccessor(); ConfigScope scope = @@ -91,9 +91,9 @@ public void TestThreadPoolSizeConfig() { for (int i = 0; i < NODE_NR; i++) { instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - _startCMResultMap.get(instanceName)._manager.getMessagingService() - .registerMessageHandlerFactory("TestMsg", new TestMessagingHandlerFactory()); - _startCMResultMap.get(instanceName)._manager.getMessagingService() + _participants[i].getMessagingService().registerMessageHandlerFactory("TestMsg", + new TestMessagingHandlerFactory()); + _participants[i].getMessagingService() .registerMessageHandlerFactory("TestMsg2", new TestMessagingHandlerFactory2()); } @@ -102,7 +102,7 @@ public void TestThreadPoolSizeConfig() { instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); DefaultMessagingService svc = - (DefaultMessagingService) (_startCMResultMap.get(instanceName)._manager + (DefaultMessagingService) (_participants[i] .getMessagingService()); HelixTaskExecutor helixExecutor = svc.getExecutor(); ThreadPoolExecutor executor = diff --git a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestHelixTaskExecutor.java b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestHelixTaskExecutor.java index 1ff6595efe..43818f7042 100644 --- a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestHelixTaskExecutor.java +++ b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestHelixTaskExecutor.java @@ -29,10 +29,8 @@ import org.apache.helix.HelixManager; import org.apache.helix.Mocks; import org.apache.helix.NotificationContext; -import org.apache.helix.messaging.handling.HelixTaskExecutor; -import org.apache.helix.messaging.handling.HelixTaskResult; -import org.apache.helix.messaging.handling.MessageHandler; -import org.apache.helix.messaging.handling.MessageHandlerFactory; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.SessionId; import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageState; import org.testng.Assert; @@ -60,7 +58,7 @@ public TestMessageHandler(Message message, NotificationContext context) { @Override public HelixTaskResult handleMessage() throws InterruptedException { HelixTaskResult result = new HelixTaskResult(); - _processedMsgIds.put(_message.getMsgId(), _message.getMsgId()); + _processedMsgIds.put(_message.getMessageId().stringify(), _message.getMessageId().stringify()); Thread.sleep(100); result.setSuccess(true); return result; @@ -127,14 +125,14 @@ public HelixTaskResult handleMessage() throws InterruptedException { if (_message.getRecord().getSimpleFields().containsKey("Cancelcount")) { sleepTimes = 10; } - _processingMsgIds.put(_message.getMsgId(), _message.getMsgId()); + _processingMsgIds.put(_message.getMessageId().stringify(), _message.getMessageId().stringify()); try { for (int i = 0; i < sleepTimes; i++) { Thread.sleep(100); } } catch (InterruptedException e) { _interrupted = true; - _timedOutMsgIds.put(_message.getMsgId(), ""); + _timedOutMsgIds.put(_message.getMessageId().stringify(), ""); result.setInterrupted(true); if (!_message.getRecord().getSimpleFields().containsKey("Cancelcount")) { _message.getRecord().setSimpleField("Cancelcount", "1"); @@ -144,7 +142,7 @@ public HelixTaskResult handleMessage() throws InterruptedException { } throw e; } - _processedMsgIds.put(_message.getMsgId(), _message.getMsgId()); + _processedMsgIds.put(_message.getMessageId().stringify(), _message.getMessageId().stringify()); result.setSuccess(true); return result; } @@ -196,8 +194,9 @@ public void testNormalMsgExecution() throws InterruptedException { int nMsgs1 = 5; for (int i = 0; i < nMsgs1; i++) { - Message msg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId(manager.getSessionId()); + Message msg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from(manager.getSessionId())); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); msg.setCorrelationId(UUID.randomUUID().toString()); @@ -206,8 +205,9 @@ public void testNormalMsgExecution() throws InterruptedException { int nMsgs2 = 6; for (int i = 0; i < nMsgs2; i++) { - Message msg = new Message(factory2.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId(manager.getSessionId()); + Message msg = + new Message(factory2.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from(manager.getSessionId())); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); msg.setCorrelationId(UUID.randomUUID().toString()); @@ -247,8 +247,9 @@ public void testUnknownTypeMsgExecution() throws InterruptedException { int nMsgs1 = 5; for (int i = 0; i < nMsgs1; i++) { - Message msg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId(manager.getSessionId()); + Message msg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from(manager.getSessionId())); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); msgList.add(msg); @@ -256,8 +257,9 @@ public void testUnknownTypeMsgExecution() throws InterruptedException { int nMsgs2 = 4; for (int i = 0; i < nMsgs2; i++) { - Message msg = new Message(factory2.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId(manager.getSessionId()); + Message msg = + new Message(factory2.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from(manager.getSessionId())); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); msgList.add(msg); @@ -294,16 +296,18 @@ public void testMsgSessionId() throws InterruptedException { int nMsgs1 = 5; for (int i = 0; i < nMsgs1; i++) { - Message msg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("*"); + Message msg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("*")); msg.setTgtName(""); msgList.add(msg); } int nMsgs2 = 4; for (int i = 0; i < nMsgs2; i++) { - Message msg = new Message(factory2.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("some other session id"); + Message msg = + new Message(factory2.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("some other session id")); msg.setTgtName(""); msgList.add(msg); } @@ -337,15 +341,17 @@ public void testCreateHandlerException() throws InterruptedException { int nMsgs1 = 5; for (int i = 0; i < nMsgs1; i++) { - Message msg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId(manager.getSessionId()); + Message msg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from(manager.getSessionId())); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); msg.setCorrelationId(UUID.randomUUID().toString()); msgList.add(msg); } - Message exceptionMsg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - exceptionMsg.setTgtSessionId(manager.getSessionId()); + Message exceptionMsg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + exceptionMsg.setTgtSessionId(SessionId.from(manager.getSessionId())); exceptionMsg.setMsgSubType("EXCEPTION"); exceptionMsg.setTgtName("Localhost_1123"); exceptionMsg.setSrcName("127.101.1.23_2234"); @@ -376,8 +382,9 @@ public void testTaskCancellation() throws InterruptedException { int nMsgs1 = 0; for (int i = 0; i < nMsgs1; i++) { - Message msg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("*"); + Message msg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("*")); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); msgList.add(msg); @@ -386,8 +393,9 @@ public void testTaskCancellation() throws InterruptedException { List msgListToCancel = new ArrayList(); int nMsgs2 = 4; for (int i = 0; i < nMsgs2; i++) { - Message msg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("*"); + Message msg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("*")); msgList.add(msg); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); @@ -432,16 +440,18 @@ public void testShutdown() throws InterruptedException { List msgList = new ArrayList(); for (int i = 0; i < nMsg1; i++) { - Message msg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("*"); + Message msg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("*")); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); msgList.add(msg); } for (int i = 0; i < nMsg2; i++) { - Message msg = new Message(factory2.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("*"); + Message msg = + new Message(factory2.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("*")); msgList.add(msg); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); @@ -449,8 +459,9 @@ public void testShutdown() throws InterruptedException { } for (int i = 0; i < nMsg3; i++) { - Message msg = new Message(factory3.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("*"); + Message msg = + new Message(factory3.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("*")); msgList.add(msg); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); @@ -486,8 +497,9 @@ public void testNoRetry() throws InterruptedException { int nMsgs2 = 4; // Test the case in which retry = 0 for (int i = 0; i < nMsgs2; i++) { - Message msg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("*"); + Message msg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("*")); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); msg.setExecutionTimeout((i + 1) * 600); @@ -530,8 +542,9 @@ public void testRetryOnce() throws InterruptedException { // Test the case that the message are executed for the second time int nMsgs2 = 4; for (int i = 0; i < nMsgs2; i++) { - Message msg = new Message(factory.getMessageType(), UUID.randomUUID().toString()); - msg.setTgtSessionId("*"); + Message msg = + new Message(factory.getMessageType(), MessageId.from(UUID.randomUUID().toString())); + msg.setTgtSessionId(SessionId.from("*")); msg.setTgtName("Localhost_1123"); msg.setSrcName("127.101.1.23_2234"); msg.setExecutionTimeout((i + 1) * 600); diff --git a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestResourceThreadpoolSize.java b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestResourceThreadpoolSize.java index 7ce677df76..a5777ab85e 100644 --- a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestResourceThreadpoolSize.java +++ b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestResourceThreadpoolSize.java @@ -22,12 +22,12 @@ import java.util.concurrent.ThreadPoolExecutor; import org.apache.helix.ConfigAccessor; -import org.apache.helix.model.ConfigScope; -import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.helix.HelixManager; import org.apache.helix.integration.ZkStandAloneCMTestBase; import org.apache.helix.messaging.DefaultMessagingService; +import org.apache.helix.model.ConfigScope; import org.apache.helix.model.Message.MessageType; +import org.apache.helix.model.builder.ConfigScopeBuilder; import org.apache.helix.tools.ClusterStateVerifier; import org.testng.Assert; import org.testng.annotations.Test; @@ -35,8 +35,7 @@ public class TestResourceThreadpoolSize extends ZkStandAloneCMTestBase { @Test public void TestThreadPoolSizeConfig() { - String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0); - HelixManager manager = _startCMResultMap.get(instanceName)._manager; + HelixManager manager = _participants[0]; ConfigAccessor accessor = manager.getConfigAccessor(); ConfigScope scope = new ConfigScopeBuilder().forCluster(manager.getClusterName()).forResource("NextDB").build(); @@ -52,11 +51,8 @@ public void TestThreadPoolSizeConfig() { long taskcount = 0; for (int i = 0; i < NODE_NR; i++) { - instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i); - DefaultMessagingService svc = - (DefaultMessagingService) (_startCMResultMap.get(instanceName)._manager - .getMessagingService()); + (DefaultMessagingService) (_participants[i].getMessagingService()); HelixTaskExecutor helixExecutor = svc.getExecutor(); ThreadPoolExecutor executor = (ThreadPoolExecutor) (helixExecutor._executorMap.get(MessageType.STATE_TRANSITION + "." diff --git a/helix-core/src/test/java/org/apache/helix/mock/controller/ClusterController.java b/helix-core/src/test/java/org/apache/helix/mock/controller/ClusterController.java deleted file mode 100644 index ae0a7fdf95..0000000000 --- a/helix-core/src/test/java/org/apache/helix/mock/controller/ClusterController.java +++ /dev/null @@ -1,129 +0,0 @@ -package org.apache.helix.mock.controller; - -/* - * 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.HelixManager; -import org.apache.helix.HelixManagerFactory; -import org.apache.helix.InstanceType; -import org.apache.helix.ZkHelixTestManager; -import org.apache.helix.controller.HelixControllerMain; -import org.apache.helix.participant.DistClusterControllerStateModelFactory; -import org.apache.helix.participant.StateMachineEngine; -import org.apache.log4j.Logger; - -public class ClusterController extends Thread { - private static Logger LOG = Logger.getLogger(ClusterController.class); - - private final CountDownLatch _startCountDown = new CountDownLatch(1); - private final CountDownLatch _stopCountDown = new CountDownLatch(1); - private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1); - private final String _controllerMode; - private final String _zkAddr; - - private ZkHelixTestManager _manager; - - public ClusterController(String clusterName, String controllerName, String zkAddr) - throws Exception { - this(clusterName, controllerName, zkAddr, HelixControllerMain.STANDALONE.toString()); - } - - public ClusterController(String clusterName, String controllerName, String zkAddr, - String controllerMode) throws Exception { - _controllerMode = controllerMode; - _zkAddr = zkAddr; - - if (_controllerMode.equals(HelixControllerMain.STANDALONE.toString())) { - _manager = - new ZkHelixTestManager(clusterName, controllerName, InstanceType.CONTROLLER, zkAddr); - } else if (_controllerMode.equals(HelixControllerMain.DISTRIBUTED.toString())) { - _manager = - new ZkHelixTestManager(clusterName, controllerName, InstanceType.CONTROLLER_PARTICIPANT, - zkAddr); - } else { - throw new IllegalArgumentException("Controller mode: " + controllerMode + " NOT recoginized"); - } - } - - public ZkHelixTestManager getManager() { - return _manager; - } - - public void syncStop() { - if (_manager == null) { - LOG.warn("manager already stopped"); - return; - } - - _stopCountDown.countDown(); - try { - _waitStopFinishCountDown.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - - public void syncStart() { - // TODO: prevent start multiple times - - super.start(); - try { - _startCountDown.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - - @Override - public void run() { - try { - try { - if (_controllerMode.equals(HelixControllerMain.STANDALONE.toString())) { - _manager.connect(); - } else if (_controllerMode.equals(HelixControllerMain.DISTRIBUTED.toString())) { - DistClusterControllerStateModelFactory stateModelFactory = - new DistClusterControllerStateModelFactory(_zkAddr); - - StateMachineEngine stateMach = _manager.getStateMachineEngine(); - stateMach.registerStateModelFactory("LeaderStandby", stateModelFactory); - _manager.connect(); - } - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } finally { - _startCountDown.countDown(); - _stopCountDown.await(); - } - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } finally { - synchronized (_manager) { - _manager.disconnect(); - _manager = null; - } - _waitStopFinishCountDown.countDown(); - } - } -} diff --git a/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java b/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java index 6ea3e36d0b..4ddaac4459 100644 --- a/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java +++ b/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java @@ -28,16 +28,20 @@ import java.util.Random; import java.util.TreeMap; -import org.apache.helix.ZNRecord; import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.ZNRecord; +import org.apache.helix.api.State; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.SessionId; 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.manager.zk.ZkClient; import org.apache.helix.model.ExternalView; -import org.apache.helix.model.Message; import org.apache.helix.model.IdealState.IdealStateProperty; import org.apache.helix.model.LiveInstance.LiveInstanceProperty; +import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageState; import org.apache.helix.model.Message.MessageType; import org.apache.helix.util.HelixUtil; @@ -57,18 +61,18 @@ public MockController(String src, String zkServer, String cluster) { client.setZkSerializer(new ZNRecordSerializer()); } - void sendMessage(String msgId, String instanceName, String fromState, String toState, + void sendMessage(MessageId msgId, String instanceName, String fromState, String toState, String partitionKey, int partitionId) throws InterruptedException, JsonGenerationException, JsonMappingException, IOException { Message message = new Message(MessageType.STATE_TRANSITION, msgId); - message.setMsgId(msgId); + message.setMessageId(msgId); message.setSrcName(srcName); message.setTgtName(instanceName); message.setMsgState(MessageState.NEW); - message.setFromState(fromState); - message.setToState(toState); + message.setFromState(State.from(fromState)); + message.setToState(State.from(toState)); // message.setPartitionId(partitionId); - message.setPartitionName(partitionKey); + message.setPartitionId(PartitionId.from(partitionKey)); String path = HelixUtil.getMessagePath(clusterName, instanceName) + "/" + message.getId(); ObjectMapper mapper = new ObjectMapper(); @@ -79,8 +83,8 @@ void sendMessage(String msgId, String instanceName, String fromState, String toS Thread.sleep(10000); ZNRecord record = client.readData(HelixUtil.getLiveInstancePath(clusterName, instanceName)); - message.setTgtSessionId(record.getSimpleField(LiveInstanceProperty.SESSION_ID.toString()) - .toString()); + message.setTgtSessionId(SessionId.from(record.getSimpleField( + LiveInstanceProperty.SESSION_ID.toString()).toString())); client.createPersistent(path, message); } diff --git a/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java b/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java index 0d0aa8407c..193abd3c9a 100644 --- a/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java +++ b/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.ArrayList; +import org.apache.helix.api.id.MessageId; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.map.JsonMappingException; @@ -53,22 +54,22 @@ public static void main(String[] args) throws JsonGenerationException, JsonMappi // Messages to initiate offline->slave->master->slave transitions - storageController.sendMessage("TestMessageId1", "localhost_8900", "Offline", "Slave", - "EspressoDB.partition-0", 0); + storageController.sendMessage(MessageId.from("TestMessageId1"), "localhost_8900", "Offline", + "Slave", "EspressoDB.partition-0", 0); Thread.sleep(10000); - storageController.sendMessage("TestMessageId2", "localhost_8900", "Slave", "Master", - "EspressoDB.partition-0", 0); + storageController.sendMessage(MessageId.from("TestMessageId2"), "localhost_8900", "Slave", + "Master", "EspressoDB.partition-0", 0); Thread.sleep(10000); - storageController.sendMessage("TestMessageId3", "localhost_8900", "Master", "Slave", - "EspressoDB.partition-0", 0); + storageController.sendMessage(MessageId.from("TestMessageId3"), "localhost_8900", "Master", + "Slave", "EspressoDB.partition-0", 0); Thread.sleep(10000); // Change the external view to trigger the consumer to listen from // another relay relayController.createExternalView(instanceNames, 10, 2, "EspressoDB", 10); - storageController.sendMessage("TestMessageId4", "localhost_8900", "Slave", "Offline", - "EspressoDB.partition-0", 0); + storageController.sendMessage(MessageId.from("TestMessageId4"), "localhost_8900", "Slave", + "Offline", "EspressoDB.partition-0", 0); Thread.sleep(10000); } diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/DummyProcess.java b/helix-core/src/test/java/org/apache/helix/mock/participant/DummyProcess.java index 2111a6528e..988060533b 100644 --- a/helix-core/src/test/java/org/apache/helix/mock/participant/DummyProcess.java +++ b/helix-core/src/test/java/org/apache/helix/mock/participant/DummyProcess.java @@ -32,6 +32,7 @@ import org.apache.helix.HelixManagerFactory; import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.model.Message; import org.apache.helix.participant.StateMachineEngine; import org.apache.helix.participant.statemachine.StateModel; @@ -162,7 +163,7 @@ public void setDelay(int delay) { } public void onBecomeSlaveFromOffline(Message message, NotificationContext context) { - String db = message.getPartitionName(); + PartitionId db = message.getPartitionId(); String instanceName = context.getManager().getInstanceName(); DummyProcess.sleep(_transDelay); @@ -207,7 +208,7 @@ public void setDelay(int delay) { } public void onBecomeOnlineFromOffline(Message message, NotificationContext context) { - String db = message.getPartitionName(); + PartitionId db = message.getPartitionId(); String instanceName = context.getManager().getInstanceName(); DummyProcess.sleep(_transDelay); @@ -238,7 +239,7 @@ public void setDelay(int delay) { } public void onBecomeLeaderFromStandby(Message message, NotificationContext context) { - String db = message.getPartitionName(); + PartitionId db = message.getPartitionId(); String instanceName = context.getManager().getInstanceName(); DummyProcess.sleep(_transDelay); logger.info("DummyLeaderStandbyStateModel.onBecomeLeaderFromStandby(), instance:" diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/ErrTransition.java b/helix-core/src/test/java/org/apache/helix/mock/participant/ErrTransition.java index 301cd62d29..a2e1414b1d 100644 --- a/helix-core/src/test/java/org/apache/helix/mock/participant/ErrTransition.java +++ b/helix-core/src/test/java/org/apache/helix/mock/participant/ErrTransition.java @@ -25,6 +25,8 @@ import java.util.Set; import org.apache.helix.NotificationContext; +import org.apache.helix.api.State; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.model.Message; // simulate error transition @@ -46,12 +48,12 @@ public ErrTransition(Map> errPartitions) { @Override public void doTransition(Message message, NotificationContext context) { - String fromState = message.getFromState(); - String toState = message.getToState(); - String partition = message.getPartitionName(); + State fromState = message.getTypedFromState(); + State toState = message.getTypedToState(); + PartitionId partition = message.getPartitionId(); String key = (fromState + "-" + toState).toUpperCase(); - if (_errPartitions.containsKey(key) && _errPartitions.get(key).contains(partition)) { + if (_errPartitions.containsKey(key) && _errPartitions.get(key).contains(partition.stringify())) { String errMsg = "IGNORABLE: test throw exception in msgId: " + message.getId() + " for " + partition + " transit from " + fromState + " to " + toState; diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockEspressoHealthReportProvider.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockEspressoHealthReportProvider.java deleted file mode 100644 index 84ba081502..0000000000 --- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockEspressoHealthReportProvider.java +++ /dev/null @@ -1,81 +0,0 @@ -package org.apache.helix.mock.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.HashMap; -import java.util.Map; - -import org.apache.helix.alerts.StatsHolder; -import org.apache.helix.healthcheck.HealthReportProvider; - -public class MockEspressoHealthReportProvider extends HealthReportProvider { - - private final String _reportName = "RestQueryStats"; - private HashMap> _statMap; - private final String DB_NAME = "DBName"; - - public MockEspressoHealthReportProvider() { - super(); - _statMap = new HashMap>(); - } - - public String buildMapKey(String dbName) { - return _reportName + "@" + DB_NAME + "=" + dbName; - } - - public void setStat(String dbName, String statName, String statVal) { - String currTime = String.valueOf(System.currentTimeMillis()); - setStat(dbName, statName, statVal, currTime); - } - - /* - * This version takes a fixed timestamp to ease with testing - */ - public void setStat(String dbName, String statName, String statVal, String timestamp) { - String key = buildMapKey(dbName); - Map dbStatMap = _statMap.get(key); - if (dbStatMap == null) { - dbStatMap = new HashMap(); - _statMap.put(key, dbStatMap); - } - dbStatMap.put(statName, statVal); - dbStatMap.put(StatsHolder.TIMESTAMP_NAME, timestamp); - } - - @Override - public Map getRecentHealthReport() { - return null; - } - - @Override - public Map> getRecentPartitionHealthReport() { - return _statMap; - } - - @Override - public void resetStats() { - _statMap.clear(); - } - - public String getReportName() { - return _reportName; - } - -} diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java deleted file mode 100644 index 59d9a0a001..0000000000 --- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java +++ /dev/null @@ -1,248 +0,0 @@ -package org.apache.helix.mock.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.HashMap; -import java.util.Map; -import java.util.Random; - -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.GnuParser; -import org.apache.commons.cli.HelpFormatter; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.OptionBuilder; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.helix.HelixManager; -import org.apache.helix.healthcheck.HealthReportProvider; -import org.apache.log4j.Logger; - -public class MockHealthReportParticipant { - private static final Logger LOG = Logger.getLogger(MockHealthReportParticipant.class); - public static final String zkServer = "zkSvr"; - public static final String cluster = "cluster"; - public static final String host = "host"; - public static final String port = "port"; - public static final String help = "help"; - - static class MockHealthReportProvider extends HealthReportProvider { - private final String _reportName = "MockRestQueryStats"; - private final Map> _mockHealthReport; - - public MockHealthReportProvider() { - _mockHealthReport = new HashMap>(); - - Map reportMap = new HashMap(); - _mockHealthReport.put("MockRestQueryStats@DBName=BizProfile", reportMap); - - reportMap.put("MeanMysqlLatency", "2.132700625"); - reportMap.put("95PercentileLatencyLucene", "108.40825525"); - reportMap.put("99PercentileLatencyMysql", "9.369827"); - reportMap.put("99PercentileLatencyServer", "167.714208"); - reportMap.put("95PercentileLatencyMysqlPool", "8.03621375"); - reportMap.put("95PercentileLatencyServer", "164.68374265"); - reportMap.put("MinLuceneLatency", "1.765908"); - reportMap.put("MaxServerLatency", "167.714208"); - reportMap.put("MeanLuceneLatency", "16.107599458333336"); - reportMap.put("CollectorName", "RestQueryStats"); - reportMap.put("MeanLucenePoolLatency", "8.120545333333332"); - reportMap.put("99PercentileLatencyLucenePool", "65.930564"); - reportMap.put("MinServerLatency", "0.425272"); - reportMap.put("IndexStoreMismatchCount", "0"); - reportMap.put("ErrorCount", "0"); - reportMap.put("MeanMysqlPoolLatency", "1.0704102916666667"); - reportMap.put("MinLucenePoolLatency", "0.008189"); - reportMap.put("MinMysqlLatency", "0.709691"); - reportMap.put("MaxMysqlPoolLatency", "8.606973"); - reportMap.put("99PercentileLatencyMysqlPool", "8.606973"); - reportMap.put("MinMysqlPoolLatency", "0.091883"); - reportMap.put("MaxLucenePoolLatency", "65.930564"); - reportMap.put("99PercentileLatencyLucene", "111.78799"); - reportMap.put("MaxMysqlLatency", "9.369827"); - reportMap.put("TimeStamp", "1332895048143"); - reportMap.put("MeanConcurrencyLevel", "1.9"); - reportMap.put("95PercentileLatencyMysql", "8.96594875"); - reportMap.put("QueryStartCount", "0"); - reportMap.put("95PercentileLatencyLucenePool", "63.518656500000006"); - reportMap.put("MeanServerLatency", "39.5451532"); - reportMap.put("MaxLuceneLatency", "111.78799"); - reportMap.put("QuerySuccessCount", "0"); - } - - @Override - public Map getRecentHealthReport() { - // TODO Auto-generated method stub - return null; - } - - @Override - public void resetStats() { - // TODO Auto-generated method stub - - } - - @Override - public Map> getRecentPartitionHealthReport() { - // tweak: randomly change the last digit - for (String key1 : _mockHealthReport.keySet()) { - Map reportMap = _mockHealthReport.get(key1); - for (String key2 : reportMap.keySet()) { - String value = reportMap.get(key2); - String lastDigit = "" + new Random().nextInt(10); - value = value.substring(0, value.length() - 1) + lastDigit; - reportMap.put(key2, value); - } - } - - return _mockHealthReport; - } - - @Override - public String getReportName() { - return _reportName; - } - } - - static class MockHealthReportJob implements MockJobIntf { - - @Override - public void doPreConnectJob(HelixManager manager) { - // TODO Auto-generated method stub - - } - - @Override - public void doPostConnectJob(HelixManager manager) { - // TODO Auto-generated method stub - manager.getHealthReportCollector().addHealthReportProvider(new MockHealthReportProvider()); - - // // set property store path for perf test - // final String setPath = "/TEST_PERF/set"; - // final String updatePath = "/TEST_PERF/update"; - // manager.getHelixPropertyStore().create(setPath, new ZNRecord(setPath), - // BaseDataAccessor.Option.PERSISTENT); - // manager.getHelixPropertyStore().set(updatePath, new ZNRecord(updatePath), - // BaseDataAccessor.Option.PERSISTENT); - } - - } - - // hack OptionBuilder is not thread safe - @SuppressWarnings("static-access") - synchronized private static Options constructCommandLineOptions() { - Option helpOption = - OptionBuilder.withLongOpt(help).withDescription("Prints command-line options info") - .create(); - - Option clusterOption = - OptionBuilder.withLongOpt(cluster).withDescription("Provide cluster name").create(); - clusterOption.setArgs(1); - clusterOption.setRequired(true); - clusterOption.setArgName("Cluster name (Required)"); - - Option hostOption = - OptionBuilder.withLongOpt(host).withDescription("Provide host name").create(); - hostOption.setArgs(1); - hostOption.setRequired(true); - hostOption.setArgName("Host name (Required)"); - - Option portOption = - OptionBuilder.withLongOpt(port).withDescription("Provide host port").create(); - portOption.setArgs(1); - portOption.setRequired(true); - portOption.setArgName("Host port (Required)"); - - Option zkServerOption = - OptionBuilder.withLongOpt(zkServer).withDescription("Provide zookeeper address").create(); - zkServerOption.setArgs(1); - zkServerOption.setRequired(true); - zkServerOption.setArgName("Zookeeper server address(Required)"); - - Options options = new Options(); - options.addOption(helpOption); - options.addOption(clusterOption); - options.addOption(hostOption); - options.addOption(portOption); - options.addOption(zkServerOption); - - return options; - } - - public static void printUsage(Options cliOptions) { - HelpFormatter helpFormatter = new HelpFormatter(); - helpFormatter.printHelp("java " + MockHealthReportParticipant.class.getName(), cliOptions); - } - - public static CommandLine processCommandLineArgs(String[] cliArgs) throws Exception { - CommandLineParser cliParser = new GnuParser(); - Options cliOptions = constructCommandLineOptions(); - - try { - - return cliParser.parse(cliOptions, cliArgs); - } catch (ParseException pe) { - System.err.println("CommandLineClient: failed to parse command-line options: " - + pe.toString()); - printUsage(cliOptions); - System.exit(1); - } - return null; - } - - // NOT working for kill -9, working for kill -2/-15 - static class MockHealthReportParticipantShutdownHook extends Thread { - final MockParticipant _participant; - - MockHealthReportParticipantShutdownHook(MockParticipant participant) { - _participant = participant; - } - - @Override - public void run() { - LOG.info("MockHealthReportParticipantShutdownHook invoked"); - _participant.syncStop(); - } - } - - public static void main(String[] args) throws Exception { - CommandLine cmd = processCommandLineArgs(args); - String zkConnectStr = cmd.getOptionValue(zkServer); - String clusterName = cmd.getOptionValue(cluster); - String hostStr = cmd.getOptionValue(host); - String portStr = cmd.getOptionValue(port); - - String instanceName = hostStr + "_" + portStr; - - MockParticipant participant = - new MockParticipant(clusterName, instanceName, zkConnectStr, null, // new - // StoreAccessDiffNodeTransition(), - // // new - // StoreAccessOneNodeTransition(), - new MockHealthReportJob()); - Runtime.getRuntime().addShutdownHook(new MockHealthReportParticipantShutdownHook(participant)); - - // Espresso_driver.py will consume this - System.out - .println("MockHealthReportParticipant process started, instanceName: " + instanceName); - - participant.run(); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockMSModelFactory.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockMSModelFactory.java index ff7a45597a..9325934b46 100644 --- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockMSModelFactory.java +++ b/helix-core/src/test/java/org/apache/helix/mock/participant/MockMSModelFactory.java @@ -19,8 +19,6 @@ * under the License. */ -import java.util.Map; - import org.apache.helix.participant.statemachine.StateModelFactory; // mock master slave state model factory diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockMSStateModel.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockMSStateModel.java index 61733ba26a..78d983272a 100644 --- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockMSStateModel.java +++ b/helix-core/src/test/java/org/apache/helix/mock/participant/MockMSStateModel.java @@ -44,6 +44,7 @@ public void setTransition(MockTransition transition) { } // overwrite default error->dropped transition + @Override @Transition(to = "DROPPED", from = "ERROR") public void onBecomeDroppedFromError(Message message, NotificationContext context) throws InterruptedException { @@ -109,6 +110,18 @@ public void onBecomeOfflineFromError(Message message, NotificationContext contex } } + @Transition(to = "LEADER", from = "MASTER") + public void onBecomeLeaderFromMaster(Message message, NotificationContext context) + throws InterruptedException { + LOG.info("Become LEADER from MASTER"); + } + + @Transition(to = "MASTER", from = "LEADER") + public void onBecomeMasterFromLeader(Message message, NotificationContext context) + throws InterruptedException { + LOG.info("Become MASTER from LEADER"); + } + @Override public void reset() { LOG.info("Default MockMSStateModel.reset() invoked"); diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockParticipant.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockParticipant.java deleted file mode 100644 index 9eb4bb932b..0000000000 --- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockParticipant.java +++ /dev/null @@ -1,197 +0,0 @@ -package org.apache.helix.mock.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.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CountDownLatch; - -import org.I0Itec.zkclient.DataUpdater; -import org.I0Itec.zkclient.exception.ZkNoNodeException; -import org.apache.helix.AccessOption; -import org.apache.helix.HelixManager; -import org.apache.helix.HelixManagerFactory; -import org.apache.helix.InstanceType; -import org.apache.helix.NotificationContext; -import org.apache.helix.ZNRecord; -import org.apache.helix.ZkHelixTestManager; -import org.apache.helix.mock.participant.DummyProcess.DummyLeaderStandbyStateModelFactory; -import org.apache.helix.mock.participant.DummyProcess.DummyOnlineOfflineStateModelFactory; -import org.apache.helix.model.Message; -import org.apache.helix.participant.StateMachineEngine; -import org.apache.helix.participant.statemachine.StateModel; -import org.apache.helix.participant.statemachine.StateModelFactory; -import org.apache.helix.participant.statemachine.StateModelInfo; -import org.apache.helix.participant.statemachine.Transition; -import org.apache.helix.store.zk.ZkHelixPropertyStore; -import org.apache.log4j.Logger; - -public class MockParticipant extends Thread { - private static Logger LOG = Logger.getLogger(MockParticipant.class); - private final String _clusterName; - private final String _instanceName; - // private final String _zkAddr; - - private final CountDownLatch _startCountDown = new CountDownLatch(1); - private final CountDownLatch _stopCountDown = new CountDownLatch(1); - private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1); - - private final ZkHelixTestManager _manager; - private final StateModelFactory _msModelFactory; - private final MockJobIntf _job; - - public MockParticipant(String clusterName, String instanceName, String zkAddr) throws Exception { - this(clusterName, instanceName, zkAddr, null, null); - } - - public MockParticipant(String clusterName, String instanceName, String zkAddr, - MockTransition transition) throws Exception { - this(clusterName, instanceName, zkAddr, transition, null); - } - - public MockParticipant(String clusterName, String instanceName, String zkAddr, - MockTransition transition, MockJobIntf job) throws Exception { - _clusterName = clusterName; - _instanceName = instanceName; - _msModelFactory = new MockMSModelFactory(transition); - - _manager = - new ZkHelixTestManager(_clusterName, _instanceName, InstanceType.PARTICIPANT, zkAddr); - _job = job; - } - - public MockParticipant(StateModelFactory factory, String clusterName, String instanceName, - String zkAddr, MockJobIntf job) throws Exception { - _clusterName = clusterName; - _instanceName = instanceName; - _msModelFactory = factory; - - _manager = - new ZkHelixTestManager(_clusterName, _instanceName, InstanceType.PARTICIPANT, zkAddr); - _job = job; - } - - public StateModelFactory getStateModelFactory() { - return _msModelFactory; - } - - public MockParticipant(ZkHelixTestManager manager, MockTransition transition) { - _clusterName = manager.getClusterName(); - _instanceName = manager.getInstanceName(); - _manager = manager; - - _msModelFactory = new MockMSModelFactory(transition); - _job = null; - } - - public void setTransition(MockTransition transition) { - if (_msModelFactory instanceof MockMSModelFactory) { - ((MockMSModelFactory) _msModelFactory).setTrasition(transition); - } - } - - public ZkHelixTestManager getManager() { - return _manager; - } - - public String getInstanceName() { - return _instanceName; - } - - public String getClusterName() { - return _clusterName; - } - - public void syncStop() { - _stopCountDown.countDown(); - try { - _waitStopFinishCountDown.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - - // synchronized (_manager) - // { - // _manager.disconnect(); - // } - } - - public void syncStart() { - super.start(); - try { - _startCountDown.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - - @Override - public void run() { - try { - StateMachineEngine stateMach = _manager.getStateMachineEngine(); - stateMach.registerStateModelFactory("MasterSlave", _msModelFactory); - - DummyLeaderStandbyStateModelFactory lsModelFactory = - new DummyLeaderStandbyStateModelFactory(10); - DummyOnlineOfflineStateModelFactory ofModelFactory = - new DummyOnlineOfflineStateModelFactory(10); - stateMach.registerStateModelFactory("LeaderStandby", lsModelFactory); - stateMach.registerStateModelFactory("OnlineOffline", ofModelFactory); - - MockSchemataModelFactory schemataFactory = new MockSchemataModelFactory(); - stateMach.registerStateModelFactory("STORAGE_DEFAULT_SM_SCHEMATA", schemataFactory); - // MockBootstrapModelFactory bootstrapFactory = new MockBootstrapModelFactory(); - // stateMach.registerStateModelFactory("Bootstrap", bootstrapFactory); - - if (_job != null) { - _job.doPreConnectJob(_manager); - } - - _manager.connect(); - _startCountDown.countDown(); - - if (_job != null) { - _job.doPostConnectJob(_manager); - } - - _stopCountDown.await(); - } catch (InterruptedException e) { - String msg = - "participant: " + _instanceName + ", " + Thread.currentThread().getName() - + " is interrupted"; - LOG.info(msg); - System.err.println(msg); - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } finally { - _startCountDown.countDown(); - - synchronized (_manager) { - _manager.disconnect(); - } - _waitStopFinishCountDown.countDown(); - } - } -} diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessDiffNodeTransition.java b/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessDiffNodeTransition.java index cee2a79926..bddc2137d4 100644 --- a/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessDiffNodeTransition.java +++ b/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessDiffNodeTransition.java @@ -34,8 +34,8 @@ public class StoreAccessDiffNodeTransition extends MockTransition { public void doTransition(Message message, NotificationContext context) { HelixManager manager = context.getManager(); ZkHelixPropertyStore store = manager.getHelixPropertyStore(); - final String setPath = "/TEST_PERF/set/" + message.getPartitionName(); - final String updatePath = "/TEST_PERF/update/" + message.getPartitionName(); + final String setPath = "/TEST_PERF/set/" + message.getPartitionId(); + final String updatePath = "/TEST_PERF/update/" + message.getPartitionId(); // final String key = message.getPartitionName(); try { // get/set once diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessOneNodeTransition.java b/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessOneNodeTransition.java index 9e1c487db5..6c7ac48656 100644 --- a/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessOneNodeTransition.java +++ b/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessOneNodeTransition.java @@ -25,6 +25,7 @@ import org.apache.helix.HelixManager; import org.apache.helix.NotificationContext; import org.apache.helix.ZNRecord; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.model.Message; import org.apache.helix.store.zk.ZkHelixPropertyStore; @@ -36,7 +37,7 @@ public void doTransition(Message message, NotificationContext context) { ZkHelixPropertyStore store = manager.getHelixPropertyStore(); final String setPath = "/TEST_PERF/set"; final String updatePath = "/TEST_PERF/update"; - final String key = message.getPartitionName(); + final PartitionId key = message.getPartitionId(); try { // get/set once ZNRecord record = null; @@ -56,7 +57,7 @@ public ZNRecord update(ZNRecord currentData) { if (currentData == null) { currentData = new ZNRecord(updatePath); } - currentData.setSimpleField(key, "" + System.currentTimeMillis()); + currentData.setSimpleField(key.stringify(), "" + System.currentTimeMillis()); return currentData; } diff --git a/helix-core/src/test/java/org/apache/helix/model/TestAlertConfig.java b/helix-core/src/test/java/org/apache/helix/model/TestAlertConfig.java new file mode 100644 index 0000000000..b2ce9e17b6 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/model/TestAlertConfig.java @@ -0,0 +1,71 @@ +package org.apache.helix.model; + +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.controller.alert.AlertAction; +import org.apache.helix.controller.alert.AlertName; +import org.testng.Assert; +import org.testng.annotations.Test; + +/* + * 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 TestAlertConfig { + @Test + public void testBasic() { + AlertConfig config = new AlertConfig("default"); + AlertName name = + new AlertName.Builder().cluster(ClusterId.from("TestCluster")).metric("latency95") + .largerThan("1000").build(); + AlertAction action = + new AlertAction.Builder().cmd("enableInstance") + .args("{cluster}", "{node}", "false").build(); + config.putConfig(name, action); + + AlertName matchingName = + new AlertName.Builder().cluster(ClusterId.from("TestCluster")) + .node(ParticipantId.from("localhost_10001")).metric("latency95").largerThan("1000") + .build(); + AlertAction matchingAction = config.findAlertAction(matchingName); + + Assert.assertNotNull(matchingAction); + Assert.assertEquals(matchingAction.toString(), + "(enableInstance)(TestCluster localhost_10001 false)"); + } + + @Test + public void testFail() { + AlertConfig config = new AlertConfig("default"); + AlertName name = + new AlertName.Builder().cluster(ClusterId.from("TestCluster")).metric("latency95") + .largerThan("1000").build(); + AlertAction action = + new AlertAction.Builder().cmd("enableInstance") + .args("{cluster}", "{node}", "false").build(); + config.putConfig(name, action); + + AlertName matchingName = + new AlertName.Builder().cluster(ClusterId.from("TestCluster")) + .resource(ResourceId.from("TestDB")).metric("latency95").largerThan("1000").build(); + AlertAction matchingAction = config.findAlertAction(matchingName); + Assert.assertNull(matchingAction, "Should fail on null scope value for {node}"); + + } +} \ No newline at end of file diff --git a/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java b/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java index f11af7fa15..691492e9c2 100644 --- a/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java +++ b/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java @@ -25,23 +25,20 @@ import java.util.Set; import java.util.TreeMap; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.api.id.MessageId; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.model.ClusterConstraints; -import org.apache.helix.model.Message; import org.apache.helix.model.ClusterConstraints.ConstraintAttribute; import org.apache.helix.model.ClusterConstraints.ConstraintType; import org.apache.helix.model.Message.MessageType; -import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.annotations.Test; public class TestConstraint extends ZkUnitTestBase { - private static Logger LOG = Logger.getLogger(TestConstraint.class); @Test public void testMsgConstraint() { @@ -122,8 +119,8 @@ record = // message1 Message msg1 = - createMessage(MessageType.STATE_TRANSITION, "msgId-001", "OFFLINE", "SLAVE", "TestDB", - "localhost_12918"); + createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-001"), "OFFLINE", + "SLAVE", "TestDB", "localhost_12918"); Map msgAttr = ClusterConstraints.toConstraintAttributes(msg1); Set matches = constraint.match(msgAttr); @@ -137,8 +134,8 @@ record = // message2 Message msg2 = - createMessage(MessageType.STATE_TRANSITION, "msgId-002", "OFFLINE", "SLAVE", "TestDB", - "localhost_12919"); + createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-002"), "OFFLINE", + "SLAVE", "TestDB", "localhost_12919"); msgAttr = ClusterConstraints.toConstraintAttributes(msg2); matches = constraint.match(msgAttr); @@ -186,7 +183,7 @@ public void testStateConstraint() { ConstraintItem constraint2 = new ConstraintItem(record.getMapField("constraint2")); ZKHelixDataAccessor accessor = - new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); Builder keyBuilder = accessor.keyBuilder(); accessor.setProperty(keyBuilder.constraint(ConstraintType.STATE_CONSTRAINT.toString()), diff --git a/helix-core/src/test/java/org/apache/helix/model/TestIdealState.java b/helix-core/src/test/java/org/apache/helix/model/TestIdealState.java index e30aff8cbf..e00f245a12 100644 --- a/helix-core/src/test/java/org/apache/helix/model/TestIdealState.java +++ b/helix-core/src/test/java/org/apache/helix/model/TestIdealState.java @@ -19,9 +19,19 @@ * under the License. */ -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.helix.TestHelper; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.model.IdealState.IdealStateModeProperty; import org.apache.helix.model.IdealState.RebalanceMode; import org.testng.Assert; @@ -45,24 +55,28 @@ public void testGetInstanceSet() { // test SEMI_AUTO mode idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); - Set instances = idealState.getInstanceSet("TestDB_0"); + Set instances = idealState.getParticipantSet(PartitionId.from("TestDB_0")); // System.out.println("instances: " + instances); Assert.assertEquals(instances.size(), 2, "Should contain node_1 and node_2"); - Assert.assertTrue(instances.contains("node_1"), "Should contain node_1 and node_2"); - Assert.assertTrue(instances.contains("node_2"), "Should contain node_1 and node_2"); + Assert.assertTrue(instances.contains(ParticipantId.from("node_1")), + "Should contain node_1 and node_2"); + Assert.assertTrue(instances.contains(ParticipantId.from("node_2")), + "Should contain node_1 and node_2"); - instances = idealState.getInstanceSet("TestDB_nonExist_auto"); + instances = idealState.getParticipantSet(PartitionId.from("TestDB_nonExist_auto")); Assert.assertEquals(instances, Collections.emptySet(), "Should get empty set"); // test CUSTOMIZED mode idealState.setRebalanceMode(RebalanceMode.CUSTOMIZED); - instances = idealState.getInstanceSet("TestDB_1"); + instances = idealState.getParticipantSet(PartitionId.from("TestDB_1")); // System.out.println("instances: " + instances); Assert.assertEquals(instances.size(), 2, "Should contain node_3 and node_4"); - Assert.assertTrue(instances.contains("node_3"), "Should contain node_3 and node_4"); - Assert.assertTrue(instances.contains("node_4"), "Should contain node_3 and node_4"); + Assert.assertTrue(instances.contains(ParticipantId.from("node_3")), + "Should contain node_3 and node_4"); + Assert.assertTrue(instances.contains(ParticipantId.from("node_4")), + "Should contain node_3 and node_4"); - instances = idealState.getInstanceSet("TestDB_nonExist_custom"); + instances = idealState.getParticipantSet(PartitionId.from("TestDB_nonExist_custom")); Assert.assertEquals(instances, Collections.emptySet(), "Should get empty set"); System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis())); @@ -73,7 +87,7 @@ public void testReplicas() { IdealState idealState = new IdealState("test-db"); idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO); idealState.setNumPartitions(4); - idealState.setStateModelDefRef("MasterSlave"); + idealState.setStateModelDefId(StateModelDefId.from("MasterSlave")); idealState.setReplicas("" + 2); diff --git a/helix-core/src/test/java/org/apache/helix/model/TestStateModelValidity.java b/helix-core/src/test/java/org/apache/helix/model/TestStateModelValidity.java new file mode 100644 index 0000000000..2e4d5f00d2 --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/model/TestStateModelValidity.java @@ -0,0 +1,252 @@ +package org.apache.helix.model; + +/* + * 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.ZNRecord; +import org.apache.helix.model.StateModelDefinition.StateModelDefinitionProperty; +import org.apache.helix.tools.StateModelConfigGenerator; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.google.common.collect.Lists; + +public class TestStateModelValidity { + /** + * Ensure that state models that we know to be good pass validation + */ + @Test + public void testValidModels() { + StateModelDefinition masterSlave = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + Assert.assertTrue(masterSlave.isValid()); + + StateModelDefinition leaderStandby = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForLeaderStandby()); + Assert.assertTrue(leaderStandby.isValid()); + + StateModelDefinition onlineOffline = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForOnlineOffline()); + Assert.assertTrue(onlineOffline.isValid()); + } + + /** + * Ensure that Helix responds negatively if DROPPED is not specified + */ + @Test + public void testNoDroppedState() { + StateModelDefinition stateModel = + new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE") + .addState("MASTER").addState("SLAVE").addTransition("OFFLINE", "SLAVE") + .addTransition("SLAVE", "MASTER").addTransition("MASTER", "SLAVE") + .addTransition("SLAVE", "OFFLINE").build(); + Assert.assertFalse(stateModel.isValid()); + } + + /** + * Ensure that Helix can catch when a state doesn't have a path to DROPPED + */ + @Test + public void testNoPathToDropped() { + StateModelDefinition stateModel = + new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE") + .addState("MASTER").addState("SLAVE").addState("DROPPED") + .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "MASTER") + .addTransition("SLAVE", "OFFLINE").addTransition("OFFLINE", "DROPPED").build(); + Assert.assertFalse(stateModel.isValid()); + + // now see that adding MASTER-DROPPED fixes the problem + stateModel = + new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE") + .addState("MASTER").addState("SLAVE").addState("DROPPED") + .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "MASTER") + .addTransition("SLAVE", "OFFLINE").addTransition("OFFLINE", "DROPPED") + .addTransition("MASTER", "DROPPED").build(); + Assert.assertTrue(stateModel.isValid()); + } + + /** + * The initial state should be added as a state, otherwise validation check should fail + */ + @Test + public void testInitialStateIsNotState() { + StateModelDefinition stateModel = + new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("MASTER") + .addState("SLAVE").addState("DROPPED").addTransition("OFFLINE", "SLAVE") + .addTransition("SLAVE", "MASTER").addTransition("SLAVE", "OFFLINE") + .addTransition("OFFLINE", "DROPPED").addTransition("MASTER", "SLAVE").build(); + Assert.assertFalse(stateModel.isValid()); + } + + /** + * There should be an initial state, otherwise instantiation should fail + */ + @Test + public void testNoInitialState() { + try { + new StateModelDefinition.Builder("stateModel").addState("OFFLINE").addState("MASTER") + .addState("SLAVE").addState("DROPPED").addTransition("OFFLINE", "SLAVE") + .addTransition("SLAVE", "MASTER").addTransition("SLAVE", "OFFLINE") + .addTransition("OFFLINE", "DROPPED").addTransition("MASTER", "SLAVE").build(); + Assert.fail("StateModelDefinition creation should fail if no initial state"); + } catch (IllegalArgumentException e) { + } + } + + /** + * SRC and DEST in a transition SRC-TEST must be valid states + */ + @Test + public void testTransitionsWithInvalidStates() { + // invalid to state + StateModelDefinition stateModel = + new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE") + .addState("MASTER").addState("SLAVE").addState("DROPPED") + .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "MASTER") + .addTransition("SLAVE", "OFFLINE").addTransition("OFFLINE", "DROPPED") + .addTransition("MASTER", "SLAVE").addTransition("OFFLINE", "INVALID").build(); + Assert.assertFalse(stateModel.isValid()); + + // invalid from state + stateModel = + new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE") + .addState("MASTER").addState("SLAVE").addState("DROPPED") + .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "MASTER") + .addTransition("SLAVE", "OFFLINE").addTransition("OFFLINE", "DROPPED") + .addTransition("MASTER", "SLAVE").addTransition("INVALID", "MASTER").build(); + Assert.assertFalse(stateModel.isValid()); + } + + /** + * The initial state should be able to reach all states, should fail validation otherwise + */ + @Test + public void testUnreachableState() { + StateModelDefinition stateModel = + new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE") + .addState("MASTER").addState("SLAVE").addState("DROPPED") + .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "OFFLINE") + .addTransition("OFFLINE", "DROPPED").addTransition("MASTER", "SLAVE") + .addTransition("MASTER", "DROPPED").build(); + Assert.assertFalse(stateModel.isValid()); + } + + /** + * The validator should fail on any detected infinite loops + */ + @Test + public void testLoopInStateModel() { + // create an infinite loop ONE --> TWO --> ONE + ZNRecord record = new ZNRecord("MasterSlave"); + record.setSimpleField(StateModelDefinitionProperty.INITIAL_STATE.toString(), "OFFLINE"); + List statePriorityList = + Lists.newArrayList("ONE", "TWO", "THREE", "OFFLINE", "DROPPED", "ERROR"); + record.setListField(StateModelDefinitionProperty.STATE_PRIORITY_LIST.toString(), + statePriorityList); + for (String state : statePriorityList) { + String key = state + ".meta"; + Map metadata = new HashMap(); + metadata.put("count", "-1"); + record.setMapField(key, metadata); + } + for (String state : statePriorityList) { + String key = state + ".next"; + if (state.equals("ONE")) { + Map metadata = new HashMap(); + metadata.put("THREE", "TWO"); + metadata.put("TWO", "TWO"); + metadata.put("OFFLINE", "OFFLINE"); + metadata.put("DROPPED", "DROPPED"); + record.setMapField(key, metadata); + } else if (state.equals("TWO")) { + Map metadata = new HashMap(); + metadata.put("THREE", "ONE"); + metadata.put("OFFLINE", "OFFLINE"); + metadata.put("DROPPED", "OFFLINE"); + record.setMapField(key, metadata); + } else if (state.equals("THREE")) { + Map metadata = new HashMap(); + metadata.put("OFFLINE", "OFFLINE"); + metadata.put("DROPPED", "OFFLINE"); + record.setMapField(key, metadata); + } else if (state.equals("OFFLINE")) { + Map metadata = new HashMap(); + metadata.put("ONE", "ONE"); + metadata.put("TWO", "TWO"); + metadata.put("THREE", "THREE"); + metadata.put("DROPPED", "DROPPED"); + record.setMapField(key, metadata); + } else if (state.equals("ERROR")) { + Map metadata = new HashMap(); + metadata.put("OFFLINE", "OFFLINE"); + record.setMapField(key, metadata); + } + } + List stateTransitionPriorityList = new ArrayList(); + record.setListField(StateModelDefinitionProperty.STATE_TRANSITION_PRIORITYLIST.toString(), + stateTransitionPriorityList); + + StateModelDefinition stateModel = new StateModelDefinition(record); + Assert.assertFalse(stateModel.isValid()); + } + + /** + * This is the example used on the website, so this must work + */ + @Test + public void testBasic() { + StateModelDefinition stateModel = new StateModelDefinition.Builder("MasterSlave") + // OFFLINE is the state that the system starts in (initial state is REQUIRED) + .initialState("OFFLINE") + + // Lowest number here indicates highest priority, no value indicates lowest priority + .addState("MASTER", 1).addState("SLAVE", 2).addState("OFFLINE") + + // Note the special inclusion of the DROPPED state (REQUIRED) + .addState(HelixDefinedState.DROPPED.toString()) + + // No more than one master allowed + .upperBound("MASTER", 1) + + // R indicates an upper bound of number of replicas for each partition + .dynamicUpperBound("SLAVE", "R") + + // Add some high-priority transitions + .addTransition("SLAVE", "MASTER", 1).addTransition("OFFLINE", "SLAVE", 2) + + // Using the same priority value indicates that these transitions can fire in any order + .addTransition("MASTER", "SLAVE", 3).addTransition("SLAVE", "OFFLINE", 3) + + // Not specifying a value defaults to lowest priority + // Notice the inclusion of the OFFLINE to DROPPED transition + // Since every state has a path to OFFLINE, they each now have a path to DROPPED (REQUIRED) + .addTransition("OFFLINE", HelixDefinedState.DROPPED.toString()) + + // Create the StateModelDefinition instance + .build(); + + Assert.assertTrue(stateModel.isValid()); + } +} diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/TestParticipantMonitor.java b/helix-core/src/test/java/org/apache/helix/monitoring/TestParticipantMonitor.java index 74b1a89279..1dc0e2dc3f 100644 --- a/helix-core/src/test/java/org/apache/helix/monitoring/TestParticipantMonitor.java +++ b/helix-core/src/test/java/org/apache/helix/monitoring/TestParticipantMonitor.java @@ -33,9 +33,6 @@ import javax.management.ObjectInstance; import javax.management.ObjectName; -import org.apache.helix.monitoring.ParticipantMonitor; -import org.apache.helix.monitoring.StateTransitionContext; -import org.apache.helix.monitoring.StateTransitionDataPoint; import org.apache.helix.monitoring.mbeans.ClusterMBeanObserver; import org.apache.log4j.Logger; import org.testng.AssertJUnit; diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/TestStatCollector.java b/helix-core/src/test/java/org/apache/helix/monitoring/TestStatCollector.java index c35c961e90..5d38db11a7 100644 --- a/helix-core/src/test/java/org/apache/helix/monitoring/TestStatCollector.java +++ b/helix-core/src/test/java/org/apache/helix/monitoring/TestStatCollector.java @@ -35,7 +35,7 @@ public void TestCollectData() { } AssertJUnit.assertEquals(collector.getNumDataPoints(), nPoints); AssertJUnit.assertEquals((long) collector.getMax(), 99000); - AssertJUnit.assertEquals((long) collector.getTotalSum(), 4950000); + AssertJUnit.assertEquals(collector.getTotalSum(), 4950000); AssertJUnit.assertEquals((long) collector.getPercentile(40), 39400); AssertJUnit.assertEquals((long) collector.getMean(), 49500); AssertJUnit.assertEquals((long) collector.getMin(), 0); @@ -44,7 +44,7 @@ public void TestCollectData() { AssertJUnit.assertEquals(collector.getNumDataPoints(), 0); AssertJUnit.assertEquals((long) collector.getMax(), 0); - AssertJUnit.assertEquals((long) collector.getTotalSum(), 0); + AssertJUnit.assertEquals(collector.getTotalSum(), 0); AssertJUnit.assertEquals((long) collector.getPercentile(40), 0); AssertJUnit.assertEquals((long) collector.getMean(), 0); AssertJUnit.assertEquals((long) collector.getMin(), 0); diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAlertItemMBeanCollection.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAlertItemMBeanCollection.java deleted file mode 100644 index e383291395..0000000000 --- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAlertItemMBeanCollection.java +++ /dev/null @@ -1,293 +0,0 @@ -package org.apache.helix.monitoring.mbeans; - -/* - * 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.io.StringReader; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import javax.management.AttributeNotFoundException; -import javax.management.InstanceNotFoundException; -import javax.management.IntrospectionException; -import javax.management.MBeanException; -import javax.management.MalformedObjectNameException; -import javax.management.ReflectionException; - -import org.apache.helix.ZNRecord; -import org.apache.helix.alerts.AlertValueAndStatus; -import org.apache.helix.alerts.Tuple; -import org.apache.helix.healthcheck.TestWildcardAlert.TestClusterMBeanObserver; -import org.apache.helix.monitoring.mbeans.ClusterAlertMBeanCollection; -import org.apache.log4j.Logger; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; -import org.testng.Assert; -import org.testng.annotations.Test; - -public class TestClusterAlertItemMBeanCollection { - private static final Logger _logger = Logger.getLogger(TestClusterAlertItemMBeanCollection.class); - - @Test - public void TestAlertReportingHistory() throws InstanceNotFoundException, - MalformedObjectNameException, NullPointerException, IOException, IntrospectionException, - AttributeNotFoundException, ReflectionException, MBeanException { - ClusterAlertMBeanCollection beanCollection = new ClusterAlertMBeanCollection(); - - String clusterName = "TestCluster"; - String originAlert1 = - "EXP(decay(1.0)(esv4-app7*.RestQueryStats@DBName=BizProfile.MinServerLatency))CMP(GREATER)CON(10)"; - Map alertResultMap1 = new HashMap(); - int nAlerts1 = 5; - - String originAlert2 = - "EXP(decay(1.0)(esv4-app9*.RestQueryStats@DBName=BizProfile.MaxServerLatency))CMP(GREATER)CON(10)"; - Map alertResultMap2 = new HashMap(); - int nAlerts2 = 3; - - TestClusterMBeanObserver jmxMBeanObserver = - new TestClusterMBeanObserver(ClusterAlertMBeanCollection.DOMAIN_ALERT); - - for (int i = 0; i < nAlerts1; i++) { - String alertName = - "esv4-app7" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true); - alertResultMap1.put(alertName, valueAndStatus); - } - - for (int i = 0; i < nAlerts2; i++) { - String alertName = - "esv4-app9" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true); - alertResultMap1.put(alertName, valueAndStatus); - } - - beanCollection.setAlerts(originAlert1, alertResultMap1, clusterName); - beanCollection.setAlerts(originAlert2, alertResultMap2, clusterName); - - beanCollection.refreshAlertDelta(clusterName); - String summaryKey = ClusterAlertMBeanCollection.ALERT_SUMMARY + "_" + clusterName; - jmxMBeanObserver.refresh(); - - // Get the history list - String beanName = "HelixAlerts:alert=" + summaryKey; - Map beanValueMap = jmxMBeanObserver._beanValueMap.get(beanName); - String history1 = (String) (beanValueMap.get("AlertFiredHistory")); - - StringReader sr = new StringReader(history1); - ObjectMapper mapper = new ObjectMapper(); - - // check the history - - Map delta = beanCollection.getRecentAlertDelta(); - Assert.assertEquals(delta.size(), nAlerts1 + nAlerts2); - for (int i = 0; i < nAlerts1; i++) { - String alertBeanName = - "(esv4-app7" - + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MinServerLatency)GREATER(10)"; - Assert.assertTrue(delta.get(alertBeanName).equals("ON")); - } - - for (int i = 0; i < nAlerts2; i++) { - String alertBeanName = - "(esv4-app9" - + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MaxServerLatency)GREATER(10)"; - Assert.assertTrue(delta.get(alertBeanName).equals("ON")); - } - - alertResultMap1 = new HashMap(); - for (int i = 0; i < 3; i++) { - String alertName = - "esv4-app7" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true); - alertResultMap1.put(alertName, valueAndStatus); - } - - for (int i = 3; i < 5; i++) { - String alertName = - "esv4-app7" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, false); - alertResultMap1.put(alertName, valueAndStatus); - } - - for (int i = 7; i < 9; i++) { - String alertName = - "esv4-app7" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true); - alertResultMap1.put(alertName, valueAndStatus); - } - - for (int i = 0; i < 2; i++) { - String alertName = - "esv4-app9" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, false); - alertResultMap1.put(alertName, valueAndStatus); - } - - for (int i = 2; i < 3; i++) { - String alertName = - "esv4-app9" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true); - alertResultMap1.put(alertName, valueAndStatus); - } - for (int i = 7; i < 9; i++) { - String alertName = - "esv4-app9" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true); - alertResultMap1.put(alertName, valueAndStatus); - } - - beanCollection.setAlerts(originAlert1, alertResultMap1, clusterName); - beanCollection.refreshAlertDelta(clusterName); - jmxMBeanObserver.refresh(); - - beanValueMap = jmxMBeanObserver._beanValueMap.get(beanName); - history1 = (String) (beanValueMap.get("AlertFiredHistory")); - - sr = new StringReader(history1); - mapper = new ObjectMapper(); - - // check the history - delta = beanCollection.getRecentAlertDelta(); - Assert.assertEquals(delta.size(), 8); - for (int i = 3; i < 5; i++) { - String alertBeanName = - "(esv4-app7" - + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MinServerLatency)GREATER(10)"; - Assert.assertTrue(delta.get(alertBeanName).equals("OFF")); - } - for (int i = 7; i < 9; i++) { - String alertBeanName = - "(esv4-app7" - + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MinServerLatency)GREATER(10)"; - Assert.assertTrue(delta.get(alertBeanName).equals("ON")); - } - - for (int i = 0; i < 2; i++) { - String alertBeanName = - "(esv4-app9" - + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MaxServerLatency)GREATER(10)"; - Assert.assertTrue(delta.get(alertBeanName).equals("OFF")); - } - for (int i = 7; i < 9; i++) { - String alertBeanName = - "(esv4-app9" - + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MaxServerLatency)GREATER(10)"; - Assert.assertTrue(delta.get(alertBeanName).equals("ON")); - } - } - - @Test - public void TestAlertRefresh() throws InstanceNotFoundException, MalformedObjectNameException, - NullPointerException, IOException, IntrospectionException, AttributeNotFoundException, - ReflectionException, MBeanException, InterruptedException { - ClusterAlertMBeanCollection beanCollection = new ClusterAlertMBeanCollection(); - - String clusterName = "TestCluster"; - String originAlert1 = - "EXP(decay(1.0)(esv4-app7*.RestQueryStats@DBName=BizProfile.MinServerLatency))CMP(GREATER)CON(10)"; - Map alertResultMap1 = new HashMap(); - int nAlerts1 = 5; - - String originAlert2 = - "EXP(decay(1.0)(esv4-app9*.RestQueryStats@DBName=BizProfile.MaxServerLatency))CMP(GREATER)CON(10)"; - Map alertResultMap2 = new HashMap(); - int nAlerts2 = 3; - - TestClusterMBeanObserver jmxMBeanObserver = - new TestClusterMBeanObserver(ClusterAlertMBeanCollection.DOMAIN_ALERT); - - for (int i = 0; i < nAlerts1; i++) { - String alertName = - "esv4-app7" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true); - alertResultMap1.put(alertName, valueAndStatus); - } - - for (int i = 0; i < nAlerts2; i++) { - String alertName = - "esv4-app9" + i - + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency"; - Tuple value = new Tuple(); - value.add("22" + i); - AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true); - alertResultMap2.put(alertName, valueAndStatus); - } - - beanCollection.setAlerts(originAlert1, alertResultMap1, clusterName); - beanCollection.setAlerts(originAlert2, alertResultMap2, clusterName); - - beanCollection.refreshAlertDelta(clusterName); - String summaryKey = ClusterAlertMBeanCollection.ALERT_SUMMARY + "_" + clusterName; - jmxMBeanObserver.refresh(); - - Assert.assertEquals(jmxMBeanObserver._beanValueMap.size(), nAlerts2 + nAlerts1 + 1); - - Thread.sleep(300); - - beanCollection.setAlerts(originAlert1, alertResultMap1, clusterName); - beanCollection.checkMBeanFreshness(200); - - for (int i = 0; i < 20; i++) { - Thread.sleep(500); - - jmxMBeanObserver.refresh(); - - if (jmxMBeanObserver._beanValueMap.size() == nAlerts1 + 1) { - break; - } - } - Assert.assertEquals(jmxMBeanObserver._beanValueMap.size(), nAlerts1 + 1); - } -} diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterStatusMonitor.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterStatusMonitor.java index 711aff2c4b..facb4ea9df 100644 --- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterStatusMonitor.java +++ b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterStatusMonitor.java @@ -29,8 +29,8 @@ import org.apache.helix.NotificationContext; import org.apache.helix.PropertyType; import org.apache.helix.ZNRecord; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; import org.apache.helix.model.LiveInstance.LiveInstanceProperty; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.testng.annotations.Test; public class TestClusterStatusMonitor { @@ -50,13 +50,11 @@ public MockDataAccessor() { _instances.add(instance); } ZNRecord externalView = - DefaultIdealStateCalculator.calculateIdealState(_instances, _partitions, _replicas, _db, + DefaultTwoStateStrategy.calculateIdealState(_instances, _partitions, _replicas, _db, "MASTER", "SLAVE"); ZNRecord externalView2 = - DefaultIdealStateCalculator.calculateIdealState(_instances, 80, 2, _db2, "MASTER", - "SLAVE"); - + DefaultTwoStateStrategy.calculateIdealState(_instances, 80, 2, _db2, "MASTER", "SLAVE"); } public ZNRecord getProperty(PropertyType type, String resource) { @@ -100,12 +98,11 @@ public void TestReportData() { _liveInstances.add(metaData); } ZNRecord externalView = - DefaultIdealStateCalculator.calculateIdealState(_instances, _partitions, _replicas, _db, + DefaultTwoStateStrategy.calculateIdealState(_instances, _partitions, _replicas, _db, "MASTER", "SLAVE"); ZNRecord externalView2 = - DefaultIdealStateCalculator.calculateIdealState(_instances, 80, 2, "TestDB", "MASTER", - "SLAVE"); + DefaultTwoStateStrategy.calculateIdealState(_instances, 80, 2, "TestDB", "MASTER", "SLAVE"); List externalViews = new ArrayList(); externalViews.add(externalView); diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResourceMonitor.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResourceMonitor.java index d631dd2e94..e8bb4b6c28 100644 --- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResourceMonitor.java +++ b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResourceMonitor.java @@ -28,14 +28,13 @@ import org.apache.helix.HelixProperty; import org.apache.helix.Mocks; import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.PropertyType; import org.apache.helix.ZNRecord; -import org.apache.helix.PropertyKey.Builder; +import org.apache.helix.controller.strategy.DefaultTwoStateStrategy; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.model.LiveInstance.LiveInstanceProperty; -import org.apache.helix.monitoring.mbeans.ResourceMonitor; -import org.apache.helix.tools.DefaultIdealStateCalculator; import org.testng.AssertJUnit; import org.testng.annotations.Test; @@ -94,8 +93,8 @@ public MockHelixManager() { } _idealState = - DefaultIdealStateCalculator.calculateIdealState(_instances, _partitions, _replicas, - _dbName, "MASTER", "SLAVE"); + DefaultTwoStateStrategy.calculateIdealState(_instances, _partitions, _replicas, _dbName, + "MASTER", "SLAVE"); _externalView = new ZNRecord(_idealState); } diff --git a/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java b/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java index 6de77b2361..7829517d45 100644 --- a/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java +++ b/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java @@ -27,7 +27,6 @@ import org.apache.helix.ControllerChangeListener; import org.apache.helix.CurrentStateChangeListener; import org.apache.helix.ExternalViewChangeListener; -import org.apache.helix.HealthStateChangeListener; import org.apache.helix.HelixAdmin; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; @@ -42,11 +41,11 @@ import org.apache.helix.PropertyKey; import org.apache.helix.ScopedConfigChangeListener; import org.apache.helix.ZNRecord; -import org.apache.helix.healthcheck.ParticipantHealthReportCollector; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.monitoring.MonitoringServer; import org.apache.helix.store.zk.ZkHelixPropertyStore; public class MockZKHelixManager implements HelixManager { @@ -169,24 +168,11 @@ public ClusterMessagingService getMessagingService() { return null; } - @Override - public ParticipantHealthReportCollector getHealthReportCollector() { - // TODO Auto-generated method stub - return null; - } - @Override public InstanceType getInstanceType() { return _type; } - @Override - public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName) - throws Exception { - // TODO Auto-generated method stub - - } - @Override public String getVersion() { // TODO Auto-generated method stub @@ -261,4 +247,10 @@ public HelixManagerProperties getProperties() { return null; } + @Override + public void addControllerMessageListener(MessageListener listener) { + // TODO Auto-generated method stub + + } + } diff --git a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerElection.java b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerElection.java index fc80d2a3f7..9ec4d90e8b 100644 --- a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerElection.java +++ b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerElection.java @@ -24,16 +24,15 @@ import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.PropertyPathConfig; import org.apache.helix.PropertyType; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.PropertyKey.Builder; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.participant.DistClusterControllerElection; +import org.apache.helix.model.Leader; import org.apache.log4j.Logger; import org.testng.AssertJUnit; import org.testng.annotations.Test; @@ -70,8 +69,8 @@ public void testController() throws Exception { // path = PropertyPathConfig.getPath(PropertyType.LEADER, clusterName); // ZNRecord leaderRecord = _gZkClient. readData(path); - LiveInstance liveInstance = accessor.getProperty(keyBuilder.controllerLeader()); - AssertJUnit.assertEquals(controllerName, liveInstance.getInstanceName()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); + AssertJUnit.assertEquals(controllerName, leader.getInstanceName()); // AssertJUnit.assertNotNull(election.getController()); // AssertJUnit.assertNull(election.getLeader()); @@ -82,8 +81,8 @@ public void testController() throws Exception { context.setType(NotificationContext.Type.INIT); election.onControllerChange(context); // leaderRecord = _gZkClient. readData(path); - liveInstance = accessor.getProperty(keyBuilder.controllerLeader()); - AssertJUnit.assertEquals(controllerName, liveInstance.getInstanceName()); + leader = accessor.getProperty(keyBuilder.controllerLeader()); + AssertJUnit.assertEquals(controllerName, leader.getInstanceName()); // AssertJUnit.assertNull(election.getController()); // AssertJUnit.assertNull(election.getLeader()); @@ -118,8 +117,8 @@ public void testControllerParticipant() throws Exception { context.setType(NotificationContext.Type.CALLBACK); election.onControllerChange(context); - LiveInstance liveInstance = accessor.getProperty(keyBuilder.controllerLeader()); - AssertJUnit.assertEquals(controllerName, liveInstance.getInstanceName()); + Leader leader = accessor.getProperty(keyBuilder.controllerLeader()); + AssertJUnit.assertEquals(controllerName, leader.getInstanceName()); // path = PropertyPathConfig.getPath(PropertyType.LEADER, clusterName); // ZNRecord leaderRecord = _gZkClient. readData(path); @@ -135,8 +134,8 @@ public void testControllerParticipant() throws Exception { context.setType(NotificationContext.Type.CALLBACK); election.onControllerChange(context); - liveInstance = accessor.getProperty(keyBuilder.controllerLeader()); - AssertJUnit.assertEquals(controllerName, liveInstance.getInstanceName()); + leader = accessor.getProperty(keyBuilder.controllerLeader()); + AssertJUnit.assertEquals(controllerName, leader.getInstanceName()); // leaderRecord = _gZkClient. readData(path); // AssertJUnit.assertEquals(controllerName, leaderRecord.getSimpleField("LEADER")); diff --git a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java index ae700a48b5..9036cf3417 100644 --- a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java +++ b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java @@ -22,13 +22,18 @@ import org.apache.helix.NotificationContext; import org.apache.helix.TestHelper; import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.id.MessageId; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageType; import org.apache.helix.participant.DistClusterControllerStateModel; +import org.apache.log4j.Logger; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class TestDistControllerStateModel extends ZkUnitTestBase { + private static Logger LOG = Logger.getLogger(TestDistControllerStateModel.class); + final String clusterName = CLUSTER_PREFIX + "_" + getShortClassName(); DistClusterControllerStateModel stateModel = null; @@ -48,30 +53,29 @@ public void testOnBecomeStandbyFromOffline() { @Test() public void testOnBecomeLeaderFromStandby() { - Message message = new Message(MessageType.STATE_TRANSITION, "0"); - message.setPartitionName(clusterName); + Message message = new Message(MessageType.STATE_TRANSITION, MessageId.from("0")); + message.setPartitionId(PartitionId.from(clusterName)); message.setTgtName("controller_0"); try { stateModel.onBecomeLeaderFromStandby(message, new NotificationContext(null)); } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Exception becoming leader from standby", e); } stateModel.onBecomeStandbyFromLeader(message, new NotificationContext(null)); } @Test() public void testOnBecomeStandbyFromLeader() { - Message message = new Message(MessageType.STATE_TRANSITION, "0"); - message.setPartitionName(clusterName); + Message message = new Message(MessageType.STATE_TRANSITION, MessageId.from("0")); + message.setPartitionId(PartitionId.from(clusterName)); message.setTgtName("controller_0"); stateModel.onBecomeStandbyFromLeader(message, new NotificationContext(null)); } @Test() public void testOnBecomeOfflineFromStandby() { - Message message = new Message(MessageType.STATE_TRANSITION, "0"); - message.setPartitionName(clusterName); + Message message = new Message(MessageType.STATE_TRANSITION, MessageId.from("0")); + message.setPartitionId(PartitionId.from(clusterName)); message.setTgtName("controller_0"); stateModel.onBecomeOfflineFromStandby(message, null); @@ -89,28 +93,26 @@ public void testOnBecomeOfflineFromDropped() { @Test() public void testRollbackOnError() { - Message message = new Message(MessageType.STATE_TRANSITION, "0"); - message.setPartitionName(clusterName); + Message message = new Message(MessageType.STATE_TRANSITION, MessageId.from("0")); + message.setPartitionId(PartitionId.from(clusterName)); message.setTgtName("controller_0"); try { stateModel.onBecomeLeaderFromStandby(message, new NotificationContext(null)); } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Exception becoming leader from standby", e); } stateModel.rollbackOnError(message, new NotificationContext(null), null); } @Test() public void testReset() { - Message message = new Message(MessageType.STATE_TRANSITION, "0"); - message.setPartitionName(clusterName); + Message message = new Message(MessageType.STATE_TRANSITION, MessageId.from("0")); + message.setPartitionId(PartitionId.from(clusterName)); message.setTgtName("controller_0"); try { stateModel.onBecomeLeaderFromStandby(message, new NotificationContext(null)); } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); + LOG.error("Exception becoming leader from standby", e); } stateModel.reset(); } diff --git a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModelFactory.java b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModelFactory.java index dae58b30ba..43ffb9a533 100644 --- a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModelFactory.java +++ b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModelFactory.java @@ -20,9 +20,6 @@ */ import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.participant.DistClusterControllerStateModel; -import org.apache.helix.participant.DistClusterControllerStateModelFactory; -import org.testng.annotations.Test; import org.testng.annotations.Test; public class TestDistControllerStateModelFactory { diff --git a/helix-core/src/test/java/org/apache/helix/store/TestJsonComparator.java b/helix-core/src/test/java/org/apache/helix/store/TestJsonComparator.java index e31fa773b0..5af3d55d10 100644 --- a/helix-core/src/test/java/org/apache/helix/store/TestJsonComparator.java +++ b/helix-core/src/test/java/org/apache/helix/store/TestJsonComparator.java @@ -22,7 +22,6 @@ import java.util.Date; import org.apache.helix.ZNRecord; -import org.apache.helix.store.PropertyJsonComparator; import org.testng.AssertJUnit; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java b/helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java index b389e1523a..b07cf8f5fe 100644 --- a/helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java +++ b/helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java @@ -34,7 +34,6 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.manager.zk.ZkClient; import org.apache.helix.store.HelixPropertyListener; -import org.apache.helix.store.zk.ZkHelixPropertyStore; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-core/src/test/java/org/apache/helix/tools/TestClusterSetup.java b/helix-core/src/test/java/org/apache/helix/tools/TestClusterSetup.java index 71909680e7..d3d6736e86 100644 --- a/helix-core/src/test/java/org/apache/helix/tools/TestClusterSetup.java +++ b/helix-core/src/test/java/org/apache/helix/tools/TestClusterSetup.java @@ -24,22 +24,19 @@ import java.util.Date; import org.apache.helix.HelixException; -import org.apache.helix.PropertyKey; +import org.apache.helix.PropertyKey.Builder; import org.apache.helix.PropertyPathConfig; import org.apache.helix.PropertyType; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.ZkUnitTestBase; -import org.apache.helix.PropertyKey.Builder; 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.manager.zk.ZkClient; -import org.apache.helix.model.LiveInstance; import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; -import org.apache.helix.tools.ClusterSetup; -import org.apache.helix.util.HelixUtil; +import org.apache.helix.model.LiveInstance; import org.apache.log4j.Logger; import org.testng.Assert; import org.testng.AssertJUnit; diff --git a/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java b/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java index c58f94d11b..d281abe4fe 100644 --- a/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java +++ b/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java @@ -29,16 +29,17 @@ import org.apache.helix.HelixDataAccessor; import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; -import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.integration.ZkIntegrationTestBase; +import org.apache.helix.integration.manager.ClusterDistributedController; +import org.apache.helix.integration.manager.MockParticipantManager; import org.apache.helix.manager.zk.ZKHelixDataAccessor; import org.apache.helix.manager.zk.ZKUtil; import org.apache.helix.manager.zk.ZkBaseDataAccessor; -import org.apache.helix.mock.controller.ClusterController; -import org.apache.helix.mock.participant.MockParticipant; import org.apache.helix.model.IdealState; import org.apache.helix.model.InstanceConfig; -import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Leader; import org.apache.helix.store.ZNRecordJsonSerializer; import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier; import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier; @@ -238,8 +239,8 @@ public void testStartCluster() throws Exception { final int n = 6; System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; - ClusterController[] controllers = new ClusterController[2]; + MockParticipantManager[] participants = new MockParticipantManager[n]; + ClusterDistributedController[] controllers = new ClusterDistributedController[2]; setupCluster(clusterName, grandClusterName, n, participants, controllers); // activate clusters @@ -282,13 +283,13 @@ public void testStartCluster() throws Exception { // verify leader node BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); HelixDataAccessor accessor = new ZKHelixDataAccessor(grandClusterName, baseAccessor); - LiveInstance controllerLeader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); + Leader controllerLeader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); Assert.assertNotNull(controllerLeader, "controllerLeader should be either controller_9000 or controller_9001"); Assert.assertTrue(controllerLeader.getInstanceName().startsWith("controller_900")); accessor = new ZKHelixDataAccessor(clusterName, baseAccessor); - LiveInstance leader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); + Leader leader = accessor.getProperty(accessor.keyBuilder().controllerLeader()); for (int i = 0; i < 20; i++) { if (leader != null) { break; @@ -309,14 +310,12 @@ public void testStartCluster() throws Exception { Assert.assertTrue(verifyResult); // clean up - // for (int i = 0; i < 2; i++) { - // controllers[i].syncStop(); - // Thread.sleep(1000); // wait for all zk callbacks done - // } - // Thread.sleep(5000); - // for (int i = 0; i < n; i++) { - // participants[i].syncStop(); - // } + for (ClusterDistributedController controller : controllers) { + controller.syncStop(); + } + for (int i = 0; i < participants.length; i++) { + participants[i].syncStop(); + } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } @@ -331,11 +330,11 @@ public void testDropAddResource() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; - ClusterController[] controllers = new ClusterController[2]; + MockParticipantManager[] participants = new MockParticipantManager[n]; + ClusterDistributedController[] controllers = new ClusterDistributedController[2]; setupCluster(clusterName, grandClusterName, n, participants, controllers); String command = - "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " true"; + "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName + " true"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); Thread.sleep(500); @@ -355,7 +354,7 @@ public void testDropAddResource() throws Exception { pw.write(new String(serializer.serialize(idealState.getRecord()))); pw.close(); - command = "-zkSvr localhost:2183 -dropResource " + clusterName + " db_11 "; + command = "-zkSvr " + ZK_ADDR + " -dropResource " + clusterName + " db_11 "; ClusterSetup.processCommandLineArgs(command.split("\\s+")); boolean verifyResult = @@ -363,7 +362,8 @@ public void testDropAddResource() throws Exception { clusterName)); Assert.assertTrue(verifyResult); - command = "-zkSvr localhost:2183 -addIdealState " + clusterName + " db_11 " + tmpIdealStateFile; + command = + "-zkSvr " + ZK_ADDR + " -addIdealState " + clusterName + " db_11 " + tmpIdealStateFile; ClusterSetup.processCommandLineArgs(command.split("\\s+")); verifyResult = @@ -375,61 +375,58 @@ public void testDropAddResource() throws Exception { Assert.assertTrue(idealState2.getRecord().equals(idealState.getRecord())); // clean up - // for (int i = 0; i < 2; i++) { - // controllers[i].syncStop(); - // Thread.sleep(1000); // wait for all zk callbacks done - // } - // Thread.sleep(5000); - // for (int i = 0; i < n; i++) { - // participants[i].syncStop(); - // } + for (int i = 0; i < controllers.length; i++) { + controllers[i].syncStop(); + } + for (int i = 0; i < participants.length; i++) { + participants[i].syncStop(); + } System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } private void setupCluster(String clusterName, String grandClusterName, final int n, - MockParticipant[] participants, ClusterController[] controllers) throws Exception, - InterruptedException { + MockParticipantManager[] participants, ClusterDistributedController[] controllers) + throws Exception, InterruptedException { // add cluster - String command = "-zkSvr localhost:2183 -addCluster " + clusterName; + String command = "-zkSvr " + ZK_ADDR + " -addCluster " + clusterName; ClusterSetup.processCommandLineArgs(command.split("\\s+")); // add grand cluster - command = "-zkSvr localhost:2183 -addCluster " + grandClusterName; + command = "-zkSvr " + ZK_ADDR + " -addCluster " + grandClusterName; ClusterSetup.processCommandLineArgs(command.split("\\s+")); // add nodes for (int i = 0; i < n; i++) { - command = "-zkSvr localhost:2183 -addNode " + clusterName + " localhost:123" + i; + command = "-zkSvr " + ZK_ADDR + " -addNode " + clusterName + " localhost:123" + i; ClusterSetup.processCommandLineArgs(command.split("\\s+")); } // add resource - command = "-zkSvr localhost:2183 -addResource " + clusterName + " db_11 48 MasterSlave"; + command = "-zkSvr " + ZK_ADDR + " -addResource " + clusterName + " db_11 48 MasterSlave"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); // rebalance with key prefix - command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 2 -key alias"; + command = "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 2 -key alias"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); // add nodes to grand cluster command = - "-zkSvr localhost:2183 -addNode " + grandClusterName + " controller:9000;controller:9001"; + "-zkSvr " + ZK_ADDR + " -addNode " + grandClusterName + " controller:9000;controller:9001"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); // start mock nodes for (int i = 0; i < n; i++) { String instanceName = "localhost_123" + i; - participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); participants[i].syncStart(); } // start controller nodes for (int i = 0; i < 2; i++) { controllers[i] = - new ClusterController(grandClusterName, "controller_900" + i, ZK_ADDR, - HelixControllerMain.DISTRIBUTED); + new ClusterDistributedController(ZK_ADDR, grandClusterName, "controller_900" + i); controllers[i].syncStart(); } @@ -446,16 +443,16 @@ public void testInstanceOperations() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; - ClusterController[] controllers = new ClusterController[2]; + MockParticipantManager[] participants = new MockParticipantManager[n]; + ClusterDistributedController[] controllers = new ClusterDistributedController[2]; setupCluster(clusterName, grandClusterName, n, participants, controllers); String command = - "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " true"; + "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName + " true"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); Thread.sleep(500); // drop node should fail if the node is not disabled - command = "-zkSvr localhost:2183 -dropNode " + clusterName + " localhost:1232"; + command = "-zkSvr " + ZK_ADDR + " -dropNode " + clusterName + " localhost:1232"; try { ClusterSetup.processCommandLineArgs(command.split("\\s+")); Assert.fail("dropNode should fail since the node is not disabled"); @@ -464,11 +461,11 @@ public void testInstanceOperations() throws Exception { } // disabled node - command = "-zkSvr localhost:2183 -enableInstance " + clusterName + " localhost:1232 false"; + command = "-zkSvr " + ZK_ADDR + " -enableInstance " + clusterName + " localhost:1232 false"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); // Cannot dropNode if the node is not disconnected - command = "-zkSvr localhost:2183 -dropNode " + clusterName + " localhost:1232"; + command = "-zkSvr " + ZK_ADDR + " -dropNode " + clusterName + " localhost:1232"; try { ClusterSetup.processCommandLineArgs(command.split("\\s+")); Assert.fail("dropNode should fail since the node is not disconnected"); @@ -478,7 +475,7 @@ public void testInstanceOperations() throws Exception { // Cannot swapNode if the node is not disconnected command = - "-zkSvr localhost:2183 -swapInstance " + clusterName + " localhost_1232 localhost_12320"; + "-zkSvr " + ZK_ADDR + " -swapInstance " + clusterName + " localhost_1232 localhost_12320"; try { ClusterSetup.processCommandLineArgs(command.split("\\s+")); Assert.fail("swapInstance should fail since the node is not disconnected"); @@ -490,12 +487,12 @@ public void testInstanceOperations() throws Exception { participants[2].syncStop(); // add new node then swap instance - command = "-zkSvr localhost:2183 -addNode " + clusterName + " localhost:12320"; + command = "-zkSvr " + ZK_ADDR + " -addNode " + clusterName + " localhost:12320"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); // swap instance. The instance get swapped out should not exist anymore command = - "-zkSvr localhost:2183 -swapInstance " + clusterName + " localhost_1232 localhost_12320"; + "-zkSvr " + ZK_ADDR + " -swapInstance " + clusterName + " localhost_1232 localhost_12320"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); @@ -504,6 +501,14 @@ public void testInstanceOperations() throws Exception { Assert.assertFalse(_gZkClient.exists(path), path + " should not exist since localhost_1232 has been swapped by localhost_12320"); + // clean up + for (int i = 0; i < controllers.length; i++) { + controllers[i].syncStop(); + } + for (int i = 0; i < participants.length; i++) { + participants[i].syncStop(); + } + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } @@ -517,26 +522,26 @@ public void testExpandCluster() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; - ClusterController[] controllers = new ClusterController[2]; + MockParticipantManager[] participants = new MockParticipantManager[n]; + ClusterDistributedController[] controllers = new ClusterDistributedController[2]; setupCluster(clusterName, grandClusterName, n, participants, controllers); String command = - "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " true"; + "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName + " true"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); Thread.sleep(500); command = - "-zkSvr localhost:2183 -addNode " + clusterName + "-zkSvr " + ZK_ADDR + " -addNode " + clusterName + " localhost:12331;localhost:12341;localhost:12351;localhost:12361"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); command = "-zkSvr localhost:2183 -expandCluster " + clusterName; ClusterSetup.processCommandLineArgs(command.split("\\s+")); - MockParticipant[] newParticipants = new MockParticipant[4]; + MockParticipantManager[] newParticipants = new MockParticipantManager[4]; for (int i = 3; i <= 6; i++) { String instanceName = "localhost_123" + i + "1"; - newParticipants[i - 3] = new MockParticipant(clusterName, instanceName, ZK_ADDR); + newParticipants[i - 3] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName); newParticipants[i - 3].syncStart(); } @@ -550,6 +555,17 @@ public void testExpandCluster() throws Exception { clusterName)); Assert.assertTrue(verifyResult); + // clean up + for (int i = 0; i < controllers.length; i++) { + controllers[i].syncStop(); + } + for (int i = 0; i < participants.length; i++) { + participants[i].syncStop(); + } + for (int i = 0; i < newParticipants.length; i++) { + newParticipants[i].syncStop(); + } + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); } @@ -563,17 +579,18 @@ public void testDeactivateCluster() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - MockParticipant[] participants = new MockParticipant[n]; - ClusterController[] controllers = new ClusterController[2]; + MockParticipantManager[] participants = new MockParticipantManager[n]; + ClusterDistributedController[] controllers = new ClusterDistributedController[2]; setupCluster(clusterName, grandClusterName, n, participants, controllers); String command = - "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " true"; + "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName + " true"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); Thread.sleep(500); // deactivate cluster command = - "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " false"; + "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName + + " false"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); @@ -588,7 +605,7 @@ public void testDeactivateCluster() throws Exception { Assert.assertFalse(_gZkClient.exists(path), "leader should be gone after deactivate the cluster"); - command = "-zkSvr localhost:2183 -dropCluster " + clusterName; + command = "-zkSvr " + ZK_ADDR + " -dropCluster " + clusterName; try { ClusterSetup.processCommandLineArgs(command.split("\\s+")); Assert.fail("dropCluster should fail since there are still instances running"); @@ -596,16 +613,15 @@ public void testDeactivateCluster() throws Exception { // OK } - for (int i = 0; i < n; i++) { + for (int i = 0; i < participants.length; i++) { participants[i].syncStop(); } command = "-zkSvr localhost:2183 -dropCluster " + clusterName; ClusterSetup.processCommandLineArgs(command.split("\\s")); - for (int i = 0; i < 2; i++) { + for (int i = 0; i < controllers.length; i++) { controllers[i].syncStop(); - Thread.sleep(1000); } command = "-zkSvr localhost:2183 -dropCluster " + grandClusterName; ClusterSetup.processCommandLineArgs(command.split("\\s+")); @@ -625,95 +641,98 @@ public void testInstanceGroupTags() throws Exception { System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); - String command = "-zkSvr localhost:2183 -addCluster " + clusterName; + String command = "-zkSvr " + ZK_ADDR + " -addCluster " + clusterName; ClusterSetup.processCommandLineArgs(command.split("\\s+")); command = "-zkSvr localhost:2183 -addResource " + clusterName + " db_11 12 MasterSlave"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); for (int i = 0; i < 6; i++) { - command = "-zkSvr localhost:2183 -addNode " + clusterName + " localhost:123" + i; + command = "-zkSvr " + ZK_ADDR + " -addNode " + clusterName + " localhost:123" + i; ClusterSetup.processCommandLineArgs(command.split("\\s+")); } for (int i = 0; i < 2; i++) { command = - "-zkSvr localhost:2183 -addInstanceTag " + clusterName + " localhost_123" + i + " tag1"; + "-zkSvr " + ZK_ADDR + " -addInstanceTag " + clusterName + " localhost_123" + i + " tag1"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); } for (int i = 2; i < 6; i++) { command = - "-zkSvr localhost:2183 -addInstanceTag " + clusterName + " localhost_123" + i + " tag2"; + "-zkSvr " + ZK_ADDR + " -addInstanceTag " + clusterName + " localhost_123" + i + " tag2"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); } - command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 2 -instanceGroupTag tag1"; + command = + "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 2 -instanceGroupTag tag1"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); IdealState dbIs = accessor.getProperty(accessor.keyBuilder().idealStates("db_11")); - Set hosts = new HashSet(); - for (String p : dbIs.getPartitionSet()) { - for (String hostName : dbIs.getInstanceStateMap(p).keySet()) { + Set hosts = new HashSet(); + for (PartitionId p : dbIs.getPartitionIdSet()) { + for (ParticipantId participantId : dbIs.getParticipantStateMap(p).keySet()) { InstanceConfig config = - accessor.getProperty(accessor.keyBuilder().instanceConfig(hostName)); + accessor.getProperty(accessor.keyBuilder().instanceConfig(participantId.stringify())); Assert.assertTrue(config.containsTag("tag1")); - hosts.add(hostName); + hosts.add(participantId); } } Assert.assertEquals(hosts.size(), 2); - command = "-zkSvr localhost:2183 -dropResource " + clusterName + " db_11 "; + command = "-zkSvr " + ZK_ADDR + " -dropResource " + clusterName + " db_11 "; ClusterSetup.processCommandLineArgs(command.split("\\s+")); // re-add and rebalance - command = "-zkSvr localhost:2183 -addResource " + clusterName + " db_11 48 MasterSlave"; + command = "-zkSvr " + ZK_ADDR + " -addResource " + clusterName + " db_11 48 MasterSlave"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); - command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 3 -instanceGroupTag tag2"; + command = + "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 3 -instanceGroupTag tag2"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); dbIs = accessor.getProperty(accessor.keyBuilder().idealStates("db_11")); - hosts = new HashSet(); - for (String p : dbIs.getPartitionSet()) { - for (String hostName : dbIs.getInstanceStateMap(p).keySet()) { + hosts = new HashSet(); + for (PartitionId p : dbIs.getPartitionIdSet()) { + for (ParticipantId participantId : dbIs.getParticipantStateMap(p).keySet()) { InstanceConfig config = - accessor.getProperty(accessor.keyBuilder().instanceConfig(hostName)); + accessor.getProperty(accessor.keyBuilder().instanceConfig(participantId.stringify())); Assert.assertTrue(config.containsTag("tag2")); - hosts.add(hostName); + hosts.add(participantId); } } Assert.assertEquals(hosts.size(), 4); - command = "-zkSvr localhost:2183 -dropResource " + clusterName + " db_11 "; + command = "-zkSvr " + ZK_ADDR + " -dropResource " + clusterName + " db_11 "; ClusterSetup.processCommandLineArgs(command.split("\\s+")); for (int i = 3; i <= 3; i++) { command = - "-zkSvr localhost:2183 -removeInstanceTag " + clusterName + " localhost_123" + i + "-zkSvr " + ZK_ADDR + " -removeInstanceTag " + clusterName + " localhost_123" + i + " tag2"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); } // re-add and rebalance - command = "-zkSvr localhost:2183 -addResource " + clusterName + " db_11 48 MasterSlave"; + command = "-zkSvr " + ZK_ADDR + " -addResource " + clusterName + " db_11 48 MasterSlave"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); - command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 3 -instanceGroupTag tag2"; + command = + "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 3 -instanceGroupTag tag2"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); dbIs = accessor.getProperty(accessor.keyBuilder().idealStates("db_11")); - hosts = new HashSet(); - for (String p : dbIs.getPartitionSet()) { - for (String hostName : dbIs.getInstanceStateMap(p).keySet()) { + hosts = new HashSet(); + for (PartitionId p : dbIs.getPartitionIdSet()) { + for (ParticipantId participantId : dbIs.getParticipantStateMap(p).keySet()) { InstanceConfig config = - accessor.getProperty(accessor.keyBuilder().instanceConfig(hostName)); + accessor.getProperty(accessor.keyBuilder().instanceConfig(participantId.stringify())); Assert.assertTrue(config.containsTag("tag2")); - hosts.add(hostName); + hosts.add(participantId); } } Assert.assertEquals(hosts.size(), 3); // rebalance with key prefix - command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 2 -key alias"; + command = "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 2 -key alias"; ClusterSetup.processCommandLineArgs(command.split("\\s+")); System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); diff --git a/helix-core/src/test/java/org/apache/helix/util/TestZKClientPool.java b/helix-core/src/test/java/org/apache/helix/util/TestZKClientPool.java index 14ad709d5f..9afca37b52 100644 --- a/helix-core/src/test/java/org/apache/helix/util/TestZKClientPool.java +++ b/helix-core/src/test/java/org/apache/helix/util/TestZKClientPool.java @@ -26,7 +26,6 @@ import org.apache.helix.TestHelper; import org.apache.helix.ZNRecord; import org.apache.helix.manager.zk.ZkClient; -import org.apache.helix.util.ZKClientPool; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/helix-examples/.gitignore b/helix-examples/.gitignore new file mode 100644 index 0000000000..2411bd8ac3 --- /dev/null +++ b/helix-examples/.gitignore @@ -0,0 +1,16 @@ +/target +/.project +/.classpath +/.settings +/zkdata +/test-output +/src/main/scripts/integration-test/var +#/src/test/java/com/linkedin/dds/ +#/src/main/scripts/integration-test/config +/src/main/scripts/target/ +/src/main/scripts/integration-test/script/.metadata_infra +#/src/main/scripts/integration-test/script/dds_driver.py +#/src/main/scripts/integration-test/script/pexpect.py +#/src/main/scripts/integration-test/script/utility.py +*.pyc +/bin diff --git a/helix-examples/DISCLAIMER b/helix-examples/DISCLAIMER new file mode 100644 index 0000000000..2001d3154d --- /dev/null +++ b/helix-examples/DISCLAIMER @@ -0,0 +1,15 @@ +Apache Helix is an effort undergoing incubation at the Apache Software +Foundation (ASF), sponsored by the Apache Incubator PMC. + +Incubation is required of all newly accepted projects until a further review +indicates that the infrastructure, communications, and decision making process +have stabilized in a manner consistent with other successful ASF projects. + +While incubation status is not necessarily a reflection of the completeness +or stability of the code, it does indicate that the project has yet to be +fully endorsed by the ASF. + +For more information about the incubation status of the Apache Helix project you +can go to the following page: + +http://incubator.apache.org/projects/helix.html diff --git a/helix-examples/LICENSE b/helix-examples/LICENSE new file mode 100644 index 0000000000..413913f132 --- /dev/null +++ b/helix-examples/LICENSE @@ -0,0 +1,273 @@ + + 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. + + + diff --git a/helix-examples/NOTICE b/helix-examples/NOTICE new file mode 100644 index 0000000000..e070e15573 --- /dev/null +++ b/helix-examples/NOTICE @@ -0,0 +1,30 @@ +Apache Helix +Copyright 2012 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 +josql (http://sourceforge.net/projects/josql/). +Licensed under the Apache License 2.0. + +This product includes software developed at +restlet (http://www.restlet.org/about/legal). +Licensed under the Apache License 2.0. + + +II. License Summary +- Apache License 2.0 +- BSD License diff --git a/helix-examples/pom.xml b/helix-examples/pom.xml new file mode 100644 index 0000000000..c3b319cd4c --- /dev/null +++ b/helix-examples/pom.xml @@ -0,0 +1,108 @@ + + + + + org.apache.helix + helix + 0.7.1-incubating-SNAPSHOT + + 4.0.0 + + helix-examples + jar + Apache Helix :: Helix Examples + + + + org.apache.helix*, + org.codehaus.jackson*, + org.apache.commons.cli*, + org.apache.log4j, + org.restlet*, + * + + org.apache.helix.webapp*;version="${project.version};-noimport:=true + + + + + org.apache.helix + helix-core + + + org.testng + testng + test + + + + + + ${basedir} + + DISCLAIMER + + + + + + org.codehaus.mojo + appassembler-maven-plugin + + + windows + unix + + + + org.apache.helix.examples.ExampleProcess + start-helix-participant + + + org.apache.helix.examples.Quickstart + quickstart + + + org.apache.helix.examples.LogicalModelExample + run-logical-model-example + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/assemble/assembly.xml + + + + + package + + single + + + + + + + diff --git a/helix-examples/src/assemble/assembly.xml b/helix-examples/src/assemble/assembly.xml new file mode 100644 index 0000000000..c2d08a1cda --- /dev/null +++ b/helix-examples/src/assemble/assembly.xml @@ -0,0 +1,60 @@ + + + + pkg + + tar + + + + ${project.build.directory}/${project.artifactId}-pkg/bin + bin + unix + 0755 + 0755 + + + ${project.build.directory}/${project.artifactId}-pkg/repo/ + repo + 0755 + 0755 + + **/*.xml + + + + ${project.build.directory}/${project.artifactId}-pkg/conf + conf + unix + 0755 + 0755 + + + ${project.basedir} + / + + LICENSE + NOTICE + DISCLAIMER + + 0755 + + + \ No newline at end of file diff --git a/helix-examples/src/main/config/log4j.properties b/helix-examples/src/main/config/log4j.properties new file mode 100644 index 0000000000..4b3dc31577 --- /dev/null +++ b/helix-examples/src/main/config/log4j.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# Set root logger level to DEBUG and its only appender to A1. +log4j.rootLogger=ERROR,A1 + +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender + +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +log4j.logger.org.I0Itec=ERROR +log4j.logger.org.apache=ERROR diff --git a/helix-core/src/main/java/org/apache/helix/examples/BootstrapHandler.java b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java similarity index 93% rename from helix-core/src/main/java/org/apache/helix/examples/BootstrapHandler.java rename to helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java index f1b37fc6bf..f0922f3cd9 100644 --- a/helix-core/src/main/java/org/apache/helix/examples/BootstrapHandler.java +++ b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java @@ -26,6 +26,7 @@ import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; +import org.apache.helix.api.id.MessageId; import org.apache.helix.model.Message; import org.apache.helix.model.Message.MessageState; import org.apache.helix.model.Message.MessageType; @@ -62,14 +63,14 @@ public void offlineToSlave(Message message, NotificationContext context) { HelixManager manager = context.getManager(); ClusterMessagingService messagingService = manager.getMessagingService(); Message requestBackupUriRequest = - new Message(MessageType.USER_DEFINE_MSG, UUID.randomUUID().toString()); + new Message(MessageType.USER_DEFINE_MSG, MessageId.from(UUID.randomUUID().toString())); requestBackupUriRequest.setMsgSubType(BootstrapProcess.REQUEST_BOOTSTRAP_URL); requestBackupUriRequest.setMsgState(MessageState.NEW); Criteria recipientCriteria = new Criteria(); recipientCriteria.setInstanceName("*"); recipientCriteria.setRecipientInstanceType(InstanceType.PARTICIPANT); - recipientCriteria.setResource(message.getResourceName()); - recipientCriteria.setPartition(message.getPartitionName()); + recipientCriteria.setResource(message.getResourceId().stringify()); + recipientCriteria.setPartition(message.getPartitionId().stringify()); recipientCriteria.setSessionSpecific(true); // wait for 30 seconds int timeout = 30000; diff --git a/helix-core/src/main/java/org/apache/helix/examples/BootstrapProcess.java b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapProcess.java similarity index 98% rename from helix-core/src/main/java/org/apache/helix/examples/BootstrapProcess.java rename to helix-examples/src/main/java/org/apache/helix/examples/BootstrapProcess.java index 73f1175344..2506c01754 100644 --- a/helix-core/src/main/java/org/apache/helix/examples/BootstrapProcess.java +++ b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapProcess.java @@ -18,7 +18,7 @@ * specific language governing permissions and limitations * under the License. */ -import java.io.File; + import java.net.InetAddress; import java.net.UnknownHostException; import java.util.Date; @@ -45,7 +45,6 @@ import org.apache.helix.participant.StateMachineEngine; import org.apache.helix.participant.statemachine.StateModel; import org.apache.helix.participant.statemachine.StateModelFactory; -import org.apache.helix.tools.ClusterStateVerifier; /** * This process does little more than handling the state transition messages. @@ -83,7 +82,6 @@ public class BootstrapProcess { // private StateMachineEngine genericStateMachineHandler; - private String _file = null; private StateModelFactory stateModelFactory; private final int delay; @@ -155,7 +153,8 @@ public HelixTaskResult handleMessage() throws InterruptedException { result.getTaskResultMap().put( "BOOTSTRAP_URL", "http://" + hostName + ":" + port + "/getFile?path=/data/bootstrap/" - + _message.getResourceName() + "/" + _message.getPartitionName() + ".tar"); + + _message.getResourceId().stringify() + "/" + + _message.getPartitionId().stringify() + ".tar"); result.getTaskResultMap().put("BOOTSTRAP_TIME", "" + new Date().getTime()); } diff --git a/helix-core/src/main/java/org/apache/helix/examples/DummyParticipant.java b/helix-examples/src/main/java/org/apache/helix/examples/DummyParticipant.java similarity index 89% rename from helix-core/src/main/java/org/apache/helix/examples/DummyParticipant.java rename to helix-examples/src/main/java/org/apache/helix/examples/DummyParticipant.java index 52a369683c..c6ab3a40b1 100644 --- a/helix-core/src/main/java/org/apache/helix/examples/DummyParticipant.java +++ b/helix-examples/src/main/java/org/apache/helix/examples/DummyParticipant.java @@ -23,6 +23,7 @@ import org.apache.helix.HelixManagerFactory; import org.apache.helix.InstanceType; import org.apache.helix.NotificationContext; +import org.apache.helix.api.id.PartitionId; import org.apache.helix.model.Message; import org.apache.helix.participant.StateMachineEngine; import org.apache.helix.participant.statemachine.StateModel; @@ -38,44 +39,44 @@ public class DummyParticipant { public static class DummyMSStateModel extends StateModel { @Transition(to = "SLAVE", from = "OFFLINE") public void onBecomeSlaveFromOffline(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - System.out.println(instanceName + " becomes SLAVE from OFFLINE for " + partitionName); + System.out.println(instanceName + " becomes SLAVE from OFFLINE for " + partitionId); } @Transition(to = "MASTER", from = "SLAVE") public void onBecomeMasterFromSlave(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - System.out.println(instanceName + " becomes MASTER from SLAVE for " + partitionName); + System.out.println(instanceName + " becomes MASTER from SLAVE for " + partitionId); } @Transition(to = "SLAVE", from = "MASTER") public void onBecomeSlaveFromMaster(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - System.out.println(instanceName + " becomes SLAVE from MASTER for " + partitionName); + System.out.println(instanceName + " becomes SLAVE from MASTER for " + partitionId); } @Transition(to = "OFFLINE", from = "SLAVE") public void onBecomeOfflineFromSlave(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - System.out.println(instanceName + " becomes OFFLINE from SLAVE for " + partitionName); + System.out.println(instanceName + " becomes OFFLINE from SLAVE for " + partitionId); } @Transition(to = "DROPPED", from = "OFFLINE") public void onBecomeDroppedFromOffline(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - System.out.println(instanceName + " becomes DROPPED from OFFLINE for " + partitionName); + System.out.println(instanceName + " becomes DROPPED from OFFLINE for " + partitionId); } @Transition(to = "OFFLINE", from = "ERROR") public void onBecomeOfflineFromError(Message message, NotificationContext context) { - String partitionName = message.getPartitionName(); + PartitionId partitionId = message.getPartitionId(); String instanceName = message.getTgtName(); - System.out.println(instanceName + " becomes OFFLINE from ERROR for " + partitionName); + System.out.println(instanceName + " becomes OFFLINE from ERROR for " + partitionId); } @Override diff --git a/helix-core/src/main/java/org/apache/helix/examples/ExampleHelper.java b/helix-examples/src/main/java/org/apache/helix/examples/ExampleHelper.java similarity index 100% rename from helix-core/src/main/java/org/apache/helix/examples/ExampleHelper.java rename to helix-examples/src/main/java/org/apache/helix/examples/ExampleHelper.java diff --git a/helix-core/src/main/java/org/apache/helix/examples/ExampleProcess.java b/helix-examples/src/main/java/org/apache/helix/examples/ExampleProcess.java similarity index 96% rename from helix-core/src/main/java/org/apache/helix/examples/ExampleProcess.java rename to helix-examples/src/main/java/org/apache/helix/examples/ExampleProcess.java index 34a13e5cd2..840a96387e 100644 --- a/helix-core/src/main/java/org/apache/helix/examples/ExampleProcess.java +++ b/helix-examples/src/main/java/org/apache/helix/examples/ExampleProcess.java @@ -33,13 +33,15 @@ import org.apache.helix.HelixManager; import org.apache.helix.HelixManagerFactory; import org.apache.helix.InstanceType; +import org.apache.helix.manager.zk.HelixManagerShutdownHook; import org.apache.helix.model.Message.MessageType; import org.apache.helix.participant.StateMachineEngine; import org.apache.helix.participant.statemachine.StateModel; import org.apache.helix.participant.statemachine.StateModelFactory; -import org.apache.helix.tools.ClusterStateVerifier; +import org.apache.log4j.Logger; public class ExampleProcess { + private static final Logger LOG = Logger.getLogger(ExampleProcess.class); public static final String zkServer = "zkSvr"; public static final String cluster = "cluster"; @@ -57,8 +59,6 @@ public class ExampleProcess { private final String stateModelType; private HelixManager manager; - // private StateMachineEngine genericStateMachineHandler; - private StateModelFactory stateModelFactory; private final int delay; @@ -98,6 +98,10 @@ public void stop() { manager.disconnect(); } + public HelixManager getManager() { + return manager; + } + @SuppressWarnings("static-access") private static Options constructCommandLineOptions() { Option helpOption = @@ -168,6 +172,7 @@ private static Options constructCommandLineOptions() { public static void printUsage(Options cliOptions) { HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setWidth(1000); helpFormatter.printHelp("java " + ExampleProcess.class.getName(), cliOptions); } @@ -232,6 +237,9 @@ public static void main(String[] args) throws Exception { new ExampleProcess(zkConnectString, clusterName, instanceName, file, stateModelValue, delay); process.start(); + + Runtime.getRuntime().addShutdownHook(new HelixManagerShutdownHook(process.getManager())); + Thread.currentThread().join(); } } diff --git a/helix-core/src/main/java/org/apache/helix/examples/IdealStateBuilderExample.java b/helix-examples/src/main/java/org/apache/helix/examples/IdealStateBuilderExample.java similarity index 100% rename from helix-core/src/main/java/org/apache/helix/examples/IdealStateBuilderExample.java rename to helix-examples/src/main/java/org/apache/helix/examples/IdealStateBuilderExample.java diff --git a/helix-core/src/main/java/org/apache/helix/examples/IdealStateExample.java b/helix-examples/src/main/java/org/apache/helix/examples/IdealStateExample.java similarity index 96% rename from helix-core/src/main/java/org/apache/helix/examples/IdealStateExample.java rename to helix-examples/src/main/java/org/apache/helix/examples/IdealStateExample.java index 7c5192ddba..a9183f5a68 100644 --- a/helix-core/src/main/java/org/apache/helix/examples/IdealStateExample.java +++ b/helix-examples/src/main/java/org/apache/helix/examples/IdealStateExample.java @@ -100,9 +100,8 @@ public static void main(String[] args) throws Exception { admin.addCluster(clusterName, true); // add MasterSlave state mode definition - StateModelConfigGenerator generator = new StateModelConfigGenerator(); - admin.addStateModelDef(clusterName, "MasterSlave", - new StateModelDefinition(generator.generateConfigForMasterSlave())); + admin.addStateModelDef(clusterName, "MasterSlave", new StateModelDefinition( + StateModelConfigGenerator.generateConfigForMasterSlave())); // add 3 participants: "localhost:{12918, 12919, 12920}" for (int i = 0; i < 3; i++) { diff --git a/helix-core/src/main/java/org/apache/helix/examples/LeaderStandbyStateModelFactory.java b/helix-examples/src/main/java/org/apache/helix/examples/LeaderStandbyStateModelFactory.java similarity index 100% rename from helix-core/src/main/java/org/apache/helix/examples/LeaderStandbyStateModelFactory.java rename to helix-examples/src/main/java/org/apache/helix/examples/LeaderStandbyStateModelFactory.java diff --git a/helix-examples/src/main/java/org/apache/helix/examples/LogicalModelExample.java b/helix-examples/src/main/java/org/apache/helix/examples/LogicalModelExample.java new file mode 100644 index 0000000000..c23341771f --- /dev/null +++ b/helix-examples/src/main/java/org/apache/helix/examples/LogicalModelExample.java @@ -0,0 +1,297 @@ +package org.apache.helix.examples; + +import java.util.List; +import java.util.Map; + +import org.apache.helix.HelixConnection; +import org.apache.helix.HelixController; +import org.apache.helix.HelixParticipant; +import org.apache.helix.NotificationContext; +import org.apache.helix.api.Partition; +import org.apache.helix.api.Scope; +import org.apache.helix.api.State; +import org.apache.helix.api.accessor.ClusterAccessor; +import org.apache.helix.api.accessor.ParticipantAccessor; +import org.apache.helix.api.accessor.ResourceAccessor; +import org.apache.helix.api.config.ClusterConfig; +import org.apache.helix.api.config.ParticipantConfig; +import org.apache.helix.api.config.ResourceConfig; +import org.apache.helix.api.config.UserConfig; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ControllerId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.ResourceId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.rebalancer.context.FullAutoRebalancerContext; +import org.apache.helix.manager.zk.ZkHelixConnection; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.Message; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.model.Transition; +import org.apache.helix.participant.statemachine.HelixStateModelFactory; +import org.apache.helix.participant.statemachine.StateModel; +import org.apache.helix.participant.statemachine.StateModelInfo; +import org.apache.log4j.Logger; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +/* + * 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. + */ + +/** + * Example showing all major interactions with the new Helix logical model + */ +public class LogicalModelExample { + private static final Logger LOG = Logger.getLogger(LogicalModelExample.class); + + public static void main(String[] args) throws InterruptedException { + if (args.length < 1) { + LOG.error("USAGE: LogicalModelExample zkAddress"); + System.exit(1); + } + + // get a state model definition + StateModelDefinition lockUnlock = getLockUnlockModel(); + + // set up a resource with the state model definition + ResourceConfig resource = getResource(lockUnlock); + + // set up a participant + ParticipantConfig participant = getParticipant(); + + // cluster id should be unique + ClusterId clusterId = ClusterId.from("exampleCluster"); + + // a user config is an object that stores arbitrary keys and values for a scope + // in this case, the scope is the cluster with id clusterId + // this is optional + UserConfig userConfig = new UserConfig(Scope.cluster(clusterId)); + userConfig.setIntField("sampleInt", 1); + + // fully specify the cluster with a ClusterConfig + ClusterConfig.Builder clusterBuilder = + new ClusterConfig.Builder(clusterId).addResource(resource).addParticipant(participant) + .addStateModelDefinition(lockUnlock).userConfig(userConfig).autoJoin(true); + + // add a state constraint that is more restrictive than what is in the state model + clusterBuilder.addStateUpperBoundConstraint(Scope.cluster(clusterId), + lockUnlock.getStateModelDefId(), State.from("LOCKED"), 1); + + // add a transition constraint (this time with a resource scope) + clusterBuilder.addTransitionConstraint(Scope.resource(resource.getId()), + lockUnlock.getStateModelDefId(), + Transition.from(State.from("RELEASED"), State.from("LOCKED")), 1); + + ClusterConfig cluster = clusterBuilder.build(); + + // set up a connection to work with ZooKeeper-persisted data + HelixConnection connection = new ZkHelixConnection(args[0]); + connection.connect(); + + // create the cluster + createCluster(cluster, connection); + + // update the resource + updateResource(resource, clusterId, connection); + + // update the participant + updateParticipant(participant, clusterId, connection); + + // start the controller + ControllerId controllerId = ControllerId.from("exampleController"); + HelixController helixController = connection.createController(clusterId, controllerId); + helixController.startAsync(); + + // start the specified participant + HelixParticipant helixParticipant = + connection.createParticipant(clusterId, participant.getId()); + helixParticipant.getStateMachineEngine().registerStateModelFactory( + lockUnlock.getStateModelDefId(), new LockUnlockFactory()); + helixParticipant.startAsync(); + + // start another participant via auto join + HelixParticipant autoJoinParticipant = + connection.createParticipant(clusterId, ParticipantId.from("localhost_12120")); + autoJoinParticipant.getStateMachineEngine().registerStateModelFactory( + lockUnlock.getStateModelDefId(), new LockUnlockFactory()); + autoJoinParticipant.startAsync(); + + Thread.sleep(5000); + printExternalView(connection, clusterId, resource.getId()); + + // stop the participants + helixParticipant.stopAsync(); + autoJoinParticipant.stopAsync(); + + // stop the controller + helixController.stopAsync(); + + // drop the cluster + dropCluster(clusterId, connection); + connection.disconnect(); + } + + private static void dropCluster(ClusterId clusterId, HelixConnection connection) { + ClusterAccessor accessor = connection.createClusterAccessor(clusterId); + accessor.dropCluster(); + } + + private static void printExternalView(HelixConnection connection, ClusterId clusterId, + ResourceId resourceId) { + ResourceAccessor accessor = connection.createResourceAccessor(clusterId); + ExternalView externalView = accessor.readExternalView(resourceId); + System.out.println("ASSIGNMENTS:"); + for (PartitionId partitionId : externalView.getPartitionIdSet()) { + System.out.println(partitionId + ": " + externalView.getStateMap(partitionId)); + } + } + + private static void updateParticipant(ParticipantConfig participant, ClusterId clusterId, + HelixConnection connection) { + // add a tag to the participant and change the hostname, then update it using a delta + ParticipantAccessor accessor = connection.createParticipantAccessor(clusterId); + ParticipantConfig.Delta delta = + new ParticipantConfig.Delta(participant.getId()).addTag("newTag").setHostName("newHost"); + accessor.updateParticipant(participant.getId(), delta); + } + + private static void updateResource(ResourceConfig resource, ClusterId clusterId, + HelixConnection connection) { + // add some fields to the resource user config, then update it using the resource config delta + ResourceAccessor accessor = connection.createResourceAccessor(clusterId); + UserConfig userConfig = resource.getUserConfig(); + Map mapField = Maps.newHashMap(); + mapField.put("k1", "v1"); + mapField.put("k2", "v2"); + userConfig.setMapField("sampleMap", mapField); + ResourceConfig.Delta delta = + new ResourceConfig.Delta(resource.getId()).setUserConfig(userConfig); + accessor.updateResource(resource.getId(), delta); + } + + private static void createCluster(ClusterConfig cluster, HelixConnection connection) { + ClusterAccessor accessor = connection.createClusterAccessor(cluster.getId()); + accessor.createCluster(cluster); + } + + private static ParticipantConfig getParticipant() { + // identify the participant + ParticipantId participantId = ParticipantId.from("localhost_0"); + + // create (optional) participant user config properties + UserConfig userConfig = new UserConfig(Scope.participant(participantId)); + List sampleList = Lists.newArrayList("elem1", "elem2"); + userConfig.setListField("sampleList", sampleList); + + // create the configuration of a new participant + ParticipantConfig.Builder participantBuilder = + new ParticipantConfig.Builder(participantId).hostName("localhost").port(0) + .userConfig(userConfig); + return participantBuilder.build(); + } + + private static ResourceConfig getResource(StateModelDefinition stateModelDef) { + // identify the resource + ResourceId resourceId = ResourceId.from("exampleResource"); + + // create a partition + Partition partition1 = new Partition(PartitionId.from(resourceId, "1")); + + // create a second partition + Partition partition2 = new Partition(PartitionId.from(resourceId, "2")); + + // specify the rebalancer configuration + // this resource will be rebalanced in FULL_AUTO mode, so use the FullAutoRebalancerContext + // builder + FullAutoRebalancerContext.Builder rebalanceContextBuilder = + new FullAutoRebalancerContext.Builder(resourceId).replicaCount(1).addPartition(partition1) + .addPartition(partition2).stateModelDefId(stateModelDef.getStateModelDefId()); + + // create (optional) user-defined configuration properties for the resource + UserConfig userConfig = new UserConfig(Scope.resource(resourceId)); + userConfig.setBooleanField("sampleBoolean", true); + + // create the configuration for a new resource + ResourceConfig.Builder resourceBuilder = + new ResourceConfig.Builder(resourceId).rebalancerContext(rebalanceContextBuilder.build()) + .userConfig(userConfig); + return resourceBuilder.build(); + } + + private static StateModelDefinition getLockUnlockModel() { + final State LOCKED = State.from("LOCKED"); + final State RELEASED = State.from("RELEASED"); + final State DROPPED = State.from("DROPPED"); + StateModelDefId stateModelId = StateModelDefId.from("LockUnlock"); + StateModelDefinition.Builder stateModelBuilder = + new StateModelDefinition.Builder(stateModelId).addState(LOCKED, 0).addState(RELEASED, 1) + .addState(DROPPED, 2).addTransition(RELEASED, LOCKED, 0) + .addTransition(LOCKED, RELEASED, 1).addTransition(RELEASED, DROPPED, 2) + .upperBound(LOCKED, 1).upperBound(RELEASED, -1).upperBound(DROPPED, -1) + .initialState(RELEASED); + return stateModelBuilder.build(); + } + + /** + * Dummy state model that just prints state transitions for the lock-unlock model + */ + @StateModelInfo(initialState = "OFFLINE", states = { + "LOCKED", "RELEASED", "DROPPED", "ERROR" + }) + public static class LockUnlockStateModel extends StateModel { + private final PartitionId _partitionId; + + /** + * Instantiate for a partition + * @param partitionId the partition for which to track state transitions + */ + public LockUnlockStateModel(PartitionId partitionId) { + _partitionId = partitionId; + } + + public void onBecomeLockedFromReleased(Message message, NotificationContext context) { + onBecomeAnyFromAny(message, context); + } + + public void onBecomeReleasedFromLocked(Message message, NotificationContext context) { + onBecomeAnyFromAny(message, context); + } + + public void onBecomeDroppedFromReleased(Message message, NotificationContext context) { + onBecomeAnyFromAny(message, context); + } + + public void onBecomeAnyFromAny(Message message, NotificationContext context) { + System.out.println("Partition " + _partitionId + " transition from " + message.getFromState() + + " to " + message.getToState()); + } + } + + /** + * State model factory for lock-unlock + */ + public static class LockUnlockFactory extends HelixStateModelFactory { + @Override + public LockUnlockStateModel createNewStateModel(PartitionId partitionId) { + return new LockUnlockStateModel(partitionId); + } + } +} diff --git a/helix-core/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java b/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java similarity index 88% rename from helix-core/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java rename to helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java index affbea8b72..71d14124a7 100644 --- a/helix-core/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java +++ b/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java @@ -24,7 +24,6 @@ import org.apache.helix.participant.statemachine.StateModel; import org.apache.helix.participant.statemachine.StateModelFactory; -@SuppressWarnings("rawtypes") public class MasterSlaveStateModelFactory extends StateModelFactory { int _delay; @@ -75,8 +74,8 @@ public void setInstanceName(String instanceName) { public void onBecomeSlaveFromOffline(Message message, NotificationContext context) { - System.out.println(_instanceName + " transitioning from " + message.getFromState() + " to " - + message.getToState() + " for " + partitionName); + System.out.println(_instanceName + " transitioning from " + message.getTypedFromState() + " to " + + message.getTypedToState() + " for " + partitionName); sleep(); } @@ -89,22 +88,22 @@ private void sleep() { } public void onBecomeSlaveFromMaster(Message message, NotificationContext context) { - System.out.println(_instanceName + " transitioning from " + message.getFromState() + " to " - + message.getToState() + " for " + partitionName); + System.out.println(_instanceName + " transitioning from " + message.getTypedFromState() + " to " + + message.getTypedToState() + " for " + partitionName); sleep(); } public void onBecomeMasterFromSlave(Message message, NotificationContext context) { - System.out.println(_instanceName + " transitioning from " + message.getFromState() + " to " - + message.getToState() + " for " + partitionName); + System.out.println(_instanceName + " transitioning from " + message.getTypedFromState() + " to " + + message.getTypedToState() + " for " + partitionName); sleep(); } public void onBecomeOfflineFromSlave(Message message, NotificationContext context) { - System.out.println(_instanceName + " transitioning from " + message.getFromState() + " to " - + message.getToState() + " for " + partitionName); + System.out.println(_instanceName + " transitioning from " + message.getTypedFromState() + " to " + + message.getTypedToState() + " for " + partitionName); sleep(); } diff --git a/helix-core/src/main/java/org/apache/helix/examples/OnlineOfflineStateModelFactory.java b/helix-examples/src/main/java/org/apache/helix/examples/OnlineOfflineStateModelFactory.java similarity index 100% rename from helix-core/src/main/java/org/apache/helix/examples/OnlineOfflineStateModelFactory.java rename to helix-examples/src/main/java/org/apache/helix/examples/OnlineOfflineStateModelFactory.java diff --git a/helix-core/src/main/java/org/apache/helix/examples/Quickstart.java b/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java similarity index 93% rename from helix-core/src/main/java/org/apache/helix/examples/Quickstart.java rename to helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java index b80d458b58..2f3a677b16 100644 --- a/helix-core/src/main/java/org/apache/helix/examples/Quickstart.java +++ b/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java @@ -32,9 +32,10 @@ import org.apache.helix.HelixManager; import org.apache.helix.HelixManagerFactory; import org.apache.helix.InstanceType; +import org.apache.helix.api.State; +import org.apache.helix.api.id.StateModelDefId; import org.apache.helix.controller.HelixControllerMain; import org.apache.helix.manager.zk.ZKHelixAdmin; -import org.apache.helix.manager.zk.ZNRecordSerializer; import org.apache.helix.model.ExternalView; import org.apache.helix.model.InstanceConfig; import org.apache.helix.model.StateModelDefinition; @@ -49,13 +50,13 @@ public class Quickstart { private static final int NUM_PARTITIONS = 6; private static final int NUM_REPLICAS = 2; - private static final String STATE_MODEL_NAME = "MyStateModel"; + private static final StateModelDefId STATE_MODEL_NAME = StateModelDefId.from("MyStateModel"); // states - private static final String SLAVE = "SLAVE"; - private static final String OFFLINE = "OFFLINE"; - private static final String MASTER = "MASTER"; - private static final String DROPPED = "DROPPED"; + private static final State SLAVE = State.from("SLAVE"); + private static final State OFFLINE = State.from("OFFLINE"); + private static final State MASTER = State.from("MASTER"); + private static final State DROPPED = State.from("DROPPED"); private static List INSTANCE_CONFIG_LIST; private static List PROCESS_LIST; @@ -90,11 +91,12 @@ public static void setup() { // Add a state model StateModelDefinition myStateModel = defineStateModel(); echo("Configuring StateModel: " + "MyStateModel with 1 Master and 1 Slave"); - admin.addStateModelDef(CLUSTER_NAME, STATE_MODEL_NAME, myStateModel); + admin.addStateModelDef(CLUSTER_NAME, STATE_MODEL_NAME.stringify(), myStateModel); // Add a resource with 6 partitions and 2 replicas echo("Adding a resource MyResource: " + "with 6 partitions and 2 replicas"); - admin.addResource(CLUSTER_NAME, RESOURCE_NAME, NUM_PARTITIONS, STATE_MODEL_NAME, "AUTO"); + admin.addResource(CLUSTER_NAME, RESOURCE_NAME, NUM_PARTITIONS, STATE_MODEL_NAME.stringify(), + "AUTO"); // this will set up the ideal state, it calculates the preference list for // each partition similar to consistent hashing admin.rebalance(CLUSTER_NAME, RESOURCE_NAME, NUM_REPLICAS); @@ -250,7 +252,7 @@ public void start() throws Exception { new MasterSlaveStateModelFactory(instanceName); StateMachineEngine stateMach = manager.getStateMachineEngine(); - stateMach.registerStateModelFactory(STATE_MODEL_NAME, stateModelFactory); + stateMach.registerStateModelFactory(STATE_MODEL_NAME.stringify(), stateModelFactory); manager.connect(); } diff --git a/helix-core/src/main/java/org/apache/helix/examples/package-info.java b/helix-examples/src/main/java/org/apache/helix/examples/package-info.java similarity index 100% rename from helix-core/src/main/java/org/apache/helix/examples/package-info.java rename to helix-examples/src/main/java/org/apache/helix/examples/package-info.java diff --git a/helix-examples/src/test/conf/testng.xml b/helix-examples/src/test/conf/testng.xml new file mode 100644 index 0000000000..07080ea5ef --- /dev/null +++ b/helix-examples/src/test/conf/testng.xml @@ -0,0 +1,27 @@ + + + + + + + + + + diff --git a/helix-monitor-client/DISCLAIMER b/helix-monitor-client/DISCLAIMER new file mode 100644 index 0000000000..2001d3154d --- /dev/null +++ b/helix-monitor-client/DISCLAIMER @@ -0,0 +1,15 @@ +Apache Helix is an effort undergoing incubation at the Apache Software +Foundation (ASF), sponsored by the Apache Incubator PMC. + +Incubation is required of all newly accepted projects until a further review +indicates that the infrastructure, communications, and decision making process +have stabilized in a manner consistent with other successful ASF projects. + +While incubation status is not necessarily a reflection of the completeness +or stability of the code, it does indicate that the project has yet to be +fully endorsed by the ASF. + +For more information about the incubation status of the Apache Helix project you +can go to the following page: + +http://incubator.apache.org/projects/helix.html diff --git a/helix-monitor-client/LICENSE b/helix-monitor-client/LICENSE new file mode 100644 index 0000000000..413913f132 --- /dev/null +++ b/helix-monitor-client/LICENSE @@ -0,0 +1,273 @@ + + 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. + + + diff --git a/helix-monitor-client/NOTICE b/helix-monitor-client/NOTICE new file mode 100644 index 0000000000..e070e15573 --- /dev/null +++ b/helix-monitor-client/NOTICE @@ -0,0 +1,30 @@ +Apache Helix +Copyright 2012 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 +josql (http://sourceforge.net/projects/josql/). +Licensed under the Apache License 2.0. + +This product includes software developed at +restlet (http://www.restlet.org/about/legal). +Licensed under the Apache License 2.0. + + +II. License Summary +- Apache License 2.0 +- BSD License diff --git a/helix-monitor-client/pom.xml b/helix-monitor-client/pom.xml new file mode 100644 index 0000000000..5857a59266 --- /dev/null +++ b/helix-monitor-client/pom.xml @@ -0,0 +1,113 @@ + + + + + org.apache.helix + helix + 0.7.1-incubating-SNAPSHOT + + 4.0.0 + + helix-monitor-client + jar + Apache Helix :: Helix Monitoring Client + + + + org.apache.helix*, + org.apache.log4j, + * + + org.apache.helix.monitoring*;version="${project.version};-noimport:=true + + + + + org.apache.helix + helix-core + + + com.aphyr + riemann-java-client + 0.2.8 + + + org.testng + testng + test + + + junit + junit + + + + + org.apache.helix + helix-core + test-jar + test + + + + + + ${basedir}/src/main/resources + true + + + ${basedir} + + DISCLAIMER + + + + + + org.codehaus.mojo + appassembler-maven-plugin + + + windows + unix + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/assemble/assembly.xml + + + + + package + + single + + + + + + + diff --git a/helix-monitor-client/src/assemble/assembly.xml b/helix-monitor-client/src/assemble/assembly.xml new file mode 100644 index 0000000000..c2d08a1cda --- /dev/null +++ b/helix-monitor-client/src/assemble/assembly.xml @@ -0,0 +1,60 @@ + + + + pkg + + tar + + + + ${project.build.directory}/${project.artifactId}-pkg/bin + bin + unix + 0755 + 0755 + + + ${project.build.directory}/${project.artifactId}-pkg/repo/ + repo + 0755 + 0755 + + **/*.xml + + + + ${project.build.directory}/${project.artifactId}-pkg/conf + conf + unix + 0755 + 0755 + + + ${project.basedir} + / + + LICENSE + NOTICE + DISCLAIMER + + 0755 + + + \ No newline at end of file diff --git a/helix-monitor-client/src/main/config/log4j.properties b/helix-monitor-client/src/main/config/log4j.properties new file mode 100644 index 0000000000..4b3dc31577 --- /dev/null +++ b/helix-monitor-client/src/main/config/log4j.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# Set root logger level to DEBUG and its only appender to A1. +log4j.rootLogger=ERROR,A1 + +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender + +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +log4j.logger.org.I0Itec=ERROR +log4j.logger.org.apache=ERROR diff --git a/helix-monitor-client/src/main/java/org/apache/helix/monitoring/riemann/ClientUtil.java b/helix-monitor-client/src/main/java/org/apache/helix/monitoring/riemann/ClientUtil.java new file mode 100644 index 0000000000..0c12ca349b --- /dev/null +++ b/helix-monitor-client/src/main/java/org/apache/helix/monitoring/riemann/ClientUtil.java @@ -0,0 +1,69 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.monitoring.MonitoringEvent; + +import com.aphyr.riemann.client.AbstractRiemannClient; +import com.aphyr.riemann.client.EventDSL; + +public class ClientUtil { + /** + * Change a helix monitoring event into a Riemann event + * @param c Riemann client + * @param helixEvent helix event + * @return Riemann EventDSL + */ + public static EventDSL convertEvent(AbstractRiemannClient c, MonitoringEvent helixEvent) { + EventDSL event = c.event(); + if (helixEvent.host() != null) { + event.host(helixEvent.host()); + } + if (helixEvent.service() != null) { + event.service(helixEvent.service()); + } + if (helixEvent.eventState() != null) { + event.state(helixEvent.eventState()); + } + if (helixEvent.description() != null) { + event.description(helixEvent.description()); + } + if (helixEvent.time() != null) { + event.time(helixEvent.time()); + } + if (helixEvent.ttl() != null) { + event.ttl(helixEvent.ttl()); + } + if (helixEvent.longMetric() != null) { + event.metric(helixEvent.longMetric()); + } else if (helixEvent.floatMetric() != null) { + event.metric(helixEvent.floatMetric()); + } else if (helixEvent.doubleMetric() != null) { + event.metric(helixEvent.doubleMetric()); + } + if (!helixEvent.tags().isEmpty()) { + event.tags(helixEvent.tags()); + } + if (!helixEvent.attributes().isEmpty()) { + event.attributes.putAll(helixEvent.attributes()); + } + return event; + } +} diff --git a/helix-monitor-client/src/main/java/org/apache/helix/monitoring/riemann/RawRiemannClient.java b/helix-monitor-client/src/main/java/org/apache/helix/monitoring/riemann/RawRiemannClient.java new file mode 100644 index 0000000000..4fe5902dd9 --- /dev/null +++ b/helix-monitor-client/src/main/java/org/apache/helix/monitoring/riemann/RawRiemannClient.java @@ -0,0 +1,229 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.net.UnknownHostException; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +import org.apache.helix.monitoring.MonitoringEvent; +import org.apache.log4j.Logger; + +import com.aphyr.riemann.client.AbstractRiemannClient; +import com.aphyr.riemann.client.RiemannBatchClient; +import com.aphyr.riemann.client.RiemannClient; +import com.aphyr.riemann.client.UnsupportedJVMException; + +/** + * Simple wrapper around RiemannClient that does auto reconnect + */ +class RawRiemannClient { + private static final Logger LOG = Logger.getLogger(RawRiemannClient.class); + private static final int HEARTBEAT_PERIOD = 10; + private static final int TIMEOUT_LIMIT = 3; + + enum State { + DISCONNECTED, + CONNECTED, + RECONNECTING + } + + private RiemannClient _rclient; + private RiemannBatchClient _brclient; + private volatile State _state = State.DISCONNECTED; + private final String _host; + private final int _port; + private int _batchSize; + private Thread _reconnectThread; + + public RawRiemannClient(String host, int port) { + this(host, port, 1); + } + + public RawRiemannClient(String host, int port, int batchSize) { + _host = host; + _port = port; + _batchSize = batchSize > 0 ? batchSize : 1; + } + + private synchronized boolean doConnect() { + if (_state == State.CONNECTED) { + return true; + } + + try { + _rclient = RiemannClient.tcp(_host, _port); + _rclient.connect(); + if (_rclient != null && _batchSize > 1) { + try { + _brclient = new RiemannBatchClient(_batchSize, _rclient); + } catch (UnknownHostException e) { + _batchSize = 1; + LOG.error("Could not resolve host", e); + } catch (UnsupportedJVMException e) { + _batchSize = 1; + LOG.warn("Batching not enabled because of incompatible JVM", e); + } + } + + Random random = new Random(); + _rclient.every(HEARTBEAT_PERIOD, random.nextInt(HEARTBEAT_PERIOD), TimeUnit.SECONDS, + new Runnable() { + + @Override + public void run() { + try { + _rclient.event().service("heartbeat").ttl(TIMEOUT_LIMIT * HEARTBEAT_PERIOD) + .sendWithAck(); + _state = State.CONNECTED; + } catch (Exception e) { + LOG.error("Exception in send heatbeat to riemann server: " + _host + ":" + _port, e); + _state = State.RECONNECTING; + } + } + }); + _state = State.CONNECTED; + return true; + } catch (IOException e) { + LOG.error("Fail to connect to riemann server: " + _host + ":" + _port); + } + + return false; + } + + /** + * Make a connection to Riemann server; if fails, start a thread for retrying + */ + public synchronized void connect() { + boolean success = doConnect(); + if (!success) { + _reconnectThread = new Thread(new Runnable() { + + @Override + public void run() { + LOG.info("Start reconnect thread"); + Random random = new Random(); + try { + while (!Thread.currentThread().isInterrupted()) { + boolean success = doConnect(); + if (success) { + break; + } + + TimeUnit.SECONDS.sleep(HEARTBEAT_PERIOD + random.nextInt() % HEARTBEAT_PERIOD); + + } + } catch (InterruptedException e) { + LOG.info("Reconnect thread is interrupted"); + } finally { + LOG.info("Terminate reconnect thread"); + } + } + }); + + _reconnectThread.start(); + } + } + + /** + * Disconnect from Riemann server + */ + public synchronized void disconnect() { + try { + if (_reconnectThread != null) { + _reconnectThread.interrupt(); + } + + if (_rclient != null) { + _rclient.scheduler().shutdown(); + _rclient.disconnect(); + } + } catch (IOException e) { + LOG.error("Fail to disconnect rclient for " + _host + ":" + _port, e); + } + _state = State.DISCONNECTED; + } + + public boolean isConnected() { + return _state == State.CONNECTED; + } + + private AbstractRiemannClient client() { + if (isBatchEnabled()) { + return _brclient; + } else { + return _rclient; + } + } + + public boolean send(MonitoringEvent event) { + if (!isConnected()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Fail to send event: " + event + " to " + _host + ":" + _port + + ", because state is not connected, was: " + _state); + } + return false; + } + + try { + ClientUtil.convertEvent(client(), event).send(); + return true; + } catch (Exception e) { + LOG.error("Fail to send event: " + event + " to " + _host + ":" + _port, e); + } + return false; + } + + public boolean flush() { + if (!isConnected()) { + return false; + } + + try { + client().flush(); + return true; + } catch (IOException e) { + LOG.error("Problem flushing the Riemann event queue", e); + } + return false; + } + + public boolean sendAndFlush(MonitoringEvent event) { + boolean success = send(event); + if (success) { + return flush(); + } + + return false; + } + + public int getBatchSize() { + return _batchSize; + } + + public boolean isBatchEnabled() { + return _batchSize > 1; + } + + public State getState() { + return _state; + } +} diff --git a/helix-monitor-client/src/main/java/org/apache/helix/monitoring/riemann/RiemannClientWrapper.java b/helix-monitor-client/src/main/java/org/apache/helix/monitoring/riemann/RiemannClientWrapper.java new file mode 100644 index 0000000000..bd6517fed2 --- /dev/null +++ b/helix-monitor-client/src/main/java/org/apache/helix/monitoring/riemann/RiemannClientWrapper.java @@ -0,0 +1,234 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.List; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.helix.monitoring.MonitoringClient; +import org.apache.helix.monitoring.MonitoringEvent; +import org.apache.log4j.Logger; + +/** + * Wrapper around a list of RawRiemannClients; if one client fails, try the next one in list + */ +public class RiemannClientWrapper implements MonitoringClient { + private static final Logger LOG = Logger.getLogger(RiemannClientWrapper.class); + + /** + * A list of "host:port" addresses for Riemann servers + */ + private final List _rsHosts; + private boolean _isConnected; + private List _rclients; + private int _batchSize; + + private ScheduledThreadPoolExecutor _pool; + + public RiemannClientWrapper(List rsHosts) { + this(rsHosts, 1); + } + + public RiemannClientWrapper(List rsHosts, int batchSize) { + _rsHosts = rsHosts; + Collections.sort(_rsHosts); + _batchSize = batchSize > 0 ? batchSize : 1; + _isConnected = false; + } + + // Returns the pool for this client. Creates the pool on first use + private synchronized ScheduledThreadPoolExecutor pool() { + if (_pool == null) { + _pool = new ScheduledThreadPoolExecutor(1); + _pool.setContinueExistingPeriodicTasksAfterShutdownPolicy(false); + _pool.setExecuteExistingDelayedTasksAfterShutdownPolicy(false); + } + + return _pool; + } + + @Override + public synchronized void connect() throws Exception { + if (_isConnected) { + LOG.warn("Client already connected"); + return; + } + + _rclients = new ArrayList(); + for (String rsHost : _rsHosts) { + String[] splits = rsHost.split(":"); + + if (splits == null || splits.length != 2) { + throw new IllegalArgumentException("Invalid Riemann server: " + rsHost); + } + + String host = splits[0]; + int port = Integer.parseInt(splits[1]); + + RawRiemannClient rclient = new RawRiemannClient(host, port, _batchSize); + rclient.connect(); + + /** + * If any Riemann client doesn't support batch, set it to 1 + */ + if (rclient.isConnected() && rclient.getBatchSize() == 1) { + _batchSize = 1; + } + _rclients.add(rclient); + } + + _isConnected = true; + } + + @Override + public synchronized void disconnect() { + if (!_isConnected || _rclients == null) { + LOG.warn("Client already disconnected"); + return; + } + + for (RawRiemannClient rclient : _rclients) { + rclient.disconnect(); + } + _rclients = null; + _isConnected = false; + } + + /** + * Get raw client based on event's sharding key + * @param event + * @return + */ + private RawRiemannClient client(MonitoringEvent event) { + String shardingKey = event.shardingKey(); + int baseIdx = shardingKey.hashCode() % _rsHosts.size(); + + // find the first rclient in CONNECTED state and send + for (int i = 0; i < _rsHosts.size(); i++) { + int idx = (baseIdx + i) % _rsHosts.size(); + RawRiemannClient rclient = _rclients.get(idx); + if (rclient.isConnected()) { + return rclient; + } + } + return null; + } + + @Override + public boolean send(MonitoringEvent event) { + if (!_isConnected || _rclients == null) { + LOG.warn("Client is not connected. Fail to send event: " + event); + return false; + } + + RawRiemannClient rclient = client(event); + if (rclient != null) { + return rclient.send(event); + } + + LOG.error("Fail to send event: " + event + ", no rclient is available"); + return false; + } + + @Override + public void every(long interval, long delay, TimeUnit unit, Runnable r) { + pool().scheduleAtFixedRate(r, delay, interval, unit); + } + + @Override + public boolean sendAndFlush(MonitoringEvent event) { + if (!_isConnected || _rclients == null) { + LOG.warn("Client is not connected. Fail to send event: " + event); + return false; + } + + RawRiemannClient rclient = client(event); + if (rclient != null) { + boolean success = rclient.send(event); + if (success) { + return rclient.flush(); + } + } + LOG.error("Fail to send event: " + event + ", no rclient is available"); + return false; + } + + @Override + public boolean isConnected() { + return _isConnected; + } + + @Override + public boolean isBatchingEnabled() { + if (!_isConnected || _rclients == null) { + LOG.warn("Client is not connected"); + return false; + } + + /** + * Batch should be enabled for all or none raw clients + */ + for (RawRiemannClient rclient : _rclients) { + if (rclient.isConnected()) { + return rclient.isBatchEnabled(); + } + } + return false; + } + + /** + * Return batch size if connected or 1 otherwise + */ + @Override + public int getBatchSize() { + if (!_isConnected || _rclients == null) { + LOG.warn("Client is not connected"); + return 1; + } + + /** + * Batch size should be the same for all raw clients + */ + for (RawRiemannClient rclient : _rclients) { + if (rclient.isConnected()) { + return rclient.getBatchSize(); + } + } + + return 1; + } + + @Override + public boolean flush() { + if (!_isConnected || _rclients == null) { + LOG.warn("Client is not connected"); + return false; + } + + boolean success = true; + for (RawRiemannClient rclient : _rclients) { + success &= rclient.flush(); + } + return success; + } +} diff --git a/helix-monitor-client/src/test/conf/testng.xml b/helix-monitor-client/src/test/conf/testng.xml new file mode 100644 index 0000000000..90910aac8d --- /dev/null +++ b/helix-monitor-client/src/test/conf/testng.xml @@ -0,0 +1,27 @@ + + + + + + + + + + diff --git a/helix-monitor-server/.gitignore b/helix-monitor-server/.gitignore new file mode 100644 index 0000000000..a2d1ebb1c2 --- /dev/null +++ b/helix-monitor-server/.gitignore @@ -0,0 +1,11 @@ +/target +/.project +/.classpath +/.settings +/zkdata +/test-output +/src/main/scripts/integration-test/var +/src/main/scripts/target/ +/src/main/scripts/integration-test/script/.metadata_infra +*.pyc +/bin diff --git a/helix-monitor-server/DISCLAIMER b/helix-monitor-server/DISCLAIMER new file mode 100644 index 0000000000..2001d3154d --- /dev/null +++ b/helix-monitor-server/DISCLAIMER @@ -0,0 +1,15 @@ +Apache Helix is an effort undergoing incubation at the Apache Software +Foundation (ASF), sponsored by the Apache Incubator PMC. + +Incubation is required of all newly accepted projects until a further review +indicates that the infrastructure, communications, and decision making process +have stabilized in a manner consistent with other successful ASF projects. + +While incubation status is not necessarily a reflection of the completeness +or stability of the code, it does indicate that the project has yet to be +fully endorsed by the ASF. + +For more information about the incubation status of the Apache Helix project you +can go to the following page: + +http://incubator.apache.org/projects/helix.html diff --git a/helix-monitor-server/LICENSE b/helix-monitor-server/LICENSE new file mode 100644 index 0000000000..413913f132 --- /dev/null +++ b/helix-monitor-server/LICENSE @@ -0,0 +1,273 @@ + + 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. + + + diff --git a/helix-monitor-server/NOTICE b/helix-monitor-server/NOTICE new file mode 100644 index 0000000000..e070e15573 --- /dev/null +++ b/helix-monitor-server/NOTICE @@ -0,0 +1,30 @@ +Apache Helix +Copyright 2012 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 +josql (http://sourceforge.net/projects/josql/). +Licensed under the Apache License 2.0. + +This product includes software developed at +restlet (http://www.restlet.org/about/legal). +Licensed under the Apache License 2.0. + + +II. License Summary +- Apache License 2.0 +- BSD License diff --git a/helix-monitor-server/pom.xml b/helix-monitor-server/pom.xml new file mode 100644 index 0000000000..041a3901d5 --- /dev/null +++ b/helix-monitor-server/pom.xml @@ -0,0 +1,123 @@ + + + + + org.apache.helix + helix + 0.7.1-incubating-SNAPSHOT + + 4.0.0 + + helix-monitor-server + jar + Apache Helix :: Helix Monitoring Server + + + + org.apache.helix*, + org.apache.log4j, + * + + org.apache.helix.monitoring*;version="${project.version};-noimport:=true + + + + + org.apache.helix + helix-core + + + org.apache.helix + helix-monitor-client + 0.7.1-incubating-SNAPSHOT + + + riemann + riemann + 0.2.4 + + + factual + clj-helix + 0.1.0 + + + org.testng + testng + test + + + junit + junit + + + + + org.apache.helix + helix-core + test-jar + test + + + + + + ${basedir}/src/main/resources + true + + + ${basedir} + + DISCLAIMER + + + + + + org.codehaus.mojo + appassembler-maven-plugin + + + windows + unix + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/assemble/assembly.xml + + + + + package + + single + + + + + + + diff --git a/helix-monitor-server/src/assemble/assembly.xml b/helix-monitor-server/src/assemble/assembly.xml new file mode 100644 index 0000000000..c2d08a1cda --- /dev/null +++ b/helix-monitor-server/src/assemble/assembly.xml @@ -0,0 +1,60 @@ + + + + pkg + + tar + + + + ${project.build.directory}/${project.artifactId}-pkg/bin + bin + unix + 0755 + 0755 + + + ${project.build.directory}/${project.artifactId}-pkg/repo/ + repo + 0755 + 0755 + + **/*.xml + + + + ${project.build.directory}/${project.artifactId}-pkg/conf + conf + unix + 0755 + 0755 + + + ${project.basedir} + / + + LICENSE + NOTICE + DISCLAIMER + + 0755 + + + \ No newline at end of file diff --git a/helix-monitor-server/src/main/config/log4j.properties b/helix-monitor-server/src/main/config/log4j.properties new file mode 100644 index 0000000000..4b3dc31577 --- /dev/null +++ b/helix-monitor-server/src/main/config/log4j.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# Set root logger level to DEBUG and its only appender to A1. +log4j.rootLogger=ERROR,A1 + +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender + +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +log4j.logger.org.I0Itec=ERROR +log4j.logger.org.apache=ERROR diff --git a/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/HelixAlertMessenger.java b/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/HelixAlertMessenger.java new file mode 100644 index 0000000000..cbc209af7f --- /dev/null +++ b/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/HelixAlertMessenger.java @@ -0,0 +1,112 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.LinkedList; +import java.util.Queue; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.ZNRecord; +import org.apache.helix.controller.alert.AlertName; +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.manager.zk.ZkClient; +import org.apache.helix.model.Message; +import org.apache.helix.model.Message.MessageType; +import org.apache.log4j.Logger; + +/** + * Accept alerts from local Riemann server and forward it to helix-controller + */ +public class HelixAlertMessenger { + private static final Logger LOG = Logger.getLogger(HelixAlertMessenger.class); + private static final int DEFAULT_MAX_ALERT_COUNT = 1; + private static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MINUTES; + + private final ZkClient _zkclient; + private final BaseDataAccessor _baseAccessor; + + /** + * A queue that keeps track of timestamps (in millisecond) of last N alerts sent + */ + private final Queue _queue; + private final int _maxAlertCount; + private final TimeUnit _timeUnit; + + public HelixAlertMessenger(String zkHosts) { + this(zkHosts, DEFAULT_MAX_ALERT_COUNT, DEFAULT_TIME_UNIT); + } + + public HelixAlertMessenger(String zkHosts, int maxAlertCount, TimeUnit timeUnit) { + _zkclient = + new ZkClient(zkHosts, ZkClient.DEFAULT_SESSION_TIMEOUT, + ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer()); + _baseAccessor = new ZkBaseDataAccessor(_zkclient); + _queue = new LinkedList(); + _maxAlertCount = maxAlertCount; + _timeUnit = timeUnit; + } + + /** + * Send alert to helix controller; throttle if necessary + * not thread-safe + * @param alertNameStr + */ + public void onAlert(String alertNameStr) { + LOG.info("Handling alert: " + alertNameStr); + + // throttling + long now = System.currentTimeMillis(); + if (_queue.size() >= _maxAlertCount) { + if (_queue.peek() + _timeUnit.toMillis(_maxAlertCount) > now) { + if (LOG.isDebugEnabled()) { + LOG.debug("Throttling alert: " + alertNameStr); + } + return; + } else { + _queue.remove(); + } + } + + // Send alert message to the controller of cluster being monitored + try { + AlertName alertName = AlertName.from(alertNameStr); + String clusterName = alertName.getScope().getClusterId().stringify(); + HelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, _baseAccessor); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + Message message = new Message(MessageType.ALERT, UUID.randomUUID().toString()); + message.setAttribute(Message.Attributes.ALERT_NAME, alertNameStr); + message.setTgtSessionId("*"); + message.setTgtName("controller"); + accessor.setProperty(keyBuilder.controllerMessage(message.getId()), message); + + // record the timestamp + _queue.add(now); + + } catch (Exception e) { + LOG.error("Fail to send alert to cluster being monitored: " + alertNameStr, e); + } + } +} diff --git a/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/RiemannAgent.java b/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/RiemannAgent.java new file mode 100644 index 0000000000..3e7efcaf9c --- /dev/null +++ b/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/RiemannAgent.java @@ -0,0 +1,169 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.net.InetAddress; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +import org.apache.helix.HelixManager; +import org.apache.helix.HelixManagerFactory; +import org.apache.helix.InstanceType; +import org.apache.log4j.Logger; + +import com.aphyr.riemann.client.RiemannClient; + +/** + * Start a Helix participant that joins cluster and represents local Riemann server + */ +public class RiemannAgent { + private static final Logger LOG = Logger.getLogger(RiemannAgent.class); + private static final int HEARTBEAT_PERIOD = 10; + private static final int TIMEOUT_LIMIT = 3; + + private final String _zkAddr; + private final String _clusterName; + private final String _instanceName; + private final int _riemannPort; + private HelixManager _participant; + private final RiemannClient _client; + private Thread _reconnectThread; + + public RiemannAgent(String zkAddr, String clusterName, int riemannPort) throws IOException { + _zkAddr = zkAddr; + _clusterName = clusterName; + _instanceName = + String.format("%s_%d", InetAddress.getLocalHost().getCanonicalHostName(), riemannPort); + _riemannPort = riemannPort; + _client = RiemannClient.tcp("localhost", riemannPort); + } + + private synchronized boolean doStart() throws Exception { + try { + _client.connect(); + _client.event().service("heartbeat").state("running").ttl(TIMEOUT_LIMIT * HEARTBEAT_PERIOD) + .sendWithAck(); + LOG.info("RiemannAgent connected to local riemann server on localhost:" + _riemannPort); + _participant = + HelixManagerFactory.getZKHelixManager(_clusterName, _instanceName, + InstanceType.PARTICIPANT, _zkAddr); + _participant.connect(); + + // Monitoring Riemann server + Random random = new Random(); + _client.every(HEARTBEAT_PERIOD, random.nextInt(HEARTBEAT_PERIOD), TimeUnit.SECONDS, + new Runnable() { + + @Override + public void run() { + try { + // Send heartbeat metrics + _client.event().service("heartbeat").state("running") + .ttl(TIMEOUT_LIMIT * HEARTBEAT_PERIOD).sendWithAck(); + if (_participant == null) { + _participant = + HelixManagerFactory.getZKHelixManager(_clusterName, _instanceName, + InstanceType.PARTICIPANT, _zkAddr); + _participant.connect(); + } + } catch (Exception e) { + LOG.error( + "Exception in send heatbeat to local riemann server, shutdown RiemannAgent: " + + _instanceName, e); + + if (_participant != null) { + _participant.disconnect(); + _participant = null; + } + } + } + }); + + return true; + } catch (IOException e) { + LOG.error("Fail to connect to Riemann server on localhost:" + _riemannPort); + } + return false; + } + + /** + * Try connect local Riemann server; if fails, start a thread to retry async + * @throws Exception + */ + public synchronized void start() throws Exception { + LOG.info("Starting RiemannAgent. zk: " + _zkAddr + ", cluster: " + _clusterName + + ", instance: " + _instanceName + ", riemannPort: " + _riemannPort); + + boolean success = doStart(); + if (!success) { + _reconnectThread = new Thread(new Runnable() { + + @Override + public void run() { + LOG.info("Start reconnect thread"); + Random random = new Random(); + try { + while (!Thread.currentThread().isInterrupted()) { + boolean success = doStart(); + if (success) { + break; + } + + TimeUnit.SECONDS.sleep(HEARTBEAT_PERIOD + random.nextInt() % HEARTBEAT_PERIOD); + + } + } catch (InterruptedException e) { + LOG.info("Reconnect thread is interrupted"); + } catch (Exception e) { + LOG.error("Fail to start RiemannAgent", e); + } finally { + LOG.info("Terminate reconnect thread"); + } + + } + }); + _reconnectThread.start(); + } + + } + + public synchronized void shutdown() { + LOG.info("Shutting down RiemannAgent. zk: " + _zkAddr + ", cluster: " + _clusterName + + ", instance: " + _instanceName + ", riemannPort: " + _riemannPort); + + if (_reconnectThread != null) { + _reconnectThread.interrupt(); + _reconnectThread = null; + } + + try { + _client.scheduler().shutdown(); + _client.disconnect(); + } catch (IOException e) { + LOG.error("Exception in disconnect riemann client", e); + } + + if (_participant != null) { + _participant.disconnect(); + _participant = null; + } + } +} diff --git a/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/RiemannConfigs.java b/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/RiemannConfigs.java new file mode 100644 index 0000000000..43b957ea91 --- /dev/null +++ b/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/RiemannConfigs.java @@ -0,0 +1,116 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.FileNotFoundException; +import java.io.PrintWriter; +import java.util.List; + +import org.apache.helix.model.MonitoringConfig; +import org.apache.log4j.Logger; + +import com.google.common.collect.Lists; + +/** + * Riemann configs + */ +public class RiemannConfigs { + private static final Logger LOG = Logger.getLogger(RiemannConfigs.class); + public static final String DEFAULT_CONFIG_DIR = "riemannconfigs"; + public static final String DEFAULT_RIEMANN_CONFIG = "riemann.config"; + + private final String _configDir; + private final List _configs; + + RiemannConfigs(String configDir, List configs) { + _configDir = configDir; + _configs = configs; + } + + /** + * persist configs to riemann config dir + */ + public void persistConfigs() { + // create the directory + File dir = new File(_configDir); + if (!dir.exists()) { + dir.mkdir(); + } + + for (MonitoringConfig config : _configs) { + String configData = config.getConfig(); + String fileName = _configDir + "/" + config.getId(); + try { + PrintWriter writer = new PrintWriter(fileName); + writer.println(configData); + writer.close(); + + // make sure this is cleaned up eventually + File file = new File(fileName); + file.deleteOnExit(); + } catch (FileNotFoundException e) { + LOG.error("Could not write " + config.getId(), e); + } + } + } + + public String getConfigDir() { + return _configDir; + } + + public static class Builder { + private final List _configs; + private final String _configDir; + + /** + * By default, configs will be placed in "{systemTmpDir}/riemannconfigs" + */ + public Builder() { + this(System.getProperty("java.io.tmpdir") + "/" + DEFAULT_CONFIG_DIR); + } + + public Builder(String configDir) { + _configDir = configDir; + _configs = Lists.newArrayList(); + } + + public Builder addConfig(MonitoringConfig monitoringConfig) { + _configs.add(monitoringConfig); + return this; + } + + public Builder addConfigs(List monitoringConfigs) { + _configs.addAll(monitoringConfigs); + return this; + } + + public RiemannConfigs build() { + // Check default riemann config exists + for (MonitoringConfig config : _configs) { + if (config.getId().equals(DEFAULT_RIEMANN_CONFIG)) { + return new RiemannConfigs(_configDir, _configs); + } + } + throw new IllegalArgumentException("Missing default riemann config: " + + DEFAULT_RIEMANN_CONFIG); + } + } +} diff --git a/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/RiemannMonitoringServer.java b/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/RiemannMonitoringServer.java new file mode 100644 index 0000000000..63e0f65abe --- /dev/null +++ b/helix-monitor-server/src/main/java/org/apache/helix/monitoring/riemann/RiemannMonitoringServer.java @@ -0,0 +1,74 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.monitoring.MonitoringServer; +import org.apache.log4j.Logger; + +import clojure.lang.RT; +import clojure.lang.Symbol; + +/** + * A monitoring server implementation that uses Riemann + */ +public class RiemannMonitoringServer implements MonitoringServer { + private static final Logger LOG = Logger.getLogger(RiemannMonitoringServer.class); + + private volatile boolean _isStarted; + private final RiemannConfigs _config; + + /** + * Create a monitoring server + * @param config + */ + public RiemannMonitoringServer(RiemannConfigs config) { + LOG.info("Construct RiemannMonitoringServer with configDir: " + config.getConfigDir()); + _config = config; + config.persistConfigs(); + _isStarted = false; + } + + @Override + public synchronized void start() { + LOG.info("Starting Riemann server with configDir: " + _config.getConfigDir()); + + // start Riemann + RT.var("clojure.core", "require").invoke(Symbol.intern("riemann.bin")); + RT.var("clojure.core", "require").invoke(Symbol.intern(RiemannConfigs.DEFAULT_RIEMANN_CONFIG)); + RT.var("riemann.bin", "-main").invoke(_config.getConfigDir()); + _isStarted = true; + } + + @Override + public synchronized void stop() { + if (!_isStarted) { + LOG.error("Tried to stop Riemann when not started!"); + return; + } + LOG.info("Stopping Riemann server"); + RT.var("riemann.config", "stop!").invoke(); + _isStarted = false; + } + + @Override + public boolean isStarted() { + return _isStarted; + } +} diff --git a/helix-monitor-server/src/main/resources/riemann.config b/helix-monitor-server/src/main/resources/riemann.config new file mode 100644 index 0000000000..bdb785ef54 --- /dev/null +++ b/helix-monitor-server/src/main/resources/riemann.config @@ -0,0 +1,76 @@ +; -*- mode: clojure; -*- +; vim: filetype=clojure + +; 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. + +(logging/init :file "/dev/null") + +(tcp-server :host "0.0.0.0") + +(instrumentation {:interval 1}) + +(udp-server :host "0.0.0.0") +(ws-server :host "0.0.0.0") +(repl-server :host "0.0.0.0") +(use 'clj-helix.admin) + +(def admin (helix-admin "localhost:2185")) + +(add-cluster admin :my-app) + +(def fsm-def + (clj-helix.fsm/fsm-definition + {:name :OnlineOffline + :states + {:DROPPED {:transitions :OFFLINE} + :ONLINE {:priority 1 :upper-bound :R :transitions :OFFLINE} + :OFFLINE {:initial? true :transitions [:ONLINE :DROPPED]}}})) + +(add-fsm-definition admin :my-app fsm-def) + +(add-resource admin :my-app {:resource :some-resouce + :partitions 128 + :replicas 5 + :state-model (:name fsm-def)}) + +(add-instance admin :my-app {:host "localhost" + :port 5555}) +(use 'clj-helix.manager) + +(def f (clj-helix.fsm/fsm fsm-def + (:OFFLINE :ONLINE [part message context] + (prn part :coming-online)) + + (:OFFLINE :DROPPED [part m c] + (prn part "dropped!")) + + (:ONLINE :OFFLINE [part m c] + (prn part "Going offline.")))) + +(def p (participant {:zookeeper "localhost:2185" + :cluster :my-app + :instance {:host "localhost" :port 5555} + :fsm f})) + +(periodically-expire 1) + +(let [index (default :ttl 3 (update-index (index)))] + (streams + (expired prn) + index)) + diff --git a/helix-monitor-server/src/test/conf/testng.xml b/helix-monitor-server/src/test/conf/testng.xml new file mode 100644 index 0000000000..90910aac8d --- /dev/null +++ b/helix-monitor-server/src/test/conf/testng.xml @@ -0,0 +1,27 @@ + + + + + + + + + + diff --git a/helix-monitor-server/src/test/java/org/apache/helix/monitoring/MonitoringTestHelper.java b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/MonitoringTestHelper.java new file mode 100644 index 0000000000..cfaa7874a6 --- /dev/null +++ b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/MonitoringTestHelper.java @@ -0,0 +1,136 @@ +package org.apache.helix.monitoring; + +/* + * 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.net.ServerSocket; + +import org.I0Itec.zkclient.NetworkUtil; +import org.apache.helix.model.MonitoringConfig; +import org.apache.helix.monitoring.riemann.RiemannConfigs; +import org.apache.helix.monitoring.riemann.RiemannMonitoringServer; + +public class MonitoringTestHelper { + static final int MAX_PORT = 65535; + + /** + * generate a default riemann.config + * @param riemannPort + * @return + */ + public static String getRiemannConfigString(int riemannPort) { + StringBuilder sb = new StringBuilder(); + sb.append("(logging/init :file \"/dev/null\")\n\n") + .append("(tcp-server :host \"0.0.0.0\" :port " + riemannPort + ")\n\n") + .append("(instrumentation {:interval 1})\n\n") + .append("; (udp-server :host \"0.0.0.0\" :port " + riemannPort + ")\n") + .append("; (ws-server :host \"0.0.0.0\")\n") + .append("; (repl-server :host \"0.0.0.0\")\n\n") + .append("(periodically-expire 1)\n\n") + .append( + "(let [index (default :ttl 3 (update-index (index)))]\n (streams\n (expired prn)\n index))\n"); + + return sb.toString(); + } + + /** + * generate a test config for checking latency + * @param proxyPort + * @return + */ + public static String getLatencyCheckConfigString(String zkAddr) { + StringBuilder sb = new StringBuilder(); + sb.append("(require 'riemann.config)\n") + .append( + "(def alert-proxy (new org.apache.helix.monitoring.riemann.HelixAlertMessenger \"" + + zkAddr + "\"))\n\n") + .append("(defn parse-double\n \"Convert a string into a double\"\n ") + .append("[instr]\n (Double/parseDouble instr))\n\n") + .append( + "(defn check-95th-latency\n \"Check if the 95th percentile latency is within expectations\"\n ") + .append("[e]\n (let [latency (parse-double (:latency95 e))]\n ") + .append( + "(if (> latency 1.0) \n ; Report if the 95th percentile latency exceeds 1.0s\n ") + .append( + "(do (prn (:host e) \"has an unacceptable 95th percentile latency of\" latency)\n ") + .append( + "(let [alert-name-str (str \"(\" (:cluster e) \".%.\" (:host e) \")(latency95)>(1000)\" )]\n ") + .append("(.onAlert alert-proxy alert-name-str))))))\n\n") + .append("(streams\n (where\n ; Only process services containing LatencyReport\n ") + .append("(and (service #\".*LatencyReport.*\") (not (state \"expired\")))\n ") + .append("check-95th-latency))\n"); + + return sb.toString(); + } + + /** + * find an available tcp port + * @return + */ + public static int availableTcpPort() { + ServerSocket ss = null; + try { + ss = new ServerSocket(0); + ss.setReuseAddress(true); + return ss.getLocalPort(); + } catch (IOException e) { + // ok + } finally { + if (ss != null) { + try { + ss.close(); + } catch (IOException e) { + // should not be thrown + } + } + } + return -1; + } + + /** + * find the first available port starting from startPort inclusive + * @param startPort + * @return + */ + public static int availableTcpPort(int startPort) { + int port = startPort; + for (; port <= MAX_PORT; port++) { + if (NetworkUtil.isPortFree(port)) + break; + } + + return port > MAX_PORT ? -1 : port; + } + + public static RiemannMonitoringServer startRiemannServer(int port) { + MonitoringConfig monitoringConfig = new MonitoringConfig(RiemannConfigs.DEFAULT_RIEMANN_CONFIG); + monitoringConfig.setConfig(MonitoringTestHelper.getRiemannConfigString(port)); + + String configDir = + String.format("%s/%s_%d", System.getProperty("java.io.tmpdir"), + RiemannConfigs.DEFAULT_CONFIG_DIR, port); + RiemannConfigs.Builder builder = + new RiemannConfigs.Builder(configDir).addConfig(monitoringConfig); + RiemannMonitoringServer server = new RiemannMonitoringServer(builder.build()); + server.start(); + + return server; + } +} diff --git a/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/IntegrationTest.java b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/IntegrationTest.java new file mode 100644 index 0000000000..f356d18d63 --- /dev/null +++ b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/IntegrationTest.java @@ -0,0 +1,199 @@ +package org.apache.helix.monitoring.riemann; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import java.util.Arrays; +import java.util.Date; +import java.util.List; + +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.ConfigAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.id.ClusterId; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.controller.alert.AlertAction; +import org.apache.helix.controller.alert.AlertName; +import org.apache.helix.integration.manager.ClusterControllerManager; +import org.apache.helix.manager.zk.DefaultAlertMsgHandlerFactory; +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.messaging.handling.MessageHandlerFactory; +import org.apache.helix.model.AlertConfig; +import org.apache.helix.model.HelixConfigScope; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.model.MonitoringConfig; +import org.apache.helix.model.builder.HelixConfigScopeBuilder; +import org.apache.helix.monitoring.MonitoringEvent; +import org.apache.helix.monitoring.MonitoringTestHelper; +import org.apache.helix.monitoring.riemann.RiemannAgent; +import org.apache.helix.monitoring.riemann.RiemannConfigs; +import org.apache.helix.monitoring.riemann.RiemannMonitoringServer; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.google.common.collect.Lists; + +public class IntegrationTest extends ZkUnitTestBase { + @Test + public void testBasic() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + // Set up monitoring cluster + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "MonitoringService", // resource name prefix + 1, // resources + 8, // partitions per resource + 0, // number of nodes + 1, // replicas + "OnlineOffline", // pick a built-in state model + RebalanceMode.FULL_AUTO, // let Helix handle rebalancing + true); // do rebalance + + // Enable auto-join + HelixConfigScope scope = + new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(clusterName).build(); + ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient); + configAccessor.set(scope, ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, "" + true); + + // Start controller + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller"); + controller.syncStart(); + + // Start helix proxy + final BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); + final HelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, baseAccessor); + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + + // Start monitoring server + int riemannPort = MonitoringTestHelper.availableTcpPort(); + MonitoringConfig riemannConfig = new MonitoringConfig(RiemannConfigs.DEFAULT_RIEMANN_CONFIG); + riemannConfig.setConfig(MonitoringTestHelper.getRiemannConfigString(riemannPort)); + + MonitoringConfig latencyCheckConfig = new MonitoringConfig("check_latency_config.clj"); + latencyCheckConfig.setConfig(MonitoringTestHelper.getLatencyCheckConfigString(ZK_ADDR)); + + // Set monitoring config on zk + accessor.setProperty(keyBuilder.monitoringConfig(riemannConfig.getId()), + riemannConfig); + accessor.setProperty(keyBuilder.monitoringConfig(latencyCheckConfig.getId()), + latencyCheckConfig); + + RiemannConfigs.Builder riemannConfigBuilder = + new RiemannConfigs.Builder().addConfigs(Lists.newArrayList(riemannConfig, + latencyCheckConfig)); + RiemannMonitoringServer server = new RiemannMonitoringServer(riemannConfigBuilder.build()); + server.start(); + + // Start Riemann agent + RiemannAgent agent = new RiemannAgent(ZK_ADDR, clusterName, riemannPort); + agent.start(); + + // Check live-instance + List liveInstances = accessor.getChildNames(keyBuilder.liveInstances()); + Assert.assertNotNull(liveInstances); + Assert.assertEquals(liveInstances.size(), 1); + + boolean result; + + // Setup mock storage cluster to be monitored + String storageClusterName = clusterName + "_storage"; + TestHelper.setupCluster(storageClusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "TestDB", // resource name prefix + 1, // resources + 8, // partitions per resource + 2, // number of nodes + 1, // replicas + "MasterSlave", // pick a built-in state model + RebalanceMode.FULL_AUTO, // let Helix handle rebalancing + true); // do rebalance + + // Add alert config + AlertConfig alertConfig = new AlertConfig("default"); + AlertName alertName = + new AlertName.Builder().cluster(ClusterId.from(storageClusterName)).metric("latency95") + .largerThan("1000").build(); + AlertAction alertAction = + new AlertAction.Builder().cmd("enableInstance").args("{cluster}", "{node}", "false") + .build(); + alertConfig.putConfig(alertName, alertAction); + final HelixDataAccessor storageAccessor = + new ZKHelixDataAccessor(storageClusterName, baseAccessor); + final PropertyKey.Builder storageKeyBuilder = storageAccessor.keyBuilder(); + storageAccessor.setProperty(storageKeyBuilder.alertConfig("default"), alertConfig); + + // Start another controller for mock storage cluster + ClusterControllerManager storageController = + new ClusterControllerManager(ZK_ADDR, storageClusterName, "controller"); + MessageHandlerFactory fty = new DefaultAlertMsgHandlerFactory(); + storageController.getMessagingService() + .registerMessageHandlerFactory(fty.getMessageType(), fty); + storageController.syncStart(); + + // Check localhost_12918 is enabled + InstanceConfig instanceConfig = + storageAccessor.getProperty(storageKeyBuilder.instanceConfig("localhost_12918")); + Assert.assertTrue(instanceConfig.getInstanceEnabled()); + + // Connect monitoring client + final RiemannClientWrapper rclient = + new RiemannClientWrapper(Arrays.asList("localhost:" + riemannPort)); + rclient.connect(); + + MonitoringEvent event = + new MonitoringEvent().participant(ParticipantId.from("localhost_12918")) + .name("LatencyReport").attribute("latency95", "" + 2) + .attribute("cluster", storageClusterName); + rclient.send(event); + + // Check localhost_12918 is disabled + result = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + InstanceConfig instanceConfig = + storageAccessor.getProperty(storageKeyBuilder.instanceConfig("localhost_12918")); + return instanceConfig.getInstanceEnabled() == false; + } + }, 10 * 1000); + Assert.assertTrue(result, "localhost_12918 should be disabled"); + + // Cleanup + rclient.disconnect(); + storageController.syncStop(); + controller.syncStop(); + + agent.shutdown(); + server.stop(); + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestClientServerMonitoring.java b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestClientServerMonitoring.java new file mode 100644 index 0000000000..0f1fb5fcdd --- /dev/null +++ b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestClientServerMonitoring.java @@ -0,0 +1,181 @@ +package org.apache.helix.monitoring.riemann; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.apache.helix.ConfigAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.api.id.ResourceId; +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.HelixConfigScope; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.builder.HelixConfigScopeBuilder; +import org.apache.helix.monitoring.MonitoringEvent; +import org.apache.helix.monitoring.MonitoringTestHelper; +import org.apache.helix.monitoring.riemann.RiemannAgent; +import org.apache.helix.monitoring.riemann.RiemannMonitoringServer; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.aphyr.riemann.Proto.Event; +import com.aphyr.riemann.client.RiemannClient; + +public class TestClientServerMonitoring extends ZkUnitTestBase { + @Test + public void test() throws Exception { + final int NUM_PARTICIPANTS = 0; + final int NUM_PARTITIONS = 8; + final int NUM_REPLICAS = 1; + + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + // Set up monitoring cluster + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "MonitoringService", // resource name prefix + 1, // resources + NUM_PARTITIONS, // partitions per resource + NUM_PARTICIPANTS, // number of nodes + NUM_REPLICAS, // replicas + "OnlineOffline", // pick a built-in state model + RebalanceMode.FULL_AUTO, // let Helix handle rebalancing + true); // do rebalance + + // Enable auto-join + HelixConfigScope scope = + new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(clusterName).build(); + ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient); + configAccessor.set(scope, ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, "" + true); + + // Start controller + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller"); + controller.syncStart(); + + // Start monitoring server + int port = MonitoringTestHelper.availableTcpPort(); + RiemannMonitoringServer server = MonitoringTestHelper.startRiemannServer(port); + + // Start Riemann agent + RiemannAgent agent = new RiemannAgent(ZK_ADDR, clusterName, port); + agent.start(); + + // Check live-instance + final HelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + List liveInstances = accessor.getChildNames(keyBuilder.liveInstances()); + Assert.assertNotNull(liveInstances); + Assert.assertEquals(liveInstances.size(), 1); + + // Connect monitoring client + final RiemannClientWrapper client = + new RiemannClientWrapper(Arrays.asList("localhost:" + port)); + client.connect(); + + final RiemannClient rclient = RiemannClient.tcp("localhost", port); + rclient.connect(); + + // Test MonitoringEvent#send() + MonitoringEvent event = new MonitoringEvent().tag("test").ttl(5); + boolean result = client.send(event); + Assert.assertTrue(result); + + // Check monitoring server has received the event with tag="test" + result = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + List events = rclient.query("tagged \"test\""); + System.out.println("events=" + events); + return (events != null) && (events.size() == 1) && (events.get(0).getTagsCount() == 1) + && (events.get(0).getTags(0).equals("test")); + } + }, 5 * 1000); + System.out.println("result=" + result); + Assert.assertTrue(result); + + // Test MonitoringEvent#sendAndFlush() + MonitoringEvent event2 = new MonitoringEvent().tag("test2").ttl(5); + client.sendAndFlush(event2); + + // Check monitoring server has received the event with tag="test2" + result = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + List events = rclient.query("tagged \"test2\""); + return (events != null) && (events.size() == 1) && (events.get(0).getTagsCount() == 1) + && (events.get(0).getTags(0).equals("test2")); + } + }, 5 * 1000); + Assert.assertTrue(result); + + // Test MonitoringEvent#every() + client.every(1, 0, TimeUnit.SECONDS, new Runnable() { + + @Override + public void run() { + MonitoringEvent event3 = + new MonitoringEvent().tag("test3").resource(ResourceId.from("db" + System.currentTimeMillis())).ttl(5); + client.send(event3); + } + }); + + // Check monitoring server has received at least 2 event2 with tag="test3" + result = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + List events = rclient.query("tagged \"test3\""); + return (events.size() > 2) && (events.get(0).getTagsCount() == 1) + && (events.get(0).getTags(0).equals("test3")); + } + }, 10 * 1000); + Assert.assertTrue(result); + + // Stop client + client.disconnect(); + rclient.disconnect(); + + // Stop controller + controller.syncStop(); + + server.stop(); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } + +} diff --git a/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestHelixAlertMessenger.java b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestHelixAlertMessenger.java new file mode 100644 index 0000000000..587e6f8e31 --- /dev/null +++ b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestHelixAlertMessenger.java @@ -0,0 +1,110 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.List; + +import org.apache.helix.BaseDataAccessor; +import org.apache.helix.HelixAdmin; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +import org.apache.helix.manager.zk.ZKHelixAdmin; +import org.apache.helix.manager.zk.ZKHelixDataAccessor; +import org.apache.helix.manager.zk.ZkBaseDataAccessor; +import org.apache.helix.model.Message; +import org.apache.helix.model.Message.Attributes; +import org.apache.helix.model.Message.MessageType; +import org.apache.helix.monitoring.riemann.HelixAlertMessenger; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestHelixAlertMessenger extends ZkUnitTestBase { + + @Test + public void testBasic() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + HelixAdmin admin = new ZKHelixAdmin(_gZkClient); + admin.addCluster(clusterName); + + HelixAlertMessenger messenger = new HelixAlertMessenger(ZK_ADDR); + + // Send a valid alert + String alertNameStr = String.format("(%s.%%.node1)(latency95)>(1000)", clusterName); + messenger.onAlert(alertNameStr); + + // Send an invalid alert + String inValidAlertNameStr = "IGNORABLE: invalid alert"; + messenger.onAlert(inValidAlertNameStr); + + // Check only 1 alert controller message is sent + BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); + HelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, baseAccessor); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + List messages = accessor.getChildValues(keyBuilder.controllerMessages()); + + Assert.assertEquals(messages.size(), 1); + Message message = messages.get(0); + Assert.assertEquals(message.getMsgType(), MessageType.ALERT.toString()); + Assert.assertEquals(message.getAttribute(Attributes.ALERT_NAME), alertNameStr); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } + + @Test + public void testThrottle() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + HelixAdmin admin = new ZKHelixAdmin(_gZkClient); + admin.addCluster(clusterName); + + HelixAlertMessenger messenger = new HelixAlertMessenger(ZK_ADDR); + + long startT = System.currentTimeMillis(); + String alertNameStr = String.format("(%s.%%.node1)(latency95)>(1000)", clusterName); + for (int i = 0; i < 10; i++) { + messenger.onAlert(alertNameStr); + } + long seconds = (System.currentTimeMillis() - startT) / 1000 + 1; + + // Check no more than 1 alert per second + BaseDataAccessor baseAccessor = new ZkBaseDataAccessor(_gZkClient); + HelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, baseAccessor); + PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + List messages = accessor.getChildValues(keyBuilder.controllerMessages()); + + Assert.assertTrue(messages.size() <= seconds, "Should not receive more than " + seconds + + " messages"); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestRiemannAgent.java b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestRiemannAgent.java new file mode 100644 index 0000000000..11e411c1f8 --- /dev/null +++ b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestRiemannAgent.java @@ -0,0 +1,117 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.List; + +import org.apache.helix.ConfigAccessor; +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.PropertyKey; +import org.apache.helix.TestHelper; +import org.apache.helix.ZNRecord; +import org.apache.helix.ZkUnitTestBase; +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.HelixConfigScope; +import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty; +import org.apache.helix.model.IdealState.RebalanceMode; +import org.apache.helix.model.MonitoringConfig; +import org.apache.helix.model.builder.HelixConfigScopeBuilder; +import org.apache.helix.monitoring.MonitoringTestHelper; +import org.apache.helix.monitoring.riemann.RiemannAgent; +import org.apache.helix.monitoring.riemann.RiemannMonitoringServer; +import org.apache.helix.monitoring.riemann.RiemannConfigs.Builder; +import org.apache.helix.tools.ClusterStateVerifier; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestRiemannAgent extends ZkUnitTestBase { + @Test + public void testStartAndStop() throws Exception { + final int NUM_PARTICIPANTS = 0; + final int NUM_PARTITIONS = 4; + final int NUM_REPLICAS = 1; + + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String clusterName = className + "_" + methodName; + System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis())); + + // Set up monitoring cluster + TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port + "localhost", // participant name prefix + "MonitoringService", // resource name prefix + 1, // resources + NUM_PARTITIONS, // partitions per resource + NUM_PARTICIPANTS, // number of nodes + NUM_REPLICAS, // replicas + "OnlineOffline", // pick a built-in state model + RebalanceMode.FULL_AUTO, // let Helix handle rebalancing + true); // do rebalance + + // Enable auto-join + HelixConfigScope scope = + new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(clusterName).build(); + ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient); + configAccessor.set(scope, ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, "" + true); + + // start controller + ClusterControllerManager controller = + new ClusterControllerManager(ZK_ADDR, clusterName, "controller"); + controller.syncStart(); + + // Start monitoring server + int port = MonitoringTestHelper.availableTcpPort(); + RiemannMonitoringServer server = MonitoringTestHelper.startRiemannServer(port); + + // Start Riemann agent + RiemannAgent agent = new RiemannAgent(ZK_ADDR, clusterName, port); + agent.start(); + + // Check live-instance + final HelixDataAccessor accessor = + new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient)); + final PropertyKey.Builder keyBuilder = accessor.keyBuilder(); + List liveInstances = accessor.getChildNames(keyBuilder.liveInstances()); + Assert.assertNotNull(liveInstances); + Assert.assertEquals(liveInstances.size(), 1); + + // Stop monitoring server + server.stop(); + + boolean result = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + List liveInstances = accessor.getChildNames(keyBuilder.liveInstances()); + return liveInstances != null && liveInstances.size() == 0; + } + }, 15 * 1000); + Assert.assertTrue(result, "RiemannAgent should be disconnected if RiemannServer is stopped"); + + agent.shutdown(); + controller.syncStop(); + + System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestRiemannClientWrapper.java b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestRiemannClientWrapper.java new file mode 100644 index 0000000000..24af69e220 --- /dev/null +++ b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestRiemannClientWrapper.java @@ -0,0 +1,134 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.List; + +import org.apache.helix.TestHelper; +import org.apache.helix.model.MonitoringConfig; +import org.apache.helix.monitoring.MonitoringEvent; +import org.apache.helix.monitoring.MonitoringTestHelper; +import org.apache.helix.monitoring.riemann.RawRiemannClient.State; +import org.apache.helix.monitoring.riemann.RiemannConfigs; +import org.apache.helix.monitoring.riemann.RiemannMonitoringServer; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.aphyr.riemann.Proto.Event; +import com.aphyr.riemann.client.RiemannClient; + +public class TestRiemannClientWrapper { + + @Test + public void testBasic() throws Exception { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String testName = className + "_" + methodName; + + System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis())); + + int port = MonitoringTestHelper.availableTcpPort(); + + MonitoringConfig monitoringConfig = new MonitoringConfig(RiemannConfigs.DEFAULT_RIEMANN_CONFIG); + monitoringConfig.setConfig(MonitoringTestHelper.getRiemannConfigString(port)); + + RiemannConfigs.Builder builder = new RiemannConfigs.Builder().addConfig(monitoringConfig); + RiemannConfigs config = builder.build(); + System.out.println("configDir: " + config.getConfigDir()); + RiemannMonitoringServer server = new RiemannMonitoringServer(config); + server.start(); + Assert.assertTrue(server.isStarted()); + + // create client + final RawRiemannClient rawRclient = new RawRiemannClient("localhost", port); + rawRclient.connect(); + + final RiemannClient rclient = RiemannClient.tcp("localhost", port); + rclient.connect(); + + MonitoringEvent event = new MonitoringEvent().tag("test").ttl(3); + boolean ret; + ret = rawRclient.send(event); + Assert.assertTrue(ret); + + // wait until we can query it from riemann server + ret = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + List events = rclient.query("tagged \"test\""); + + return events != null && events.size() == 1 && events.get(0).getTagsCount() == 1 + && events.get(0).getTags(0).equals("test"); + } + }, 5 * 1000); + Assert.assertTrue(ret); + + server.stop(); + + // wait until heartbeat detects server is down + ret = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + return rawRclient.getState() == State.RECONNECTING; + } + }, 20 * 1000); + Assert.assertTrue(ret); + + ret = rawRclient.send(event); + Assert.assertFalse(ret); + + server.start(); + // wait until heartbeat detects server is up again + ret = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + return rawRclient.getState() == State.CONNECTED; + } + }, 20 * 1000); + Assert.assertTrue(ret); + + MonitoringEvent event2 = new MonitoringEvent().tag("test2").ttl(3); + ret = rawRclient.send(event2); + Assert.assertTrue(ret); + + // wait until we can query it from riemann server + ret = TestHelper.verify(new TestHelper.Verifier() { + + @Override + public boolean verify() throws Exception { + List events = rclient.query("tagged \"test2\""); + + return events != null && events.size() == 1 && events.get(0).getTagsCount() == 1 + && events.get(0).getTags(0).equals("test2"); + } + }, 5 * 1000); + Assert.assertTrue(ret); + + // clean up + rawRclient.disconnect(); + server.stop(); + + System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestRiemannMonitoringServer.java b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestRiemannMonitoringServer.java new file mode 100644 index 0000000000..37234e0ab3 --- /dev/null +++ b/helix-monitor-server/src/test/java/org/apache/helix/monitoring/riemann/TestRiemannMonitoringServer.java @@ -0,0 +1,82 @@ +package org.apache.helix.monitoring.riemann; + +/* + * 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.Date; + +import org.apache.helix.TestHelper; +import org.apache.helix.model.MonitoringConfig; +import org.apache.helix.monitoring.MonitoringTestHelper; +import org.apache.helix.monitoring.riemann.RiemannConfigs; +import org.apache.helix.monitoring.riemann.RiemannMonitoringServer; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.aphyr.riemann.client.RiemannClient; + +public class TestRiemannMonitoringServer { + + @Test + public void testBasic() throws IOException { + String className = TestHelper.getTestClassName(); + String methodName = TestHelper.getTestMethodName(); + String testName = className + "_" + methodName; + + System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis())); + + int port = MonitoringTestHelper.availableTcpPort(); + MonitoringConfig monitoringConfig = new MonitoringConfig(RiemannConfigs.DEFAULT_RIEMANN_CONFIG); + monitoringConfig.setConfig(MonitoringTestHelper.getRiemannConfigString(port)); + + RiemannConfigs.Builder builder = new RiemannConfigs.Builder().addConfig(monitoringConfig); + RiemannMonitoringServer server = new RiemannMonitoringServer(builder.build()); + + // Check server starts + server.start(); + Assert.assertTrue(server.isStarted()); + + RiemannClient rclient = null; + try { + rclient = RiemannClient.tcp("localhost", port); + rclient.connect(); + rclient.event().sendWithAck(); + } catch (IOException e) { + Assert.fail("Riemann server should start on port: " + port); + } + + // Check server stops + Assert.assertNotNull(rclient); + rclient.disconnect(); + server.stop(); + Assert.assertFalse(server.isStarted()); + + try { + rclient = RiemannClient.tcp("localhost", port); + rclient.connect(); + rclient.event().sendWithAck(); + Assert.fail("Riemann server should be stopped on port: " + port); + } catch (IOException e) { + // ok + } + + System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis())); + } +} diff --git a/hpost-review.sh b/hpost-review.sh index 0f1f5a1fe6..7ccc41c239 100755 --- a/hpost-review.sh +++ b/hpost-review.sh @@ -35,7 +35,19 @@ REVLIST=$1; JIRA=$2 shift 2; +# Run the rat plugin +echo 'Checking source for license headers' +mvn -Prat -DskipTests > /dev/null +RAT_STATUS=$? + +if [[ $RAT_STATUS -ne 0 ]] ; then + echo "Maven rat plugin failed. Add license headers and try again." + exit 1; +fi; +echo 'Checking source for license headers: PASSED' + # Check if the commit is prefixed with [HELIX-NNN] +echo 'Checking commit message format' BUG_NAME=HELIX-$JIRA COMMIT_PREFIX=\[$BUG_NAME\] DESCRIPTION=$(git log --pretty=format:%s $REVLIST) @@ -44,8 +56,10 @@ if [[ "$DESCRIPTION" != "$COMMIT_PREFIX"* ]] ; then echo "Commit message must start with $COMMIT_PREFIX" usage fi; +echo 'Checking commit message format: PASSED' # Check if HELIX-NNN is a valid bug +echo 'Checking JIRA existence' JIRA_URL=https://issues.apache.org/jira/rest/api/latest/issue/$BUG_NAME JIRA_STATUS=$(curl -o /dev/null --silent --head --write-out '%{http_code}\n' $JIRA_URL) @@ -53,6 +67,7 @@ if [[ $JIRA_STATUS -eq 404 ]]; then echo "$BUG_NAME does not exist in JIRA" usage fi; +echo 'Checking JIRA existence: PASSED' post-review --server="https://reviews.apache.org" --target-groups=helix --summary="$(git log --pretty=format:%s $REVLIST)" --description="$(git whatchanged $REVLIST)" --diff-filename=<(git diff --no-prefix $REVLIST) -o --bugs-closed=$BUG_NAME $* diff --git a/pom.xml b/pom.xml index 76474fd9b1..e7309a0551 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ under the License. org.apache.helix helix - 0.6.2-incubating-SNAPSHOT + 0.7.1-incubating-SNAPSHOT pom Apache Helix @@ -44,7 +44,7 @@ under the License. Olivier Lamy olamy@apache.org - PMC Member + Mentor Australia/Melbourne @@ -53,16 +53,16 @@ under the License. Patrick Hunt phunt@apache.org - PMC Member + Mentor -8 - Mahadev + mahadev Mahadev Konar mahadev@apache.org - PMC Member + Mentor -8 @@ -71,7 +71,7 @@ under the License. Owen O'Malley omalley@apache.org - PMC Member + Mentor -8 @@ -102,6 +102,15 @@ under the License. -8 + + TBA + Adam Silberstein + TBA@apache.org + + Committer + + -8 + ksurlaker Kapil Surlaker @@ -110,13 +119,89 @@ under the License. Committer -8 - + + + rms + Bob Schulman + rms@apache.org + + Committer + + -8 + + + swaroop-aj + Swaroop Jagadish + swaroop-aj@apache.org + + Committer + + -8 + + + rahula + Rahul Aggarwal + rahula@apache.org + + Committer + + -8 + + + chtyim + Terence Yim + chtyim@apache.org + + Committer + + -8 + + + santip + Santiago Perez + santip@apache.org + + Committer + + -8 + + + vinayakb + Vinayak Borkar + vinayakb@apache.org + + Committer + + -8 + + + sdas + Shirshanka Das + sdas@apache.org + + Committer + + -8 + + + kanak + Kanak Biscuitwala + kanak@apache.org + + Committer + + -8 + helix-core helix-admin-webapp helix-agent + helix-examples + helix-monitor-server + helix-monitor-client recipes + site-releases @@ -163,9 +248,23 @@ under the License. false + + Sonatype-public + SnakeYAML repository + http://oss.sonatype.org/content/groups/public/ + + + jboss-fs-public + JBoss FuseSource repository + http://repository.jboss.org/nexus/content/groups/fs-public/ + + + clojars.org + http://clojars.org/repo + - + apache.snapshots @@ -218,21 +317,21 @@ under the License. -Papache-release - 0.6.1-incubating + 0.7.0-incubating - - - - - - - - - + + + + + + + + + - + +Distributed lock manager with a user-defined rebalancer and YAML configuration +------------------------------------------------------------------------------ +This recipe is a second take on the distributed lock manager example with two key differences + * Instead of specifying the cluster using the HelixAdmin Java API, a YAML file indicates the cluster, its resources, and its participants. This is a simplified way to bootstrap cluster creation with a compact, logical hierarchy. + * The rebalancing process (i.e. the algorithm that uses the cluster state to determine an assignment of locks to participants) is specified in a class defined by the recipe itself, completely independent of Helix. + +For additional background and motivation, see the distributed-lock-manager recipe. + +### YAML Cluster Setup +The YAML configuration below specifies a state model for a lock in which it can be locked and unlocked. At most one participant can hold the lock at any time, and there are 12 locks to distribute across 4 participants. + +``` +clusterName: lock-manager-custom-rebalancer # unique name for the cluster +resources: + - name: lock-group # unique resource name + rebalancer: # we will provide our own rebalancer + mode: USER_DEFINED + class: org.apache.helix.userdefinedrebalancer.LockManagerRebalancer + partitions: + count: 12 # number of locks + replicas: 1 # number of simultaneous holders for each lock + stateModel: + name: lock-unlock # unique model name + states: [LOCKED, RELEASED, DROPPED] # the list of possible states + transitions: # the list of possible transitions + - name: Unlock + from: LOCKED + to: RELEASED + - name: Lock + from: RELEASED + to: LOCKED + - name: DropLock + from: LOCKED + to: DROPPED + - name: DropUnlock + from: RELEASED + to: DROPPED + - name: Undrop + from: DROPPED + to: RELEASED + initialState: RELEASED + constraints: + state: + counts: # maximum number of replicas of a partition that can be in each state + - name: LOCKED + count: "1" + - name: RELEASED + count: "-1" + - name: DROPPED + count: "-1" + priorityList: [LOCKED, RELEASED, DROPPED] # states in order of priority + transition: # transitions priority to enforce order that transitions occur + priorityList: [Unlock, Lock, Undrop, DropUnlock, DropLock] +participants: # list of nodes that can acquire locks + - name: localhost_12001 + host: localhost + port: 12001 + - name: localhost_12002 + host: localhost + port: 12002 + - name: localhost_12003 + host: localhost + port: 12003 +``` + +### User-Defined Rebalancer +The implementation of the Rebalancer interface is quite simple. It assumes a Lock/Unlock model where the lock state has highest priority. It uses a mod-based approach to fairly assign locks to participants so that no participant holds more than one instance of a lock, and each lock is only assigned to as many participants as can hold the same lock simultaneously. In the configuration above, only one participant can hold a given lock in the locked state. + +The result is a ResourceMapping, which maps each lock to its holder and its lock state. In Helix terminology, the lock manager is the resource, a lock is a partition, its holder is a participant, and the lock state is the current state of the lock based on one of the pre-defined states in the state model. + +``` +@Override +public ResourceAssignment computeResourceMapping(Resource resource, IdealState currentIdealState, + CurrentStateOutput currentStateOutput, ClusterDataCache clusterData) { + // Initialize an empty mapping of locks to participants + ResourceAssignment assignment = new ResourceAssignment(resource.getResourceName()); + + // Get the list of live participants in the cluster + List liveParticipants = new ArrayList(clusterData.getLiveInstances().keySet()); + + // Get the state model (should be a simple lock/unlock model) and the highest-priority state + String stateModelName = currentIdealState.getStateModelDefRef(); + StateModelDefinition stateModelDef = clusterData.getStateModelDef(stateModelName); + if (stateModelDef.getStatesPriorityList().size() < 1) { + LOG.error("Invalid state model definition. There should be at least one state."); + return assignment; + } + String lockState = stateModelDef.getStatesPriorityList().get(0); + + // Count the number of participants allowed to lock each lock + String stateCount = stateModelDef.getNumInstancesPerState(lockState); + int lockHolders = 0; + try { + // a numeric value is a custom-specified number of participants allowed to lock the lock + lockHolders = Integer.parseInt(stateCount); + } catch (NumberFormatException e) { + LOG.error("Invalid state model definition. The lock state does not have a valid count"); + return assignment; + } + + // Fairly assign the lock state to the participants using a simple mod-based sequential + // assignment. For instance, if each lock can be held by 3 participants, lock 0 would be held + // by participants (0, 1, 2), lock 1 would be held by (1, 2, 3), and so on, wrapping around the + // number of participants as necessary. + // This assumes a simple lock-unlock model where the only state of interest is which nodes have + // acquired each lock. + int i = 0; + for (Partition partition : resource.getPartitions()) { + Map replicaMap = new HashMap(); + for (int j = i; j < i + lockHolders; j++) { + int participantIndex = j % liveParticipants.size(); + String participant = liveParticipants.get(participantIndex); + // enforce that a participant can only have one instance of a given lock + if (!replicaMap.containsKey(participant)) { + replicaMap.put(participant, lockState); + } + } + assignment.addReplicaMap(partition, replicaMap); + i++; + } + return assignment; +} +``` +---------------------------------------------------------------------------------------- + +#### In Action + +##### Specifying a Lock StateModel +In our configuration file, we indicated a special state model with two key states: LOCKED and RELEASED. Thus, we need to provide for the participant a subclass of StateModel that can respond to transitions between those states. + +``` +public class Lock extends StateModel { + private String lockName; + + public Lock(String lockName) { + this.lockName = lockName; + } + + @Transition(from = "RELEASED", to = "LOCKED") + public void lock(Message m, NotificationContext context) { + System.out.println(context.getManager().getInstanceName() + " acquired lock:" + lockName); + } + + @Transition(from = "LOCKED", to = "RELEASED") + public void release(Message m, NotificationContext context) { + System.out.println(context.getManager().getInstanceName() + " releasing lock:" + lockName); + } +} +``` + +##### Loading the configuration file +We include a YAML file parser that will set up the cluster according to the specifications of the file. Here is the code that this example uses to set up the cluster: + +``` +YAMLClusterSetup setup = new YAMLClusterSetup(zkAddress); +InputStream input = + Thread.currentThread().getContextClassLoader() + .getResourceAsStream("lock-manager-config.yaml"); +YAMLClusterSetup.YAMLClusterConfig config = setup.setupCluster(input); +``` +At this point, the cluster is set up and the configuration is persisted on Zookeeper. The config variable contains a snapshot of this configuration for further access. + +##### Building +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/user-rebalanced-lock-manager/target/user-rebalanced-lock-manager-pkg/bin +chmod +x * +./lock-manager-demo +``` + +##### Output + +``` +./lock-manager-demo +STARTING localhost_12002 +STARTING localhost_12001 +STARTING localhost_12003 +STARTED localhost_12001 +STARTED localhost_12003 +STARTED localhost_12002 +localhost_12003 acquired lock:lock-group_4 +localhost_12002 acquired lock:lock-group_8 +localhost_12001 acquired lock:lock-group_10 +localhost_12001 acquired lock:lock-group_3 +localhost_12001 acquired lock:lock-group_6 +localhost_12003 acquired lock:lock-group_0 +localhost_12002 acquired lock:lock-group_5 +localhost_12001 acquired lock:lock-group_9 +localhost_12002 acquired lock:lock-group_2 +localhost_12003 acquired lock:lock-group_7 +localhost_12003 acquired lock:lock-group_11 +localhost_12002 acquired lock:lock-group_1 +lockName acquired By +====================================== +lock-group_0 localhost_12003 +lock-group_1 localhost_12002 +lock-group_10 localhost_12001 +lock-group_11 localhost_12003 +lock-group_2 localhost_12002 +lock-group_3 localhost_12001 +lock-group_4 localhost_12003 +lock-group_5 localhost_12002 +lock-group_6 localhost_12001 +lock-group_7 localhost_12003 +lock-group_8 localhost_12002 +lock-group_9 localhost_12001 +Stopping the first participant +localhost_12001 Interrupted +localhost_12002 acquired lock:lock-group_3 +localhost_12003 acquired lock:lock-group_6 +localhost_12003 acquired lock:lock-group_10 +localhost_12002 acquired lock:lock-group_9 +lockName acquired By +====================================== +lock-group_0 localhost_12003 +lock-group_1 localhost_12002 +lock-group_10 localhost_12003 +lock-group_11 localhost_12003 +lock-group_2 localhost_12002 +lock-group_3 localhost_12002 +lock-group_4 localhost_12003 +lock-group_5 localhost_12002 +lock-group_6 localhost_12003 +lock-group_7 localhost_12003 +lock-group_8 localhost_12002 +lock-group_9 localhost_12002 +``` + +---------------------------------------------------------------------------------------- + + + + + diff --git a/recipes/user-defined-rebalancer/pom.xml b/recipes/user-defined-rebalancer/pom.xml new file mode 100644 index 0000000000..8eba0353fd --- /dev/null +++ b/recipes/user-defined-rebalancer/pom.xml @@ -0,0 +1,139 @@ + + + + 4.0.0 + + + org.apache.helix.recipes + recipes + 0.7.1-incubating-SNAPSHOT + + + user-defined-rebalancer + bundle + Apache Helix :: Recipes :: user-defined-rebalancer + + + + org.apache.helix*, + org.apache.log4j, + * + + org.apache.helix.userdefinedrebalancer*;version="${project.version};-noimport:=true + + + + + org.testng + testng + 6.0.1 + + + org.apache.helix + helix-core + + + log4j + log4j + + + javax.mail + mail + + + javax.jms + jms + + + com.sun.jdmk + jmxtools + + + com.sun.jmx + jmxri + + + + + + + + ${basedir}/src/main/resources + true + + + + + + org.codehaus.mojo + appassembler-maven-plugin + + + + + + + true + ${project.build.directory}/${project.artifactId}-pkg + + -Xms512m -Xmx512m + + + windows + unix + + + + + package + + assemble + + + + + + org.apache.rat + apache-rat-plugin + + + + + + + + + + org.codehaus.mojo + appassembler-maven-plugin + + + + org.apache.helix.userdefinedrebalancer.LockManagerDemo + lock-manager-demo + + + + + + + diff --git a/recipes/user-defined-rebalancer/src/main/config/log4j.properties b/recipes/user-defined-rebalancer/src/main/config/log4j.properties new file mode 100644 index 0000000000..4b3dc31577 --- /dev/null +++ b/recipes/user-defined-rebalancer/src/main/config/log4j.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# Set root logger level to DEBUG and its only appender to A1. +log4j.rootLogger=ERROR,A1 + +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender + +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +log4j.logger.org.I0Itec=ERROR +log4j.logger.org.apache=ERROR diff --git a/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java new file mode 100644 index 0000000000..308ae14c72 --- /dev/null +++ b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java @@ -0,0 +1,53 @@ +package org.apache.helix.userdefinedrebalancer; + +/* + * 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; + +@StateModelInfo(initialState = "RELEASED", states = { + "RELEASED", "LOCKED" +}) +public class Lock extends StateModel { + private String lockName; + + public Lock(String lockName) { + this.lockName = lockName; + } + + @Transition(from = "RELEASED", to = "LOCKED") + public void lock(Message m, NotificationContext context) { + System.out.println(context.getManager().getInstanceName() + " acquired lock:" + lockName); + } + + @Transition(from = "LOCKED", to = "RELEASED") + public void release(Message m, NotificationContext context) { + System.out.println(context.getManager().getInstanceName() + " releasing lock:" + lockName); + } + + @Transition(from = "*", to = "DROPPED") + public void drop(Message m, NotificationContext context) { + System.out.println(context.getManager().getInstanceName() + " dropping lock:" + lockName); + } + +} diff --git a/helix-core/src/main/java/org/apache/helix/alerts/ExpandOperator.java b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockFactory.java similarity index 55% rename from helix-core/src/main/java/org/apache/helix/alerts/ExpandOperator.java rename to recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockFactory.java index ecd47917f3..c607b1b77d 100644 --- a/helix-core/src/main/java/org/apache/helix/alerts/ExpandOperator.java +++ b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockFactory.java @@ -1,4 +1,4 @@ -package org.apache.helix.alerts; +package org.apache.helix.userdefinedrebalancer; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -19,21 +19,18 @@ * under the License. */ -import java.util.Iterator; -import java.util.List; - -public class ExpandOperator extends Operator { - - public ExpandOperator() { - minInputTupleLists = 1; - maxInputTupleLists = Integer.MAX_VALUE; - inputOutputTupleListsCountsEqual = true; - } +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.participant.statemachine.HelixStateModelFactory; +/** + * This factory allows a participant to get the appropriate state model callbacks for the lock + * manager state model. This is used exactly once per participant to get a valid instance of a Lock, + * and then the same Lock instance is used for all state transition callbacks. + */ +public class LockFactory extends HelixStateModelFactory { @Override - public List>> execute(List>> input) { - // TODO: confirm this is a no-op operator - return input; + public Lock createNewStateModel(PartitionId partitionId) { + // TODO Auto-generated method stub + return new Lock(partitionId.stringify()); } - } diff --git a/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockManagerDemo.java b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockManagerDemo.java new file mode 100644 index 0000000000..727c5b7ce8 --- /dev/null +++ b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockManagerDemo.java @@ -0,0 +1,192 @@ +package org.apache.helix.userdefinedrebalancer; + +/* + * 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.InputStream; +import java.util.Map; +import java.util.TreeSet; + +import org.I0Itec.zkclient.IDefaultNameSpace; +import org.I0Itec.zkclient.ZkClient; +import org.I0Itec.zkclient.ZkServer; +import org.apache.commons.io.FileUtils; +import org.apache.helix.HelixAdmin; +import org.apache.helix.HelixManager; +import org.apache.helix.controller.HelixControllerMain; +import org.apache.helix.manager.zk.ZKHelixAdmin; +import org.apache.helix.model.ExternalView; +import org.apache.helix.tools.YAMLClusterSetup; +import org.apache.log4j.Logger; + +public class LockManagerDemo { + private static final Logger LOG = Logger.getLogger(LockManagerDemo.class); + + /** + * LockManagerDemo clusterName, numInstances, lockGroupName, numLocks + * @param args + * @throws Exception + */ + public static void main(String[] args) throws Exception { + final String zkAddress = "localhost:2199"; + + // default participant parameters in case the config does not specify them + int numInstances = 3; + boolean instancesSpecified = false; + Thread[] processArray = new Thread[numInstances]; + + // HelixManager for setting up the controller + HelixManager controllerManager = null; + + // Name of the lock group resource (specified by the config file) + String lockGroupName = null; + try { + startLocalZookeeper(2199); + YAMLClusterSetup setup = new YAMLClusterSetup(zkAddress); + InputStream input = + Thread.currentThread().getContextClassLoader() + .getResourceAsStream("lock-manager-config.yaml"); + final YAMLClusterSetup.YAMLClusterConfig config = setup.setupCluster(input); + if (config == null) { + LOG.error("Invalid YAML configuration"); + return; + } + if (config.resources == null || config.resources.isEmpty()) { + LOG.error("Need to specify a resource!"); + return; + } + + // save resource name + lockGroupName = config.resources.get(0).name; + + // save participants if specified + if (config.participants != null && config.participants.size() > 0) { + numInstances = config.participants.size(); + instancesSpecified = true; + processArray = new Thread[numInstances]; + } + + // run each participant + for (int i = 0; i < numInstances; i++) { + String participantName; + if (instancesSpecified) { + participantName = config.participants.get(i).name; + } else { + participantName = "localhost_" + (12000 + i); + } + final String instanceName = participantName; + processArray[i] = new Thread(new Runnable() { + + @Override + public void run() { + LockProcess lockProcess = null; + + try { + lockProcess = + new LockProcess(config.clusterName, zkAddress, instanceName, + config.resources.get(0).stateModel.name); + lockProcess.start(); + Thread.currentThread().join(); + } catch (InterruptedException e) { + System.out.println(instanceName + " Interrupted"); + if (lockProcess != null) { + lockProcess.stop(); + } + } catch (Exception e) { + e.printStackTrace(); + } + } + + }); + processArray[i].start(); + } + Thread.sleep(3000); + + // start the controller + controllerManager = + HelixControllerMain.startHelixController(zkAddress, config.clusterName, "controller", + HelixControllerMain.STANDALONE); + Thread.sleep(5000); + + // HelixAdmin for querying cluster state + HelixAdmin admin = new ZKHelixAdmin(zkAddress); + + printStatus(admin, config.clusterName, lockGroupName); + + // stop one participant + System.out.println("Stopping the first participant"); + processArray[0].interrupt(); + Thread.sleep(3000); + printStatus(admin, config.clusterName, lockGroupName); + Thread.currentThread().join(); + } catch (Exception e) { + e.printStackTrace(); + } finally { + if (controllerManager != null) { + controllerManager.disconnect(); + } + for (Thread process : processArray) { + if (process != null) { + process.interrupt(); + } + } + } + } + + private static void printStatus(HelixAdmin admin, String cluster, String resource) { + ExternalView externalView = admin.getResourceExternalView(cluster, resource); + TreeSet treeSet = new TreeSet(externalView.getPartitionSet()); + System.out.println("lockName" + "\t" + "acquired By"); + System.out.println("======================================"); + for (String lockName : treeSet) { + Map stateMap = externalView.getStateMap(lockName); + String acquiredBy = null; + if (stateMap != null) { + for (String instanceName : stateMap.keySet()) { + if ("LOCKED".equals(stateMap.get(instanceName))) { + acquiredBy = instanceName; + break; + } + } + } + System.out.println(lockName + "\t" + ((acquiredBy != null) ? acquiredBy : "NONE")); + } + } + + private static void startLocalZookeeper(int port) throws Exception { + ZkServer server = null; + String baseDir = "/tmp/IntegrationTest/"; + final String dataDir = baseDir + "zk/dataDir"; + final String logDir = baseDir + "/tmp/logDir"; + FileUtils.deleteDirectory(new File(dataDir)); + FileUtils.deleteDirectory(new File(logDir)); + + IDefaultNameSpace defaultNameSpace = new IDefaultNameSpace() { + @Override + public void createDefaultNameSpace(ZkClient zkClient) { + + } + }; + int zkPort = 2199; + server = new ZkServer(dataDir, logDir, defaultNameSpace, zkPort); + server.start(); + } + +} diff --git a/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockManagerRebalancer.java b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockManagerRebalancer.java new file mode 100644 index 0000000000..ff98cd2360 --- /dev/null +++ b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockManagerRebalancer.java @@ -0,0 +1,111 @@ +package org.apache.helix.userdefinedrebalancer; + +/* + * 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.HelixManager; +import org.apache.helix.api.Cluster; +import org.apache.helix.api.State; +import org.apache.helix.api.id.ParticipantId; +import org.apache.helix.api.id.PartitionId; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.controller.rebalancer.HelixRebalancer; +import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext; +import org.apache.helix.controller.rebalancer.context.RebalancerConfig; +import org.apache.helix.controller.stages.ResourceCurrentState; +import org.apache.helix.model.ResourceAssignment; +import org.apache.helix.model.StateModelDefinition; +import org.apache.log4j.Logger; + +public class LockManagerRebalancer implements HelixRebalancer { + private static final Logger LOG = Logger.getLogger(LockManagerRebalancer.class); + + @Override + public void init(HelixManager manager) { + // do nothing; this rebalancer is independent of the manager + } + + /** + * This rebalancer is invoked whenever there is a change in the cluster, including when new + * participants join or leave, or the configuration of any participant changes. It is written + * specifically to handle assignment of locks to nodes under the very simple lock-unlock state + * model. + */ + @Override + public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, + Cluster cluster, ResourceCurrentState currentState) { + // get a typed context + PartitionedRebalancerContext context = + rebalancerConfig.getRebalancerContext(PartitionedRebalancerContext.class); + + // Initialize an empty mapping of locks to participants + ResourceAssignment assignment = new ResourceAssignment(context.getResourceId()); + + // Get the list of live participants in the cluster + List liveParticipants = + new ArrayList(cluster.getLiveParticipantMap().keySet()); + + // Get the state model (should be a simple lock/unlock model) and the highest-priority state + StateModelDefId stateModelDefId = context.getStateModelDefId(); + StateModelDefinition stateModelDef = cluster.getStateModelMap().get(stateModelDefId); + if (stateModelDef.getStatesPriorityList().size() < 1) { + LOG.error("Invalid state model definition. There should be at least one state."); + return assignment; + } + State lockState = stateModelDef.getTypedStatesPriorityList().get(0); + + // Count the number of participants allowed to lock each lock + String stateCount = stateModelDef.getNumParticipantsPerState(lockState); + int lockHolders = 0; + try { + // a numeric value is a custom-specified number of participants allowed to lock the lock + lockHolders = Integer.parseInt(stateCount); + } catch (NumberFormatException e) { + LOG.error("Invalid state model definition. The lock state does not have a valid count"); + return assignment; + } + + // Fairly assign the lock state to the participants using a simple mod-based sequential + // assignment. For instance, if each lock can be held by 3 participants, lock 0 would be held + // by participants (0, 1, 2), lock 1 would be held by (1, 2, 3), and so on, wrapping around the + // number of participants as necessary. + // This assumes a simple lock-unlock model where the only state of interest is which nodes have + // acquired each lock. + int i = 0; + for (PartitionId partition : context.getPartitionSet()) { + Map replicaMap = new HashMap(); + for (int j = i; j < i + lockHolders; j++) { + int participantIndex = j % liveParticipants.size(); + ParticipantId participant = liveParticipants.get(participantIndex); + // enforce that a participant can only have one instance of a given lock + if (!replicaMap.containsKey(participant)) { + replicaMap.put(participant, lockState); + } + } + assignment.addReplicaMap(partition, replicaMap); + i++; + } + return assignment; + } +} diff --git a/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockProcess.java b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockProcess.java new file mode 100644 index 0000000000..723f9f24e1 --- /dev/null +++ b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/LockProcess.java @@ -0,0 +1,80 @@ +package org.apache.helix.userdefinedrebalancer; + +/* + * 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 org.apache.helix.HelixManager; +import org.apache.helix.HelixManagerFactory; +import org.apache.helix.InstanceType; +import org.apache.helix.api.id.StateModelDefId; +import org.apache.helix.manager.zk.ZKHelixAdmin; +import org.apache.helix.model.InstanceConfig; + +public class LockProcess { + private final String clusterName; + private final String zkAddress; + private final String instanceName; + private final String stateModelName; + private HelixManager participantManager; + + LockProcess(String clusterName, String zkAddress, String instanceName, String stateModelName) { + this.clusterName = clusterName; + this.zkAddress = zkAddress; + this.instanceName = instanceName; + this.stateModelName = stateModelName; + + } + + public void start() throws Exception { + System.out.println("STARTING " + instanceName); + configureInstance(instanceName); + participantManager = + HelixManagerFactory.getZKHelixManager(clusterName, instanceName, InstanceType.PARTICIPANT, + zkAddress); + participantManager.getStateMachineEngine().registerStateModelFactory( + StateModelDefId.from(stateModelName), new LockFactory()); + participantManager.connect(); + System.out.println("STARTED " + instanceName); + } + + /** + * Configure the instance, the configuration of each node is available to + * other nodes. + * @param instanceName + */ + private void configureInstance(String instanceName) { + ZKHelixAdmin helixAdmin = new ZKHelixAdmin(zkAddress); + + List instancesInCluster = helixAdmin.getInstancesInCluster(clusterName); + if (instancesInCluster == null || !instancesInCluster.contains(instanceName)) { + InstanceConfig config = new InstanceConfig(instanceName); + config.setHostName("localhost"); + config.setPort("12000"); + helixAdmin.addInstance(clusterName, config); + } + } + + public void stop() { + if (participantManager != null) { + participantManager.disconnect(); + } + } +} diff --git a/recipes/user-defined-rebalancer/src/main/resources/lock-manager-config.yaml b/recipes/user-defined-rebalancer/src/main/resources/lock-manager-config.yaml new file mode 100644 index 0000000000..b3128779c6 --- /dev/null +++ b/recipes/user-defined-rebalancer/src/main/resources/lock-manager-config.yaml @@ -0,0 +1,69 @@ +# +# 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. +# +clusterName: lock-manager-custom-rebalancer # unique name for the cluster +resources: + - name: lock-group # unique resource name + rebalancer: # we will provide our own rebalancer + mode: USER_DEFINED + class: org.apache.helix.userdefinedrebalancer.LockManagerRebalancer + partitions: + count: 12 # number of locks + replicas: 1 # number of simultaneous holders for each lock + stateModel: + name: lock-unlock # unique model name + states: [LOCKED, RELEASED, DROPPED] # the list of possible states + transitions: # the list of possible transitions + - name: Unlock + from: LOCKED + to: RELEASED + - name: Lock + from: RELEASED + to: LOCKED + - name: DropLock + from: LOCKED + to: DROPPED + - name: DropUnlock + from: RELEASED + to: DROPPED + - name: Undrop + from: DROPPED + to: RELEASED + initialState: RELEASED + constraints: + state: + counts: # maximum number of replicas of a partition that can be in each state + - name: LOCKED + count: "1" + - name: RELEASED + count: "-1" + - name: DROPPED + count: "-1" + priorityList: [LOCKED, RELEASED, DROPPED] # states in order of priority + transition: # transitions priority to enforce order that transitions occur + priorityList: [Unlock, Lock, Undrop, DropUnlock, DropLock] +participants: # list of nodes that can acquire locks + - name: localhost_12001 + host: localhost + port: 12001 + - name: localhost_12002 + host: localhost + port: 12002 + - name: localhost_12003 + host: localhost + port: 12003 \ No newline at end of file diff --git a/recipes/user-defined-rebalancer/src/test/conf/testng.xml b/recipes/user-defined-rebalancer/src/test/conf/testng.xml new file mode 100644 index 0000000000..58f0803678 --- /dev/null +++ b/recipes/user-defined-rebalancer/src/test/conf/testng.xml @@ -0,0 +1,27 @@ + + + + + + + + + + diff --git a/site-releases/0.6.1-incubating/pom.xml b/site-releases/0.6.1-incubating/pom.xml new file mode 100644 index 0000000000..7efc0192f3 --- /dev/null +++ b/site-releases/0.6.1-incubating/pom.xml @@ -0,0 +1,51 @@ + + + + 4.0.0 + + + org.apache.helix + site-releases + 0.7.1-incubating-SNAPSHOT + + + 0.6.1-incubating-site + bundle + Apache Helix :: Site :: 0.6.1-incubating + + + + + + + org.testng + testng + 6.0.1 + + + + + + + + + + + diff --git a/site-releases/0.6.1-incubating/src/site/apt/privacy-policy.apt b/site-releases/0.6.1-incubating/src/site/apt/privacy-policy.apt new file mode 100644 index 0000000000..ada93631d1 --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/apt/privacy-policy.apt @@ -0,0 +1,52 @@ + ---- + Privacy Policy + ----- + Olivier Lamy + ----- + 2013-02-04 + ----- + +~~ 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. + +Privacy Policy + + Information about your use of this website is collected using server access logs and a tracking cookie. The + collected information consists of the following: + + [[1]] The IP address from which you access the website; + + [[2]] The type of browser and operating system you use to access our site; + + [[3]] The date and time you access our site; + + [[4]] The pages you visit; and + + [[5]] The addresses of pages from where you followed a link to our site. + + [] + + Part of this information is gathered using a tracking cookie set by the + {{{http://www.google.com/analytics/}Google Analytics}} service and handled by Google as described in their + {{{http://www.google.com/privacy.html}privacy policy}}. See your browser documentation for instructions on how to + disable the cookie if you prefer not to share this data with Google. + + We use the gathered information to help us make our site more useful to visitors and to better understand how and + when our site is used. We do not track or collect personally identifiable information or associate gathered data + with any personally identifying information from other sources. + + By using this website, you consent to the collection of this data in the manner and for the purpose described above. diff --git a/site-releases/0.6.1-incubating/src/site/apt/releasenotes/release-0.6.1-incubating.apt b/site-releases/0.6.1-incubating/src/site/apt/releasenotes/release-0.6.1-incubating.apt new file mode 100644 index 0000000000..93052149a9 --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/apt/releasenotes/release-0.6.1-incubating.apt @@ -0,0 +1,110 @@ + ----- + Release Notes for 0.6.1-incubating Apache Helix + ----- + +~~ 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. + +~~ NOTE: For help with the syntax of this file, see: +~~ http://maven.apache.org/guides/mini/guide-apt-format.html + +Release Notes for 0.6.1-incubating Apache Helix + + The Apache Helix would like to announce the release of Apache Helix 0.6.1-incubating + + This is the second release in Apache umbrella. + + Helix is a generic cluster management framework used for the automatic management of partitioned, replicated and distributed resources hosted on a cluster of nodes. Helix provides the following features: + + * Automatic assignment of resource/partition to nodes + + * Node failure detection and recovery + + * Dynamic addition of Resources + + * Dynamic addition of nodes to the cluster + + * Pluggable distributed state machine to manage the state of a resource via state transitions + + * Automatic load balancing and throttling of transitions + + [] + +* Changes + +** Bug + + * [HELIX-25] - setConfig should check if instance exist or not when setting PARTICIPANT config + + * [HELIX-29] - Not receiving transitions after participant reconnection + + * [HELIX-30] - ZkHelixManager.carryOverPreviousCurrentState() should use a special merge logic + + * [HELIX-34] - Remove watches after the node /resource is deleted + + * [HELIX-35] - Support custom instance id format in CLI + + * [HELIX-41] - fix intermittent test failures + + * [HELIX-44] - ZkHelix property store is not checking the validity of path + + * [HELIX-55] - Session timeout setting not honoured + + * [HELIX-75] - ZKHelixManager declares throws Exception + + * [HELIX-76] - ChangeLogGenerator split bug + + +** Improvements + + * [HELIX-31] - Detect flapping and disable the participant/controller/spectator + + * [HELIX-32] - Flapping detection: if a helix manager starts connect/disconnect frequently it should be disconnected + + * [HELIX-64] - Allow application to provide additional metadata while connecting to cluster + + * [HELIX-73] - Remove assumption that Instance.id is always host_port + + * [HELIX-81] - org.apache.helix.manager.zk.ZKUtil#isClusterSetup() should not log error + + +** New Features + + * [HELIX-19] - Allow process to join the cluster dynamically + + * [HELIX-43] - Add support for error->dropped transition + + * [HELIX-45] - Standalone helix agent + + * [HELIX-63] - Make the idealstate computation code pluggable + + * [HELIX-72] - Allow pluggable rebalancer in controller + + +** Tasks + + * [HELIX-16] - Distributed task execution framework sample app + + + [] + + Cheers, + -- + The Apache Helix Team + + + diff --git a/site-releases/0.6.1-incubating/src/site/apt/releasing.apt b/site-releases/0.6.1-incubating/src/site/apt/releasing.apt new file mode 100644 index 0000000000..11d0cd92aa --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/apt/releasing.apt @@ -0,0 +1,107 @@ + ----- + Helix release process + ----- + ----- + 2012-12-15 + ----- + +~~ 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. + +~~ NOTE: For help with the syntax of this file, see: +~~ http://maven.apache.org/guides/mini/guide-apt-format.html + +Helix release process + + [[1]] Post to the dev list a few days before you plan to do an Helix release + + [[2]] Your maven setting must contains the entry to be able to deploy. + + ~/.m2/settings.xml + ++------------- + + apache.releases.https + + + ++------------- + + [[3]] Apache DAV passwords + ++------------- + Add the following info into your ~/.netrc + machine git-wip-us.apache.org login + ++------------- + [[4]] Release Helix + You should have a GPG agent running in the session you will run the maven release commands(preferred), and confirm it works by running "gpg -ab" (type some text and press Ctrl-D). + If you do not have a GPG agent running, make sure that you have the "apache-release" profile set in your settings.xml as shown below. + + Run the release + ++------------- +mvn release:prepare release:perform -B ++------------- + + GPG configuration in maven settings xml: + ++------------- + + apache-release + + [GPG_PASSWORD] + + ++------------- + + [[4]] go to https://repository.apache.org and close your staged repository. Note the repository url (format https://repository.apache.org/content/repositories/orgapachehelix-019/org/apache/helix/helix/0.6-incubating/) + ++------------- +svn co https://dist.apache.org/repos/dist/dev/incubator/helix helix-dev-release +cd helix-dev-release +sh ./release-script-svn.sh version stagingRepoUrl +then svn add +then svn ci ++------------- + + [[5]] Validating the release + ++------------- + * Download sources, extract, build and run tests - mvn clean package + * Verify license headers - mvn -Prat -DskipTests + * Download binaries and .asc files + * Download release manager's public key - From the KEYS file, get the release manager's public key finger print and run gpg --keyserver pgpkeys.mit.edu --recv-key + * Validate authenticity of key - run gpg --fingerprint + * Check signatures of all the binaries using gpg ++------------- + + [[6]] Call for a vote in the dev list and wait for 72 hrs. for the vote results. 3 binding votes are necessary for the release to be finalized. example + After the vote has passed, move the files from dist dev to dist release: svn mv https://dist.apache.org/repos/dist/dev/incubator/helix/version to https://dist.apache.org/repos/dist/release/incubator/helix/ + + [[7]] Prepare release note. Add a page in src/site/apt/releasenotes/ and change value of \ in parent pom. + + + [[8]] Send out an announcement of the release to: + + * users@helix.incubator.apache.org + + * dev@helix.incubator.apache.org + + [[9]] Celebrate ! + + diff --git a/site-releases/0.6.1-incubating/src/site/markdown/Architecture.md b/site-releases/0.6.1-incubating/src/site/markdown/Architecture.md new file mode 100644 index 0000000000..7acf590fbf --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/markdown/Architecture.md @@ -0,0 +1,248 @@ + + + +Helix aims to provide the following abilities to a distributed system: + +* Automatic management of a cluster hosting partitioned, replicated resources. +* Soft and hard failure detection and handling. +* Automatic load balancing via smart placement of resources on servers(nodes) based on server capacity and resource profile (size of partition, access patterns, etc). +* Centralized config management and self discovery. Eliminates the need to modify config on each node. +* Fault tolerance and optimized rebalancing during cluster expansion. +* Manages entire operational lifecycle of a node. Addition, start, stop, enable/disable without downtime. +* Monitor cluster health and provide alerts on SLA violation. +* Service discovery mechanism to route requests. + +To build such a system, we need a mechanism to co-ordinate between different nodes/components in the system. This mechanism can be achieved with a software that reacts to any change in the cluster and comes up with a set of tasks needed to bring the cluster to a stable state. The set of tasks will be assigned to one or more nodes in the cluster. Helix serves this purpose of managing the various components in the cluster. + +![Helix Design](images/system.png) + +Distributed System Components + +In general any distributed system cluster will have the following + +* Set of nodes also referred to as an instance. +* Set of resources which can be a database, lucene index or a task. +* Each resource is also partitioned into one or more Partitions. +* Each partition may have one or more copies called replicas. +* Each replica can have a state associated with it. For example Master, Slave, Leader, Standby, Online, Offline etc + +Roles +----- + +![Helix Design](images/HELIX-components.png) + +Not all nodes in a distributed system will perform similar functionality. For e.g, a few nodes might be serving requests, few nodes might be sending the request and some nodes might be controlling the nodes in the cluster. Based on functionality we have grouped them into + +We have divided Helix in 3 logical components based on their responsibility + +1. PARTICIPANT: The nodes that actually host the distributed resources. +2. SPECTATOR: The nodes that simply observe the PARTICIPANT State and route the request accordingly. Routers, for example, need to know the Instance on which a partition is hosted and its state in order to route the request to the appropriate end point. +3. CONTROLLER: The controller observes and controls the PARTICIPANT nodes. It is responsible for coordinating all transitions in the cluster and ensuring that state constraints are satisfied and cluster stability is maintained. + + +These are simply logical components and can be deployed as per the system requirements. For example: + +1. Controller can be deployed as a separate service +2. Controller can be deployed along with a Participant but only one Controller will be active at any given time. + +Both have pros and cons, which will be discussed later and one can chose the mode of deployment as per system needs. + + +## Cluster state/metadata store + +We need a distributed store to maintain the state of the cluster and a notification system to notify if there is any change in the cluster state. Helix uses Zookeeper to achieve this functionality. + +Zookeeper provides: + +* A way to represent PERSISTENT state which basically remains until its deleted. +* A way to represent TRANSIENT/EPHEMERAL state which vanishes when the process that created the STATE dies. +* Notification mechanism when there is a change in PERSISTENT/EPHEMERAL STATE + +The namespace provided by ZooKeeper is much like that of a standard file system. A name is a sequence of path elements separated by a slash (/). Every node[ZNODE] in ZooKeeper\'s namespace is identified by a path. + +More info on Zookeeper can be found here http://zookeeper.apache.org + +## Statemachine and constraints + +Even though the concept of Resource, Partition, Replicas is common to most distributed systems, one thing that differentiates one distributed system from another is the way each partition is assigned a state and the constraints on each state. + +For example: + +1. If a system is serving READ ONLY data then all partition\'s replicas are equal and they can either be ONLINE or OFFLINE. +2. If a system takes BOTH READ and WRITES but ensure that WRITES go through only one partition then the states will be MASTER, SLAVE and OFFLINE. Writes go through the MASTER and is replicated to the SLAVES. Optionally, READS can go through SLAVES. + +Apart from defining STATE for each partition, the transition path to each STATE can be application specific. For example, in order to become MASTER it might be a requirement to first become a SLAVE. This ensures that if the SLAVE does not have the data as part of OFFLINE-SLAVE transition it can bootstrap data from other nodes in the system. + +Helix provides a way to configure an application specific state machine along with constraints on each state. Along with constraints on STATE, Helix also provides a way to specify constraints on transitions. (More on this later.) + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +![Helix Design](images/statemachine.png) + +## Concepts + +The following terminologies are used in Helix to model a state machine. + +* IDEALSTATE: The state in which we need the cluster to be in if all nodes are up and running. In other words, all state constraints are satisfied. +* CURRENTSTATE: Represents the current state of each node in the cluster +* EXTERNALVIEW: Represents the combined view of CURRENTSTATE of all nodes. + +The goal of Helix is always to make the CURRENTSTATE of the system same as the IDEALSTATE. Some scenarios where this may not be true are: + +* When all nodes are down +* When one or more nodes fail +* New nodes are added and the partitions need to be reassigned + +### IDEALSTATE + +Helix lets the application define the IdealState on a resource basis which basically consists of: + +* List of partitions. Example: 64 +* Number of replicas for each partition. Example: 3 +* Node and State for each replica. + +Example: + +* Partition-1, replica-1, Master, Node-1 +* Partition-1, replica-2, Slave, Node-2 +* Partition-1, replica-3, Slave, Node-3 +* ..... +* ..... +* Partition-p, replica-3, Slave, Node-n + +Helix comes with various algorithms to automatically assign the partitions to nodes. The default algorithm minimizes the number of shuffles that happen when new nodes are added to the system + +### CURRENTSTATE + +Every instance in the cluster hosts one or more partitions of a resource. Each of the partitions has a State associated with it. + +Example Node-1 + +* Partition-1, Master +* Partition-2, Slave +* .... +* .... +* Partition-p, Slave + +### EXTERNALVIEW + +External clients needs to know the state of each partition in the cluster and the Node hosting that partition. Helix provides one view of the system to SPECTATORS as EXTERNAL VIEW. EXTERNAL VIEW is simply an aggregate of all CURRENTSTATE + +* Partition-1, replica-1, Master, Node-1 +* Partition-1, replica-2, Slave, Node-2 +* Partition-1, replica-3, Slave, Node-3 +* ..... +* ..... +* Partition-p, replica-3, Slave, Node-n + +## Process Workflow + +Mode of operation in a cluster + +A node process can be one of the following: + +* PARTICIPANT: The process registers itself in the cluster and acts on the messages received in its queue and updates the current state. Example: Storage Node +* SPECTATOR: The process is simply interested in the changes in the Externalview. The Router is a spectator of the Storage cluster. +* CONTROLLER: This process actively controls the cluster by reacting to changes in Cluster State and sending messages to PARTICIPANTS. + + +### Participant Node Process + +* When Node starts up, it registers itself under LIVEINSTANCES +* After registering, it waits for new Messages in the message queue +* When it receives a message, it will perform the required task as indicated in the message +* After the task is completed, depending on the task outcome it updates the CURRENTSTATE + +### Controller Process + +* Watches IDEALSTATE +* Node goes down/comes up or Node is added/removed. Watches LIVEINSTANCES and CURRENTSTATE of each Node in the cluster +* Triggers appropriate state transition by sending message to PARTICIPANT + +### Spectator Process + +* When the process starts, it asks cluster manager agent to be notified of changes in ExternalView +* Whenever it receives a notification, it reads the Externalview and performs required duties. For the Router, it updates its routing table. + +#### Interaction between controller, participant and spectator + +The following picture shows how controllers, participants and spectators interact with each other. + +![Helix Architecture](images/helix-architecture.png) + +## Core algorithm + +* Controller gets the IdealState and the CurrentState of active storage nodes from Zookeeper +* Compute the delta between IdealState and CurrentState for each partition across all participant nodes +* For each partition compute tasks based on the State Machine Table. It\'s possible to configure priority on the state Transition. For example, in case of Master-Slave: + * Attempt mastership transfer if possible without violating constraint. + * Partition Addition + * Drop Partition +* Add the tasks in parallel if possible to the respective queue for each storage node (if the tasks added are mutually independent) +* If a task is dependent on another task being completed, do not add that task +* After any task is completed by a Participant, Controllers gets notified of the change and the State Transition algorithm is re-run until the CurrentState is same as IdealState. + +## Helix znode layout + +Helix organizes znodes under clusterName in multiple levels. + +The top level (under clusterName) znodes are all Helix defined and in upper case: + +* PROPERTYSTORE: application property store +* STATEMODELDEFES: state model definitions +* INSTANCES: instance runtime information including current state and messages +* CONFIGS: configurations +* IDEALSTATES: ideal states +* EXTERNALVIEW: external views +* LIVEINSTANCES: live instances +* CONTROLLER: cluster controller runtime information + +Under INSTANCES, there are runtime znodes for each instance. An instance organizes znodes as follows: + +* CURRENTSTATES + * sessionId + * resourceName +* ERRORS +* STATUSUPDATES +* MESSAGES +* HEALTHREPORT + +Under CONFIGS, there are different scopes of configurations: + +* RESOURCE: contains resource scope configurations +* CLUSTER: contains cluster scope configurations +* PARTICIPANT: contains participant scope configurations + +The following image shows an example of Helix znodes layout for a cluster named "test-cluster": + +![Helix znode layout](images/helix-znode-layout.png) diff --git a/site-releases/0.6.1-incubating/src/site/markdown/Building.md b/site-releases/0.6.1-incubating/src/site/markdown/Building.md new file mode 100644 index 0000000000..f79193e2f3 --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/markdown/Building.md @@ -0,0 +1,46 @@ + + +Build Instructions +------------------ + +Requirements: Jdk 1.6+, Maven 2.0.8+ + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +git checkout tags/helix-0.6.1-incubating +mvn install package -DskipTests +``` + +Maven dependency + +``` + + org.apache.helix + helix-core + 0.6.1-incubating + +``` + +Download +-------- + +[0.6.1-incubating](./download.html) + diff --git a/site-releases/0.6.1-incubating/src/site/markdown/Concepts.md b/site-releases/0.6.1-incubating/src/site/markdown/Concepts.md new file mode 100644 index 0000000000..02d74065e9 --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/markdown/Concepts.md @@ -0,0 +1,268 @@ + + +Helix is based on the idea that a given task has the following attributes associated with it: + +* _Location of the task_. For example it runs on Node N1 +* _State_. For example, it is running, stopped etc. + +In Helix terminology, a task is referred to as a _resource_. + +### IdealState + +IdealState simply allows one to map tasks to location and state. A standard way of expressing this in Helix: + +``` + "TASK_NAME" : { + "LOCATION" : "STATE" + } + +``` +Consider a simple case where you want to launch a task \'myTask\' on node \'N1\'. The IdealState for this can be expressed as follows: + +``` +{ + "id" : "MyTask", + "mapFields" : { + "myTask" : { + "N1" : "ONLINE", + } + } +} +``` +### Partition + +If this task get too big to fit on one box, you might want to divide it into subTasks. Each subTask is referred to as a _partition_ in Helix. Let\'s say you want to divide the task into 3 subTasks/partitions, the IdealState can be changed as shown below. + +\'myTask_0\', \'myTask_1\', \'myTask_2\' are logical names representing the partitions of myTask. Each tasks runs on N1, N2 and N3 respectively. + +``` +{ + "id" : "myTask", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + } + "mapFields" : { + "myTask_0" : { + "N1" : "ONLINE", + }, + "myTask_1" : { + "N2" : "ONLINE", + }, + "myTask_2" : { + "N3" : "ONLINE", + } + } +} +``` + +### Replica + +Partitioning allows one to split the data/task into multiple subparts. But let\'s say the request rate each partition increases. The common solution is to have multiple copies for each partition. Helix refers to the copy of a partition as a _replica_. Adding a replica also increases the availability of the system during failures. One can see this methodology employed often in Search systems. The index is divided into shards, and each shard has multiple copies. + +Let\'s say you want to add one additional replica for each task. The IdealState can simply be changed as shown below. + +For increasing the availability of the system, it\'s better to place the replica of a given partition on different nodes. + +``` +{ + "id" : "myIndex", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + }, + "mapFields" : { + "myIndex_0" : { + "N1" : "ONLINE", + "N2" : "ONLINE" + }, + "myIndex_1" : { + "N2" : "ONLINE", + "N3" : "ONLINE" + }, + "myIndex_2" : { + "N3" : "ONLINE", + "N1" : "ONLINE" + } + } +} +``` + +### State + +Now let\'s take a slightly complicated scenario where a task represents a database. Unlike an index which is in general read-only, a database supports both reads and writes. Keeping the data consistent among the replicas is crucial in distributed data stores. One commonly applied technique is to assign one replica as MASTER and remaining replicas as SLAVE. All writes go to the MASTER and are then replicated to the SLAVE replicas. + +Helix allows one to assign different states to each replica. Let\'s say you have two MySQL instances N1 and N2, where one will serve as MASTER and another as SLAVE. The IdealState can be changed to: + +``` +{ + "id" : "myDB", + "simpleFields" : { + "NUM_PARTITIONS" : "1", + "REPLICAS" : "2", + }, + "mapFields" : { + "myDB" : { + "N1" : "MASTER", + "N2" : "SLAVE", + } + } +} + +``` + + +### State Machine and Transitions + +IdealState allows one to exactly specify the desired state of the cluster. Given an IdealState, Helix takes up the responsibility of ensuring that the cluster reaches the IdealState. The Helix _controller_ reads the IdealState and then commands the Participant to take appropriate actions to move from one state to another until it matches the IdealState. These actions are referred to as _transitions_ in Helix. + +The next logical question is: how does the _controller_ compute the transitions required to get to IdealState? This is where the finite state machine concept comes in. Helix allows applications to plug in a finite state machine. A state machine consists of the following: + +* State: Describes the role of a replica +* Transition: An action that allows a replica to move from one State to another, thus changing its role. + +Here is an example of MASTERSLAVE state machine, + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +Helix allows each resource to be associated with one state machine. This means you can have one resource as an index and another as a database in the same cluster. One can associate each resource with a state machine as follows: + +``` +{ + "id" : "myDB", + "simpleFields" : { + "NUM_PARTITIONS" : "1", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "myDB" : { + "N1" : "MASTER", + "N2" : "SLAVE", + } + } +} + +``` + +### Current State + +CurrentState of a resource simply represents its actual state at a PARTICIPANT. In the below example: + +* INSTANCE_NAME: Unique name representing the process +* SESSION_ID: ID that is automatically assigned every time a process joins the cluster + +``` +{ + "id":"MyResource" + ,"simpleFields":{ + ,"SESSION_ID":"13d0e34675e0002" + ,"INSTANCE_NAME":"node1" + ,"STATE_MODEL_DEF":"MasterSlave" + } + ,"mapFields":{ + "MyResource_0":{ + "CURRENT_STATE":"SLAVE" + } + ,"MyResource_1":{ + "CURRENT_STATE":"MASTER" + } + ,"MyResource_2":{ + "CURRENT_STATE":"MASTER" + } + } +} +``` +Each node in the cluster has its own CurrentState. + +### External View + +In order to communicate with the PARTICIPANTs, external clients need to know the current state of each of the PARTICIPANTs. The external clients are referred to as SPECTATORS. In order to make the life of SPECTATOR simple, Helix provides an EXTERNALVIEW that is an aggregated view of the current state across all nodes. The EXTERNALVIEW has a similar format as IDEALSTATE. + +``` +{ + "id":"MyResource", + "mapFields":{ + "MyResource_0":{ + "N1":"SLAVE", + "N2":"MASTER", + "N3":"OFFLINE" + }, + "MyResource_1":{ + "N1":"MASTER", + "N2":"SLAVE", + "N3":"ERROR" + }, + "MyResource_2":{ + "N1":"MASTER", + "N2":"SLAVE", + "N3":"SLAVE" + } + } +} +``` + +### Rebalancer + +The core component of Helix is the CONTROLLER which runs the REBALANCER algorithm on every cluster event. Cluster events can be one of the following: + +* Nodes start/stop and soft/hard failures +* New nodes are added/removed +* Ideal state changes + +There are few more such as config changes, etc. The key takeaway: there are many ways to trigger the rebalancer. + +When a rebalancer is run it simply does the following: + +* Compares the IdealState and current state +* Computes the transitions required to reach the IdealState +* Issues the transitions to each PARTICIPANT + +The above steps happen for every change in the system. Once the current state matches the IdealState, the system is considered stable which implies \'IDEALSTATE = CURRENTSTATE = EXTERNALVIEW\' + +### Dynamic IdealState + +One of the things that makes Helix powerful is that IdealState can be changed dynamically. This means one can listen to cluster events like node failures and dynamically change the ideal state. Helix will then take care of triggering the respective transitions in the system. + +Helix comes with a few algorithms to automatically compute the IdealState based on the constraints. For example, if you have a resource of 3 partitions and 2 replicas, Helix can automatically compute the IdealState based on the nodes that are currently active. See the [tutorial](./tutorial_rebalance.html) to find out more about various execution modes of Helix like AUTO_REBALANCE, AUTO and CUSTOM. + + + + + + + + + + + + diff --git a/src/site/markdown/Features.md b/site-releases/0.6.1-incubating/src/site/markdown/Features.md similarity index 100% rename from src/site/markdown/Features.md rename to site-releases/0.6.1-incubating/src/site/markdown/Features.md diff --git a/src/site/markdown/Quickstart.md b/site-releases/0.6.1-incubating/src/site/markdown/Quickstart.md similarity index 99% rename from src/site/markdown/Quickstart.md rename to site-releases/0.6.1-incubating/src/site/markdown/Quickstart.md index 4e09d24ecb..96c4efb4fa 100644 --- a/src/site/markdown/Quickstart.md +++ b/site-releases/0.6.1-incubating/src/site/markdown/Quickstart.md @@ -26,13 +26,14 @@ First, let\'s get Helix, either build it, or download. git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git cd incubator-helix + git checkout tags/helix-0.6.1-incubating mvn install package -DskipTests cd helix-core/target/helix-core-pkg/bin //This folder contains all the scripts used in following sections chmod +x * ### Download -Download the 0.6.1-incubating release package [here](https://dist.apache.org/repos/dist/dev/incubator/helix/0.6.1-incubating/binaries/helix-core-0.6.1-incubating-pkg.tar) +Download the 0.6.1-incubating release package [here](./download.html) Overview -------- diff --git a/src/site/markdown/Tutorial.md b/site-releases/0.6.1-incubating/src/site/markdown/Tutorial.md similarity index 100% rename from src/site/markdown/Tutorial.md rename to site-releases/0.6.1-incubating/src/site/markdown/Tutorial.md diff --git a/site-releases/0.6.1-incubating/src/site/markdown/UseCases.md b/site-releases/0.6.1-incubating/src/site/markdown/UseCases.md new file mode 100644 index 0000000000..d46b37221a --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/markdown/UseCases.md @@ -0,0 +1,109 @@ + + + +# Use cases at LinkedIn + +At LinkedIn Helix framework is used to manage 3 distributed data systems which are quite different from each other. + +* Espresso +* Databus +* Search As A Service + +## Espresso + +Espresso is a distributed, timeline consistent, scal- able, document store that supports local secondary indexing and local transactions. +Espresso databases are horizontally partitioned into a number of partitions, with each partition having a certain number of replicas +distributed across the storage nodes. +Espresso designates one replica of each partition as master and the rest as slaves; only one master may exist for each partition at any time. +Espresso enforces timeline consistency where only the master of a partition can accept writes to its records, and all slaves receive and +apply the same writes through a replication stream. +For load balancing, both master and slave partitions are assigned evenly across all storage nodes. +For fault tolerance, it adds the constraint that no two replicas of the same partition may be located on the same node. + +### State model +Espresso follows a Master-Slave state model. A replica can be in Offline,Slave or Master state. +The state machine table describes the next state given the Current State, Final State + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +### Constraints +* Max number of replicas in Master state:1 +* Execution mode AUTO. i.e on node failure no new replicas will be created. Only the State of remaining replicas will be changed. +* Number of mastered partitions on each node must be approximately same. +* The above constraint must be satisfied when a node fails or a new node is added. +* When new nodes are added the number of partitions moved must be minimized. +* When new nodes are added the max number of OFFLINE-SLAVE transitions that can happen concurrently on new node is X. + +## Databus + +Databus is a change data capture (CDC) system that provides a common pipeline for transporting events +from LinkedIn primary databases to caches within various applications. +Databus deploys a cluster of relays that pull the change log from multiple databases and +let consumers subscribe to the change log stream. Each Databus relay connects to one or more database servers and +hosts a certain subset of databases (and partitions) from those database servers. + +For a large partitioned database (e.g. Espresso), the change log is consumed by a bank of consumers. +Each databus partition is assigned to a consumer such that partitions are evenly distributed across consumers and each partition is +assigned to exactly one consumer at a time. The set of consumers may grow over time, and consumers may leave the group due to planned or unplanned +outages. In these cases, partitions must be reassigned, while maintaining balance and the single consumer-per-partition invariant. + +### State model +Databus consumers follow a simple Offline-Online state model. +The state machine table describes the next state given the Current State, Final State + +


+          OFFLINE  | ONLINE |   
+         ___________________|
+        |          |        |
+OFFLINE |   N/A    | ONLINE |
+        |__________|________|
+        |          |        |
+ONLINE  |  OFFLINE |   N/A  |
+        |__________|________|
+
+
+
+ + +## Search As A Service + +LinkedIn�s Search-as-a-service lets internal customers define custom indexes on a chosen dataset +and then makes those indexes searchable via a service API. The index service runs on a cluster of machines. +The index is broken into partitions and each partition has a configured number of replicas. +Each cluster server runs an instance of the Sensei system (an online index store) and hosts index partitions. +Each new indexing service gets assigned to a set of servers, and the partition replicas must be evenly distributed across those servers. + +### State model +![Helix Design](images/bootstrap_statemodel.gif) + + diff --git a/site-releases/0.6.1-incubating/src/site/markdown/index.md b/site-releases/0.6.1-incubating/src/site/markdown/index.md new file mode 100644 index 0000000000..a358d885e4 --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/markdown/index.md @@ -0,0 +1,54 @@ + + +Navigating the Documentation +---------------------------- + +### Conceptual Understanding + +[Concepts / Terminology](./Concepts.html) + +[Architecture](./Architecture.html) + +### Hands-on Helix + +[Getting Helix](./Building.html) + +[Quickstart](./Quickstart.html) + +[Tutorial](./Tutorial.html) + +[Javadocs](http://helix.incubator.apache.org/javadocs/0.6.1-incubating/index.html) + +### Recipes + +[Distributed lock manager](./recipes/lock_manager.html) + +[Rabbit MQ consumer group](./recipes/rabbitmq_consumer_group.html) + +[Rsync replicated file store](./recipes/rsync_replicated_file_store.html) + +[Service discovery](./recipes/service_discovery.html) + +[Distributed Task DAG Execution](./recipes/task_dag_execution.html) + +### Download + +[0.6.1-incubating](./download.html) + diff --git a/src/site/markdown/recipes/lock_manager.md b/site-releases/0.6.1-incubating/src/site/markdown/recipes/lock_manager.md similarity index 100% rename from src/site/markdown/recipes/lock_manager.md rename to site-releases/0.6.1-incubating/src/site/markdown/recipes/lock_manager.md diff --git a/src/site/markdown/recipes/rabbitmq_consumer_group.md b/site-releases/0.6.1-incubating/src/site/markdown/recipes/rabbitmq_consumer_group.md similarity index 100% rename from src/site/markdown/recipes/rabbitmq_consumer_group.md rename to site-releases/0.6.1-incubating/src/site/markdown/recipes/rabbitmq_consumer_group.md diff --git a/src/site/markdown/recipes/rsync_replicated_file_store.md b/site-releases/0.6.1-incubating/src/site/markdown/recipes/rsync_replicated_file_store.md similarity index 100% rename from src/site/markdown/recipes/rsync_replicated_file_store.md rename to site-releases/0.6.1-incubating/src/site/markdown/recipes/rsync_replicated_file_store.md diff --git a/src/site/markdown/recipes/service_discovery.md b/site-releases/0.6.1-incubating/src/site/markdown/recipes/service_discovery.md similarity index 100% rename from src/site/markdown/recipes/service_discovery.md rename to site-releases/0.6.1-incubating/src/site/markdown/recipes/service_discovery.md diff --git a/src/site/markdown/recipes/task_dag_execution.md b/site-releases/0.6.1-incubating/src/site/markdown/recipes/task_dag_execution.md similarity index 100% rename from src/site/markdown/recipes/task_dag_execution.md rename to site-releases/0.6.1-incubating/src/site/markdown/recipes/task_dag_execution.md diff --git a/src/site/markdown/tutorial_admin.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_admin.md similarity index 100% rename from src/site/markdown/tutorial_admin.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_admin.md diff --git a/src/site/markdown/tutorial_controller.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_controller.md similarity index 100% rename from src/site/markdown/tutorial_controller.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_controller.md diff --git a/src/site/markdown/tutorial_health.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_health.md similarity index 100% rename from src/site/markdown/tutorial_health.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_health.md diff --git a/src/site/markdown/tutorial_messaging.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_messaging.md similarity index 96% rename from src/site/markdown/tutorial_messaging.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_messaging.md index f3fef109fa..4b46671268 100644 --- a/src/site/markdown/tutorial_messaging.md +++ b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_messaging.md @@ -63,5 +63,5 @@ System Admins can also perform ad-hoc tasks, such as on-demand backups or a syst requestBackupUriRequest, responseHandler, timeout); ``` -See HelixManager.DefaultMessagingService in [Javadocs](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/messaging/DefaultMessagingService.html) for more info. +See HelixManager.DefaultMessagingService in [Javadocs](http://helix.incubator.apache.org/javadocs/0.6.1-incubating/reference/org/apache/helix/messaging/DefaultMessagingService.html) for more info. diff --git a/src/site/markdown/tutorial_participant.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_participant.md similarity index 100% rename from src/site/markdown/tutorial_participant.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_participant.md diff --git a/src/site/markdown/tutorial_propstore.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_propstore.md similarity index 92% rename from src/site/markdown/tutorial_propstore.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_propstore.md index 6af3c289f5..4ee9299faa 100644 --- a/src/site/markdown/tutorial_propstore.md +++ b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_propstore.md @@ -27,4 +27,4 @@ It is common that an application needs support for distributed, shared data stru While you could use Zookeeper directly, Helix supports caching the data and a write-through cache. This is far more efficient than reading from Zookeeper for every access. -See [HelixManager.getHelixPropertyStore](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/store/package-summary.html) for details. +See [HelixManager.getHelixPropertyStore](http://helix.incubator.apache.org/javadocs/0.6.1-incubating/reference/org/apache/helix/store/package-summary.html) for details. diff --git a/src/site/markdown/tutorial_rebalance.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_rebalance.md similarity index 100% rename from src/site/markdown/tutorial_rebalance.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_rebalance.md diff --git a/src/site/markdown/tutorial_spectator.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_spectator.md similarity index 100% rename from src/site/markdown/tutorial_spectator.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_spectator.md diff --git a/src/site/markdown/tutorial_state.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_state.md similarity index 100% rename from src/site/markdown/tutorial_state.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_state.md diff --git a/src/site/markdown/tutorial_throttling.md b/site-releases/0.6.1-incubating/src/site/markdown/tutorial_throttling.md similarity index 100% rename from src/site/markdown/tutorial_throttling.md rename to site-releases/0.6.1-incubating/src/site/markdown/tutorial_throttling.md diff --git a/site-releases/0.6.1-incubating/src/site/resources/.htaccess b/site-releases/0.6.1-incubating/src/site/resources/.htaccess new file mode 100644 index 0000000000..d5c7bf3f94 --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/resources/.htaccess @@ -0,0 +1,20 @@ +# +# 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. +# + +Redirect /download.html /download.cgi diff --git a/site-releases/0.6.1-incubating/src/site/resources/download.cgi b/site-releases/0.6.1-incubating/src/site/resources/download.cgi new file mode 100644 index 0000000000..f9a0e3007c --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/resources/download.cgi @@ -0,0 +1,22 @@ +#!/bin/sh +# Just call the standard mirrors.cgi script. It will use download.html +# as the input template. +# +# 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. +# +exec /www/www.apache.org/dyn/mirrors/mirrors.cgi $* diff --git a/site-releases/0.6.1-incubating/src/site/resources/images/HELIX-components.png b/site-releases/0.6.1-incubating/src/site/resources/images/HELIX-components.png new file mode 100644 index 0000000000..c0c35aee2c Binary files /dev/null and b/site-releases/0.6.1-incubating/src/site/resources/images/HELIX-components.png differ diff --git a/src/site/resources/images/PFS-Generic.png b/site-releases/0.6.1-incubating/src/site/resources/images/PFS-Generic.png similarity index 100% rename from src/site/resources/images/PFS-Generic.png rename to site-releases/0.6.1-incubating/src/site/resources/images/PFS-Generic.png diff --git a/src/site/resources/images/RSYNC_BASED_PFS.png b/site-releases/0.6.1-incubating/src/site/resources/images/RSYNC_BASED_PFS.png similarity index 100% rename from src/site/resources/images/RSYNC_BASED_PFS.png rename to site-releases/0.6.1-incubating/src/site/resources/images/RSYNC_BASED_PFS.png diff --git a/site-releases/0.6.1-incubating/src/site/resources/images/bootstrap_statemodel.gif b/site-releases/0.6.1-incubating/src/site/resources/images/bootstrap_statemodel.gif new file mode 100644 index 0000000000..b8f8a42589 Binary files /dev/null and b/site-releases/0.6.1-incubating/src/site/resources/images/bootstrap_statemodel.gif differ diff --git a/site-releases/0.6.1-incubating/src/site/resources/images/helix-architecture.png b/site-releases/0.6.1-incubating/src/site/resources/images/helix-architecture.png new file mode 100644 index 0000000000..6f69a2db34 Binary files /dev/null and b/site-releases/0.6.1-incubating/src/site/resources/images/helix-architecture.png differ diff --git a/site-releases/0.6.1-incubating/src/site/resources/images/helix-logo.jpg b/site-releases/0.6.1-incubating/src/site/resources/images/helix-logo.jpg new file mode 100644 index 0000000000..d6428f600f Binary files /dev/null and b/site-releases/0.6.1-incubating/src/site/resources/images/helix-logo.jpg differ diff --git a/site-releases/0.6.1-incubating/src/site/resources/images/helix-znode-layout.png b/site-releases/0.6.1-incubating/src/site/resources/images/helix-znode-layout.png new file mode 100644 index 0000000000..5bafc45f21 Binary files /dev/null and b/site-releases/0.6.1-incubating/src/site/resources/images/helix-znode-layout.png differ diff --git a/site-releases/0.6.1-incubating/src/site/resources/images/statemachine.png b/site-releases/0.6.1-incubating/src/site/resources/images/statemachine.png new file mode 100644 index 0000000000..43d27ecfbc Binary files /dev/null and b/site-releases/0.6.1-incubating/src/site/resources/images/statemachine.png differ diff --git a/site-releases/0.6.1-incubating/src/site/resources/images/system.png b/site-releases/0.6.1-incubating/src/site/resources/images/system.png new file mode 100644 index 0000000000..f8a05c8764 Binary files /dev/null and b/site-releases/0.6.1-incubating/src/site/resources/images/system.png differ diff --git a/site-releases/0.6.1-incubating/src/site/site.xml b/site-releases/0.6.1-incubating/src/site/site.xml new file mode 100644 index 0000000000..732616281e --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/site.xml @@ -0,0 +1,119 @@ + + + + + images/helix-logo.jpg + http://helix.incubator.apache.org/site-releases/0.6.1-incubating-site + + + http://incubator.apache.org/images/egg-logo.png + http://incubator.apache.org/ + + + + + + + org.apache.maven.skins + maven-fluido-skin + 1.3.0 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+
Apache Helix, Apache, the Apache feather logo, and the Apache Helix project logos are trademarks of The Apache Software Foundation. + All other marks mentioned may be trademarks or registered trademarks of their respective owners.
+ Privacy Policy +
+
+ + + + + + + true + + true + + + ApacheHelix + true + false + + + + +
diff --git a/site-releases/0.6.1-incubating/src/site/xdoc/download.xml.vm b/site-releases/0.6.1-incubating/src/site/xdoc/download.xml.vm new file mode 100644 index 0000000000..dabe9ec325 --- /dev/null +++ b/site-releases/0.6.1-incubating/src/site/xdoc/download.xml.vm @@ -0,0 +1,193 @@ + + +#set( $releaseName = "0.6.1-incubating" ) +#set( $releaseDate = "05/29/2013" ) + + + + Apache Incubator Helix Downloads + Apache Helix Documentation Team + + + +
+ + + +
+ +
+

Apache Helix artifacts are distributed in source and binary form under the terms of the + Apache License, Version 2.0. + See the included LICENSE and NOTICE files included in each artifact for additional license + information. +

+

Use the links below to download a source distribution of Apache Helix. + It is good practice to verify the integrity of the distribution files.

+
+ +
+

Release date: ${releaseDate}

+

${releaseName} Release notes

+ + + +

+ [if-any logo] + + logo + + [end] + The currently selected mirror is + [preferred]. + If you encounter a problem with this mirror, + please select another mirror. + If all mirrors are failing, there are + backup + mirrors + (at the end of the mirrors list) that should be available. +

+ +
+ Other mirrors: + + +
+ +

+ You may also consult the + complete list of mirrors. +

+ + + + + + + + + + + + + + + + +
ArtifactSignatures
+ helix-${releaseName}-src.zip + + asc + md5 + sha1 +
+
+ + + + + + + + + + + + + + + + + + +
ArtifactSignatures
+ helix-core-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+ helix-admin-webapp-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+
+
+ + + +
+

We strongly recommend you verify the integrity of the downloaded files with both PGP and MD5.

+ +

The PGP signatures can be verified using PGP or + GPG. + First download the KEYS as well as the + *.asc signature file for the particular distribution. Make sure you get these files from the main + distribution directory, rather than from a mirror. Then verify the signatures using one of the following sets of + commands: + + $ pgp -ka KEYS +$ pgp helix-*.zip.asc + + $ gpg --import KEYS +$ gpg --verify helix-*.zip.asc +

+

Alternatively, you can verify the MD5 signature on the files. A Unix/Linux program called + md5 or + md5sum is included in most distributions. It is also available as part of + GNU Textutils. + Windows users can get binary md5 programs from these (and likely other) places: +

+

+
+ +
diff --git a/site-releases/0.6.1-incubating/src/test/conf/testng.xml b/site-releases/0.6.1-incubating/src/test/conf/testng.xml new file mode 100644 index 0000000000..58f0803678 --- /dev/null +++ b/site-releases/0.6.1-incubating/src/test/conf/testng.xml @@ -0,0 +1,27 @@ + + + + + + + + + + diff --git a/site-releases/0.6.2-incubating/pom.xml b/site-releases/0.6.2-incubating/pom.xml new file mode 100644 index 0000000000..471ea4ccfa --- /dev/null +++ b/site-releases/0.6.2-incubating/pom.xml @@ -0,0 +1,51 @@ + + + + 4.0.0 + + + org.apache.helix + site-releases + 0.7.1-incubating-SNAPSHOT + + + 0.6.2-incubating-site + bundle + Apache Helix :: Site :: 0.6.2-incubating + + + + + + + org.testng + testng + 6.0.1 + + + + + + + + + + + diff --git a/site-releases/0.6.2-incubating/src/site/apt/privacy-policy.apt b/site-releases/0.6.2-incubating/src/site/apt/privacy-policy.apt new file mode 100644 index 0000000000..ada93631d1 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/apt/privacy-policy.apt @@ -0,0 +1,52 @@ + ---- + Privacy Policy + ----- + Olivier Lamy + ----- + 2013-02-04 + ----- + +~~ 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. + +Privacy Policy + + Information about your use of this website is collected using server access logs and a tracking cookie. The + collected information consists of the following: + + [[1]] The IP address from which you access the website; + + [[2]] The type of browser and operating system you use to access our site; + + [[3]] The date and time you access our site; + + [[4]] The pages you visit; and + + [[5]] The addresses of pages from where you followed a link to our site. + + [] + + Part of this information is gathered using a tracking cookie set by the + {{{http://www.google.com/analytics/}Google Analytics}} service and handled by Google as described in their + {{{http://www.google.com/privacy.html}privacy policy}}. See your browser documentation for instructions on how to + disable the cookie if you prefer not to share this data with Google. + + We use the gathered information to help us make our site more useful to visitors and to better understand how and + when our site is used. We do not track or collect personally identifiable information or associate gathered data + with any personally identifying information from other sources. + + By using this website, you consent to the collection of this data in the manner and for the purpose described above. diff --git a/site-releases/0.6.2-incubating/src/site/apt/releasenotes/release-0.6.2-incubating.apt b/site-releases/0.6.2-incubating/src/site/apt/releasenotes/release-0.6.2-incubating.apt new file mode 100644 index 0000000000..51afc62361 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/apt/releasenotes/release-0.6.2-incubating.apt @@ -0,0 +1,181 @@ + ----- + Release Notes for Apache Helix 0.6.2-incubating + ----- + +~~ 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. + +~~ NOTE: For help with the syntax of this file, see: +~~ http://maven.apache.org/guides/mini/guide-apt-format.html + +Release Notes for Apache Helix 0.6.2-incubating + + The Apache Helix team would like to announce the release of Apache Helix 0.6.2-incubating + + This is the third release under the Apache umbrella. + + Helix is a generic cluster management framework used for the automatic management of partitioned, replicated and distributed resources hosted on a cluster of nodes. Helix provides the following features: + + * Automatic assignment of resource/partition to nodes + + * Node failure detection and recovery + + * Dynamic addition of Resources + + * Dynamic addition of nodes to the cluster + + * Pluggable distributed state machine to manage the state of a resource via state transitions + + * Automatic load balancing and throttling of transitions + + [] + +* Changes + +** Sub-task + + * [HELIX-28] - ZkHelixManager.handleNewSession() can happen when a liveinstance already exists + + * [HELIX-85] - Remove mock service module + + * [HELIX-106] - Remove all string constants in the code + + * [HELIX-107] - Add support to set custom objects into ZNRecord + + * [HELIX-124] - race condition in ZkHelixManager.handleNewSession() + + * [HELIX-165] - Add dependency for Guava libraries + + * [HELIX-169] - Take care of consecutive handleNewSession() and session expiry during handleNewSession() + + * [HELIX-170] - HelixManager#isLeader() should compare both instanceName and sessionId + + * [HELIX-195] - Race condition between FINALIZE callbacks and Zk Callbacks + + * [HELIX-207] - Add javadocs to classes and public methods in the top-level package + + * [HELIX-208] - Add javadocs to classes and public methods in the model package + + * [HELIX-277] - FULL_AUTO rebalancer should not prefer nodes that are just coming up + +** Bug + + * [HELIX-7] - Tune test parameters to fix random test failures + + * [HELIX-87] - Bad repository links in website + + * [HELIX-117] - backward incompatibility problem in accessing zkPath vis HelixWebAdmin + + * [HELIX-118] - PropertyStore -> HelixPropertyStore backwards incompatible location + + * [HELIX-119] - HelixManager serializer no longer needs ByteArraySerializer for /PROPERTYSTORE + + * [HELIX-129] - ZKDumper should use byte[] instead of String to read/write file/zk + + * [HELIX-131] - Connection timeout not set while connecting to zookeeper via zkHelixAdmin + + * [HELIX-133] - Cluster-admin command parsing does not work with removeConfig + + * [HELIX-140] - In ClusterSetup.java, the removeConfig is wrong wired to getConfig + + * [HELIX-141] - Autorebalance does not work reliably and fails when replica>1 + + * [HELIX-144] - Need to validate StateModelDefinition when adding new StateModelDefinition to Cluster + + * [HELIX-147] - Fix typo in Idealstate property max_partitions_per_instance + + * [HELIX-148] - Current preferred placement for auto rebalace is suboptimal for n > p + + * [HELIX-150] - Auto rebalance might not evenly distribute states across nodes + + * [HELIX-151] - Auto rebalance doesn't assign some replicas when other nodes could make room + + * [HELIX-153] - Auto rebalance tester uses the returned map fields, but production uses only list fields + + * [HELIX-155] - PropertyKey.instances() is wrongly wired to CONFIG type instead of INSTANCES type + + * [HELIX-197] - state model leak + + * [HELIX-199] - ZNRecord should not publish rawPayload unless it exists + + * [HELIX-216] - Allow HelixAdmin addResource to accept the old rebalancing types + + * [HELIX-221] - Can't find default error->dropped transition method using name convention + + * [HELIX-257] - Upgrade Restlet to 2.1.4 - due security flaw + + * [HELIX-258] - Upgrade Apache Camel due to CVE-2013-4330 + + * [HELIX-264] - fix zkclient#close() bug + + * [HELIX-279] - Apply gc handling fixes to main ZKHelixManager class + + * [HELIX-280] - Full auto rebalancer should check for resource tag first + + * [HELIX-288] - helix-core uses an old version of guava + + * [HELIX-299] - Some files in 0.6.2 are missing license headers + +** Improvement + + * [HELIX-20] - AUTO-REBALANCE helix controller should re-assign disabled partitions on a node to other available nodes + + * [HELIX-70] - Make Helix OSGi ready + + * [HELIX-149] - Allow clients to pass in preferred placement strategies + + * [HELIX-198] - Unify helix code style + + * [HELIX-218] - Add a reviewboard submission script + + * [HELIX-284] - Support participant auto join in YAML cluster setup + +** New Feature + + * [HELIX-215] - Allow setting up the cluster with a YAML file + +** Task + + * [HELIX-95] - Tracker for 0.6.2 release + + * [HELIX-154] - Auto rebalance algorithm should not depend on state + + * [HELIX-166] - Rename modes to auto, semi-auto, and custom + + * [HELIX-173] - Move rebalancing strategies to separate classes that implement the Rebalancer interface + + * [HELIX-188] - Add admin command line / REST API documentations + + * [HELIX-194] - ZNRecord has too many constructors + + * [HELIX-205] - Have user-defined rebalancers use RebalanceMode.USER_DEFINED + + * [HELIX-210] - Add support to set data with expect version in BaseDataAccessor + + * [HELIX-217] - Remove mock service module + + * [HELIX-273] - Rebalancer interface should remain unchanged in 0.6.2 + + * [HELIX-274] - Verify FULL_AUTO tagged node behavior + + * [HELIX-285] - add integration test util's + + [] + + Cheers, + -- + The Apache Helix Team diff --git a/site-releases/0.6.2-incubating/src/site/apt/releasing.apt b/site-releases/0.6.2-incubating/src/site/apt/releasing.apt new file mode 100644 index 0000000000..11d0cd92aa --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/apt/releasing.apt @@ -0,0 +1,107 @@ + ----- + Helix release process + ----- + ----- + 2012-12-15 + ----- + +~~ 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. + +~~ NOTE: For help with the syntax of this file, see: +~~ http://maven.apache.org/guides/mini/guide-apt-format.html + +Helix release process + + [[1]] Post to the dev list a few days before you plan to do an Helix release + + [[2]] Your maven setting must contains the entry to be able to deploy. + + ~/.m2/settings.xml + ++------------- + + apache.releases.https + + + ++------------- + + [[3]] Apache DAV passwords + ++------------- + Add the following info into your ~/.netrc + machine git-wip-us.apache.org login + ++------------- + [[4]] Release Helix + You should have a GPG agent running in the session you will run the maven release commands(preferred), and confirm it works by running "gpg -ab" (type some text and press Ctrl-D). + If you do not have a GPG agent running, make sure that you have the "apache-release" profile set in your settings.xml as shown below. + + Run the release + ++------------- +mvn release:prepare release:perform -B ++------------- + + GPG configuration in maven settings xml: + ++------------- + + apache-release + + [GPG_PASSWORD] + + ++------------- + + [[4]] go to https://repository.apache.org and close your staged repository. Note the repository url (format https://repository.apache.org/content/repositories/orgapachehelix-019/org/apache/helix/helix/0.6-incubating/) + ++------------- +svn co https://dist.apache.org/repos/dist/dev/incubator/helix helix-dev-release +cd helix-dev-release +sh ./release-script-svn.sh version stagingRepoUrl +then svn add +then svn ci ++------------- + + [[5]] Validating the release + ++------------- + * Download sources, extract, build and run tests - mvn clean package + * Verify license headers - mvn -Prat -DskipTests + * Download binaries and .asc files + * Download release manager's public key - From the KEYS file, get the release manager's public key finger print and run gpg --keyserver pgpkeys.mit.edu --recv-key + * Validate authenticity of key - run gpg --fingerprint + * Check signatures of all the binaries using gpg ++------------- + + [[6]] Call for a vote in the dev list and wait for 72 hrs. for the vote results. 3 binding votes are necessary for the release to be finalized. example + After the vote has passed, move the files from dist dev to dist release: svn mv https://dist.apache.org/repos/dist/dev/incubator/helix/version to https://dist.apache.org/repos/dist/release/incubator/helix/ + + [[7]] Prepare release note. Add a page in src/site/apt/releasenotes/ and change value of \ in parent pom. + + + [[8]] Send out an announcement of the release to: + + * users@helix.incubator.apache.org + + * dev@helix.incubator.apache.org + + [[9]] Celebrate ! + + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/Architecture.md b/site-releases/0.6.2-incubating/src/site/markdown/Architecture.md new file mode 100644 index 0000000000..933e917c7e --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/Architecture.md @@ -0,0 +1,252 @@ + + + + Architecture + + +Architecture +---------------------------- +Helix aims to provide the following abilities to a distributed system: + +* Automatic management of a cluster hosting partitioned, replicated resources. +* Soft and hard failure detection and handling. +* Automatic load balancing via smart placement of resources on servers(nodes) based on server capacity and resource profile (size of partition, access patterns, etc). +* Centralized config management and self discovery. Eliminates the need to modify config on each node. +* Fault tolerance and optimized rebalancing during cluster expansion. +* Manages entire operational lifecycle of a node. Addition, start, stop, enable/disable without downtime. +* Monitor cluster health and provide alerts on SLA violation. +* Service discovery mechanism to route requests. + +To build such a system, we need a mechanism to co-ordinate between different nodes and other components in the system. This mechanism can be achieved with software that reacts to any change in the cluster and comes up with a set of tasks needed to bring the cluster to a stable state. The set of tasks will be assigned to one or more nodes in the cluster. Helix serves this purpose of managing the various components in the cluster. + +![Helix Design](images/system.png) + +Distributed System Components + +In general any distributed system cluster will have the following components and properties: + +* A set of nodes also referred to as instances. +* A set of resources which can be databases, lucene indexes or tasks. +* Each resource is also partitioned into one or more Partitions. +* Each partition may have one or more copies called replicas. +* Each replica can have a state associated with it. For example Master, Slave, Leader, Standby, Online, Offline etc + +Roles +----- + +![Helix Design](images/HELIX-components.png) + +Not all nodes in a distributed system will perform similar functionalities. For example, a few nodes might be serving requests and a few nodes might be sending requests, and some nodes might be controlling the nodes in the cluster. Thus, Helix categorizes nodes by their specific roles in the system. + +We have divided Helix nodes into 3 logical components based on their responsibilities: + +1. Participant: The nodes that actually host the distributed resources. +2. Spectator: The nodes that simply observe the Participant state and route the request accordingly. Routers, for example, need to know the instance on which a partition is hosted and its state in order to route the request to the appropriate end point. +3. Controller: The controller observes and controls the Participant nodes. It is responsible for coordinating all transitions in the cluster and ensuring that state constraints are satisfied and cluster stability is maintained. + +These are simply logical components and can be deployed as per the system requirements. For example: + +1. The controller can be deployed as a separate service +2. The controller can be deployed along with a Participant but only one Controller will be active at any given time. + +Both have pros and cons, which will be discussed later and one can chose the mode of deployment as per system needs. + + +## Cluster state metadata store + +We need a distributed store to maintain the state of the cluster and a notification system to notify if there is any change in the cluster state. Helix uses Zookeeper to achieve this functionality. + +Zookeeper provides: + +* A way to represent PERSISTENT state which basically remains until its deleted. +* A way to represent TRANSIENT/EPHEMERAL state which vanishes when the process that created the state dies. +* Notification mechanism when there is a change in PERSISTENT and EPHEMERAL state + +The namespace provided by ZooKeeper is much like that of a standard file system. A name is a sequence of path elements separated by a slash (/). Every node[ZNode] in ZooKeeper\'s namespace is identified by a path. + +More info on Zookeeper can be found at http://zookeeper.apache.org + +## State machine and constraints + +Even though the concepts of Resources, Partitions, and Replicas are common to most distributed systems, one thing that differentiates one distributed system from another is the way each partition is assigned a state and the constraints on each state. + +For example: + +1. If a system is serving read-only data then all partition\'s replicas are equal and they can either be ONLINE or OFFLINE. +2. If a system takes _both_ reads and writes but ensure that writes go through only one partition, the states will be MASTER, SLAVE, and OFFLINE. Writes go through the MASTER and replicate to the SLAVEs. Optionally, reads can go through SLAVES. + +Apart from defining state for each partition, the transition path to each state can be application specific. For example, in order to become MASTER it might be a requirement to first become a SLAVE. This ensures that if the SLAVE does not have the data as part of OFFLINE-SLAVE transition it can bootstrap data from other nodes in the system. + +Helix provides a way to configure an application specific state machine along with constraints on each state. Along with constraints on STATE, Helix also provides a way to specify constraints on transitions. (More on this later.) + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +![Helix Design](images/statemachine.png) + +## Concepts + +The following terminologies are used in Helix to model a state machine. + +* IdealState: The state in which we need the cluster to be in if all nodes are up and running. In other words, all state constraints are satisfied. +* CurrentState: Represents the actual current state of each node in the cluster +* ExternalView: Represents the combined view of CurrentState of all nodes. + +The goal of Helix is always to make the CurrentState of the system same as the IdealState. Some scenarios where this may not be true are: + +* When all nodes are down +* When one or more nodes fail +* New nodes are added and the partitions need to be reassigned + +### IdealState + +Helix lets the application define the IdealState on a resource basis which basically consists of: + +* List of partitions. Example: 64 +* Number of replicas for each partition. Example: 3 +* Node and State for each replica. + +Example: + +* Partition-1, replica-1, Master, Node-1 +* Partition-1, replica-2, Slave, Node-2 +* Partition-1, replica-3, Slave, Node-3 +* ..... +* ..... +* Partition-p, replica-3, Slave, Node-n + +Helix comes with various algorithms to automatically assign the partitions to nodes. The default algorithm minimizes the number of shuffles that happen when new nodes are added to the system. + +### CurrentState + +Every instance in the cluster hosts one or more partitions of a resource. Each of the partitions has a state associated with it. + +Example Node-1 + +* Partition-1, Master +* Partition-2, Slave +* .... +* .... +* Partition-p, Slave + +### ExternalView + +External clients needs to know the state of each partition in the cluster and the Node hosting that partition. Helix provides one view of the system to Spectators as _ExternalView_. ExternalView is simply an aggregate of all node CurrentStates. + +* Partition-1, replica-1, Master, Node-1 +* Partition-1, replica-2, Slave, Node-2 +* Partition-1, replica-3, Slave, Node-3 +* ..... +* ..... +* Partition-p, replica-3, Slave, Node-n + +## Process Workflow + +Mode of operation in a cluster + +A node process can be one of the following: + +* Participant: The process registers itself in the cluster and acts on the messages received in its queue and updates the current state. Example: a storage node in a distributed database +* Spectator: The process is simply interested in the changes in the Externalview. +* Controller: This process actively controls the cluster by reacting to changes in cluster state and sending messages to Participants. + + +### Participant Node Process + +* When Node starts up, it registers itself under _LiveInstances_ +* After registering, it waits for new _Messages_ in the message queue +* When it receives a message, it will perform the required task as indicated in the message +* After the task is completed, depending on the task outcome it updates the CurrentState + +### Controller Process + +* Watches IdealState +* Notified when a node goes down/comes up or node is added/removed. Watches LiveInstances and CurrentState of each node in the cluster +* Triggers appropriate state transitions by sending message to Participants + +### Spectator Process + +* When the process starts, it asks the Helix agent to be notified of changes in ExternalView +* Whenever it receives a notification, it reads the Externalview and performs required duties. + +#### Interaction between controller, participant and spectator + +The following picture shows how controllers, participants and spectators interact with each other. + +![Helix Architecture](images/helix-architecture.png) + +## Core algorithm + +* Controller gets the IdealState and the CurrentState of active storage nodes from Zookeeper +* Compute the delta between IdealState and CurrentState for each partition across all participant nodes +* For each partition compute tasks based on the State Machine Table. It\'s possible to configure priority on the state Transition. For example, in case of Master-Slave: + * Attempt mastership transfer if possible without violating constraint. + * Partition Addition + * Drop Partition +* Add the tasks in parallel if possible to the respective queue for each storage node (if the tasks added are mutually independent) +* If a task is dependent on another task being completed, do not add that task +* After any task is completed by a Participant, Controllers gets notified of the change and the State Transition algorithm is re-run until the CurrentState is same as IdealState. + +## Helix ZNode layout + +Helix organizes znodes under clusterName in multiple levels. + +The top level (under the cluster name) ZNodes are all Helix-defined and in upper case: + +* PROPERTYSTORE: application property store +* STATEMODELDEFES: state model definitions +* INSTANCES: instance runtime information including current state and messages +* CONFIGS: configurations +* IDEALSTATES: ideal states +* EXTERNALVIEW: external views +* LIVEINSTANCES: live instances +* CONTROLLER: cluster controller runtime information + +Under INSTANCES, there are runtime ZNodes for each instance. An instance organizes ZNodes as follows: + +* CURRENTSTATES + * sessionId + * resourceName +* ERRORS +* STATUSUPDATES +* MESSAGES +* HEALTHREPORT + +Under CONFIGS, there are different scopes of configurations: + +* RESOURCE: contains resource scope configurations +* CLUSTER: contains cluster scope configurations +* PARTICIPANT: contains participant scope configurations + +The following image shows an example of Helix znodes layout for a cluster named "test-cluster": + +![Helix znode layout](images/helix-znode-layout.png) diff --git a/site-releases/0.6.2-incubating/src/site/markdown/Building.md b/site-releases/0.6.2-incubating/src/site/markdown/Building.md new file mode 100644 index 0000000000..bf9462b083 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/Building.md @@ -0,0 +1,46 @@ + + +Build Instructions +------------------ + +Requirements: Jdk 1.6+, Maven 2.0.8+ + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +git checkout tags/helix-0.6.2-incubating +mvn install package -DskipTests +``` + +Maven dependency + +``` + + org.apache.helix + helix-core + 0.6.2-incubating + +``` + +Download +-------- + +[0.6.2-incubating](./download.html) + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/Concepts.md b/site-releases/0.6.2-incubating/src/site/markdown/Concepts.md new file mode 100644 index 0000000000..fa5d0ba75f --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/Concepts.md @@ -0,0 +1,275 @@ + + + + Concepts + + +Concepts +---------------------------- + +Helix is based on the idea that a given task has the following attributes associated with it: + +* _Location of the task_. For example it runs on Node N1 +* _State_. For example, it is running, stopped etc. + +In Helix terminology, a task is referred to as a _resource_. + +### IdealState + +IdealState simply allows one to map tasks to location and state. A standard way of expressing this in Helix: + +``` + "TASK_NAME" : { + "LOCATION" : "STATE" + } + +``` +Consider a simple case where you want to launch a task \'myTask\' on node \'N1\'. The IdealState for this can be expressed as follows: + +``` +{ + "id" : "MyTask", + "mapFields" : { + "myTask" : { + "N1" : "ONLINE", + } + } +} +``` +### Partition + +If this task get too big to fit on one box, you might want to divide it into subtasks. Each subtask is referred to as a _partition_ in Helix. Let\'s say you want to divide the task into 3 subtasks/partitions, the IdealState can be changed as shown below. + +\'myTask_0\', \'myTask_1\', \'myTask_2\' are logical names representing the partitions of myTask. Each tasks runs on N1, N2 and N3 respectively. + +``` +{ + "id" : "myTask", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + } + "mapFields" : { + "myTask_0" : { + "N1" : "ONLINE", + }, + "myTask_1" : { + "N2" : "ONLINE", + }, + "myTask_2" : { + "N3" : "ONLINE", + } + } +} +``` + +### Replica + +Partitioning allows one to split the data/task into multiple subparts. But let\'s say the request rate for each partition increases. The common solution is to have multiple copies for each partition. Helix refers to the copy of a partition as a _replica_. Adding a replica also increases the availability of the system during failures. One can see this methodology employed often in search systems. The index is divided into shards, and each shard has multiple copies. + +Let\'s say you want to add one additional replica for each task. The IdealState can simply be changed as shown below. + +For increasing the availability of the system, it\'s better to place the replica of a given partition on different nodes. + +``` +{ + "id" : "myIndex", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + }, + "mapFields" : { + "myIndex_0" : { + "N1" : "ONLINE", + "N2" : "ONLINE" + }, + "myIndex_1" : { + "N2" : "ONLINE", + "N3" : "ONLINE" + }, + "myIndex_2" : { + "N3" : "ONLINE", + "N1" : "ONLINE" + } + } +} +``` + +### State + +Now let\'s take a slightly more complicated scenario where a task represents a database. Unlike an index which is in general read-only, a database supports both reads and writes. Keeping the data consistent among the replicas is crucial in distributed data stores. One commonly applied technique is to assign one replica as the MASTER and remaining replicas as SLAVEs. All writes go to the MASTER and are then replicated to the SLAVE replicas. + +Helix allows one to assign different states to each replica. Let\'s say you have two MySQL instances N1 and N2, where one will serve as MASTER and another as SLAVE. The IdealState can be changed to: + +``` +{ + "id" : "myDB", + "simpleFields" : { + "NUM_PARTITIONS" : "1", + "REPLICAS" : "2", + }, + "mapFields" : { + "myDB" : { + "N1" : "MASTER", + "N2" : "SLAVE", + } + } +} + +``` + + +### State Machine and Transitions + +IdealState allows one to exactly specify the desired state of the cluster. Given an IdealState, Helix takes up the responsibility of ensuring that the cluster reaches the IdealState. The Helix _controller_ reads the IdealState and then commands each Participant to take appropriate actions to move from one state to another until it matches the IdealState. These actions are referred to as _transitions_ in Helix. + +The next logical question is: how does the _controller_ compute the transitions required to get to IdealState? This is where the finite state machine concept comes in. Helix allows applications to plug in a finite state machine. A state machine consists of the following: + +* State: Describes the role of a replica +* Transition: An action that allows a replica to move from one state to another, thus changing its role. + +Here is an example of MasterSlave state machine: + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +Helix allows each resource to be associated with one state machine. This means you can have one resource as an index and another as a database in the same cluster. One can associate each resource with a state machine as follows: + +``` +{ + "id" : "myDB", + "simpleFields" : { + "NUM_PARTITIONS" : "1", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "myDB" : { + "N1" : "MASTER", + "N2" : "SLAVE", + } + } +} + +``` + +### Current State + +CurrentState of a resource simply represents its actual state at a Participant. In the below example: + +* INSTANCE_NAME: Unique name representing the process +* SESSION_ID: ID that is automatically assigned every time a process joins the cluster + +``` +{ + "id":"MyResource" + ,"simpleFields":{ + ,"SESSION_ID":"13d0e34675e0002" + ,"INSTANCE_NAME":"node1" + ,"STATE_MODEL_DEF":"MasterSlave" + } + ,"mapFields":{ + "MyResource_0":{ + "CURRENT_STATE":"SLAVE" + } + ,"MyResource_1":{ + "CURRENT_STATE":"MASTER" + } + ,"MyResource_2":{ + "CURRENT_STATE":"MASTER" + } + } +} +``` +Each node in the cluster has its own CurrentState. + +### External View + +In order to communicate with the Participants, external clients need to know the current state of each of the Participants. The external clients are referred to as Spectators. In order to make the life of Spectator simple, Helix provides an ExternalView that is an aggregated view of the current state across all nodes. The ExternalView has a similar format as IdealState. + +``` +{ + "id":"MyResource", + "mapFields":{ + "MyResource_0":{ + "N1":"SLAVE", + "N2":"MASTER", + "N3":"OFFLINE" + }, + "MyResource_1":{ + "N1":"MASTER", + "N2":"SLAVE", + "N3":"ERROR" + }, + "MyResource_2":{ + "N1":"MASTER", + "N2":"SLAVE", + "N3":"SLAVE" + } + } +} +``` + +### Rebalancer + +The core component of Helix is the Controller which runs the Rebalancer algorithm on every cluster event. Cluster events can be one of the following: + +* Nodes start/stop and soft/hard failures +* New nodes are added/removed +* Ideal state changes + +There are few more examples such as configuration changes, etc. The key takeaway: there are many ways to trigger the rebalancer. + +When a rebalancer is run it simply does the following: + +* Compares the IdealState and current state +* Computes the transitions required to reach the IdealState +* Issues the transitions to each Participant + +The above steps happen for every change in the system. Once the current state matches the IdealState, the system is considered stable which implies \'IdealState = CurrentState = ExternalView\' + +### Dynamic IdealState + +One of the things that makes Helix powerful is that IdealState can be changed dynamically. This means one can listen to cluster events like node failures and dynamically change the ideal state. Helix will then take care of triggering the respective transitions in the system. + +Helix comes with a few algorithms to automatically compute the IdealState based on the constraints. For example, if you have a resource of 3 partitions and 2 replicas, Helix can automatically compute the IdealState based on the nodes that are currently active. See the [tutorial](./tutorial_rebalance.html) to find out more about various execution modes of Helix like FULL_AUTO, SEMI_AUTO and CUSTOMIZED. + + + + + + + + + + + + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/Features.md b/site-releases/0.6.2-incubating/src/site/markdown/Features.md new file mode 100644 index 0000000000..ba9d0e72d0 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/Features.md @@ -0,0 +1,313 @@ + + + + Features + + +Features +---------------------------- + + +### CONFIGURING IDEALSTATE + + +Read concepts page for definition of Idealstate. + +The placement of partitions in a DDS is very critical for reliability and scalability of the system. +For example, when a node fails, it is important that the partitions hosted on that node are reallocated evenly among the remaining nodes. Consistent hashing is one such algorithm that can guarantee this. +Helix by default comes with a variant of consistent hashing based of the RUSH algorithm. + +This means given a number of partitions, replicas and number of nodes Helix does the automatic assignment of partition to nodes such that + +* Each node has the same number of partitions and replicas of the same partition do not stay on the same node. +* When a node fails, the partitions will be equally distributed among the remaining nodes +* When new nodes are added, the number of partitions moved will be minimized along with satisfying the above two criteria. + + +Helix provides multiple ways to control the placement and state of a replica. + +``` + + |AUTO REBALANCE| AUTO | CUSTOM | + ----------------------------------------- + LOCATION | HELIX | APP | APP | + ----------------------------------------- + STATE | HELIX | HELIX | APP | + ----------------------------------------- +``` + +#### HELIX EXECUTION MODE + + +Idealstate is defined as the state of the DDS when all nodes are up and running and healthy. +Helix uses this as the target state of the system and computes the appropriate transitions needed in the system to bring it to a stable state. + +Helix supports 3 different execution modes which allows application to explicitly control the placement and state of the replica. + +##### AUTO_REBALANCE + +When the idealstate mode is set to AUTO_REBALANCE, Helix controls both the location of the replica along with the state. This option is useful for applications where creation of a replica is not expensive. Example + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "IDEAL_STATE_MODE" : "AUTO_REBALANCE", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [], + "MyResource_1" : [], + "MyResource_2" : [] + }, + "mapFields" : { + } +} +``` + +If there are 3 nodes in the cluster, then Helix will internally compute the ideal state as + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +Another typical example is evenly distributing a group of tasks among the currently alive processes. For example, if there are 60 tasks and 4 nodes, Helix assigns 15 tasks to each node. +When one node fails Helix redistributes its 15 tasks to the remaining 3 nodes. Similarly, if a node is added, Helix re-allocates 3 tasks from each of the 4 nodes to the 5th node. + +#### AUTO + +When the idealstate mode is set to AUTO, Helix only controls STATE of the replicas where as the location of the partition is controlled by application. Example: The below idealstate indicates thats 'MyResource_0' must be only on node1 and node2. But gives the control of assigning the STATE to Helix. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "IDEAL_STATE_MODE" : "AUTO", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [node1, node2], + "MyResource_1" : [node2, node3], + "MyResource_2" : [node3, node1] + }, + "mapFields" : { + } +} +``` +In this mode when node1 fails, unlike in AUTO-REBALANCE mode the partition is not moved from node1 to others nodes in the cluster. Instead, Helix will decide to change the state of MyResource_0 in N2 based on the system constraints. For example, if a system constraint specified that there should be 1 Master and if the Master failed, then node2 will be made the new master. + +#### CUSTOM + +Helix offers a third mode called CUSTOM, in which application can completely control the placement and state of each replica. Applications will have to implement an interface that Helix will invoke when the cluster state changes. +Within this callback, the application can recompute the idealstate. Helix will then issue appropriate transitions such that Idealstate and Currentstate converges. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "IDEAL_STATE_MODE" : "CUSTOM", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +For example, the current state of the system might be 'MyResource_0' -> {N1:MASTER,N2:SLAVE} and the application changes the ideal state to 'MyResource_0' -> {N1:SLAVE,N2:MASTER}. Helix will not blindly issue MASTER-->SLAVE to N1 and SLAVE-->MASTER to N2 in parallel since it might result in a transient state where both N1 and N2 are masters. +Helix will first issue MASTER-->SLAVE to N1 and after its completed it will issue SLAVE-->MASTER to N2. + + +### State Machine Configuration + +Helix comes with 3 default state models that are most commonly used. Its possible to have multiple state models in a cluster. +Every resource that is added should have a reference to the state model. + +* MASTER-SLAVE: Has 3 states OFFLINE,SLAVE,MASTER. Max masters is 1. Slaves will be based on the replication factor. Replication factor can be specified while adding the resource +* ONLINE-OFFLINE: Has 2 states OFFLINE and ONLINE. Very simple state model and most applications start off with this state model. +* LEADER-STANDBY:1 Leader and many stand bys. In general the standby's are idle. + +Apart from providing the state machine configuration, one can specify the constraints of states and transitions. + +For example one can say +Master:1. Max number of replicas in Master state at any time is 1. +OFFLINE-SLAVE:5 Max number of Offline-Slave transitions that can happen concurrently in the system + +STATE PRIORITY +Helix uses greedy approach to satisfy the state constraints. For example if the state machine configuration says it needs 1 master and 2 slaves but only 1 node is active, Helix must promote it to master. This behavior is achieved by providing the state priority list as MASTER,SLAVE. + +STATE TRANSITION PRIORITY +Helix tries to fire as many transitions as possible in parallel to reach the stable state without violating constraints. By default Helix simply sorts the transitions alphabetically and fires as many as it can without violating the constraints. +One can control this by overriding the priority order. + +### Config management + +Helix allows applications to store application specific properties. The configuration can have different scopes. + +* Cluster +* Node specific +* Resource specific +* Partition specific + +Helix also provides notifications when any configs are changed. This allows applications to support dynamic configuration changes. + +See HelixManager.getConfigAccessor for more info + +### Intra cluster messaging api + +This is an interesting feature which is quite useful in practice. Often times, nodes in DDS requires a mechanism to interact with each other. One such requirement is a process of bootstrapping a replica. + +Consider a search system use case where the index replica starts up and it does not have an index. One of the commonly used solutions is to get the index from a common location or to copy the index from another replica. +Helix provides a messaging api, that can be used to talk to other nodes in the system. The value added that Helix provides here is, message recipient can be specified in terms of resource, +partition, state and Helix ensures that the message is delivered to all of the required recipients. In this particular use case, the instance can specify the recipient criteria as all replicas of P1. +Since Helix is aware of the global state of the system, it can send the message to appropriate nodes. Once the nodes respond Helix provides the bootstrapping replica with all the responses. + +This is a very generic api and can also be used to schedule various periodic tasks in the cluster like data backups etc. +System Admins can also perform adhoc tasks like on demand backup or execute a system command(like rm -rf ;-)) across all nodes. + +``` + ClusterMessagingService messagingService = manager.getMessagingService(); + //CONSTRUCT THE MESSAGE + Message requestBackupUriRequest = new Message( + MessageType.USER_DEFINE_MSG, UUID.randomUUID().toString()); + requestBackupUriRequest + .setMsgSubType(BootstrapProcess.REQUEST_BOOTSTRAP_URL); + requestBackupUriRequest.setMsgState(MessageState.NEW); + //SET THE RECIPIENT CRITERIA, All nodes that satisfy the criteria will receive the message + Criteria recipientCriteria = new Criteria(); + recipientCriteria.setInstanceName("%"); + recipientCriteria.setRecipientInstanceType(InstanceType.PARTICIPANT); + recipientCriteria.setResource("MyDB"); + recipientCriteria.setPartition(""); + //Should be processed only the process that is active at the time of sending the message. + //This means if the recipient is restarted after message is sent, it will not be processed. + recipientCriteria.setSessionSpecific(true); + // wait for 30 seconds + int timeout = 30000; + //The handler that will be invoked when any recipient responds to the message. + BootstrapReplyHandler responseHandler = new BootstrapReplyHandler(); + //This will return only after all recipients respond or after timeout. + int sentMessageCount = messagingService.sendAndWait(recipientCriteria, + requestBackupUriRequest, responseHandler, timeout); +``` + +See HelixManager.getMessagingService for more info. + + +### Application specific property storage + +There are several usecases where applications needs support for distributed data structures. Helix uses Zookeeper to store the application data and hence provides notifications when the data changes. +One value add Helix provides is the ability to specify cache the data and also write through cache. This is more efficient than reading from ZK every time. + +See HelixManager.getHelixPropertyStore + +### Throttling + +Since all state changes in the system are triggered through transitions, Helix can control the number of transitions that can happen in parallel. Some of the transitions may be light weight but some might involve moving data around which is quite expensive. +Helix allows applications to set threshold on transitions. The threshold can be set at the multiple scopes. + +* MessageType e.g STATE_TRANSITION +* TransitionType e.g SLAVE-MASTER +* Resource e.g database +* Node i.e per node max transitions in parallel. + +See HelixManager.getHelixAdmin.addMessageConstraint() + +### Health monitoring and alerting + +This in currently in development mode, not yet productionized. + +Helix provides ability for each node in the system to report health metrics on a periodic basis. +Helix supports multiple ways to aggregate these metrics like simple SUM, AVG, EXPONENTIAL DECAY, WINDOW. Helix will only persist the aggregated value. +Applications can define threshold on the aggregate values according to the SLA's and when the SLA is violated Helix will fire an alert. +Currently Helix only fires an alert but eventually we plan to use this metrics to either mark the node dead or load balance the partitions. +This feature will be valuable in for distributed systems that support multi-tenancy and have huge variation in work load patterns. Another place this can be used is to detect skewed partitions and rebalance the cluster. + +This feature is not yet stable and do not recommend to be used in production. + + +### Controller deployment modes + +Read Architecture wiki for more details on the Role of a controller. In simple words, it basically controls the participants in the cluster by issuing transitions. + +Helix provides multiple options to deploy the controller. + +#### STANDALONE + +Controller can be started as a separate process to manage a cluster. This is the recommended approach. How ever since one controller can be a single point of failure, multiple controller processes are required for reliability. +Even if multiple controllers are running only one will be actively managing the cluster at any time and is decided by a leader election process. If the leader fails, another leader will resume managing the cluster. + +Even though we recommend this method of deployment, it has the drawback of having to manage an additional service for each cluster. See Controller As a Service option. + +#### EMBEDDED + +If setting up a separate controller process is not viable, then it is possible to embed the controller as a library in each of the participant. + +#### CONTROLLER AS A SERVICE + +One of the cool feature we added in helix was use a set of controllers to manage a large number of clusters. +For example if you have X clusters to be managed, instead of deploying X*3(3 controllers for fault tolerance) controllers for each cluster, one can deploy only 3 controllers. Each controller can manage X/3 clusters. +If any controller fails the remaining two will manage X/2 clusters. At LinkedIn, we always deploy controllers in this mode. + + + + + + + + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/Quickstart.md b/site-releases/0.6.2-incubating/src/site/markdown/Quickstart.md new file mode 100644 index 0000000000..533a48c052 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/Quickstart.md @@ -0,0 +1,626 @@ + + + + Quickstart + + +Get Helix +--------- + +First, let\'s get Helix, either build it, or download. + +### Build + + git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git + cd incubator-helix + git checkout tags/helix-0.6.2-incubating + ./build + cd helix-core/target/helix-core-pkg/bin //This folder contains all the scripts used in following sections + chmod +x * + +### Download + +Download the 0.6.2-incubating release package [here](./download.html) + +Overview +-------- + +In this Quickstart, we\'ll set up a master-slave replicated, partitioned system. Then we\'ll demonstrate how to add a node, rebalance the partitions, and show how Helix manages failover. + + +Let\'s Do It +------------ + +Helix provides command line interfaces to set up the cluster and view the cluster state. The best way to understand how Helix views a cluster is to build a cluster. + +#### First, get to the tools directory + +If you built the code + +``` +cd incubator-helix/helix-core/target/helix-core-pkg/bin +``` + +If you downloaded the release package, extract it. + + +Short Version +------------- +You can observe the components working together in this demo, which does the following: + +* Create a cluster +* Add 2 nodes (participants) to the cluster +* Set up a resource with 6 partitions and 2 replicas: 1 Master, and 1 Slave per partition +* Show the cluster state after Helix balances the partitions +* Add a third node +* Show the cluster state. Note that the third node has taken mastership of 2 partitions. +* Kill the third node (Helix takes care of failover) +* Show the cluster state. Note that the two surviving nodes take over mastership of the partitions from the failed node + +##### Run the demo + +``` +cd incubator-helix/helix-core/target/helix-core-pkg/bin +./quickstart.sh +``` + +##### 2 nodes are set up and the partitions rebalanced + +The cluster state is as follows: + +``` +CLUSTER STATE: After starting 2 nodes + localhost_12000 localhost_12001 + MyResource_0 M S + MyResource_1 S M + MyResource_2 M S + MyResource_3 M S + MyResource_4 S M + MyResource_5 S M +``` + +Note there is one master and one slave per partition. + +##### A third node is added and the cluster rebalanced + +The cluster state changes to: + +``` +CLUSTER STATE: After adding a third node + localhost_12000 localhost_12001 localhost_12002 + MyResource_0 S M S + MyResource_1 S S M + MyResource_2 M S S + MyResource_3 S S M + MyResource_4 M S S + MyResource_5 S M S +``` + +Note there is one master and _two_ slaves per partition. This is expected because there are three nodes. + +##### Finally, a node is killed to simulate a failure + +Helix makes sure each partition has a master. The cluster state changes to: + +``` +CLUSTER STATE: After the 3rd node stops/crashes + localhost_12000 localhost_12001 localhost_12002 + MyResource_0 S M - + MyResource_1 S M - + MyResource_2 M S - + MyResource_3 M S - + MyResource_4 M S - + MyResource_5 S M - +``` + + +Long Version +------------ +Now you can run the same steps by hand. In the detailed version, we\'ll do the following: + +* Define a cluster +* Add two nodes to the cluster +* Add a 6-partition resource with 1 master and 2 slave replicas per partition +* Verify that the cluster is healthy and inspect the Helix view +* Expand the cluster: add a few nodes and rebalance the partitions +* Failover: stop a node and verify the mastership transfer + +### Install and Start Zookeeper + +Zookeeper can be started in standalone mode or replicated mode. + +More info is available at + +* http://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html +* http://zookeeper.apache.org/doc/trunk/zookeeperAdmin.html#sc_zkMulitServerSetup + +In this example, let\'s start zookeeper in local mode. + +##### start zookeeper locally on port 2199 + + ./start-standalone-zookeeper.sh 2199 & + +### Define the Cluster + +The helix-admin tool is used for cluster administration tasks. In the Quickstart, we\'ll use the command line interface. Helix supports a REST interface as well. + +zookeeper_address is of the format host:port e.g localhost:2199 for standalone or host1:port,host2:port for multi-node. + +Next, we\'ll set up a cluster MYCLUSTER cluster with these attributes: + +* 3 instances running on localhost at ports 12913,12914,12915 +* One database named myDB with 6 partitions +* Each partition will have 3 replicas with 1 master, 2 slaves +* zookeeper running locally at localhost:2199 + +##### Create the cluster MYCLUSTER + ## helix-admin.sh --zkSvr --addCluster + ./helix-admin.sh --zkSvr localhost:2199 --addCluster MYCLUSTER + +##### Add nodes to the cluster + +In this case we\'ll add three nodes: localhost:12913, localhost:12914, localhost:12915 + + ## helix-admin.sh --zkSvr --addNode + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12913 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12914 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12915 + +#### Define the resource and partitioning + +In this example, the resource is a database, partitioned 6 ways. (In a production system, it\'s common to over-partition for better load balancing. Helix has been used in production to manage hundreds of databases each with 10s or 100s of partitions running on 10s of physical nodes.) + +##### Create a database with 6 partitions using the MasterSlave state model. + +Helix ensures there will be exactly one master for each partition. + + ## helix-admin.sh --zkSvr --addResource + ./helix-admin.sh --zkSvr localhost:2199 --addResource MYCLUSTER myDB 6 MasterSlave + +##### Now we can let Helix assign partitions to nodes. + +This command will distribute the partitions amongst all the nodes in the cluster. In this example, each partition has 3 replicas. + + ## helix-admin.sh --zkSvr --rebalance + ./helix-admin.sh --zkSvr localhost:2199 --rebalance MYCLUSTER myDB 3 + +Now the cluster is defined in Zookeeper. The nodes (localhost:12913, localhost:12914, localhost:12915) and resource (myDB, with 6 partitions using the MasterSlave model). And the _ideal state_ has been calculated, assuming a replication factor of 3. + +##### Start the Helix Controller + +Now that the cluster is defined in Zookeeper, the Helix controller can manage the cluster. + + ## Start the cluster manager, which will manage MYCLUSTER + ./run-helix-controller.sh --zkSvr localhost:2199 --cluster MYCLUSTER 2>&1 > /tmp/controller.log & + +##### Start up the cluster to be managed + +We\'ve started up Zookeeper, defined the cluster, the resources, the partitioning, and started up the Helix controller. Next, we\'ll start up the nodes of the system to be managed. Each node is a Participant, which is an instance of the system component to be managed. Helix assigns work to Participants, keeps track of their roles and health, and takes action when a node fails. + + # start up each instance. These are mock implementations that are actively managed by Helix + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12913 --stateModelType MasterSlave 2>&1 > /tmp/participant_12913.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12914 --stateModelType MasterSlave 2>&1 > /tmp/participant_12914.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12915 --stateModelType MasterSlave 2>&1 > /tmp/participant_12915.log + + +#### Inspect the Cluster + +Now, let\'s see the Helix view of our cluster. We\'ll work our way down as follows: + +``` +Clusters -> MYCLUSTER -> instances -> instance detail + -> resources -> resource detail + -> partitions +``` + +A single Helix controller can manage multiple clusters, though so far, we\'ve only defined one cluster. Let\'s see: + +``` +## List existing clusters +./helix-admin.sh --zkSvr localhost:2199 --listClusters + +Existing clusters: +MYCLUSTER +``` + +Now, let\'s see the Helix view of MYCLUSTER + +``` +## helix-admin.sh --zkSvr --listClusterInfo +./helix-admin.sh --zkSvr localhost:2199 --listClusterInfo MYCLUSTER + +Existing resources in cluster MYCLUSTER: +myDB +Instances in cluster MYCLUSTER: +localhost_12915 +localhost_12914 +localhost_12913 +``` + + +Let\'s look at the details of an instance + +``` +## ./helix-admin.sh --zkSvr --listInstanceInfo +./helix-admin.sh --zkSvr localhost:2199 --listInstanceInfo MYCLUSTER localhost_12913 + +InstanceConfig: { + "id" : "localhost_12913", + "mapFields" : { + }, + "listFields" : { + }, + "simpleFields" : { + "HELIX_ENABLED" : "true", + "HELIX_HOST" : "localhost", + "HELIX_PORT" : "12913" + } +} +``` + + +##### Query info of a resource + +``` +## helix-admin.sh --zkSvr --listResourceInfo +./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB + +IdealState for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + }, + "myDB_1" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_3" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_4" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + "myDB_0" : [ "localhost_12914", "localhost_12913", "localhost_12915" ], + "myDB_1" : [ "localhost_12915", "localhost_12913", "localhost_12914" ], + "myDB_2" : [ "localhost_12913", "localhost_12915", "localhost_12914" ], + "myDB_3" : [ "localhost_12915", "localhost_12913", "localhost_12914" ], + "myDB_4" : [ "localhost_12913", "localhost_12914", "localhost_12915" ], + "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ] + }, + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "6", + "REPLICAS" : "3", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + } +} + +ExternalView for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + }, + "myDB_1" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_3" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_4" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + }, + "simpleFields" : { + "BUCKET_SIZE" : "0" + } +} +``` + +Now, let\'s look at one of the partitions: + + ## helix-admin.sh --zkSvr --listPartitionInfo + ./helix-admin.sh --zkSvr localhost:2199 --listPartitionInfo MYCLUSTER myDB myDB_0 + +#### Expand the Cluster + +Next, we\'ll show how Helix does the work that you\'d otherwise have to build into your system. When you add capacity to your cluster, you want the work to be evenly distributed. In this example, we started with 3 nodes, with 6 partitions. The partitions were evenly balanced, 2 masters and 4 slaves per node. Let\'s add 3 more nodes: localhost:12916, localhost:12917, localhost:12918 + + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12916 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12917 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12918 + +And start up these instances: + + # start up each instance. These are mock implementations that are actively managed by Helix + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12916 --stateModelType MasterSlave 2>&1 > /tmp/participant_12916.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12917 --stateModelType MasterSlave 2>&1 > /tmp/participant_12917.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12918 --stateModelType MasterSlave 2>&1 > /tmp/participant_12918.log + + +And now, let Helix do the work for you. To shift the work, simply rebalance. After the rebalance, each node will have one master and two slaves. + + ./helix-admin.sh --zkSvr localhost:2199 --rebalance MYCLUSTER myDB 3 + +#### View the cluster + +OK, let\'s see how it looks: + + +``` +./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB + +IdealState for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "SLAVE", + "localhost_12918" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + "myDB_0" : [ "localhost_12917", "localhost_12913", "localhost_12914" ], + "myDB_1" : [ "localhost_12918", "localhost_12917", "localhost_12916" ], + "myDB_2" : [ "localhost_12913", "localhost_12917", "localhost_12918" ], + "myDB_3" : [ "localhost_12915", "localhost_12917", "localhost_12918" ], + "myDB_4" : [ "localhost_12916", "localhost_12917", "localhost_12918" ], + "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ] + }, + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "6", + "REPLICAS" : "3", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + } +} + +ExternalView for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "SLAVE", + "localhost_12918" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + }, + "simpleFields" : { + "BUCKET_SIZE" : "0" + } +} +``` + +Mission accomplished. The partitions are nicely balanced. + +#### How about Failover? + +Building a fault tolerant system isn\'t trivial, but with Helix, it\'s easy. Helix detects a failed instance, and triggers mastership transfer automatically. + +First, let's fail an instance. In this example, we\'ll kill localhost:12918 to simulate a failure. + +We lost localhost:12918, so myDB_1 lost its MASTER. Helix can fix that, it will transfer mastership to a healthy node that is currently a SLAVE, say localhost:12197. Helix balances the load as best as it can, given there are 6 partitions on 5 nodes. Let\'s see: + + +``` +./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB + +IdealState for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "SLAVE", + "localhost_12918" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + "myDB_0" : [ "localhost_12917", "localhost_12913", "localhost_12914" ], + "myDB_1" : [ "localhost_12918", "localhost_12917", "localhost_12916" ], + "myDB_2" : [ "localhost_12913", "localhost_12918", "localhost_12917" ], + "myDB_3" : [ "localhost_12915", "localhost_12918", "localhost_12917" ], + "myDB_4" : [ "localhost_12916", "localhost_12917", "localhost_12918" ], + "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ] + }, + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "6", + "REPLICAS" : "3", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + } +} + +ExternalView for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + }, + "simpleFields" : { + "BUCKET_SIZE" : "0" + } +} +``` + +As we\'ve seen in this Quickstart, Helix takes care of partitioning, load balancing, elasticity, failure detection and recovery. + +##### ZooInspector + +You can view all of the underlying data by going direct to zookeeper. Use ZooInspector that comes with zookeeper to browse the data. This is a java applet (make sure you have X windows) + +To start zooinspector run the following command from /contrib/ZooInspector + + java -cp zookeeper-3.3.3-ZooInspector.jar:lib/jtoaster-1.0.4.jar:../../lib/log4j-1.2.15.jar:../../zookeeper-3.3.3.jar org.apache.zookeeper.inspector.ZooInspector + +#### Next + +Now that you understand the idea of Helix, read the [tutorial](./tutorial.html) to learn how to choose the right state model and constraints for your system, and how to implement it. In many cases, the built-in features meet your requirements. And best of all, Helix is a customizable framework, so you can plug in your own behavior, while retaining the automation provided by Helix. + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/Tutorial.md b/site-releases/0.6.2-incubating/src/site/markdown/Tutorial.md new file mode 100644 index 0000000000..61221b71d4 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/Tutorial.md @@ -0,0 +1,205 @@ + + + + Tutorial + + +# Helix Tutorial + +In this tutorial, we will cover the roles of a Helix-managed cluster, and show the code you need to write to integrate with it. In many cases, there is a simple default behavior that is often appropriate, but you can also customize the behavior. + +Convention: we first cover the _basic_ approach, which is the easiest to implement. Then, we'll describe _advanced_ options, which give you more control over the system behavior, but require you to write more code. + + +### Prerequisites + +1. Read [Concepts/Terminology](./Concepts.html) and [Architecture](./Architecture.html) +2. Read the [Quickstart guide](./Quickstart.html) to learn how Helix models and manages a cluster +3. Install Helix source. See: [Quickstart](./Quickstart.html) for the steps. + +### Tutorial Outline + +1. [Participant](./tutorial_participant.html) +2. [Spectator](./tutorial_spectator.html) +3. [Controller](./tutorial_controller.html) +4. [Rebalancing Algorithms](./tutorial_rebalance.html) +5. [User-Defined Rebalancing](./tutorial_user_def_rebalancer.html) +6. [State Machines](./tutorial_state.html) +7. [Messaging](./tutorial_messaging.html) +8. [Customized health check](./tutorial_health.html) +9. [Throttling](./tutorial_throttling.html) +10. [Application Property Store](./tutorial_propstore.html) +11. [Admin Interface](./tutorial_admin.html) +12. [YAML Cluster Setup](./tutorial_yaml.html) + +### Preliminaries + +First, we need to set up the system. Let\'s walk through the steps in building a distributed system using Helix. + +### Start Zookeeper + +This starts a zookeeper in standalone mode. For production deployment, see [Apache Zookeeper](http://zookeeper.apache.org) for instructions. + +``` + ./start-standalone-zookeeper.sh 2199 & +``` + +### Create a cluster + +Creating a cluster will define the cluster in appropriate znodes on zookeeper. + +Using the java API: + +``` + // Create setup tool instance + // Note: ZK_ADDRESS is the host:port of Zookeeper + String ZK_ADDRESS = "localhost:2199"; + admin = new ZKHelixAdmin(ZK_ADDRESS); + + String CLUSTER_NAME = "helix-demo"; + //Create cluster namespace in zookeeper + admin.addCluster(CLUSTER_NAME); +``` + +OR + +Using the command-line interface: + +``` + ./helix-admin.sh --zkSvr localhost:2199 --addCluster helix-demo +``` + + +### Configure the nodes of the cluster + +First we\'ll add new nodes to the cluster, then configure the nodes in the cluster. Each node in the cluster must be uniquely identifiable. +The most commonly used convention is hostname:port. + +``` + String CLUSTER_NAME = "helix-demo"; + int NUM_NODES = 2; + String hosts[] = new String[]{"localhost","localhost"}; + String ports[] = new String[]{7000,7001}; + for (int i = 0; i < NUM_NODES; i++) + { + + InstanceConfig instanceConfig = new InstanceConfig(hosts[i]+ "_" + ports[i]); + instanceConfig.setHostName(hosts[i]); + instanceConfig.setPort(ports[i]); + instanceConfig.setInstanceEnabled(true); + + //Add additional system specific configuration if needed. These can be accessed during the node start up. + instanceConfig.getRecord().setSimpleField("key", "value"); + admin.addInstance(CLUSTER_NAME, instanceConfig); + + } +``` + +### Configure the resource + +A _resource_ represents the actual task performed by the nodes. It can be a database, index, topic, queue or any other processing entity. +A _resource_ can be divided into many sub-parts known as _partitions_. + + +#### Define the _state model_ and _constraints_ + +For scalability and fault tolerance, each partition can have one or more replicas. +The _state model_ allows one to declare the system behavior by first enumerating the various STATES, and the TRANSITIONS between them. +A simple model is ONLINE-OFFLINE where ONLINE means the task is active and OFFLINE means it\'s not active. +You can also specify how many replicas must be in each state, these are known as _constraints_. +For example, in a search system, one might need more than one node serving the same index to handle the load. + +The allowed states: + +* MASTER +* SLAVE +* OFFLINE + +The allowed transitions: + +* OFFLINE to SLAVE +* SLAVE to OFFLINE +* SLAVE to MASTER +* MASTER to SLAVE + +The constraints: + +* no more than 1 MASTER per partition +* the rest of the replicas should be slaves + +The following snippet shows how to declare the _state model_ and _constraints_ for the MASTER-SLAVE model. + +``` + + StateModelDefinition.Builder builder = new StateModelDefinition.Builder(STATE_MODEL_NAME); + + // Add states and their rank to indicate priority. A lower rank corresponds to a higher priority + builder.addState(MASTER, 1); + builder.addState(SLAVE, 2); + builder.addState(OFFLINE); + + // Set the initial state when the node starts + builder.initialState(OFFLINE); + + // Add transitions between the states. + builder.addTransition(OFFLINE, SLAVE); + builder.addTransition(SLAVE, OFFLINE); + builder.addTransition(SLAVE, MASTER); + builder.addTransition(MASTER, SLAVE); + + // set constraints on states. + + // static constraint: upper bound of 1 MASTER + builder.upperBound(MASTER, 1); + + // dynamic constraint: R means it should be derived based on the replication factor for the cluster + // this allows a different replication factor for each resource without + // having to define a new state model + // + builder.dynamicUpperBound(SLAVE, "R"); + + StateModelDefinition statemodelDefinition = builder.build(); + admin.addStateModelDef(CLUSTER_NAME, STATE_MODEL_NAME, myStateModel); +``` + +#### Assigning partitions to nodes + +The final goal of Helix is to ensure that the constraints on the state model are satisfied. +Helix does this by assigning a STATE to a partition (such as MASTER, SLAVE), and placing it on a particular node. + +There are 3 assignment modes Helix can operate on + +* FULL_AUTO: Helix decides the placement and state of a partition. +* SEMI_AUTO: Application decides the placement but Helix decides the state of a partition. +* CUSTOMIZED: Application controls the placement and state of a partition. + +For more info on the assignment modes, see [Rebalancing Algorithms](./tutorial_rebalance.html) section of the tutorial. + +``` + String RESOURCE_NAME = "MyDB"; + int NUM_PARTITIONS = 6; + STATE_MODEL_NAME = "MasterSlave"; + String MODE = "SEMI_AUTO"; + int NUM_REPLICAS = 2; + + admin.addResource(CLUSTER_NAME, RESOURCE_NAME, NUM_PARTITIONS, STATE_MODEL_NAME, MODE); + admin.rebalance(CLUSTER_NAME, RESOURCE_NAME, NUM_REPLICAS); +``` + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/index.md b/site-releases/0.6.2-incubating/src/site/markdown/index.md new file mode 100644 index 0000000000..a09a70d562 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/index.md @@ -0,0 +1,58 @@ + + + + Home + + +Navigating the Documentation +---------------------------- + +### Conceptual Understanding + +[Concepts / Terminology](./Concepts.html) + +[Architecture](./Architecture.html) + +### Hands-on Helix + +[Getting Helix](./Building.html) + +[Quickstart](./Quickstart.html) + +[Tutorial](./Tutorial.html) + +[Javadocs](http://helix.incubator.apache.org/javadocs/0.6.2-incubating/index.html) + +### Recipes + +[Distributed lock manager](./recipes/lock_manager.html) + +[Rabbit MQ consumer group](./recipes/rabbitmq_consumer_group.html) + +[Rsync replicated file store](./recipes/rsync_replicated_file_store.html) + +[Service discovery](./recipes/service_discovery.html) + +[Distributed Task DAG Execution](./recipes/task_dag_execution.html) + +### Download + +[0.6.2-incubating](./download.html) + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/recipes/lock_manager.md b/site-releases/0.6.2-incubating/src/site/markdown/recipes/lock_manager.md new file mode 100644 index 0000000000..252ace7eab --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/recipes/lock_manager.md @@ -0,0 +1,253 @@ + +Distributed lock manager +------------------------ +Distributed locks are used to synchronize accesses shared resources. Most applications use Zookeeper to model the distributed locks. + +The simplest way to model a lock using zookeeper is (See Zookeeper leader recipe for an exact and more advanced solution) + +* Each process tries to create an emphemeral node. +* If can successfully create it then, it acquires the lock +* Else it will watch on the znode and try to acquire the lock again if the current lock holder disappears + +This is good enough if there is only one lock. But in practice, an application will need many such locks. Distributing and managing the locks among difference process becomes challenging. Extending such a solution to many locks will result in + +* Uneven distribution of locks among nodes, the node that starts first will acquire all the lock. Nodes that start later will be idle. +* When a node fails, how the locks will be distributed among remaining nodes is not predicable. +* When new nodes are added the current nodes dont relinquish the locks so that new nodes can acquire some locks + +In other words we want a system to satisfy the following requirements. + +* Distribute locks evenly among all nodes to get better hardware utilization +* If a node fails, the locks that were acquired by that node should be evenly distributed among other nodes +* If nodes are added, locks must be evenly re-distributed among nodes. + +Helix provides a simple and elegant solution to this problem. Simply specify the number of locks and Helix will ensure that above constraints are satisfied. + +To quickly see this working run the lock-manager-demo script where 12 locks are evenly distributed among three nodes, and when a node fails, the locks get re-distributed among remaining two nodes. Note that Helix does not re-shuffle the locks completely, instead it simply distributes the locks relinquished by dead node among 2 remaining nodes evenly. + +---------------------------------------------------------------------------------------- + +#### Short version + This version starts multiple threads with in same process to simulate a multi node deployment. Try the long version to get a better idea of how it works. + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/distributed-lock-manager/target/distributed-lock-manager-pkg/bin +chmod +x * +./lock-manager-demo +``` + +##### Output + +``` +./lock-manager-demo +STARTING localhost_12000 +STARTING localhost_12002 +STARTING localhost_12001 +STARTED localhost_12000 +STARTED localhost_12002 +STARTED localhost_12001 +localhost_12001 acquired lock:lock-group_3 +localhost_12000 acquired lock:lock-group_8 +localhost_12001 acquired lock:lock-group_2 +localhost_12001 acquired lock:lock-group_4 +localhost_12002 acquired lock:lock-group_1 +localhost_12002 acquired lock:lock-group_10 +localhost_12000 acquired lock:lock-group_7 +localhost_12001 acquired lock:lock-group_5 +localhost_12002 acquired lock:lock-group_11 +localhost_12000 acquired lock:lock-group_6 +localhost_12002 acquired lock:lock-group_0 +localhost_12000 acquired lock:lock-group_9 +lockName acquired By +====================================== +lock-group_0 localhost_12002 +lock-group_1 localhost_12002 +lock-group_10 localhost_12002 +lock-group_11 localhost_12002 +lock-group_2 localhost_12001 +lock-group_3 localhost_12001 +lock-group_4 localhost_12001 +lock-group_5 localhost_12001 +lock-group_6 localhost_12000 +lock-group_7 localhost_12000 +lock-group_8 localhost_12000 +lock-group_9 localhost_12000 +Stopping localhost_12000 +localhost_12000 Interrupted +localhost_12001 acquired lock:lock-group_9 +localhost_12001 acquired lock:lock-group_8 +localhost_12002 acquired lock:lock-group_6 +localhost_12002 acquired lock:lock-group_7 +lockName acquired By +====================================== +lock-group_0 localhost_12002 +lock-group_1 localhost_12002 +lock-group_10 localhost_12002 +lock-group_11 localhost_12002 +lock-group_2 localhost_12001 +lock-group_3 localhost_12001 +lock-group_4 localhost_12001 +lock-group_5 localhost_12001 +lock-group_6 localhost_12002 +lock-group_7 localhost_12002 +lock-group_8 localhost_12001 +lock-group_9 localhost_12001 + +``` + +---------------------------------------------------------------------------------------- + +#### Long version +This provides more details on how to setup the cluster and where to plugin application code. + +##### start zookeeper + +``` +./start-standalone-zookeeper 2199 +``` + +##### Create a cluster + +``` +./helix-admin --zkSvr localhost:2199 --addCluster lock-manager-demo +``` + +##### Create a lock group + +Create a lock group and specify the number of locks in the lock group. + +``` +./helix-admin --zkSvr localhost:2199 --addResource lock-manager-demo lock-group 6 OnlineOffline FULL_AUTO +``` + +##### Start the nodes + +Create a Lock class that handles the callbacks. + +``` + +public class Lock extends StateModel +{ + private String lockName; + + public Lock(String lockName) + { + this.lockName = lockName; + } + + public void lock(Message m, NotificationContext context) + { + System.out.println(" acquired lock:"+ lockName ); + } + + public void release(Message m, NotificationContext context) + { + System.out.println(" releasing lock:"+ lockName ); + } + +} + +``` + +LockFactory that creates the lock + +``` +public class LockFactory extends StateModelFactory{ + + /* Instantiates the lock handler, one per lockName*/ + public Lock create(String lockName) + { + return new Lock(lockName); + } +} +``` + +At node start up, simply join the cluster and helix will invoke the appropriate callbacks on Lock instance. One can start any number of nodes and Helix detects that a new node has joined the cluster and re-distributes the locks automatically. + +``` +public class LockProcess{ + + public static void main(String args){ + String zkAddress= "localhost:2199"; + String clusterName = "lock-manager-demo"; + //Give a unique id to each process, most commonly used format hostname_port + String instanceName ="localhost_12000"; + ZKHelixAdmin helixAdmin = new ZKHelixAdmin(zkAddress); + //configure the instance and provide some metadata + InstanceConfig config = new InstanceConfig(instanceName); + config.setHostName("localhost"); + config.setPort("12000"); + admin.addInstance(clusterName, config); + //join the cluster + HelixManager manager; + manager = HelixManagerFactory.getHelixManager(clusterName, + instanceName, + InstanceType.PARTICIPANT, + zkAddress); + manager.getStateMachineEngine().registerStateModelFactory("OnlineOffline", modelFactory); + manager.connect(); + Thread.currentThread.join(); + } + +} +``` + +##### Start the controller + +Controller can be started either as a separate process or can be embedded within each node process + +###### Separate process +This is recommended when number of nodes in the cluster >100. For fault tolerance, you can run multiple controllers on different boxes. + +``` +./run-helix-controller --zkSvr localhost:2199 --cluster lock-manager-demo 2>&1 > /tmp/controller.log & +``` + +###### Embedded within the node process +This is recommended when the number of nodes in the cluster is less than 100. To start a controller from each process, simply add the following lines to MyClass + +``` +public class LockProcess{ + + public static void main(String args){ + String zkAddress= "localhost:2199"; + String clusterName = "lock-manager-demo"; + . + . + manager.connect(); + HelixManager controller; + controller = HelixControllerMain.startHelixController(zkAddress, + clusterName, + "controller", + HelixControllerMain.STANDALONE); + Thread.currentThread.join(); + } +} +``` + +---------------------------------------------------------------------------------------- + + + + + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/recipes/rabbitmq_consumer_group.md b/site-releases/0.6.2-incubating/src/site/markdown/recipes/rabbitmq_consumer_group.md new file mode 100644 index 0000000000..9edc2cb242 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/recipes/rabbitmq_consumer_group.md @@ -0,0 +1,227 @@ + + + +RabbitMQ Consumer Group +======================= + +[RabbitMQ](http://www.rabbitmq.com/) is a well known Open source software the provides robust messaging for applications. + +One of the commonly implemented recipes using this software is a work queue. http://www.rabbitmq.com/tutorials/tutorial-four-java.html describes the use case where + +* A producer sends a message with a routing key. +* The message is routed to the queue whose binding key exactly matches the routing key of the message. +* There are multiple consumers and each consumer is interested in processing only a subset of the messages by binding to the interested keys + +The example provided [here](http://www.rabbitmq.com/tutorials/tutorial-four-java.html) describes how multiple consumers can be started to process all the messages. + +While this works, in production systems one needs the following + +* Ability to handle failures: when a consumers fails another consumer must be started or the other consumers must start processing these messages that should have been processed by the failed consumer. +* When the existing consumers cannot keep up with the task generation rate, new consumers will be added. The tasks must be redistributed among all the consumers. + +In this recipe, we demonstrate handling of consumer failures and new consumer additions using Helix. + +Mapping this usecase to Helix is pretty easy as the binding key/routing key is equivalent to a partition. + +Let's take an example. Lets say the queue has 6 partitions, and we have 2 consumers to process all the queues. +What we want is all 6 queues to be evenly divided among 2 consumers. +Eventually when the system scales, we add more consumers to keep up. This will make each consumer process tasks from 2 queues. +Now let's say that a consumer failed which reduces the number of active consumers to 2. This means each consumer must process 3 queues. + +We showcase how such a dynamic App can be developed using Helix. Even though we use rabbitmq as the pub/sub system one can extend this solution to other pub/sub systems. + +Try it +====== + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/rabbitmq-consumer-group/bin +chmod +x * +export HELIX_PKG_ROOT=`pwd`/helix-core/target/helix-core-pkg +export HELIX_RABBITMQ_ROOT=`pwd`/recipes/rabbitmq-consumer-group/ +chmod +x $HELIX_PKG_ROOT/bin/* +chmod +x $HELIX_RABBITMQ_ROOT/bin/* +``` + + +Install Rabbit MQ +---------------- + +Setting up RabbitMQ on a local box is straightforward. You can find the instructions here +http://www.rabbitmq.com/download.html + +Start ZK +-------- +Start zookeeper at port 2199 + +``` +$HELIX_PKG_ROOT/bin/start-standalone-zookeeper 2199 +``` + +Setup the consumer group cluster +-------------------------------- +This will setup the cluster by creating a "rabbitmq-consumer-group" cluster and adds a "topic" with "6" queues. + +``` +$HELIX_RABBITMQ_ROOT/bin/setup-cluster.sh localhost:2199 +``` + +Add consumers +------------- +Start 2 consumers in 2 different terminals. Each consumer is given a unique id. + +``` +//start-consumer.sh zookeeperAddress (e.g. localhost:2181) consumerId , rabbitmqServer (e.g. localhost) +$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 0 localhost +$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 1 localhost + +``` + +Start HelixController +-------------------- +Now start a Helix controller that starts managing the "rabbitmq-consumer-group" cluster. + +``` +$HELIX_RABBITMQ_ROOT/bin/start-cluster-manager.sh localhost:2199 +``` + +Send messages to the Topic +-------------------------- + +Start sending messages to the topic. This script randomly selects a routing key (1-6) and sends the message to topic. +Based on the key, messages gets routed to the appropriate queue. + +``` +$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 20 +``` + +After running this, you should see all 20 messages being processed by 2 consumers. + +Add another consumer +-------------------- +Once a new consumer is started, helix detects it. In order to balance the load between 3 consumers, it deallocates 1 partition from the existing consumers and allocates it to the new consumer. We see that +each consumer is now processing only 2 queues. +Helix makes sure that old nodes are asked to stop consuming before the new consumer is asked to start consuming for a given partition. But the transitions for each partition can happen in parallel. + +``` +$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 2 localhost +``` + +Send messages again to the topic. + +``` +$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 100 +``` + +You should see that messages are now received by all 3 consumers. + +Stop a consumer +--------------- +In any terminal press CTRL^C and notice that Helix detects the consumer failure and distributes the 2 partitions that were processed by failed consumer to the remaining 2 active consumers. + + +How does it work +================ + +Find the entire code [here](https://git-wip-us.apache.org/repos/asf?p=incubator-helix.git;a=tree;f=recipes/rabbitmq-consumer-group/src/main/java/org/apache/helix/recipes/rabbitmq). + +Cluster setup +------------- +This step creates znode on zookeeper for the cluster and adds the state model. We use online offline state model since there is no need for other states. The consumer is either processing a queue or it is not. + +It creates a resource called "rabbitmq-consumer-group" with 6 partitions. The execution mode is set to FULL_AUTO. This means that the Helix controls the assignment of partition to consumers and automatically distributes the partitions evenly among the active consumers. When a consumer is added or removed, it ensures that a minimum number of partitions are shuffled. + +``` + zkclient = new ZkClient(zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT, + ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer()); + ZKHelixAdmin admin = new ZKHelixAdmin(zkclient); + + // add cluster + admin.addCluster(clusterName, true); + + // add state model definition + StateModelConfigGenerator generator = new StateModelConfigGenerator(); + admin.addStateModelDef(clusterName, "OnlineOffline", + new StateModelDefinition(generator.generateConfigForOnlineOffline())); + + // add resource "topic" which has 6 partitions + String resourceName = "rabbitmq-consumer-group"; + admin.addResource(clusterName, resourceName, 6, "OnlineOffline", "FULL_AUTO"); +``` + +Starting the consumers +---------------------- +The only thing consumers need to know is the zkaddress, cluster name and consumer id. It does not need to know anything else. + +``` + _manager = + HelixManagerFactory.getZKHelixManager(_clusterName, + _consumerId, + InstanceType.PARTICIPANT, + _zkAddr); + + StateMachineEngine stateMach = _manager.getStateMachineEngine(); + ConsumerStateModelFactory modelFactory = + new ConsumerStateModelFactory(_consumerId, _mqServer); + stateMach.registerStateModelFactory("OnlineOffline", modelFactory); + + _manager.connect(); + +``` + +Once the consumer has registered the statemodel and the controller is started, the consumer starts getting callbacks (onBecomeOnlineFromOffline) for the partition it needs to host. All it needs to do as part of the callback is to start consuming messages from the appropriate queue. Similarly, when the controller deallocates a partitions from a consumer, it fires onBecomeOfflineFromOnline for the same partition. +As a part of this transition, the consumer will stop consuming from a that queue. + +``` + @Transition(to = "ONLINE", from = "OFFLINE") + public void onBecomeOnlineFromOffline(Message message, NotificationContext context) + { + LOG.debug(_consumerId + " becomes ONLINE from OFFLINE for " + _partition); + + if (_thread == null) + { + LOG.debug("Starting ConsumerThread for " + _partition + "..."); + _thread = new ConsumerThread(_partition, _mqServer, _consumerId); + _thread.start(); + LOG.debug("Starting ConsumerThread for " + _partition + " done"); + + } + } + + @Transition(to = "OFFLINE", from = "ONLINE") + public void onBecomeOfflineFromOnline(Message message, NotificationContext context) + throws InterruptedException + { + LOG.debug(_consumerId + " becomes OFFLINE from ONLINE for " + _partition); + + if (_thread != null) + { + LOG.debug("Stopping " + _consumerId + " for " + _partition + "..."); + + _thread.interrupt(); + _thread.join(2000); + _thread = null; + LOG.debug("Stopping " + _consumerId + " for " + _partition + " done"); + + } + } +``` \ No newline at end of file diff --git a/site-releases/0.6.2-incubating/src/site/markdown/recipes/rsync_replicated_file_store.md b/site-releases/0.6.2-incubating/src/site/markdown/recipes/rsync_replicated_file_store.md new file mode 100644 index 0000000000..f8a74a0abc --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/recipes/rsync_replicated_file_store.md @@ -0,0 +1,165 @@ + + +Near real time rsync replicated file system +=========================================== + +Quickdemo +--------- + +* This demo starts 3 instances with id's as ```localhost_12001, localhost_12002, localhost_12003``` +* Each instance stores its files under ```/tmp//filestore``` +* ``` localhost_12001 ``` is designated as the master and ``` localhost_12002 and localhost_12003``` are the slaves. +* Files written to master are replicated to the slaves automatically. In this demo, a.txt and b.txt are written to ```/tmp/localhost_12001/filestore``` and it gets replicated to other folders. +* When the master is stopped, ```localhost_12002``` is promoted to master. +* The other slave ```localhost_12003``` stops replicating from ```localhost_12001``` and starts replicating from new master ```localhost_12002``` +* Files written to new master ```localhost_12002``` are replicated to ```localhost_12003``` +* In the end state of this quick demo, ```localhost_12002``` is the master and ```localhost_12003``` is the slave. Manually create files under ```/tmp/localhost_12002/filestore``` and see that appears in ```/tmp/localhost_12003/filestore``` +* Ignore the interrupted exceptions on the console :-). + + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd recipes/rsync-replicated-file-system/ +mvn clean install package -DskipTests +cd target/rsync-replicated-file-system-pkg/bin +chmod +x * +./quickdemo + +``` + +Overview +-------- + +There are many applications that require storage for storing large number of relatively small data files. Examples include media stores to store small videos, images, mail attachments etc. Each of these objects is typically kilobytes, often no larger than a few megabytes. An additional distinguishing feature of these usecases is also that files are typically only added or deleted, rarely updated. When there are updates, they are rare and do not have any concurrency requirements. + +These are much simpler requirements than what general purpose distributed file system have to satisfy including concurrent access to files, random access for reads and updates, posix compliance etc. To satisfy those requirements, general DFSs are also pretty complex that are expensive to build and maintain. + +A different implementation of a distributed file system includes HDFS which is inspired by Google's GFS. This is one of the most widely used distributed file system that forms the main data storage platform for Hadoop. HDFS is primary aimed at processing very large data sets and distributes files across a cluster of commodity servers by splitting up files in fixed size chunks. HDFS is not particularly well suited for storing a very large number of relatively tiny files. + +### File Store + +It's possible to build a vastly simpler system for the class of applications that have simpler requirements as we have pointed out. + +* Large number of files but each file is relatively small. +* Access is limited to create, delete and get entire files. +* No updates to files that are already created (or it's feasible to delete the old file and create a new one). + + +We call this system a Partitioned File Store (PFS) to distinguish it from other distributed file systems. This system needs to provide the following features: + +* CRD access to large number of small files +* Scalability: Files should be distributed across a large number of commodity servers based on the storage requirement. +* Fault-tolerance: Each file should be replicated on multiple servers so that individual server failures do not reduce availability. +* Elasticity: It should be possible to add capacity to the cluster easily. + + +Apache Helix is a generic cluster management framework that makes it very easy to provide the scalability, fault-tolerance and elasticity features. +Rsync can be easily used as a replication channel between servers so that each file gets replicated on multiple servers. + +Design +------ + +High level + +* Partition the file system based on the file name. +* At any time a single writer can write, we call this a master. +* For redundancy, we need to have additional replicas called slave. Slaves can optionally serve reads. +* Slave replicates data from the master. +* When a master fails, slave gets promoted to master. + +### Transaction log + +Every write on the master will result in creation/deletion of one or more files. In order to maintain timeline consistency slaves need to apply the changes in the same order. +To facilitate this, the master logs each transaction in a file and each transaction is associated with an 64 bit id in which the 32 LSB represents a sequence number and MSB represents the generation number. +Sequence gets incremented on every transaction and and generation is increment when a new master is elected. + +### Replication + +Replication is required to slave to keep up with the changes on the master. Every time the slave applies a change it checkpoints the last applied transaction id. +During restarts, this allows the slave to pull changes from the last checkpointed id. Similar to master, the slave logs each transaction to the transaction logs but instead of generating new transaction id, it uses the same id generated by the master. + + +### Fail over + +When a master fails, a new slave will be promoted to master. If the prev master node is reachable, then the new master will flush all the +changes from previous master before taking up mastership. The new master will record the end transaction id of the current generation and then starts new generation +with sequence starting from 1. After this the master will begin accepting writes. + + +![Partitioned File Store](../images/PFS-Generic.png) + + + +Rsync based solution +------------------- + +![Rsync based File Store](../images/RSYNC_BASED_PFS.png) + + +This application demonstrate a file store that uses rsync as the replication mechanism. One can envision a similar system where instead of using rsync, +can implement a custom solution to notify the slave of the changes and also provide an api to pull the change files. +#### Concept +* file_store_dir: Root directory for the actual data files +* change_log_dir: The transaction logs are generated under this folder. +* check_point_dir: The slave stores the check points ( last processed transaction) here. + +#### Master +* File server: This component support file uploads and downloads and writes the files to ```file_store_dir```. This is not included in this application. Idea is that most applications have different ways of implementing this component and has some business logic associated with it. It is not hard to come up with such a component if needed. +* File store watcher: This component watches the ```file_store_dir``` directory on the local file system for any changes and notifies the registered listeners of the changes. +* Change Log Generator: This registers as a listener of File System Watcher and on each notification logs the changes into a file under ```change_log_dir```. + +####Slave +* File server: This component on the slave will only support reads. +* Cluster state observer: Slave observes the cluster state and is able to know who is the current master. +* Replicator: This has two subcomponents + - Periodic rsync of change log: This is a background process that periodically rsyncs the ```change_log_dir``` of the master to its local directory + - Change Log Watcher: This watches the ```change_log_dir``` for changes and notifies the registered listeners of the change + - On demand rsync invoker: This is registered as a listener to change log watcher and on every change invokes rsync to sync only the changed file. + + +#### Coordination + +The coordination between nodes is done by Helix. Helix does the partition management and assigns the partition to multiple nodes based on the replication factor. It elects one the nodes as master and designates others as slaves. +It provides notifications to each node in the form of state transitions ( Offline to Slave, Slave to Master). It also provides notification when there is change is cluster state. +This allows the slave to stop replicating from current master and start replicating from new master. + +In this application, we have only one partition but its very easy to extend it to support multiple partitions. By partitioning the file store, one can add new nodes and Helix will automatically +re-distribute partitions among the nodes. To summarize, Helix provides partition management, fault tolerance and facilitates automated cluster expansion. + + + + + + + + + + + + + + + + + + + + + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/recipes/service_discovery.md b/site-releases/0.6.2-incubating/src/site/markdown/recipes/service_discovery.md new file mode 100644 index 0000000000..8e06ead472 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/recipes/service_discovery.md @@ -0,0 +1,191 @@ + +Service Discovery +----------------- + +One of the common usage of zookeeper is enable service discovery. +The basic idea is that when a server starts up it advertises its configuration/metadata such as host name port etc on zookeeper. +This allows clients to dynamically discover the servers that are currently active. One can think of this like a service registry to which a server registers when it starts and +is automatically deregistered when it shutdowns or crashes. In many cases it serves as an alternative to vips. + +The core idea behind this is to use zookeeper ephemeral nodes. The ephemeral nodes are created when the server registers and all its metadata is put into a znode. +When the server shutdowns, zookeeper automatically removes this znode. + +There are two ways the clients can dynamically discover the active servers + +#### ZOOKEEPER WATCH + +Clients can set a child watch under specific path on zookeeper. +When a new service is registered/deregistered, zookeeper notifies the client via watchevent and the client can read the list of services. Even though this looks trivial, +there are lot of things one needs to keep in mind like ensuring that you first set the watch back on zookeeper before reading data from zookeeper. + + +#### POLL + +Another approach is for the client to periodically read the zookeeper path and get the list of services. + + +Both approaches have pros and cons, for example setting a watch might trigger herd effect if there are large number of clients. This is worst especially when servers are starting up. +But good thing about setting watch is that clients are immediately notified of a change which is not true in case of polling. +In some cases, having both WATCH and POLL makes sense, WATCH allows one to get notifications as soon as possible while POLL provides a safety net if a watch event is missed because of code bug or zookeeper fails to notify. + +##### Other important scenarios to take care of +* What happens when zookeeper session expires. All the watches/ephemeral nodes previously added/created by this server are lost. +One needs to add the watches again , recreate the ephemeral nodes etc. +* Due to network issues or java GC pauses session expiry might happen again and again also known as flapping. Its important for the server to detect this and deregister itself. + +##### Other operational things to consider +* What if the node is behaving badly, one might kill the server but will lose the ability to debug. +It would be nice to have the ability to mark a server as disabled and clients know that a node is disabled and will not contact that node. + +#### Configuration ownership + +This is an important aspect that is often ignored in the initial stages of your development. In common, service discovery pattern means that servers start up with some configuration and then simply puts its configuration/metadata in zookeeper. While this works well in the beginning, +configuration management becomes very difficult since the servers themselves are statically configured. Any change in server configuration implies restarting of the server. Ideally, it will be nice to have the ability to change configuration dynamically without having to restart a server. + +Ideally you want a hybrid solution, a node starts with minimal configuration and gets the rest of configuration from zookeeper. + +h3. How to use Helix to achieve this + +Even though Helix has higher level abstraction in terms of statemachine, constraints and objectives, +service discovery is one of things that existed since we started. +The controller uses the exact mechanism we described above to discover when new servers join the cluster. +We create these znodes under /CLUSTERNAME/LIVEINSTANCES. +Since at any time there is only one controller, we use ZK watch to track the liveness of a server. + +This recipe, simply demonstrate how one can re-use that part for implementing service discovery. This demonstrates multiple MODE's of service discovery + +* POLL: The client reads from zookeeper at regular intervals 30 seconds. Use this if you have 100's of clients +* WATCH: The client sets up watcher and gets notified of the changes. Use this if you have 10's of clients. +* NONE: This does neither of the above, but reads directly from zookeeper when ever needed. + +Helix provides these additional features compared to other implementations available else where + +* It has the concept of disabling a node which means that a badly behaving node, can be disabled using helix admin api. +* It automatically detects if a node connects/disconnects from zookeeper repeatedly and disables the node. +* Configuration management + * Allows one to set configuration via admin api at various granulaties like cluster, instance, resource, partition + * Configuration can be dynamically changed. + * Notifies the server when configuration changes. + + +##### checkout and build + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/service-discovery/target/service-discovery-pkg/bin +chmod +x * +``` + +##### start zookeeper + +``` +./start-standalone-zookeeper 2199 +``` + +#### Run the demo + +``` +./service-discovery-demo.sh +``` + +#### Output + +``` +START:Service discovery demo mode:WATCH + Registering service + host.x.y.z_12000 + host.x.y.z_12001 + host.x.y.z_12002 + host.x.y.z_12003 + host.x.y.z_12004 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Deregistering service: + host.x.y.z_12002 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Registering service:host.x.y.z_12002 +END:Service discovery demo mode:WATCH +============================================= +START:Service discovery demo mode:POLL + Registering service + host.x.y.z_12000 + host.x.y.z_12001 + host.x.y.z_12002 + host.x.y.z_12003 + host.x.y.z_12004 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Deregistering service: + host.x.y.z_12002 + Sleeping for poll interval:30000 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Registering service:host.x.y.z_12002 +END:Service discovery demo mode:POLL +============================================= +START:Service discovery demo mode:NONE + Registering service + host.x.y.z_12000 + host.x.y.z_12001 + host.x.y.z_12002 + host.x.y.z_12003 + host.x.y.z_12004 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Deregistering service: + host.x.y.z_12000 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Registering service:host.x.y.z_12000 +END:Service discovery demo mode:NONE +============================================= + +``` + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/recipes/task_dag_execution.md b/site-releases/0.6.2-incubating/src/site/markdown/recipes/task_dag_execution.md new file mode 100644 index 0000000000..f0474e445e --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/recipes/task_dag_execution.md @@ -0,0 +1,204 @@ + + +# Distributed task execution + + +This recipe is intended to demonstrate how task dependencies can be modeled using primitives provided by Helix. A given task can be run with desired parallelism and will start only when up-stream dependencies are met. The demo executes the task DAG described below using 10 workers. Although the demo starts the workers as threads, there is no requirement that all the workers need to run in the same process. In reality, these workers run on many different boxes on a cluster. When worker fails, Helix takes care of +re-assigning a failed task partition to a new worker. + +Redis is used as a result store. Any other suitable implementation for TaskResultStore can be plugged in. + +### Workflow + + +#### Input + +10000 impression events and around 100 click events are pre-populated in task result store (redis). + +* **ImpEvent**: format: id,isFraudulent,country,gender + +* **ClickEvent**: format: id,isFraudulent,impEventId + +#### Stages + ++ **FilterImps**: Filters impression where isFraudulent=true. + ++ **FilterClicks**: Filters clicks where isFraudulent=true + ++ **impCountsByGender**: Generates impression counts grouped by gender. It does this by incrementing the count for 'impression_gender_counts:' in the task result store (redis hash). Depends on: **FilterImps** + ++ **impCountsByCountry**: Generates impression counts grouped by country. It does this by incrementing the count for 'impression_country_counts:' in the task result store (redis hash). Depends on: **FilterClicks** + ++ **impClickJoin**: Joins clicks with corresponding impression event using impEventId as the join key. Join is needed to pull dimensions not present in click event. Depends on: **FilterImps, FilterClicks** + ++ **clickCountsByGender**: Generates click counts grouped by gender. It does this by incrementing the count for click_gender_counts: in the task result store (redis hash). Depends on: **impClickJoin** + ++ **clickCountsByGender**: Generates click counts grouped by country. It does this by incrementing the count for click_country_counts: in the task result store (redis hash). Depends on: **impClickJoin** + ++ **report**: Reads from all aggregates generated by previous stages and prints them. Depends on: **impCountsByGender, impCountsByCountry, clickCountsByGender,clickCountsByGender** + + +### Creating DAG + +Each stage is represented as a Node along with the upstream dependency and desired parallelism. Each stage is modelled as a resource in Helix using OnlineOffline state model. As part of Offline to Online transition, we watch the external view of upstream resources and wait for them to transition to online state. See Task.java for additional info. + +``` + + Dag dag = new Dag(); + dag.addNode(new Node("filterImps", 10, "")); + dag.addNode(new Node("filterClicks", 5, "")); + dag.addNode(new Node("impClickJoin", 10, "filterImps,filterClicks")); + dag.addNode(new Node("impCountsByGender", 10, "filterImps")); + dag.addNode(new Node("impCountsByCountry", 10, "filterImps")); + dag.addNode(new Node("clickCountsByGender", 5, "impClickJoin")); + dag.addNode(new Node("clickCountsByCountry", 5, "impClickJoin")); + dag.addNode(new Node("report",1,"impCountsByGender,impCountsByCountry,clickCountsByGender,clickCountsByCountry")); + + +``` + +### DEMO + +In order to run the demo, use the following steps + +See http://redis.io/topics/quickstart on how to install redis server + +``` + +Start redis e.g: +./redis-server --port 6379 + +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd recipes/task-execution +mvn clean install package -DskipTests +cd target/task-execution-pkg/bin +chmod +x task-execution-demo.sh +./task-execution-demo.sh 2181 localhost 6379 + +``` + +``` + + + + + + +-----------------+ +----------------+ + | filterImps | | filterClicks | + | (parallelism=10)| | (parallelism=5)| + +----------+-----++ +-------+--------+ + | | | | + | | | | + | | | | + | | +------->--------v------------+ + +--------------<-+ +------v-------+ | impClickJoin | + |impCountsByGender |impCountsByCountry | (parallelism=10) | + |(parallelism=10) |(parallelism=10) ++-------------------+-+ + +-----------+--+ +---+----------+ | | + | | | | + | | | | + | | +--------v---------+ +-v-------------------+ + | | |clickCountsByGender |clickCountsByCountry | + | | |(parallelism=5) | |(parallelism=5) | + | | +----+-------------+ +---------------------+ + | | | | + | | | | + | | | | + +----->+-----+>-----------v----+<---------------+ + | report | + |(parallelism=1) | + +-----------------------+ + +``` + +(credit for above ascii art: http://www.asciiflow.com) + +### OUTPUT + +``` +Done populating dummy data +Executing filter task for filterImps_3 for impressions_demo +Executing filter task for filterImps_2 for impressions_demo +Executing filter task for filterImps_0 for impressions_demo +Executing filter task for filterImps_1 for impressions_demo +Executing filter task for filterImps_4 for impressions_demo +Executing filter task for filterClicks_3 for clicks_demo +Executing filter task for filterClicks_1 for clicks_demo +Executing filter task for filterImps_8 for impressions_demo +Executing filter task for filterImps_6 for impressions_demo +Executing filter task for filterClicks_2 for clicks_demo +Executing filter task for filterClicks_0 for clicks_demo +Executing filter task for filterImps_7 for impressions_demo +Executing filter task for filterImps_5 for impressions_demo +Executing filter task for filterClicks_4 for clicks_demo +Executing filter task for filterImps_9 for impressions_demo +Running AggTask for impCountsByGender_3 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_2 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_0 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_9 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_1 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_4 for filtered_impressions_demo gender +Running AggTask for impCountsByCountry_4 for filtered_impressions_demo country +Running AggTask for impCountsByGender_5 for filtered_impressions_demo gender +Executing JoinTask for impClickJoin_2 +Running AggTask for impCountsByCountry_3 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_1 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_0 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_2 for filtered_impressions_demo country +Running AggTask for impCountsByGender_6 for filtered_impressions_demo gender +Executing JoinTask for impClickJoin_1 +Executing JoinTask for impClickJoin_0 +Executing JoinTask for impClickJoin_3 +Running AggTask for impCountsByGender_8 for filtered_impressions_demo gender +Executing JoinTask for impClickJoin_4 +Running AggTask for impCountsByGender_7 for filtered_impressions_demo gender +Running AggTask for impCountsByCountry_5 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_6 for filtered_impressions_demo country +Executing JoinTask for impClickJoin_9 +Running AggTask for impCountsByCountry_8 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_7 for filtered_impressions_demo country +Executing JoinTask for impClickJoin_5 +Executing JoinTask for impClickJoin_6 +Running AggTask for impCountsByCountry_9 for filtered_impressions_demo country +Executing JoinTask for impClickJoin_8 +Executing JoinTask for impClickJoin_7 +Running AggTask for clickCountsByCountry_1 for joined_clicks_demo country +Running AggTask for clickCountsByCountry_0 for joined_clicks_demo country +Running AggTask for clickCountsByCountry_2 for joined_clicks_demo country +Running AggTask for clickCountsByCountry_3 for joined_clicks_demo country +Running AggTask for clickCountsByGender_1 for joined_clicks_demo gender +Running AggTask for clickCountsByCountry_4 for joined_clicks_demo country +Running AggTask for clickCountsByGender_3 for joined_clicks_demo gender +Running AggTask for clickCountsByGender_2 for joined_clicks_demo gender +Running AggTask for clickCountsByGender_4 for joined_clicks_demo gender +Running AggTask for clickCountsByGender_0 for joined_clicks_demo gender +Running reports task +Impression counts per country +{CANADA=1940, US=1958, CHINA=2014, UNKNOWN=2022, UK=1946} +Click counts per country +{US=24, CANADA=14, CHINA=26, UNKNOWN=14, UK=22} +Impression counts per gender +{F=3325, UNKNOWN=3259, M=3296} +Click counts per gender +{F=33, UNKNOWN=32, M=35} + + +``` + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_admin.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_admin.md new file mode 100644 index 0000000000..9c24b439d0 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_admin.md @@ -0,0 +1,407 @@ + + + + Tutorial - Admin Operations + + +# [Helix Tutorial](./Tutorial.html): Admin Operations + +Helix provides a set of admin api for cluster management operations. They are supported via: + +* _Java API_ +* _Commandline interface_ +* _REST interface via helix-admin-webapp_ + +### Java API +See interface [_org.apache.helix.HelixAdmin_](http://helix.incubator.apache.org/javadocs/0.6.2-incubating/reference/org/apache/helix/HelixAdmin.html) + +### Command-line interface +The command-line tool comes with helix-core package: + +Get the command-line tool: + +``` + - git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git + - cd incubator-helix + - ./build + - cd helix-core/target/helix-core-pkg/bin + - chmod +x *.sh +``` + +Get help: + +``` + - ./helix-admin.sh --help +``` + +All other commands have this form: + +``` + ./helix-admin.sh --zkSvr +``` + +Admin commands and brief description: + +| Command syntax | Description | +| -------------- | ----------- | +| _\-\-activateCluster \_ | Enable/disable a cluster in distributed controller mode | +| _\-\-addCluster \_ | Add a new cluster | +| _\-\-addIdealState \_ | Add an ideal state to a cluster | +| _\-\-addInstanceTag \_ | Add a tag to an instance | +| _\-\-addNode \_ | Add an instance to a cluster | +| _\-\-addResource \_ | Add a new resource to a cluster | +| _\-\-addResourceProperty \_ | Add a resource property | +| _\-\-addStateModelDef \_ | Add a State model definition to a cluster | +| _\-\-dropCluster \_ | Delete a cluster | +| _\-\-dropNode \_ | Remove a node from a cluster | +| _\-\-dropResource \_ | Remove an existing resource from a cluster | +| _\-\-enableCluster \_ | Enable/disable a cluster | +| _\-\-enableInstance \_ | Enable/disable an instance | +| _\-\-enablePartition \_ | Enable/disable a partition | +| _\-\-getConfig \_ | Get user configs | +| _\-\-getConstraints \_ | Get constraints | +| _\-\-help_ | print help information | +| _\-\-instanceGroupTag \_ | Specify instance group tag, used with rebalance command | +| _\-\-listClusterInfo \_ | Show information of a cluster | +| _\-\-listClusters_ | List all clusters | +| _\-\-listInstanceInfo \_ | Show information of an instance | +| _\-\-listInstances \_ | List all instances in a cluster | +| _\-\-listPartitionInfo \_ | Show information of a partition | +| _\-\-listResourceInfo \_ | Show information of a resource | +| _\-\-listResources \_ | List all resources in a cluster | +| _\-\-listStateModel \_ | Show information of a state model | +| _\-\-listStateModels \_ | List all state models in a cluster | +| _\-\-maxPartitionsPerNode \_ | Specify the max partitions per instance, used with addResourceGroup command | +| _\-\-rebalance \_ | Rebalance a resource | +| _\-\-removeConfig \_ | Remove user configs | +| _\-\-removeConstraint \_ | Remove a constraint | +| _\-\-removeInstanceTag \_ | Remove a tag from an instance | +| _\-\-removeResourceProperty \_ | Remove a resource property | +| _\-\-resetInstance \_ | Reset all erroneous partitions on an instance | +| _\-\-resetPartition \_ | Reset an erroneous partition | +| _\-\-resetResource \_ | Reset all erroneous partitions of a resource | +| _\-\-setConfig \_ | Set user configs | +| _\-\-setConstraint \_ | Set a constraint | +| _\-\-swapInstance \_ | Swap an old instance with a new instance | +| _\-\-zkSvr \_ | Provide zookeeper address | + +### REST interface + +The REST interface comes wit helix-admin-webapp package: + +``` + - git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git + - cd incubator-helix + - ./build + - cd helix-admin-webapp/target/helix-admin-webapp-pkg/bin + - chmod +x *.sh + - ./run-rest-admin.sh --zkSvr --port // make sure zookeeper is running +``` + +#### URL and support methods + +* _/clusters_ + * List all clusters + + ``` + curl http://localhost:8100/clusters + ``` + + * Add a cluster + + ``` + curl -d 'jsonParameters={"command":"addCluster","clusterName":"MyCluster"}' -H "Content-Type: application/json" http://localhost:8100/clusters + ``` + +* _/clusters/{clusterName}_ + * List cluster information + + ``` + curl http://localhost:8100/clusters/MyCluster + ``` + + * Enable/disable a cluster in distributed controller mode + + ``` + curl -d 'jsonParameters={"command":"activateCluster","grandCluster":"MyControllerCluster","enabled":"true"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster + ``` + + * Remove a cluster + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster + ``` + +* _/clusters/{clusterName}/resourceGroups_ + * List all resources in a cluster + + ``` + curl http://localhost:8100/clusters/MyCluster/resourceGroups + ``` + + * Add a resource to cluster + + ``` + curl -d 'jsonParameters={"command":"addResource","resourceGroupName":"MyDB","partitions":"8","stateModelDefRef":"MasterSlave" }' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups + ``` + +* _/clusters/{clusterName}/resourceGroups/{resourceName}_ + * List resource information + + ``` + curl http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB + ``` + + * Drop a resource + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB + ``` + + * Reset all erroneous partitions of a resource + + ``` + curl -d 'jsonParameters={"command":"resetResource"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB + ``` + +* _/clusters/{clusterName}/resourceGroups/{resourceName}/idealState_ + * Rebalance a resource + + ``` + curl -d 'jsonParameters={"command":"rebalance","replicas":"3"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState + ``` + + * Add an ideal state + + ``` + echo jsonParameters={ + "command":"addIdealState" + }&newIdealState={ + "id" : "MyDB", + "simpleFields" : { + "IDEAL_STATE_MODE" : "AUTO", + "NUM_PARTITIONS" : "8", + "REBALANCE_MODE" : "SEMI_AUTO", + "REPLICAS" : "0", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + }, + "listFields" : { + }, + "mapFields" : { + "MyDB_0" : { + "localhost_1001" : "MASTER", + "localhost_1002" : "SLAVE" + } + } + } + > newIdealState.json + curl -d @'./newIdealState.json' -H 'Content-Type: application/json' http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState + ``` + + * Add resource property + + ``` + curl -d 'jsonParameters={"command":"addResourceProperty","REBALANCE_TIMER_PERIOD":"500"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState + ``` + +* _/clusters/{clusterName}/resourceGroups/{resourceName}/externalView_ + * Show resource external view + + ``` + curl http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/externalView + ``` +* _/clusters/{clusterName}/instances_ + * List all instances + + ``` + curl http://localhost:8100/clusters/MyCluster/instances + ``` + + * Add an instance + + ``` + curl -d 'jsonParameters={"command":"addInstance","instanceNames":"localhost_1001"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances + ``` + + * Swap an instance + + ``` + curl -d 'jsonParameters={"command":"swapInstance","oldInstance":"localhost_1001", "newInstance":"localhost_1002"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances + ``` +* _/clusters/{clusterName}/instances/{instanceName}_ + * Show instance information + + ``` + curl http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Enable/disable an instance + + ``` + curl -d 'jsonParameters={"command":"enableInstance","enabled":"false"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Drop an instance + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Disable/enable partitions on an instance + + ``` + curl -d 'jsonParameters={"command":"enablePartition","resource": "MyDB","partition":"MyDB_0", "enabled" : "false"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Reset an erroneous partition on an instance + + ``` + curl -d 'jsonParameters={"command":"resetPartition","resource": "MyDB","partition":"MyDB_0"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Reset all erroneous partitions on an instance + + ``` + curl -d 'jsonParameters={"command":"resetInstance"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + +* _/clusters/{clusterName}/configs_ + * Get user cluster level config + + ``` + curl http://localhost:8100/clusters/MyCluster/configs/cluster + ``` + + * Set user cluster level config + + ``` + curl -d 'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/cluster + ``` + + * Remove user cluster level config + + ``` + curl -d 'jsonParameters={"command":"removeConfig","configs":"key1,key2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/cluster + ``` + + * Get/set/remove user participant level config + + ``` + curl -d 'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/participant/localhost_1001 + ``` + + * Get/set/remove resource level config + + ``` + curl -d 'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/resource/MyDB + ``` + +* _/clusters/{clusterName}/controller_ + * Show controller information + + ``` + curl http://localhost:8100/clusters/MyCluster/Controller + ``` + + * Enable/disable cluster + + ``` + curl -d 'jsonParameters={"command":"enableCluster","enabled":"false"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/Controller + ``` + +* _/zkPath/{path}_ + * Get information for zookeeper path + + ``` + curl http://localhost:8100/zkPath/MyCluster + ``` + +* _/clusters/{clusterName}/StateModelDefs_ + * Show all state model definitions + + ``` + curl http://localhost:8100/clusters/MyCluster/StateModelDefs + ``` + + * Add a state mdoel definition + + ``` + echo jsonParameters={ + "command":"addStateModelDef" + }&newStateModelDef={ + "id" : "OnlineOffline", + "simpleFields" : { + "INITIAL_STATE" : "OFFLINE" + }, + "listFields" : { + "STATE_PRIORITY_LIST" : [ "ONLINE", "OFFLINE", "DROPPED" ], + "STATE_TRANSITION_PRIORITYLIST" : [ "OFFLINE-ONLINE", "ONLINE-OFFLINE", "OFFLINE-DROPPED" ] + }, + "mapFields" : { + "DROPPED.meta" : { + "count" : "-1" + }, + "OFFLINE.meta" : { + "count" : "-1" + }, + "OFFLINE.next" : { + "DROPPED" : "DROPPED", + "ONLINE" : "ONLINE" + }, + "ONLINE.meta" : { + "count" : "R" + }, + "ONLINE.next" : { + "DROPPED" : "OFFLINE", + "OFFLINE" : "OFFLINE" + } + } + } + > newStateModelDef.json + curl -d @'./untitled.txt' -H 'Content-Type: application/json' http://localhost:8100/clusters/MyCluster/StateModelDefs + ``` + +* _/clusters/{clusterName}/StateModelDefs/{stateModelDefName}_ + * Show a state model definition + + ``` + curl http://localhost:8100/clusters/MyCluster/StateModelDefs/OnlineOffline + ``` + +* _/clusters/{clusterName}/constraints/{constraintType}_ + * Show all contraints + + ``` + curl http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT + ``` + + * Set a contraint + + ``` + curl -d 'jsonParameters={"constraintAttributes":"RESOURCE=MyDB,CONSTRAINT_VALUE=1"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT/MyConstraint + ``` + + * Remove a constraint + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT/MyConstraint + ``` + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_controller.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_controller.md new file mode 100644 index 0000000000..8e7e7ad2fc --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_controller.md @@ -0,0 +1,94 @@ + + + + Tutorial - Controller + + +# [Helix Tutorial](./Tutorial.html): Controller + +Next, let\'s implement the controller. This is the brain of the cluster. Helix makes sure there is exactly one active controller running the cluster. + +### Start the Helix agent + + +It requires the following parameters: + +* clusterName: A logical name to represent the group of nodes +* instanceName: A logical name of the process creating the manager instance. Generally this is host:port. +* instanceType: Type of the process. This can be one of the following types, in this case use CONTROLLER: + * CONTROLLER: Process that controls the cluster, any number of controllers can be started but only one will be active at any given time. + * PARTICIPANT: Process that performs the actual task in the distributed system. + * SPECTATOR: Process that observes the changes in the cluster. + * ADMIN: To carry out system admin actions. +* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. + +``` + manager = HelixManagerFactory.getZKHelixManager(clusterName, + instanceName, + instanceType, + zkConnectString); +``` + +### Controller Code + +The Controller needs to know about all changes in the cluster. Helix takes care of this with the default implementation. +If you need additional functionality, see GenericHelixController on how to configure the pipeline. + +``` + manager = HelixManagerFactory.getZKHelixManager(clusterName, + instanceName, + InstanceType.CONTROLLER, + zkConnectString); + manager.connect(); + GenericHelixController controller = new GenericHelixController(); + manager.addConfigChangeListener(controller); + manager.addLiveInstanceChangeListener(controller); + manager.addIdealStateChangeListener(controller); + manager.addExternalViewChangeListener(controller); + manager.addControllerListener(controller); +``` +The snippet above shows how the controller is started. You can also start the controller using command line interface. + +``` +cd helix/helix-core/target/helix-core-pkg/bin +./run-helix-controller.sh --zkSvr --cluster +``` + +### Controller deployment modes + +Helix provides multiple options to deploy the controller. + +#### STANDALONE + +The Controller can be started as a separate process to manage a cluster. This is the recommended approach. However, since one controller can be a single point of failure, multiple controller processes are required for reliability. Even if multiple controllers are running, only one will be actively managing the cluster at any time and is decided by a leader-election process. If the leader fails, another leader will take over managing the cluster. + +Even though we recommend this method of deployment, it has the drawback of having to manage an additional service for each cluster. See Controller As a Service option. + +#### EMBEDDED + +If setting up a separate controller process is not viable, then it is possible to embed the controller as a library in each of the participants. + +#### CONTROLLER AS A SERVICE + +One of the cool features we added in Helix is to use a set of controllers to manage a large number of clusters. + +For example if you have X clusters to be managed, instead of deploying X*3 (3 controllers for fault tolerance) controllers for each cluster, one can deploy just 3 controllers. Each controller can manage X/3 clusters. If any controller fails, the remaining two will manage X/2 clusters. + + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_health.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_health.md new file mode 100644 index 0000000000..e1a7f3c2d3 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_health.md @@ -0,0 +1,46 @@ + + + + Tutorial - Customizing Heath Checks + + +# [Helix Tutorial](./Tutorial.html): Customizing Health Checks + +In this chapter, we\'ll learn how to customize the health check, based on metrics of your distributed system. + +### Health Checks + +Note: _this in currently in development mode, not yet ready for production._ + +Helix provides the ability for each node in the system to report health metrics on a periodic basis. + +Helix supports multiple ways to aggregate these metrics: + +* SUM +* AVG +* EXPONENTIAL DECAY +* WINDOW + +Helix persists the aggregated value only. + +Applications can define a threshold on the aggregate values according to the SLAs, and when the SLA is violated Helix will fire an alert. +Currently Helix only fires an alert, but in a future release we plan to use these metrics to either mark the node dead or load balance the partitions. +This feature will be valuable for distributed systems that support multi-tenancy and have a large variation in work load patterns. In addition, this can be used to detect skewed partitions (hotspots) and rebalance the cluster. + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_messaging.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_messaging.md new file mode 100644 index 0000000000..e1f0385761 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_messaging.md @@ -0,0 +1,71 @@ + + + + Tutorial - Messaging + + +# [Helix Tutorial](./Tutorial.html): Messaging + +In this chapter, we\'ll learn about messaging, a convenient feature in Helix for sending messages between nodes of a cluster. This is an interesting feature which is quite useful in practice. It is common that nodes in a distributed system require a mechanism to interact with each other. + +### Example: Bootstrapping a Replica + +Consider a search system where the index replica starts up and it does not have an index. A typical solution is to get the index from a common location, or to copy the index from another replica. + +Helix provides a messaging API for intra-cluster communication between nodes in the system. Helix provides a mechanism to specify the message recipient in terms of resource, partition, and state rather than specifying hostnames. Helix ensures that the message is delivered to all of the required recipients. In this particular use case, the instance can specify the recipient criteria as all replicas of the desired partition to bootstrap. +Since Helix is aware of the global state of the system, it can send the message to appropriate nodes. Once the nodes respond, Helix provides the bootstrapping replica with all the responses. + +This is a very generic API and can also be used to schedule various periodic tasks in the cluster, such as data backups, log cleanup, etc. +System Admins can also perform ad-hoc tasks, such as on-demand backups or a system command (such as rm -rf ;) across all nodes of the cluster + +``` + ClusterMessagingService messagingService = manager.getMessagingService(); + + // Construct the Message + Message requestBackupUriRequest = new Message( + MessageType.USER_DEFINE_MSG, UUID.randomUUID().toString()); + requestBackupUriRequest + .setMsgSubType(BootstrapProcess.REQUEST_BOOTSTRAP_URL); + requestBackupUriRequest.setMsgState(MessageState.NEW); + + // Set the Recipient criteria: all nodes that satisfy the criteria will receive the message + Criteria recipientCriteria = new Criteria(); + recipientCriteria.setInstanceName("%"); + recipientCriteria.setRecipientInstanceType(InstanceType.PARTICIPANT); + recipientCriteria.setResource("MyDB"); + recipientCriteria.setPartition(""); + + // Should be processed only by process(es) that are active at the time of sending the message + // This means if the recipient is restarted after message is sent, it will not be processe. + recipientCriteria.setSessionSpecific(true); + + // wait for 30 seconds + int timeout = 30000; + + // the handler that will be invoked when any recipient responds to the message. + BootstrapReplyHandler responseHandler = new BootstrapReplyHandler(); + + // this will return only after all recipients respond or after timeout + int sentMessageCount = messagingService.sendAndWait(recipientCriteria, + requestBackupUriRequest, responseHandler, timeout); +``` + +See HelixManager.DefaultMessagingService in [Javadocs](http://helix.incubator.apache.org/javadocs/0.6.2-incubating/reference/org/apache/helix/messaging/DefaultMessagingService.html) for more info. + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_participant.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_participant.md new file mode 100644 index 0000000000..d2812dab2f --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_participant.md @@ -0,0 +1,105 @@ + + + + Tutorial - Participant + + +# [Helix Tutorial](./Tutorial.html): Participant + +In this chapter, we\'ll learn how to implement a Participant, which is a primary functional component of a distributed system. + + +### Start the Helix agent + +The Helix agent is a common component that connects each system component with the controller. + +It requires the following parameters: + +* clusterName: A logical name to represent the group of nodes +* instanceName: A logical name of the process creating the manager instance. Generally this is host:port. +* instanceType: Type of the process. This can be one of the following types, in this case, use PARTICIPANT + * CONTROLLER: Process that controls the cluster, any number of controllers can be started but only one will be active at any given time. + * PARTICIPANT: Process that performs the actual task in the distributed system. + * SPECTATOR: Process that observes the changes in the cluster. + * ADMIN: To carry out system admin actions. +* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. + +After the Helix manager instance is created, only thing that needs to be registered is the state model factory. +The methods of the State Model will be called when controller sends transitions to the Participant. In this example, we'll use the OnlineOffline factory. Other options include: + +* MasterSlaveStateModelFactory +* LeaderStandbyStateModelFactory +* BootstrapHandler +* _An application defined state model factory_ + + +``` + manager = HelixManagerFactory.getZKHelixManager(clusterName, + instanceName, + InstanceType.PARTICIPANT, + zkConnectString); + StateMachineEngine stateMach = manager.getStateMachineEngine(); + + //create a stateModelFactory that returns a statemodel object for each partition. + stateModelFactory = new OnlineOfflineStateModelFactory(); + stateMach.registerStateModelFactory(stateModelType, stateModelFactory); + manager.connect(); +``` + +Helix doesn\'t know what it means to change from OFFLINE\-\-\>ONLINE or ONLINE\-\-\>OFFLINE. The following code snippet shows where you insert your system logic for these two state transitions. + +``` +public class OnlineOfflineStateModelFactory extends + StateModelFactory { + @Override + public StateModel createNewStateModel(String stateUnitKey) { + OnlineOfflineStateModel stateModel = new OnlineOfflineStateModel(); + return stateModel; + } + @StateModelInfo(states = "{'OFFLINE','ONLINE'}", initialState = "OFFINE") + public static class OnlineOfflineStateModel extends StateModel { + + @Transition(from = "OFFLINE", to = "ONLINE") + public void onBecomeOnlineFromOffline(Message message, + NotificationContext context) { + + System.out.println("OnlineOfflineStateModel.onBecomeOnlineFromOffline()"); + + //////////////////////////////////////////////////////////////////////////////////////////////// + // Application logic to handle transition // + // For example, you might start a service, run initialization, etc // + //////////////////////////////////////////////////////////////////////////////////////////////// + } + + @Transition(from = "ONLINE", to = "OFFLINE") + public void onBecomeOfflineFromOnline(Message message, + NotificationContext context) { + + System.out.println("OnlineOfflineStateModel.onBecomeOfflineFromOnline()"); + + //////////////////////////////////////////////////////////////////////////////////////////////// + // Application logic to handle transition // + // For example, you might shutdown a service, log this event, or change monitoring settings // + //////////////////////////////////////////////////////////////////////////////////////////////// + } + } +} +``` + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_propstore.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_propstore.md new file mode 100644 index 0000000000..8e7e5b5728 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_propstore.md @@ -0,0 +1,34 @@ + + + + Tutorial - Application Property Store + + +# [Helix Tutorial](./Tutorial.html): Application Property Store + +In this chapter, we\'ll learn how to use the application property store. + +### Property Store + +It is common that an application needs support for distributed, shared data structures. Helix uses Zookeeper to store the application data and hence provides notifications when the data changes. + +While you could use Zookeeper directly, Helix supports caching the data and a write-through cache. This is far more efficient than reading from Zookeeper for every access. + +See [HelixManager.getHelixPropertyStore](http://helix.incubator.apache.org/javadocs/0.6.2-incubating/reference/org/apache/helix/store/package-summary.html) for details. diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_rebalance.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_rebalance.md new file mode 100644 index 0000000000..8f42a5ab52 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_rebalance.md @@ -0,0 +1,181 @@ + + + + Tutorial - Rebalancing Algorithms + + +# [Helix Tutorial](./Tutorial.html): Rebalancing Algorithms + +The placement of partitions in a distributed system is essential for the reliability and scalability of the system. For example, when a node fails, it is important that the partitions hosted on that node are reallocated evenly among the remaining nodes. Consistent hashing is one such algorithm that can satisfy this guarantee. Helix provides a variant of consistent hashing based on the RUSH algorithm, among others. + +This means given a number of partitions, replicas and number of nodes, Helix does the automatic assignment of partition to nodes such that: + +* Each node has the same number of partitions +* Replicas of the same partition do not stay on the same node +* When a node fails, the partitions will be equally distributed among the remaining nodes +* When new nodes are added, the number of partitions moved will be minimized along with satisfying the above criteria + +Helix employs a rebalancing algorithm to compute the _ideal state_ of the system. When the _current state_ differs from the _ideal state_, Helix uses it as the target state of the system and computes the appropriate transitions needed to bring it to the _ideal state_. + +Helix makes it easy to perform this operation, while giving you control over the algorithm. In this section, we\'ll see how to implement the desired behavior. + +Helix has four options for rebalancing, in increasing order of customization by the system builder: + +* FULL_AUTO +* SEMI_AUTO +* CUSTOMIZED +* USER_DEFINED + +``` + |FULL_AUTO | SEMI_AUTO | CUSTOMIZED| USER_DEFINED | + ---------------------------------------------------------| + LOCATION | HELIX | APP | APP | APP | + ---------------------------------------------------------| + STATE | HELIX | HELIX | APP | APP | + ---------------------------------------------------------- +``` + + +### FULL_AUTO + +When the rebalance mode is set to FULL_AUTO, Helix controls both the location of the replica along with the state. This option is useful for applications where creation of a replica is not expensive. + +For example, consider this system that uses a MasterSlave state model, with 3 partitions and 2 replicas in the ideal state. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "REBALANCE_MODE" : "FULL_AUTO", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [], + "MyResource_1" : [], + "MyResource_2" : [] + }, + "mapFields" : { + } +} +``` + +If there are 3 nodes in the cluster, then Helix will balance the masters and slaves equally. The ideal state is therefore: + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +Another typical example is evenly distributing a group of tasks among the currently healthy processes. For example, if there are 60 tasks and 4 nodes, Helix assigns 15 tasks to each node. +When one node fails, Helix redistributes its 15 tasks to the remaining 3 nodes, resulting in a balanced 20 tasks per node. Similarly, if a node is added, Helix re-allocates 3 tasks from each of the 4 nodes to the 5th node, resulting in a balanced distribution of 12 tasks per node.. + +#### SEMI_AUTO + +When the application needs to control the placement of the replicas, use the SEMI_AUTO rebalance mode. + +Example: In the ideal state below, the partition \'MyResource_0\' is constrained to be placed only on node1 or node2. The choice of _state_ is still controlled by Helix. That means MyResource_0.MASTER could be on node1 and MyResource_0.SLAVE on node2, or vice-versa but neither would be placed on node3. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [node1, node2], + "MyResource_1" : [node2, node3], + "MyResource_2" : [node3, node1] + }, + "mapFields" : { + } +} +``` + +The MasterSlave state model requires that a partition has exactly one MASTER at all times, and the other replicas should be SLAVEs. In this simple example with 2 replicas per partition, there would be one MASTER and one SLAVE. Upon failover, a SLAVE has to assume mastership, and a new SLAVE will be generated. + +In this mode when node1 fails, unlike in FULL_AUTO mode the partition is _not_ moved from node1 to node3. Instead, Helix will decide to change the state of MyResource_0 on node2 from SLAVE to MASTER, based on the system constraints. + +#### CUSTOMIZED + +Helix offers a third mode called CUSTOMIZED, in which the application controls the placement _and_ state of each replica. The application needs to implement a callback interface that Helix invokes when the cluster state changes. +Within this callback, the application can recompute the idealstate. Helix will then issue appropriate transitions such that _Idealstate_ and _Currentstate_ converges. + +Here\'s an example, again with 3 partitions, 2 replicas per partition, and the MasterSlave state model: + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "REBALANCE_MODE" : "CUSTOMIZED", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +Suppose the current state of the system is 'MyResource_0' -> {N1:MASTER, N2:SLAVE} and the application changes the ideal state to 'MyResource_0' -> {N1:SLAVE,N2:MASTER}. While the application decides which node is MASTER and which is SLAVE, Helix will not blindly issue MASTER-->SLAVE to N1 and SLAVE-->MASTER to N2 in parallel, since that might result in a transient state where both N1 and N2 are masters, which violates the MasterSlave constraint that there is exactly one MASTER at a time. Helix will first issue MASTER-->SLAVE to N1 and after it is completed, it will issue SLAVE-->MASTER to N2. + +#### USER_DEFINED + +For maximum flexibility, Helix exposes an interface that can allow applications to plug in custom rebalancing logic. By providing the name of a class that implements the Rebalancer interface, Helix will automatically call the contained method whenever there is a change to the live participants in the cluster. For more, see [User-Defined Rebalancer](./tutorial_user_def_rebalancer.html). + +#### Backwards Compatibility + +In previous versions, FULL_AUTO was called AUTO_REBALANCE and SEMI_AUTO was called AUTO. Furthermore, they were presented as the IDEAL_STATE_MODE. Helix supports both IDEAL_STATE_MODE and REBALANCE_MODE, but IDEAL_STATE_MODE is now deprecated and may be phased out in future versions. diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_spectator.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_spectator.md new file mode 100644 index 0000000000..24c1cf484e --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_spectator.md @@ -0,0 +1,76 @@ + + + + Tutorial - Spectator + + +# [Helix Tutorial](./Tutorial.html): Spectator + +Next, we\'ll learn how to implement a Spectator. Typically, a spectator needs to react to changes within the distributed system. Examples: a client that needs to know where to send a request, a topic consumer in a consumer group. The spectator is automatically informed of changes in the _external state_ of the cluster, but it does not have to add any code to keep track of other components in the system. + +### Start the Helix agent + +Same as for a Participant, The Helix agent is the common component that connects each system component with the controller. + +It requires the following parameters: + +* clusterName: A logical name to represent the group of nodes +* instanceName: A logical name of the process creating the manager instance. Generally this is host:port. +* instanceType: Type of the process. This can be one of the following types, in this case, use SPECTATOR: + * CONTROLLER: Process that controls the cluster, any number of controllers can be started but only one will be active at any given time. + * PARTICIPANT: Process that performs the actual task in the distributed system. + * SPECTATOR: Process that observes the changes in the cluster. + * ADMIN: To carry out system admin actions. +* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. + +After the Helix manager instance is created, only thing that needs to be registered is the listener. When the ExternalView changes, the listener is notified. + +### Spectator Code + +A spectator observes the cluster and is notified when the state of the system changes. Helix consolidates the state of entire cluster in one Znode called ExternalView. +Helix provides a default implementation RoutingTableProvider that caches the cluster state and updates it when there is a change in the cluster. + +``` +manager = HelixManagerFactory.getZKHelixManager(clusterName, + instanceName, + InstanceType.PARTICIPANT, + zkConnectString); +manager.connect(); +RoutingTableProvider routingTableProvider = new RoutingTableProvider(); +manager.addExternalViewChangeListener(routingTableProvider); +``` + +In the following code snippet, the application sends the request to a valid instance by interrogating the external view. Suppose the desired resource for this request is in the partition myDB_1. + +``` +## instances = routingTableProvider.getInstances(, "PARTITION_NAME", "PARTITION_STATE"); +instances = routingTableProvider.getInstances("myDB", "myDB_1", "ONLINE"); + +//////////////////////////////////////////////////////////////////////////////////////////////// +// Application-specific code to send a request to one of the instances // +//////////////////////////////////////////////////////////////////////////////////////////////// + +theInstance = instances.get(0); // should choose an instance and throw an exception if none are available +result = theInstance.sendRequest(yourApplicationRequest, responseObject); + +``` + +When the external view changes, the application needs to react by sending requests to a different instance. + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_state.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_state.md new file mode 100644 index 0000000000..4f7b1b587a --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_state.md @@ -0,0 +1,131 @@ + + + + Tutorial - State Machine Configuration + + +# [Helix Tutorial](./Tutorial.html): State Machine Configuration + +In this chapter, we\'ll learn about the state models provided by Helix, and how to create your own custom state model. + +## State Models + +Helix comes with 3 default state models that are commonly used. It is possible to have multiple state models in a cluster. +Every resource that is added should be configured to use a state model that govern its _ideal state_. + +### MASTER-SLAVE + +* 3 states: OFFLINE, SLAVE, MASTER +* Maximum number of masters: 1 +* Slaves are based on the replication factor. The replication factor can be specified while adding the resource. + + +### ONLINE-OFFLINE + +* Has 2 states: OFFLINE and ONLINE. This simple state model is a good starting point for most applications. + +### LEADER-STANDBY + +* 1 Leader and multiple stand-bys. The idea is that exactly one leader accomplishes a designated task, the stand-bys are ready to take over if the leader fails. + +## Constraints + +In addition to the state machine configuration, one can specify the constraints of states and transitions. + +For example, one can say: + +* MASTER:1 +
Maximum number of replicas in MASTER state at any time is 1 + +* OFFLINE-SLAVE:5 +
Maximum number of OFFLINE-SLAVE transitions that can happen concurrently in the system is 5 in this example. + +### Dynamic State Constraints + +We also support two dynamic upper bounds for the number of replicas in each state: + +* N: The number of replicas in the state is at most the number of live participants in the cluster +* R: The number of replicas in the state is at most the specified replica count for the partition + +### State Priority + +Helix uses a greedy approach to satisfy the state constraints. For example, if the state machine configuration says it needs 1 MASTER and 2 SLAVES, but only 1 node is active, Helix must promote it to MASTER. This behavior is achieved by providing the state priority list as \[MASTER, SLAVE\]. + +### State Transition Priority + +Helix tries to fire as many transitions as possible in parallel to reach the stable state without violating constraints. By default, Helix simply sorts the transitions alphabetically and fires as many as it can without violating the constraints. You can control this by overriding the priority order. + +## Special States + +### DROPPED + +The DROPPED state is used to signify a replica that was served by a given participant, but is no longer served. This allows Helix and its participants to effectively clean up. There are two requirements that every new state model should follow with respect to the DROPPED state: + +* The DROPPED state must be defined +* There must be a path to DROPPED for every state in the model + +### ERROR + +The ERROR state is used whenever the participant serving a partition encountered an error and cannot continue to serve the partition. HelixAdmin has \"reset\" functionality to allow for participants to recover from the ERROR state. + +## Annotated Example + +Below is a complete definition of a Master-Slave state model. Notice the fields marked REQUIRED; these are essential for any state model definition. + +``` +StateModelDefinition stateModel = new StateModelDefinition.Builder("MasterSlave") + // OFFLINE is the state that the system starts in (initial state is REQUIRED) + .initialState("OFFLINE") + + // Lowest number here indicates highest priority, no value indicates lowest priority + .addState("MASTER", 1) + .addState("SLAVE", 2) + .addState("OFFLINE") + + // Note the special inclusion of the DROPPED state (REQUIRED) + .addState(HelixDefinedState.DROPPED.toString()) + + // No more than one master allowed + .upperBound("MASTER", 1) + + // R indicates an upper bound of number of replicas for each partition + .dynamicUpperBound("SLAVE", "R") + + // Add some high-priority transitions + .addTransition("SLAVE", "MASTER", 1) + .addTransition("OFFLINE", "SLAVE", 2) + + // Using the same priority value indicates that these transitions can fire in any order + .addTransition("MASTER", "SLAVE", 3) + .addTransition("SLAVE", "OFFLINE", 3) + + // Not specifying a value defaults to lowest priority + // Notice the inclusion of the OFFLINE to DROPPED transition + // Since every state has a path to OFFLINE, they each now have a path to DROPPED (REQUIRED) + .addTransition("OFFLINE", HelixDefinedState.DROPPED.toString()) + + // Create the StateModelDefinition instance + .build(); + + // Use the isValid() function to make sure the StateModelDefinition will work without issues + Assert.assertTrue(stateModel.isValid()); +``` + + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_throttling.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_throttling.md new file mode 100644 index 0000000000..2317cf1f6f --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_throttling.md @@ -0,0 +1,38 @@ + + + + Tutorial - Throttling + + +# [Helix Tutorial](./Tutorial.html): Throttling + +In this chapter, we\'ll learn how to control the parallel execution of cluster tasks. Only a centralized cluster manager with global knowledge is capable of coordinating this decision. + +### Throttling + +Since all state changes in the system are triggered through transitions, Helix can control the number of transitions that can happen in parallel. Some of the transitions may be light weight, but some might involve moving data, which is quite expensive from a network and iops perspective. + +Helix allows applications to set a threshold on transitions. The threshold can be set at multiple scopes: + +* MessageType e.g STATE_TRANSITION +* TransitionType e.g SLAVE-MASTER +* Resource e.g database +* Node i.e per-node maximum transitions in parallel + diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_user_def_rebalancer.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_user_def_rebalancer.md new file mode 100644 index 0000000000..7590002f96 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_user_def_rebalancer.md @@ -0,0 +1,172 @@ + + + + Tutorial - User-Defined Rebalancing + + +# [Helix Tutorial](./Tutorial.html): User-Defined Rebalancing + +Even though Helix can compute both the location and the state of replicas internally using a default fully-automatic rebalancer, specific applications may require rebalancing strategies that optimize for different requirements. Thus, Helix allows applications to plug in arbitrary rebalancer algorithms that implement a provided interface. One of the main design goals of Helix is to provide maximum flexibility to any distributed application. Thus, it allows applications to fully implement the rebalancer, which is the core constraint solver in the system, if the application developer so chooses. + +Whenever the state of the cluster changes, as is the case when participants join or leave the cluster, Helix automatically calls the rebalancer to compute a new mapping of all the replicas in the resource. When using a pluggable rebalancer, the only required step is to register it with Helix. Subsequently, no additional bootstrapping steps are necessary. Helix uses reflection to look up and load the class dynamically at runtime. As a result, it is also technically possible to change the rebalancing strategy used at any time. + +The Rebalancer interface is as follows: + +``` +void init(HelixManager manager); + +IdealState computeNewIdealState(String resourceName, IdealState currentIdealState, + final CurrentStateOutput currentStateOutput, final ClusterDataCache clusterData); +``` +The first parameter is the resource to rebalance, the second is pre-existing ideal mappings, the third is a snapshot of the actual placements and state assignments, and the fourth is a full cache of all of the cluster data available to Helix. Internally, Helix implements the same interface for its own rebalancing routines, so a user-defined rebalancer will be cognizant of the same information about the cluster as an internal implementation. Helix strives to provide applications the ability to implement algorithms that may require a large portion of the entire state of the cluster to make the best placement and state assignment decisions possible. + +An IdealState is a full representation of the location of each replica of each partition of a given resource. This is a simple representation of the placement that the algorithm believes is the best possible. If the placement meets all defined constraints, this is what will become the actual state of the distributed system. + +### Specifying a Rebalancer +For implementations that set up the cluster through existing code, the following HelixAdmin calls will update the Rebalancer class: + +``` +IdealState idealState = helixAdmin.getResourceIdealState(clusterName, resourceName); +idealState.setRebalanceMode(RebalanceMode.USER_DEFINED); +idealState.setRebalancerClassName(className); +helixAdmin.setResourceIdealState(clusterName, resourceName, idealState); +``` + +There are two key fields to set to specify that a pluggable rebalancer should be used. First, the rebalance mode should be set to USER_DEFINED, and second the rebalancer class name should be set to a class that implements Rebalancer and is within the scope of the project. The class name is a fully-qualified class name consisting of its package and its name. Without specification of the USER_DEFINED mode, the user-defined rebalancer class will not be used even if specified. Furthermore, Helix will not attempt to rebalance the resources through its standard routines if its mode is USER_DEFINED, regardless of whether or not a rebalancer class is registered. + +### Example + +In the next release (0.7.0), we will provide a full example of a user-defined rebalancer in action. + +Consider the case where partitions are locks in a lock manager and 6 locks are to be distributed evenly to a set of participants, and only one participant can hold each lock. We can define a rebalancing algorithm that simply takes the modulus of the lock number and the number of participants to evenly distribute the locks across participants. Helix allows capping the number of partitions a participant can accept, but since locks are lightweight, we do not need to define a restriction in this case. The following is a succinct implementation of this algorithm. + +``` +@Override +IdealState computeNewIdealState(String resourceName, IdealState currentIdealState, + final CurrentStateOutput currentStateOutput, final ClusterDataCache clusterData) { + // Get the list of live participants in the cluster + List liveParticipants = new ArrayList(clusterData.getLiveInstances().keySet()); + + // Count the number of participants allowed to lock each lock (in this example, this is 1) + int lockHolders = Integer.parseInt(currentIdealState.getReplicas()); + + // Fairly assign the lock state to the participants using a simple mod-based sequential + // assignment. For instance, if each lock can be held by 3 participants, lock 0 would be held + // by participants (0, 1, 2), lock 1 would be held by (1, 2, 3), and so on, wrapping around the + // number of participants as necessary. + int i = 0; + for (String partition : currentIdealState.getPartitionSet()) { + List preferenceList = new ArrayList(); + for (int j = i; j < i + lockHolders; j++) { + int participantIndex = j % liveParticipants.size(); + String participant = liveParticipants.get(participantIndex); + // enforce that a participant can only have one instance of a given lock + if (!preferenceList.contains(participant)) { + preferenceList.add(participant); + } + } + currentIdealState.setPreferenceList(partition, preferenceList); + i++; + } + return assignment; +} +``` + +Here are the IdealState preference lists emitted by the user-defined rebalancer for a 3-participant system whenever there is a change to the set of participants. + +* Participant_A joins + +``` +{ + "lock_0": ["Participant_A"], + "lock_1": ["Participant_A"], + "lock_2": ["Participant_A"], + "lock_3": ["Participant_A"], + "lock_4": ["Participant_A"], + "lock_5": ["Participant_A"], +} +``` + +A preference list is a mapping for each resource of partition to the participants serving each replica. The state model is a simple LOCKED/RELEASED model, so participant A holds all lock partitions in the LOCKED state. + +* Participant_B joins + +``` +{ + "lock_0": ["Participant_A"], + "lock_1": ["Participant_B"], + "lock_2": ["Participant_A"], + "lock_3": ["Participant_B"], + "lock_4": ["Participant_A"], + "lock_5": ["Participant_B"], +} +``` + +Now that there are two participants, the simple mod-based function assigns every other lock to the second participant. On any system change, the rebalancer is invoked so that the application can define how to redistribute its resources. + +* Participant_C joins (steady state) + +``` +{ + "lock_0": ["Participant_A"], + "lock_1": ["Participant_B"], + "lock_2": ["Participant_C"], + "lock_3": ["Participant_A"], + "lock_4": ["Participant_B"], + "lock_5": ["Participant_C"], +} +``` + +This is the steady state of the system. Notice that four of the six locks now have a different owner. That is because of the naïve modulus-based assignmemt approach used by the user-defined rebalancer. However, the interface is flexible enough to allow you to employ consistent hashing or any other scheme if minimal movement is a system requirement. + +* Participant_B fails + +``` +{ + "lock_0": ["Participant_A"], + "lock_1": ["Participant_C"], + "lock_2": ["Participant_A"], + "lock_3": ["Participant_C"], + "lock_4": ["Participant_A"], + "lock_5": ["Participant_C"], +} +``` + +On any node failure, as in the case of node addition, the rebalancer is invoked automatically so that it can generate a new mapping as a response to the change. Helix ensures that the Rebalancer has the opportunity to reassign locks as required by the application. + +* Participant_B (or the replacement for the original Participant_B) rejoins + +``` +{ + "lock_0": ["Participant_A"], + "lock_1": ["Participant_B"], + "lock_2": ["Participant_C"], + "lock_3": ["Participant_A"], + "lock_4": ["Participant_B"], + "lock_5": ["Participant_C"], +} +``` + +The rebalancer was invoked once again and the resulting IdealState preference lists reflect the steady state. + +### Caveats +- The rebalancer class must be available at runtime, or else Helix will not attempt to rebalance at all +- The Helix controller will only take into account the preference lists in the new IdealState for this release. In 0.7.0, Helix rebalancers will be able to compute the full resource assignment, including the states. +- Helix does not currently persist the new IdealState computed by the user-defined rebalancer. However, the Helix property store is available for saving any computed state. In 0.7.0, Helix will persist the result of running the rebalancer. \ No newline at end of file diff --git a/site-releases/0.6.2-incubating/src/site/markdown/tutorial_yaml.md b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_yaml.md new file mode 100644 index 0000000000..0f8e0cce11 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/markdown/tutorial_yaml.md @@ -0,0 +1,102 @@ + + + + Tutorial - YAML Cluster Setup + + +# [Helix Tutorial](./Tutorial.html): YAML Cluster Setup + +As an alternative to using Helix Admin to set up the cluster, its resources, constraints, and the state model, Helix supports bootstrapping a cluster configuration based on a YAML file. Below is an annotated example of such a file for a simple distributed lock manager where a lock can only be LOCKED or RELEASED, and each lock only allows a single participant to hold it in the LOCKED state. + +``` +clusterName: lock-manager-custom-rebalancer # unique name for the cluster (required) +resources: + - name: lock-group # unique resource name (required) + rebalancer: # required + mode: USER_DEFINED # required - USER_DEFINED means we will provide our own rebalancer + class: org.apache.helix.userdefinedrebalancer.LockManagerRebalancer # required for USER_DEFINED + partitions: + count: 12 # number of partitions for the resource (default is 1) + replicas: 1 # number of replicas per partition (default is 1) + stateModel: + name: lock-unlock # model name (required) + states: [LOCKED, RELEASED, DROPPED] # the list of possible states (required if model not built-in) + transitions: # the list of possible transitions (required if model not built-in) + - name: Unlock + from: LOCKED + to: RELEASED + - name: Lock + from: RELEASED + to: LOCKED + - name: DropLock + from: LOCKED + to: DROPPED + - name: DropUnlock + from: RELEASED + to: DROPPED + - name: Undrop + from: DROPPED + to: RELEASED + initialState: RELEASED # (required if model not built-in) + constraints: + state: + counts: # maximum number of replicas of a partition that can be in each state (required if model not built-in) + - name: LOCKED + count: "1" + - name: RELEASED + count: "-1" + - name: DROPPED + count: "-1" + priorityList: [LOCKED, RELEASED, DROPPED] # states in order of priority (all priorities equal if not specified) + transition: # transitions priority to enforce order that transitions occur + priorityList: [Unlock, Lock, Undrop, DropUnlock, DropLock] # all priorities equal if not specified +participants: # list of nodes that can serve replicas (optional if dynamic joining is active, required otherwise) + - name: localhost_12001 + host: localhost + port: 12001 + - name: localhost_12002 + host: localhost + port: 12002 + - name: localhost_12003 + host: localhost + port: 12003 +``` + +Using a file like the one above, the cluster can be set up either with the command line: + +``` +incubator-helix/helix-core/target/helix-core/pkg/bin/YAMLClusterSetup.sh localhost:2199 lock-manager-config.yaml +``` + +or with code: + +``` +YAMLClusterSetup setup = new YAMLClusterSetup(zkAddress); +InputStream input = + Thread.currentThread().getContextClassLoader() + .getResourceAsStream("lock-manager-config.yaml"); +YAMLClusterSetup.YAMLClusterConfig config = setup.setupCluster(input); +``` + +Some notes: + +- A rebalancer class is only required for the USER_DEFINED mode. It is ignored otherwise. + +- Built-in state models, like OnlineOffline, LeaderStandby, and MasterSlave, or state models that have already been added only require a name for stateModel. If partition and/or replica counts are not provided, a value of 1 is assumed. \ No newline at end of file diff --git a/site-releases/0.6.2-incubating/src/site/resources/.htaccess b/site-releases/0.6.2-incubating/src/site/resources/.htaccess new file mode 100644 index 0000000000..d5c7bf3f94 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/resources/.htaccess @@ -0,0 +1,20 @@ +# +# 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. +# + +Redirect /download.html /download.cgi diff --git a/site-releases/0.6.2-incubating/src/site/resources/download.cgi b/site-releases/0.6.2-incubating/src/site/resources/download.cgi new file mode 100644 index 0000000000..f9a0e3007c --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/resources/download.cgi @@ -0,0 +1,22 @@ +#!/bin/sh +# Just call the standard mirrors.cgi script. It will use download.html +# as the input template. +# +# 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. +# +exec /www/www.apache.org/dyn/mirrors/mirrors.cgi $* diff --git a/site-releases/0.6.2-incubating/src/site/resources/images/HELIX-components.png b/site-releases/0.6.2-incubating/src/site/resources/images/HELIX-components.png new file mode 100644 index 0000000000..c0c35aee2c Binary files /dev/null and b/site-releases/0.6.2-incubating/src/site/resources/images/HELIX-components.png differ diff --git a/site-releases/0.6.2-incubating/src/site/resources/images/PFS-Generic.png b/site-releases/0.6.2-incubating/src/site/resources/images/PFS-Generic.png new file mode 100644 index 0000000000..7eea3a0497 Binary files /dev/null and b/site-releases/0.6.2-incubating/src/site/resources/images/PFS-Generic.png differ diff --git a/site-releases/0.6.2-incubating/src/site/resources/images/RSYNC_BASED_PFS.png b/site-releases/0.6.2-incubating/src/site/resources/images/RSYNC_BASED_PFS.png new file mode 100644 index 0000000000..0cc55ae100 Binary files /dev/null and b/site-releases/0.6.2-incubating/src/site/resources/images/RSYNC_BASED_PFS.png differ diff --git a/site-releases/0.6.2-incubating/src/site/resources/images/bootstrap_statemodel.gif b/site-releases/0.6.2-incubating/src/site/resources/images/bootstrap_statemodel.gif new file mode 100644 index 0000000000..b8f8a42589 Binary files /dev/null and b/site-releases/0.6.2-incubating/src/site/resources/images/bootstrap_statemodel.gif differ diff --git a/site-releases/0.6.2-incubating/src/site/resources/images/helix-architecture.png b/site-releases/0.6.2-incubating/src/site/resources/images/helix-architecture.png new file mode 100644 index 0000000000..6f69a2db34 Binary files /dev/null and b/site-releases/0.6.2-incubating/src/site/resources/images/helix-architecture.png differ diff --git a/site-releases/0.6.2-incubating/src/site/resources/images/helix-logo.jpg b/site-releases/0.6.2-incubating/src/site/resources/images/helix-logo.jpg new file mode 100644 index 0000000000..d6428f600f Binary files /dev/null and b/site-releases/0.6.2-incubating/src/site/resources/images/helix-logo.jpg differ diff --git a/site-releases/0.6.2-incubating/src/site/resources/images/helix-znode-layout.png b/site-releases/0.6.2-incubating/src/site/resources/images/helix-znode-layout.png new file mode 100644 index 0000000000..5bafc45f21 Binary files /dev/null and b/site-releases/0.6.2-incubating/src/site/resources/images/helix-znode-layout.png differ diff --git a/site-releases/0.6.2-incubating/src/site/resources/images/statemachine.png b/site-releases/0.6.2-incubating/src/site/resources/images/statemachine.png new file mode 100644 index 0000000000..43d27ecfbc Binary files /dev/null and b/site-releases/0.6.2-incubating/src/site/resources/images/statemachine.png differ diff --git a/site-releases/0.6.2-incubating/src/site/resources/images/system.png b/site-releases/0.6.2-incubating/src/site/resources/images/system.png new file mode 100644 index 0000000000..f8a05c8764 Binary files /dev/null and b/site-releases/0.6.2-incubating/src/site/resources/images/system.png differ diff --git a/site-releases/0.6.2-incubating/src/site/site.xml b/site-releases/0.6.2-incubating/src/site/site.xml new file mode 100644 index 0000000000..68cba65a21 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/site.xml @@ -0,0 +1,119 @@ + + + + + images/helix-logo.jpg + http://helix.incubator.apache.org/site-releases/0.6.2-incubating-site + + + http://incubator.apache.org/images/egg-logo.png + http://incubator.apache.org/ + + + + + + + org.apache.maven.skins + maven-fluido-skin + 1.3.0 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+
Apache Helix, Apache, the Apache feather logo, and the Apache Helix project logos are trademarks of The Apache Software Foundation. + All other marks mentioned may be trademarks or registered trademarks of their respective owners.
+ Privacy Policy +
+
+ + + + + + + true + + true + + + ApacheHelix + true + false + + + + +
diff --git a/site-releases/0.6.2-incubating/src/site/xdoc/download.xml.vm b/site-releases/0.6.2-incubating/src/site/xdoc/download.xml.vm new file mode 100644 index 0000000000..97e872763b --- /dev/null +++ b/site-releases/0.6.2-incubating/src/site/xdoc/download.xml.vm @@ -0,0 +1,203 @@ + + +#set( $releaseName = "0.6.2-incubating" ) +#set( $releaseDate = "11/22/2013" ) + + + + Apache Incubator Helix Downloads + Apache Helix Documentation Team + + + +
+ + + +
+ +
+

Apache Helix artifacts are distributed in source and binary form under the terms of the + Apache License, Version 2.0. + See the included LICENSE and NOTICE files included in each artifact for additional license + information. +

+

Use the links below to download a source distribution of Apache Helix. + It is good practice to verify the integrity of the distribution files.

+
+ +
+

Release date: ${releaseDate}

+

${releaseName} Release notes

+ + + +

+ [if-any logo] + + logo + + [end] + The currently selected mirror is + [preferred]. + If you encounter a problem with this mirror, + please select another mirror. + If all mirrors are failing, there are + backup + mirrors + (at the end of the mirrors list) that should be available. +

+ +
+ Other mirrors: + + +
+ +

+ You may also consult the + complete list of mirrors. +

+ + + + + + + + + + + + + + + + +
ArtifactSignatures
+ helix-${releaseName}-src.zip + + asc + md5 + sha1 +
+
+ + + + + + + + + + + + + + + + + + + + + + +
ArtifactSignatures
+ helix-core-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+ helix-admin-webapp-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+ helix-agent-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+
+
+ + + +
+

We strongly recommend you verify the integrity of the downloaded files with both PGP and MD5.

+ +

The PGP signatures can be verified using PGP or + GPG. + First download the KEYS as well as the + *.asc signature file for the particular distribution. Make sure you get these files from the main + distribution directory, rather than from a mirror. Then verify the signatures using one of the following sets of + commands: + + $ pgp -ka KEYS +$ pgp helix-*.zip.asc + + $ gpg --import KEYS +$ gpg --verify helix-*.zip.asc +

+

Alternatively, you can verify the MD5 signature on the files. A Unix/Linux program called + md5 or + md5sum is included in most distributions. It is also available as part of + GNU Textutils. + Windows users can get binary md5 programs from these (and likely other) places: +

+

+
+ +
diff --git a/site-releases/0.6.2-incubating/src/test/conf/testng.xml b/site-releases/0.6.2-incubating/src/test/conf/testng.xml new file mode 100644 index 0000000000..58f0803678 --- /dev/null +++ b/site-releases/0.6.2-incubating/src/test/conf/testng.xml @@ -0,0 +1,27 @@ + + + + + + + + + + diff --git a/site-releases/0.7.0-incubating/pom.xml b/site-releases/0.7.0-incubating/pom.xml new file mode 100644 index 0000000000..f3abc170a8 --- /dev/null +++ b/site-releases/0.7.0-incubating/pom.xml @@ -0,0 +1,51 @@ + + + + 4.0.0 + + + org.apache.helix + site-releases + 0.7.1-incubating-SNAPSHOT + + + 0.7.0-incubating-site + bundle + Apache Helix :: Site :: 0.7.0-incubating + + + + + + + org.testng + testng + 6.0.1 + + + + + + + + + + + diff --git a/site-releases/0.7.0-incubating/src/site/apt/privacy-policy.apt b/site-releases/0.7.0-incubating/src/site/apt/privacy-policy.apt new file mode 100644 index 0000000000..ada93631d1 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/apt/privacy-policy.apt @@ -0,0 +1,52 @@ + ---- + Privacy Policy + ----- + Olivier Lamy + ----- + 2013-02-04 + ----- + +~~ 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. + +Privacy Policy + + Information about your use of this website is collected using server access logs and a tracking cookie. The + collected information consists of the following: + + [[1]] The IP address from which you access the website; + + [[2]] The type of browser and operating system you use to access our site; + + [[3]] The date and time you access our site; + + [[4]] The pages you visit; and + + [[5]] The addresses of pages from where you followed a link to our site. + + [] + + Part of this information is gathered using a tracking cookie set by the + {{{http://www.google.com/analytics/}Google Analytics}} service and handled by Google as described in their + {{{http://www.google.com/privacy.html}privacy policy}}. See your browser documentation for instructions on how to + disable the cookie if you prefer not to share this data with Google. + + We use the gathered information to help us make our site more useful to visitors and to better understand how and + when our site is used. We do not track or collect personally identifiable information or associate gathered data + with any personally identifying information from other sources. + + By using this website, you consent to the collection of this data in the manner and for the purpose described above. diff --git a/site-releases/0.7.0-incubating/src/site/apt/releasenotes/release-0.7.0-incubating.apt b/site-releases/0.7.0-incubating/src/site/apt/releasenotes/release-0.7.0-incubating.apt new file mode 100644 index 0000000000..7661df00d3 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/apt/releasenotes/release-0.7.0-incubating.apt @@ -0,0 +1,174 @@ + ----- + Release Notes for Apache Helix 0.7.0-incubating + ----- + +~~ 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. + +~~ NOTE: For help with the syntax of this file, see: +~~ http://maven.apache.org/guides/mini/guide-apt-format.html + +Release Notes for Apache Helix 0.7.0-incubating + + The Apache Helix team would like to announce the release of Apache Helix 0.7.0-incubating + + This is the fourth release and second major release under the Apache umbrella. + + Helix is a generic cluster management framework used for the automatic management of partitioned, replicated and distributed resources hosted on a cluster of nodes. Helix provides the following features: + + * Automatic assignment of resource/partition to nodes + + * Node failure detection and recovery + + * Dynamic addition of Resources + + * Dynamic addition of nodes to the cluster + + * Pluggable distributed state machine to manage the state of a resource via state transitions + + * Automatic load balancing and throttling of transitions + + * Configurable, pluggable rebalancing + + [] + +* Changes + +** Sub-task + + * [HELIX-18] - Unify cluster setup and helixadmin + + * [HELIX-79] - consecutive GC may mess up helix session ids + + * [HELIX-83] - Add typed classes to denote helix ids + + * [HELIX-90] - Clean up Api's + + * [HELIX-98] - clean up setting constraint api + + * [HELIX-100] - Improve the helix config api + + * [HELIX-102] - Add new wrapper classes for Participant, Controller, Spectator, Administrator + + * [HELIX-104] - Add support to reuse zkclient + + * [HELIX-123] - ZkHelixManager.isLeader() should check session id in addition to instance name + + * [HELIX-139] - Need to double check the logic to prevent 2 controllers to control the same cluster + + * [HELIX-168] - separate HelixManager implementation for participant, controller, and distributed controller + + * [HELIX-176] - Need a list of tests that must pass to certify a release + + * [HELIX-224] - Move helix examples to separate module + + * [HELIX-233] - Ensure that website and wiki fully capture the updated changes in 0.7.0 + + * [HELIX-234] - Create concrete id classes for constructs, replacing strings + + * [HELIX-235] - Create a hierarchical logical model for the cluster + + * [HELIX-236] - Create a hierarchical cluster snapshot to replace ClusterDataCache + + * [HELIX-237] - Create helix-internal config classes for the hierarchical model + + * [HELIX-238] - Create accessors for the logical model + + * [HELIX-239] - List use cases for the logical model + + * [HELIX-240] - Write an example of the key use cases for the logical model + + * [HELIX-241] - Write the controller pipeline with the logical model + + * [HELIX-242] - Re-integrate the scheduler rebalancing into the new controller pipeline + + * [HELIX-243] - Fix failing tests related to helix model overhaul + + * [HELIX-244] - Redesign rebalancers using rebalancer-specific configs + + * [HELIX-246] - Refactor scheduler task config to comply with new rebalancer config and fix related scheduler task tests + + * [HELIX-248] - Resource logical model should be general enough to handle various resource types + + * [HELIX-268] - Atomic API + + * [HELIX-297] - Make 0.7.0 backward compatible for user-defined rebalancing + + +** Bug + + * [HELIX-40] - fix zkclient subscribe path leaking and zk callback-handler leaking in case of session expiry + + * [HELIX-46] - Add REST/cli admin command for message selection constraints + + * [HELIX-47] - when drop resource, remove resource-level config also + + * [HELIX-48] - use resource instead of db in output messages + + * [HELIX-50] - Ensure num replicas and preference list size in idealstate matches + + * [HELIX-59] - controller not cleaning dead external view generated from old sessions + + * [HELIX-136] - Write IdealState back to ZK when computed by custom Rebalancer + + * [HELIX-200] - helix controller send ERROR->DROPPED transition infinitely + + * [HELIX-214] - User-defined rebalancer should never use SEMI_AUTO code paths + + * [HELIX-225] - fix helix-example package build error + + * [HELIX-271] - ZkHelixAdmin#addResource() backward compatible problem + + * [HELIX-292] - ZNRecordStreamingSerializer should not assume id comes first + + * [HELIX-296] - HelixConnection in 0.7.0 does not remove LiveInstance znode + + * [HELIX-300] - Some files in 0.7.0 are missing license headers + + * [HELIX-302] - fix helix version compare bug + +** Improvement + + * [HELIX-37] - Cleanup CallbackHandler + + * [HELIX-202] - Ideal state should be a full mapping, not just a set of instance preferences + +** Task + + * [HELIX-109] - Review Helix model package + + * [HELIX-174] - Clean up ideal state calculators, move them to the controller rebalancer package + + * [HELIX-212] - Rebalancer interface should have 1 function to compute the entire ideal state + + * [HELIX-232] - Validation of 0.7.0 + + * [HELIX-290] - Ensure 0.7.0 can respond correctly to ideal state changes + + * [HELIX-295] - Upgrade or remove xstream dependency + + * [HELIX-301] - Update integration test utils for 0.7.0 + +** Test + + * [HELIX-286] - add a test for redefine state model definition + + [] + + Cheers, + -- + The Apache Helix Team diff --git a/site-releases/0.7.0-incubating/src/site/apt/releasing.apt b/site-releases/0.7.0-incubating/src/site/apt/releasing.apt new file mode 100644 index 0000000000..11d0cd92aa --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/apt/releasing.apt @@ -0,0 +1,107 @@ + ----- + Helix release process + ----- + ----- + 2012-12-15 + ----- + +~~ 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. + +~~ NOTE: For help with the syntax of this file, see: +~~ http://maven.apache.org/guides/mini/guide-apt-format.html + +Helix release process + + [[1]] Post to the dev list a few days before you plan to do an Helix release + + [[2]] Your maven setting must contains the entry to be able to deploy. + + ~/.m2/settings.xml + ++------------- + + apache.releases.https + + + ++------------- + + [[3]] Apache DAV passwords + ++------------- + Add the following info into your ~/.netrc + machine git-wip-us.apache.org login + ++------------- + [[4]] Release Helix + You should have a GPG agent running in the session you will run the maven release commands(preferred), and confirm it works by running "gpg -ab" (type some text and press Ctrl-D). + If you do not have a GPG agent running, make sure that you have the "apache-release" profile set in your settings.xml as shown below. + + Run the release + ++------------- +mvn release:prepare release:perform -B ++------------- + + GPG configuration in maven settings xml: + ++------------- + + apache-release + + [GPG_PASSWORD] + + ++------------- + + [[4]] go to https://repository.apache.org and close your staged repository. Note the repository url (format https://repository.apache.org/content/repositories/orgapachehelix-019/org/apache/helix/helix/0.6-incubating/) + ++------------- +svn co https://dist.apache.org/repos/dist/dev/incubator/helix helix-dev-release +cd helix-dev-release +sh ./release-script-svn.sh version stagingRepoUrl +then svn add +then svn ci ++------------- + + [[5]] Validating the release + ++------------- + * Download sources, extract, build and run tests - mvn clean package + * Verify license headers - mvn -Prat -DskipTests + * Download binaries and .asc files + * Download release manager's public key - From the KEYS file, get the release manager's public key finger print and run gpg --keyserver pgpkeys.mit.edu --recv-key + * Validate authenticity of key - run gpg --fingerprint + * Check signatures of all the binaries using gpg ++------------- + + [[6]] Call for a vote in the dev list and wait for 72 hrs. for the vote results. 3 binding votes are necessary for the release to be finalized. example + After the vote has passed, move the files from dist dev to dist release: svn mv https://dist.apache.org/repos/dist/dev/incubator/helix/version to https://dist.apache.org/repos/dist/release/incubator/helix/ + + [[7]] Prepare release note. Add a page in src/site/apt/releasenotes/ and change value of \ in parent pom. + + + [[8]] Send out an announcement of the release to: + + * users@helix.incubator.apache.org + + * dev@helix.incubator.apache.org + + [[9]] Celebrate ! + + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/Architecture.md b/site-releases/0.7.0-incubating/src/site/markdown/Architecture.md new file mode 100644 index 0000000000..933e917c7e --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/Architecture.md @@ -0,0 +1,252 @@ + + + + Architecture + + +Architecture +---------------------------- +Helix aims to provide the following abilities to a distributed system: + +* Automatic management of a cluster hosting partitioned, replicated resources. +* Soft and hard failure detection and handling. +* Automatic load balancing via smart placement of resources on servers(nodes) based on server capacity and resource profile (size of partition, access patterns, etc). +* Centralized config management and self discovery. Eliminates the need to modify config on each node. +* Fault tolerance and optimized rebalancing during cluster expansion. +* Manages entire operational lifecycle of a node. Addition, start, stop, enable/disable without downtime. +* Monitor cluster health and provide alerts on SLA violation. +* Service discovery mechanism to route requests. + +To build such a system, we need a mechanism to co-ordinate between different nodes and other components in the system. This mechanism can be achieved with software that reacts to any change in the cluster and comes up with a set of tasks needed to bring the cluster to a stable state. The set of tasks will be assigned to one or more nodes in the cluster. Helix serves this purpose of managing the various components in the cluster. + +![Helix Design](images/system.png) + +Distributed System Components + +In general any distributed system cluster will have the following components and properties: + +* A set of nodes also referred to as instances. +* A set of resources which can be databases, lucene indexes or tasks. +* Each resource is also partitioned into one or more Partitions. +* Each partition may have one or more copies called replicas. +* Each replica can have a state associated with it. For example Master, Slave, Leader, Standby, Online, Offline etc + +Roles +----- + +![Helix Design](images/HELIX-components.png) + +Not all nodes in a distributed system will perform similar functionalities. For example, a few nodes might be serving requests and a few nodes might be sending requests, and some nodes might be controlling the nodes in the cluster. Thus, Helix categorizes nodes by their specific roles in the system. + +We have divided Helix nodes into 3 logical components based on their responsibilities: + +1. Participant: The nodes that actually host the distributed resources. +2. Spectator: The nodes that simply observe the Participant state and route the request accordingly. Routers, for example, need to know the instance on which a partition is hosted and its state in order to route the request to the appropriate end point. +3. Controller: The controller observes and controls the Participant nodes. It is responsible for coordinating all transitions in the cluster and ensuring that state constraints are satisfied and cluster stability is maintained. + +These are simply logical components and can be deployed as per the system requirements. For example: + +1. The controller can be deployed as a separate service +2. The controller can be deployed along with a Participant but only one Controller will be active at any given time. + +Both have pros and cons, which will be discussed later and one can chose the mode of deployment as per system needs. + + +## Cluster state metadata store + +We need a distributed store to maintain the state of the cluster and a notification system to notify if there is any change in the cluster state. Helix uses Zookeeper to achieve this functionality. + +Zookeeper provides: + +* A way to represent PERSISTENT state which basically remains until its deleted. +* A way to represent TRANSIENT/EPHEMERAL state which vanishes when the process that created the state dies. +* Notification mechanism when there is a change in PERSISTENT and EPHEMERAL state + +The namespace provided by ZooKeeper is much like that of a standard file system. A name is a sequence of path elements separated by a slash (/). Every node[ZNode] in ZooKeeper\'s namespace is identified by a path. + +More info on Zookeeper can be found at http://zookeeper.apache.org + +## State machine and constraints + +Even though the concepts of Resources, Partitions, and Replicas are common to most distributed systems, one thing that differentiates one distributed system from another is the way each partition is assigned a state and the constraints on each state. + +For example: + +1. If a system is serving read-only data then all partition\'s replicas are equal and they can either be ONLINE or OFFLINE. +2. If a system takes _both_ reads and writes but ensure that writes go through only one partition, the states will be MASTER, SLAVE, and OFFLINE. Writes go through the MASTER and replicate to the SLAVEs. Optionally, reads can go through SLAVES. + +Apart from defining state for each partition, the transition path to each state can be application specific. For example, in order to become MASTER it might be a requirement to first become a SLAVE. This ensures that if the SLAVE does not have the data as part of OFFLINE-SLAVE transition it can bootstrap data from other nodes in the system. + +Helix provides a way to configure an application specific state machine along with constraints on each state. Along with constraints on STATE, Helix also provides a way to specify constraints on transitions. (More on this later.) + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +![Helix Design](images/statemachine.png) + +## Concepts + +The following terminologies are used in Helix to model a state machine. + +* IdealState: The state in which we need the cluster to be in if all nodes are up and running. In other words, all state constraints are satisfied. +* CurrentState: Represents the actual current state of each node in the cluster +* ExternalView: Represents the combined view of CurrentState of all nodes. + +The goal of Helix is always to make the CurrentState of the system same as the IdealState. Some scenarios where this may not be true are: + +* When all nodes are down +* When one or more nodes fail +* New nodes are added and the partitions need to be reassigned + +### IdealState + +Helix lets the application define the IdealState on a resource basis which basically consists of: + +* List of partitions. Example: 64 +* Number of replicas for each partition. Example: 3 +* Node and State for each replica. + +Example: + +* Partition-1, replica-1, Master, Node-1 +* Partition-1, replica-2, Slave, Node-2 +* Partition-1, replica-3, Slave, Node-3 +* ..... +* ..... +* Partition-p, replica-3, Slave, Node-n + +Helix comes with various algorithms to automatically assign the partitions to nodes. The default algorithm minimizes the number of shuffles that happen when new nodes are added to the system. + +### CurrentState + +Every instance in the cluster hosts one or more partitions of a resource. Each of the partitions has a state associated with it. + +Example Node-1 + +* Partition-1, Master +* Partition-2, Slave +* .... +* .... +* Partition-p, Slave + +### ExternalView + +External clients needs to know the state of each partition in the cluster and the Node hosting that partition. Helix provides one view of the system to Spectators as _ExternalView_. ExternalView is simply an aggregate of all node CurrentStates. + +* Partition-1, replica-1, Master, Node-1 +* Partition-1, replica-2, Slave, Node-2 +* Partition-1, replica-3, Slave, Node-3 +* ..... +* ..... +* Partition-p, replica-3, Slave, Node-n + +## Process Workflow + +Mode of operation in a cluster + +A node process can be one of the following: + +* Participant: The process registers itself in the cluster and acts on the messages received in its queue and updates the current state. Example: a storage node in a distributed database +* Spectator: The process is simply interested in the changes in the Externalview. +* Controller: This process actively controls the cluster by reacting to changes in cluster state and sending messages to Participants. + + +### Participant Node Process + +* When Node starts up, it registers itself under _LiveInstances_ +* After registering, it waits for new _Messages_ in the message queue +* When it receives a message, it will perform the required task as indicated in the message +* After the task is completed, depending on the task outcome it updates the CurrentState + +### Controller Process + +* Watches IdealState +* Notified when a node goes down/comes up or node is added/removed. Watches LiveInstances and CurrentState of each node in the cluster +* Triggers appropriate state transitions by sending message to Participants + +### Spectator Process + +* When the process starts, it asks the Helix agent to be notified of changes in ExternalView +* Whenever it receives a notification, it reads the Externalview and performs required duties. + +#### Interaction between controller, participant and spectator + +The following picture shows how controllers, participants and spectators interact with each other. + +![Helix Architecture](images/helix-architecture.png) + +## Core algorithm + +* Controller gets the IdealState and the CurrentState of active storage nodes from Zookeeper +* Compute the delta between IdealState and CurrentState for each partition across all participant nodes +* For each partition compute tasks based on the State Machine Table. It\'s possible to configure priority on the state Transition. For example, in case of Master-Slave: + * Attempt mastership transfer if possible without violating constraint. + * Partition Addition + * Drop Partition +* Add the tasks in parallel if possible to the respective queue for each storage node (if the tasks added are mutually independent) +* If a task is dependent on another task being completed, do not add that task +* After any task is completed by a Participant, Controllers gets notified of the change and the State Transition algorithm is re-run until the CurrentState is same as IdealState. + +## Helix ZNode layout + +Helix organizes znodes under clusterName in multiple levels. + +The top level (under the cluster name) ZNodes are all Helix-defined and in upper case: + +* PROPERTYSTORE: application property store +* STATEMODELDEFES: state model definitions +* INSTANCES: instance runtime information including current state and messages +* CONFIGS: configurations +* IDEALSTATES: ideal states +* EXTERNALVIEW: external views +* LIVEINSTANCES: live instances +* CONTROLLER: cluster controller runtime information + +Under INSTANCES, there are runtime ZNodes for each instance. An instance organizes ZNodes as follows: + +* CURRENTSTATES + * sessionId + * resourceName +* ERRORS +* STATUSUPDATES +* MESSAGES +* HEALTHREPORT + +Under CONFIGS, there are different scopes of configurations: + +* RESOURCE: contains resource scope configurations +* CLUSTER: contains cluster scope configurations +* PARTICIPANT: contains participant scope configurations + +The following image shows an example of Helix znodes layout for a cluster named "test-cluster": + +![Helix znode layout](images/helix-znode-layout.png) diff --git a/site-releases/0.7.0-incubating/src/site/markdown/Building.md b/site-releases/0.7.0-incubating/src/site/markdown/Building.md new file mode 100644 index 0000000000..06046d521c --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/Building.md @@ -0,0 +1,46 @@ + + +Build Instructions +------------------ + +Requirements: Jdk 1.6+, Maven 2.0.8+ + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +git checkout tags/helix-0.7.0-incubating +mvn install package -DskipTests +``` + +Maven dependency + +``` + + org.apache.helix + helix-core + 0.7.0-incubating + +``` + +Download +-------- + +[0.7.0-incubating](./download.html) + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/Concepts.md b/site-releases/0.7.0-incubating/src/site/markdown/Concepts.md new file mode 100644 index 0000000000..fa5d0ba75f --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/Concepts.md @@ -0,0 +1,275 @@ + + + + Concepts + + +Concepts +---------------------------- + +Helix is based on the idea that a given task has the following attributes associated with it: + +* _Location of the task_. For example it runs on Node N1 +* _State_. For example, it is running, stopped etc. + +In Helix terminology, a task is referred to as a _resource_. + +### IdealState + +IdealState simply allows one to map tasks to location and state. A standard way of expressing this in Helix: + +``` + "TASK_NAME" : { + "LOCATION" : "STATE" + } + +``` +Consider a simple case where you want to launch a task \'myTask\' on node \'N1\'. The IdealState for this can be expressed as follows: + +``` +{ + "id" : "MyTask", + "mapFields" : { + "myTask" : { + "N1" : "ONLINE", + } + } +} +``` +### Partition + +If this task get too big to fit on one box, you might want to divide it into subtasks. Each subtask is referred to as a _partition_ in Helix. Let\'s say you want to divide the task into 3 subtasks/partitions, the IdealState can be changed as shown below. + +\'myTask_0\', \'myTask_1\', \'myTask_2\' are logical names representing the partitions of myTask. Each tasks runs on N1, N2 and N3 respectively. + +``` +{ + "id" : "myTask", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + } + "mapFields" : { + "myTask_0" : { + "N1" : "ONLINE", + }, + "myTask_1" : { + "N2" : "ONLINE", + }, + "myTask_2" : { + "N3" : "ONLINE", + } + } +} +``` + +### Replica + +Partitioning allows one to split the data/task into multiple subparts. But let\'s say the request rate for each partition increases. The common solution is to have multiple copies for each partition. Helix refers to the copy of a partition as a _replica_. Adding a replica also increases the availability of the system during failures. One can see this methodology employed often in search systems. The index is divided into shards, and each shard has multiple copies. + +Let\'s say you want to add one additional replica for each task. The IdealState can simply be changed as shown below. + +For increasing the availability of the system, it\'s better to place the replica of a given partition on different nodes. + +``` +{ + "id" : "myIndex", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + }, + "mapFields" : { + "myIndex_0" : { + "N1" : "ONLINE", + "N2" : "ONLINE" + }, + "myIndex_1" : { + "N2" : "ONLINE", + "N3" : "ONLINE" + }, + "myIndex_2" : { + "N3" : "ONLINE", + "N1" : "ONLINE" + } + } +} +``` + +### State + +Now let\'s take a slightly more complicated scenario where a task represents a database. Unlike an index which is in general read-only, a database supports both reads and writes. Keeping the data consistent among the replicas is crucial in distributed data stores. One commonly applied technique is to assign one replica as the MASTER and remaining replicas as SLAVEs. All writes go to the MASTER and are then replicated to the SLAVE replicas. + +Helix allows one to assign different states to each replica. Let\'s say you have two MySQL instances N1 and N2, where one will serve as MASTER and another as SLAVE. The IdealState can be changed to: + +``` +{ + "id" : "myDB", + "simpleFields" : { + "NUM_PARTITIONS" : "1", + "REPLICAS" : "2", + }, + "mapFields" : { + "myDB" : { + "N1" : "MASTER", + "N2" : "SLAVE", + } + } +} + +``` + + +### State Machine and Transitions + +IdealState allows one to exactly specify the desired state of the cluster. Given an IdealState, Helix takes up the responsibility of ensuring that the cluster reaches the IdealState. The Helix _controller_ reads the IdealState and then commands each Participant to take appropriate actions to move from one state to another until it matches the IdealState. These actions are referred to as _transitions_ in Helix. + +The next logical question is: how does the _controller_ compute the transitions required to get to IdealState? This is where the finite state machine concept comes in. Helix allows applications to plug in a finite state machine. A state machine consists of the following: + +* State: Describes the role of a replica +* Transition: An action that allows a replica to move from one state to another, thus changing its role. + +Here is an example of MasterSlave state machine: + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +Helix allows each resource to be associated with one state machine. This means you can have one resource as an index and another as a database in the same cluster. One can associate each resource with a state machine as follows: + +``` +{ + "id" : "myDB", + "simpleFields" : { + "NUM_PARTITIONS" : "1", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "myDB" : { + "N1" : "MASTER", + "N2" : "SLAVE", + } + } +} + +``` + +### Current State + +CurrentState of a resource simply represents its actual state at a Participant. In the below example: + +* INSTANCE_NAME: Unique name representing the process +* SESSION_ID: ID that is automatically assigned every time a process joins the cluster + +``` +{ + "id":"MyResource" + ,"simpleFields":{ + ,"SESSION_ID":"13d0e34675e0002" + ,"INSTANCE_NAME":"node1" + ,"STATE_MODEL_DEF":"MasterSlave" + } + ,"mapFields":{ + "MyResource_0":{ + "CURRENT_STATE":"SLAVE" + } + ,"MyResource_1":{ + "CURRENT_STATE":"MASTER" + } + ,"MyResource_2":{ + "CURRENT_STATE":"MASTER" + } + } +} +``` +Each node in the cluster has its own CurrentState. + +### External View + +In order to communicate with the Participants, external clients need to know the current state of each of the Participants. The external clients are referred to as Spectators. In order to make the life of Spectator simple, Helix provides an ExternalView that is an aggregated view of the current state across all nodes. The ExternalView has a similar format as IdealState. + +``` +{ + "id":"MyResource", + "mapFields":{ + "MyResource_0":{ + "N1":"SLAVE", + "N2":"MASTER", + "N3":"OFFLINE" + }, + "MyResource_1":{ + "N1":"MASTER", + "N2":"SLAVE", + "N3":"ERROR" + }, + "MyResource_2":{ + "N1":"MASTER", + "N2":"SLAVE", + "N3":"SLAVE" + } + } +} +``` + +### Rebalancer + +The core component of Helix is the Controller which runs the Rebalancer algorithm on every cluster event. Cluster events can be one of the following: + +* Nodes start/stop and soft/hard failures +* New nodes are added/removed +* Ideal state changes + +There are few more examples such as configuration changes, etc. The key takeaway: there are many ways to trigger the rebalancer. + +When a rebalancer is run it simply does the following: + +* Compares the IdealState and current state +* Computes the transitions required to reach the IdealState +* Issues the transitions to each Participant + +The above steps happen for every change in the system. Once the current state matches the IdealState, the system is considered stable which implies \'IdealState = CurrentState = ExternalView\' + +### Dynamic IdealState + +One of the things that makes Helix powerful is that IdealState can be changed dynamically. This means one can listen to cluster events like node failures and dynamically change the ideal state. Helix will then take care of triggering the respective transitions in the system. + +Helix comes with a few algorithms to automatically compute the IdealState based on the constraints. For example, if you have a resource of 3 partitions and 2 replicas, Helix can automatically compute the IdealState based on the nodes that are currently active. See the [tutorial](./tutorial_rebalance.html) to find out more about various execution modes of Helix like FULL_AUTO, SEMI_AUTO and CUSTOMIZED. + + + + + + + + + + + + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/Features.md b/site-releases/0.7.0-incubating/src/site/markdown/Features.md new file mode 100644 index 0000000000..ba9d0e72d0 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/Features.md @@ -0,0 +1,313 @@ + + + + Features + + +Features +---------------------------- + + +### CONFIGURING IDEALSTATE + + +Read concepts page for definition of Idealstate. + +The placement of partitions in a DDS is very critical for reliability and scalability of the system. +For example, when a node fails, it is important that the partitions hosted on that node are reallocated evenly among the remaining nodes. Consistent hashing is one such algorithm that can guarantee this. +Helix by default comes with a variant of consistent hashing based of the RUSH algorithm. + +This means given a number of partitions, replicas and number of nodes Helix does the automatic assignment of partition to nodes such that + +* Each node has the same number of partitions and replicas of the same partition do not stay on the same node. +* When a node fails, the partitions will be equally distributed among the remaining nodes +* When new nodes are added, the number of partitions moved will be minimized along with satisfying the above two criteria. + + +Helix provides multiple ways to control the placement and state of a replica. + +``` + + |AUTO REBALANCE| AUTO | CUSTOM | + ----------------------------------------- + LOCATION | HELIX | APP | APP | + ----------------------------------------- + STATE | HELIX | HELIX | APP | + ----------------------------------------- +``` + +#### HELIX EXECUTION MODE + + +Idealstate is defined as the state of the DDS when all nodes are up and running and healthy. +Helix uses this as the target state of the system and computes the appropriate transitions needed in the system to bring it to a stable state. + +Helix supports 3 different execution modes which allows application to explicitly control the placement and state of the replica. + +##### AUTO_REBALANCE + +When the idealstate mode is set to AUTO_REBALANCE, Helix controls both the location of the replica along with the state. This option is useful for applications where creation of a replica is not expensive. Example + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "IDEAL_STATE_MODE" : "AUTO_REBALANCE", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [], + "MyResource_1" : [], + "MyResource_2" : [] + }, + "mapFields" : { + } +} +``` + +If there are 3 nodes in the cluster, then Helix will internally compute the ideal state as + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +Another typical example is evenly distributing a group of tasks among the currently alive processes. For example, if there are 60 tasks and 4 nodes, Helix assigns 15 tasks to each node. +When one node fails Helix redistributes its 15 tasks to the remaining 3 nodes. Similarly, if a node is added, Helix re-allocates 3 tasks from each of the 4 nodes to the 5th node. + +#### AUTO + +When the idealstate mode is set to AUTO, Helix only controls STATE of the replicas where as the location of the partition is controlled by application. Example: The below idealstate indicates thats 'MyResource_0' must be only on node1 and node2. But gives the control of assigning the STATE to Helix. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "IDEAL_STATE_MODE" : "AUTO", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [node1, node2], + "MyResource_1" : [node2, node3], + "MyResource_2" : [node3, node1] + }, + "mapFields" : { + } +} +``` +In this mode when node1 fails, unlike in AUTO-REBALANCE mode the partition is not moved from node1 to others nodes in the cluster. Instead, Helix will decide to change the state of MyResource_0 in N2 based on the system constraints. For example, if a system constraint specified that there should be 1 Master and if the Master failed, then node2 will be made the new master. + +#### CUSTOM + +Helix offers a third mode called CUSTOM, in which application can completely control the placement and state of each replica. Applications will have to implement an interface that Helix will invoke when the cluster state changes. +Within this callback, the application can recompute the idealstate. Helix will then issue appropriate transitions such that Idealstate and Currentstate converges. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "IDEAL_STATE_MODE" : "CUSTOM", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +For example, the current state of the system might be 'MyResource_0' -> {N1:MASTER,N2:SLAVE} and the application changes the ideal state to 'MyResource_0' -> {N1:SLAVE,N2:MASTER}. Helix will not blindly issue MASTER-->SLAVE to N1 and SLAVE-->MASTER to N2 in parallel since it might result in a transient state where both N1 and N2 are masters. +Helix will first issue MASTER-->SLAVE to N1 and after its completed it will issue SLAVE-->MASTER to N2. + + +### State Machine Configuration + +Helix comes with 3 default state models that are most commonly used. Its possible to have multiple state models in a cluster. +Every resource that is added should have a reference to the state model. + +* MASTER-SLAVE: Has 3 states OFFLINE,SLAVE,MASTER. Max masters is 1. Slaves will be based on the replication factor. Replication factor can be specified while adding the resource +* ONLINE-OFFLINE: Has 2 states OFFLINE and ONLINE. Very simple state model and most applications start off with this state model. +* LEADER-STANDBY:1 Leader and many stand bys. In general the standby's are idle. + +Apart from providing the state machine configuration, one can specify the constraints of states and transitions. + +For example one can say +Master:1. Max number of replicas in Master state at any time is 1. +OFFLINE-SLAVE:5 Max number of Offline-Slave transitions that can happen concurrently in the system + +STATE PRIORITY +Helix uses greedy approach to satisfy the state constraints. For example if the state machine configuration says it needs 1 master and 2 slaves but only 1 node is active, Helix must promote it to master. This behavior is achieved by providing the state priority list as MASTER,SLAVE. + +STATE TRANSITION PRIORITY +Helix tries to fire as many transitions as possible in parallel to reach the stable state without violating constraints. By default Helix simply sorts the transitions alphabetically and fires as many as it can without violating the constraints. +One can control this by overriding the priority order. + +### Config management + +Helix allows applications to store application specific properties. The configuration can have different scopes. + +* Cluster +* Node specific +* Resource specific +* Partition specific + +Helix also provides notifications when any configs are changed. This allows applications to support dynamic configuration changes. + +See HelixManager.getConfigAccessor for more info + +### Intra cluster messaging api + +This is an interesting feature which is quite useful in practice. Often times, nodes in DDS requires a mechanism to interact with each other. One such requirement is a process of bootstrapping a replica. + +Consider a search system use case where the index replica starts up and it does not have an index. One of the commonly used solutions is to get the index from a common location or to copy the index from another replica. +Helix provides a messaging api, that can be used to talk to other nodes in the system. The value added that Helix provides here is, message recipient can be specified in terms of resource, +partition, state and Helix ensures that the message is delivered to all of the required recipients. In this particular use case, the instance can specify the recipient criteria as all replicas of P1. +Since Helix is aware of the global state of the system, it can send the message to appropriate nodes. Once the nodes respond Helix provides the bootstrapping replica with all the responses. + +This is a very generic api and can also be used to schedule various periodic tasks in the cluster like data backups etc. +System Admins can also perform adhoc tasks like on demand backup or execute a system command(like rm -rf ;-)) across all nodes. + +``` + ClusterMessagingService messagingService = manager.getMessagingService(); + //CONSTRUCT THE MESSAGE + Message requestBackupUriRequest = new Message( + MessageType.USER_DEFINE_MSG, UUID.randomUUID().toString()); + requestBackupUriRequest + .setMsgSubType(BootstrapProcess.REQUEST_BOOTSTRAP_URL); + requestBackupUriRequest.setMsgState(MessageState.NEW); + //SET THE RECIPIENT CRITERIA, All nodes that satisfy the criteria will receive the message + Criteria recipientCriteria = new Criteria(); + recipientCriteria.setInstanceName("%"); + recipientCriteria.setRecipientInstanceType(InstanceType.PARTICIPANT); + recipientCriteria.setResource("MyDB"); + recipientCriteria.setPartition(""); + //Should be processed only the process that is active at the time of sending the message. + //This means if the recipient is restarted after message is sent, it will not be processed. + recipientCriteria.setSessionSpecific(true); + // wait for 30 seconds + int timeout = 30000; + //The handler that will be invoked when any recipient responds to the message. + BootstrapReplyHandler responseHandler = new BootstrapReplyHandler(); + //This will return only after all recipients respond or after timeout. + int sentMessageCount = messagingService.sendAndWait(recipientCriteria, + requestBackupUriRequest, responseHandler, timeout); +``` + +See HelixManager.getMessagingService for more info. + + +### Application specific property storage + +There are several usecases where applications needs support for distributed data structures. Helix uses Zookeeper to store the application data and hence provides notifications when the data changes. +One value add Helix provides is the ability to specify cache the data and also write through cache. This is more efficient than reading from ZK every time. + +See HelixManager.getHelixPropertyStore + +### Throttling + +Since all state changes in the system are triggered through transitions, Helix can control the number of transitions that can happen in parallel. Some of the transitions may be light weight but some might involve moving data around which is quite expensive. +Helix allows applications to set threshold on transitions. The threshold can be set at the multiple scopes. + +* MessageType e.g STATE_TRANSITION +* TransitionType e.g SLAVE-MASTER +* Resource e.g database +* Node i.e per node max transitions in parallel. + +See HelixManager.getHelixAdmin.addMessageConstraint() + +### Health monitoring and alerting + +This in currently in development mode, not yet productionized. + +Helix provides ability for each node in the system to report health metrics on a periodic basis. +Helix supports multiple ways to aggregate these metrics like simple SUM, AVG, EXPONENTIAL DECAY, WINDOW. Helix will only persist the aggregated value. +Applications can define threshold on the aggregate values according to the SLA's and when the SLA is violated Helix will fire an alert. +Currently Helix only fires an alert but eventually we plan to use this metrics to either mark the node dead or load balance the partitions. +This feature will be valuable in for distributed systems that support multi-tenancy and have huge variation in work load patterns. Another place this can be used is to detect skewed partitions and rebalance the cluster. + +This feature is not yet stable and do not recommend to be used in production. + + +### Controller deployment modes + +Read Architecture wiki for more details on the Role of a controller. In simple words, it basically controls the participants in the cluster by issuing transitions. + +Helix provides multiple options to deploy the controller. + +#### STANDALONE + +Controller can be started as a separate process to manage a cluster. This is the recommended approach. How ever since one controller can be a single point of failure, multiple controller processes are required for reliability. +Even if multiple controllers are running only one will be actively managing the cluster at any time and is decided by a leader election process. If the leader fails, another leader will resume managing the cluster. + +Even though we recommend this method of deployment, it has the drawback of having to manage an additional service for each cluster. See Controller As a Service option. + +#### EMBEDDED + +If setting up a separate controller process is not viable, then it is possible to embed the controller as a library in each of the participant. + +#### CONTROLLER AS A SERVICE + +One of the cool feature we added in helix was use a set of controllers to manage a large number of clusters. +For example if you have X clusters to be managed, instead of deploying X*3(3 controllers for fault tolerance) controllers for each cluster, one can deploy only 3 controllers. Each controller can manage X/3 clusters. +If any controller fails the remaining two will manage X/2 clusters. At LinkedIn, we always deploy controllers in this mode. + + + + + + + + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/Quickstart.md b/site-releases/0.7.0-incubating/src/site/markdown/Quickstart.md new file mode 100644 index 0000000000..b4f095b180 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/Quickstart.md @@ -0,0 +1,626 @@ + + + + Quickstart + + +Get Helix +--------- + +First, let\'s get Helix, either build it, or download. + +### Build + + git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git + cd incubator-helix + git checkout tags/helix-0.7.0-incubating + ./build + cd helix-core/target/helix-core-pkg/bin //This folder contains all the scripts used in following sections + chmod +x * + +### Download + +Download the 0.7.0-incubating release package [here](./download.html) + +Overview +-------- + +In this Quickstart, we\'ll set up a master-slave replicated, partitioned system. Then we\'ll demonstrate how to add a node, rebalance the partitions, and show how Helix manages failover. + + +Let\'s Do It +------------ + +Helix provides command line interfaces to set up the cluster and view the cluster state. The best way to understand how Helix views a cluster is to build a cluster. + +#### First, get to the tools directory + +If you built the code + +``` +cd incubator-helix/helix-core/target/helix-core-pkg/bin +``` + +If you downloaded the release package, extract it. + + +Short Version +------------- +You can observe the components working together in this demo, which does the following: + +* Create a cluster +* Add 2 nodes (participants) to the cluster +* Set up a resource with 6 partitions and 2 replicas: 1 Master, and 1 Slave per partition +* Show the cluster state after Helix balances the partitions +* Add a third node +* Show the cluster state. Note that the third node has taken mastership of 2 partitions. +* Kill the third node (Helix takes care of failover) +* Show the cluster state. Note that the two surviving nodes take over mastership of the partitions from the failed node + +##### Run the demo + +``` +cd incubator-helix/helix-core/target/helix-core-pkg/bin +./quickstart.sh +``` + +##### 2 nodes are set up and the partitions rebalanced + +The cluster state is as follows: + +``` +CLUSTER STATE: After starting 2 nodes + localhost_12000 localhost_12001 + MyResource_0 M S + MyResource_1 S M + MyResource_2 M S + MyResource_3 M S + MyResource_4 S M + MyResource_5 S M +``` + +Note there is one master and one slave per partition. + +##### A third node is added and the cluster rebalanced + +The cluster state changes to: + +``` +CLUSTER STATE: After adding a third node + localhost_12000 localhost_12001 localhost_12002 + MyResource_0 S M S + MyResource_1 S S M + MyResource_2 M S S + MyResource_3 S S M + MyResource_4 M S S + MyResource_5 S M S +``` + +Note there is one master and _two_ slaves per partition. This is expected because there are three nodes. + +##### Finally, a node is killed to simulate a failure + +Helix makes sure each partition has a master. The cluster state changes to: + +``` +CLUSTER STATE: After the 3rd node stops/crashes + localhost_12000 localhost_12001 localhost_12002 + MyResource_0 S M - + MyResource_1 S M - + MyResource_2 M S - + MyResource_3 M S - + MyResource_4 M S - + MyResource_5 S M - +``` + + +Long Version +------------ +Now you can run the same steps by hand. In the detailed version, we\'ll do the following: + +* Define a cluster +* Add two nodes to the cluster +* Add a 6-partition resource with 1 master and 2 slave replicas per partition +* Verify that the cluster is healthy and inspect the Helix view +* Expand the cluster: add a few nodes and rebalance the partitions +* Failover: stop a node and verify the mastership transfer + +### Install and Start Zookeeper + +Zookeeper can be started in standalone mode or replicated mode. + +More info is available at + +* http://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html +* http://zookeeper.apache.org/doc/trunk/zookeeperAdmin.html#sc_zkMulitServerSetup + +In this example, let\'s start zookeeper in local mode. + +##### start zookeeper locally on port 2199 + + ./start-standalone-zookeeper.sh 2199 & + +### Define the Cluster + +The helix-admin tool is used for cluster administration tasks. In the Quickstart, we\'ll use the command line interface. Helix supports a REST interface as well. + +zookeeper_address is of the format host:port e.g localhost:2199 for standalone or host1:port,host2:port for multi-node. + +Next, we\'ll set up a cluster MYCLUSTER cluster with these attributes: + +* 3 instances running on localhost at ports 12913,12914,12915 +* One database named myDB with 6 partitions +* Each partition will have 3 replicas with 1 master, 2 slaves +* zookeeper running locally at localhost:2199 + +##### Create the cluster MYCLUSTER + ## helix-admin.sh --zkSvr --addCluster + ./helix-admin.sh --zkSvr localhost:2199 --addCluster MYCLUSTER + +##### Add nodes to the cluster + +In this case we\'ll add three nodes: localhost:12913, localhost:12914, localhost:12915 + + ## helix-admin.sh --zkSvr --addNode + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12913 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12914 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12915 + +#### Define the resource and partitioning + +In this example, the resource is a database, partitioned 6 ways. (In a production system, it\'s common to over-partition for better load balancing. Helix has been used in production to manage hundreds of databases each with 10s or 100s of partitions running on 10s of physical nodes.) + +##### Create a database with 6 partitions using the MasterSlave state model. + +Helix ensures there will be exactly one master for each partition. + + ## helix-admin.sh --zkSvr --addResource + ./helix-admin.sh --zkSvr localhost:2199 --addResource MYCLUSTER myDB 6 MasterSlave + +##### Now we can let Helix assign partitions to nodes. + +This command will distribute the partitions amongst all the nodes in the cluster. In this example, each partition has 3 replicas. + + ## helix-admin.sh --zkSvr --rebalance + ./helix-admin.sh --zkSvr localhost:2199 --rebalance MYCLUSTER myDB 3 + +Now the cluster is defined in Zookeeper. The nodes (localhost:12913, localhost:12914, localhost:12915) and resource (myDB, with 6 partitions using the MasterSlave model). And the _ideal state_ has been calculated, assuming a replication factor of 3. + +##### Start the Helix Controller + +Now that the cluster is defined in Zookeeper, the Helix controller can manage the cluster. + + ## Start the cluster manager, which will manage MYCLUSTER + ./run-helix-controller.sh --zkSvr localhost:2199 --cluster MYCLUSTER 2>&1 > /tmp/controller.log & + +##### Start up the cluster to be managed + +We\'ve started up Zookeeper, defined the cluster, the resources, the partitioning, and started up the Helix controller. Next, we\'ll start up the nodes of the system to be managed. Each node is a Participant, which is an instance of the system component to be managed. Helix assigns work to Participants, keeps track of their roles and health, and takes action when a node fails. + + # start up each instance. These are mock implementations that are actively managed by Helix + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12913 --stateModelType MasterSlave 2>&1 > /tmp/participant_12913.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12914 --stateModelType MasterSlave 2>&1 > /tmp/participant_12914.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12915 --stateModelType MasterSlave 2>&1 > /tmp/participant_12915.log + + +#### Inspect the Cluster + +Now, let\'s see the Helix view of our cluster. We\'ll work our way down as follows: + +``` +Clusters -> MYCLUSTER -> instances -> instance detail + -> resources -> resource detail + -> partitions +``` + +A single Helix controller can manage multiple clusters, though so far, we\'ve only defined one cluster. Let\'s see: + +``` +## List existing clusters +./helix-admin.sh --zkSvr localhost:2199 --listClusters + +Existing clusters: +MYCLUSTER +``` + +Now, let\'s see the Helix view of MYCLUSTER + +``` +## helix-admin.sh --zkSvr --listClusterInfo +./helix-admin.sh --zkSvr localhost:2199 --listClusterInfo MYCLUSTER + +Existing resources in cluster MYCLUSTER: +myDB +Instances in cluster MYCLUSTER: +localhost_12915 +localhost_12914 +localhost_12913 +``` + + +Let\'s look at the details of an instance + +``` +## ./helix-admin.sh --zkSvr --listInstanceInfo +./helix-admin.sh --zkSvr localhost:2199 --listInstanceInfo MYCLUSTER localhost_12913 + +InstanceConfig: { + "id" : "localhost_12913", + "mapFields" : { + }, + "listFields" : { + }, + "simpleFields" : { + "HELIX_ENABLED" : "true", + "HELIX_HOST" : "localhost", + "HELIX_PORT" : "12913" + } +} +``` + + +##### Query info of a resource + +``` +## helix-admin.sh --zkSvr --listResourceInfo +./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB + +IdealState for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + }, + "myDB_1" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_3" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_4" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + "myDB_0" : [ "localhost_12914", "localhost_12913", "localhost_12915" ], + "myDB_1" : [ "localhost_12915", "localhost_12913", "localhost_12914" ], + "myDB_2" : [ "localhost_12913", "localhost_12915", "localhost_12914" ], + "myDB_3" : [ "localhost_12915", "localhost_12913", "localhost_12914" ], + "myDB_4" : [ "localhost_12913", "localhost_12914", "localhost_12915" ], + "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ] + }, + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "6", + "REPLICAS" : "3", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + } +} + +ExternalView for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + }, + "myDB_1" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_3" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_4" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + }, + "simpleFields" : { + "BUCKET_SIZE" : "0" + } +} +``` + +Now, let\'s look at one of the partitions: + + ## helix-admin.sh --zkSvr --listPartitionInfo + ./helix-admin.sh --zkSvr localhost:2199 --listPartitionInfo MYCLUSTER myDB myDB_0 + +#### Expand the Cluster + +Next, we\'ll show how Helix does the work that you\'d otherwise have to build into your system. When you add capacity to your cluster, you want the work to be evenly distributed. In this example, we started with 3 nodes, with 6 partitions. The partitions were evenly balanced, 2 masters and 4 slaves per node. Let\'s add 3 more nodes: localhost:12916, localhost:12917, localhost:12918 + + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12916 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12917 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12918 + +And start up these instances: + + # start up each instance. These are mock implementations that are actively managed by Helix + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12916 --stateModelType MasterSlave 2>&1 > /tmp/participant_12916.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12917 --stateModelType MasterSlave 2>&1 > /tmp/participant_12917.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12918 --stateModelType MasterSlave 2>&1 > /tmp/participant_12918.log + + +And now, let Helix do the work for you. To shift the work, simply rebalance. After the rebalance, each node will have one master and two slaves. + + ./helix-admin.sh --zkSvr localhost:2199 --rebalance MYCLUSTER myDB 3 + +#### View the cluster + +OK, let\'s see how it looks: + + +``` +./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB + +IdealState for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "SLAVE", + "localhost_12918" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + "myDB_0" : [ "localhost_12917", "localhost_12913", "localhost_12914" ], + "myDB_1" : [ "localhost_12918", "localhost_12917", "localhost_12916" ], + "myDB_2" : [ "localhost_12913", "localhost_12917", "localhost_12918" ], + "myDB_3" : [ "localhost_12915", "localhost_12917", "localhost_12918" ], + "myDB_4" : [ "localhost_12916", "localhost_12917", "localhost_12918" ], + "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ] + }, + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "6", + "REPLICAS" : "3", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + } +} + +ExternalView for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "SLAVE", + "localhost_12918" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + }, + "simpleFields" : { + "BUCKET_SIZE" : "0" + } +} +``` + +Mission accomplished. The partitions are nicely balanced. + +#### How about Failover? + +Building a fault tolerant system isn\'t trivial, but with Helix, it\'s easy. Helix detects a failed instance, and triggers mastership transfer automatically. + +First, let's fail an instance. In this example, we\'ll kill localhost:12918 to simulate a failure. + +We lost localhost:12918, so myDB_1 lost its MASTER. Helix can fix that, it will transfer mastership to a healthy node that is currently a SLAVE, say localhost:12197. Helix balances the load as best as it can, given there are 6 partitions on 5 nodes. Let\'s see: + + +``` +./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB + +IdealState for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "SLAVE", + "localhost_12918" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + "myDB_0" : [ "localhost_12917", "localhost_12913", "localhost_12914" ], + "myDB_1" : [ "localhost_12918", "localhost_12917", "localhost_12916" ], + "myDB_2" : [ "localhost_12913", "localhost_12918", "localhost_12917" ], + "myDB_3" : [ "localhost_12915", "localhost_12918", "localhost_12917" ], + "myDB_4" : [ "localhost_12916", "localhost_12917", "localhost_12918" ], + "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ] + }, + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "6", + "REPLICAS" : "3", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + } +} + +ExternalView for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + }, + "simpleFields" : { + "BUCKET_SIZE" : "0" + } +} +``` + +As we\'ve seen in this Quickstart, Helix takes care of partitioning, load balancing, elasticity, failure detection and recovery. + +##### ZooInspector + +You can view all of the underlying data by going direct to zookeeper. Use ZooInspector that comes with zookeeper to browse the data. This is a java applet (make sure you have X windows) + +To start zooinspector run the following command from /contrib/ZooInspector + + java -cp zookeeper-3.3.3-ZooInspector.jar:lib/jtoaster-1.0.4.jar:../../lib/log4j-1.2.15.jar:../../zookeeper-3.3.3.jar org.apache.zookeeper.inspector.ZooInspector + +#### Next + +Now that you understand the idea of Helix, read the [tutorial](./tutorial.html) to learn how to choose the right state model and constraints for your system, and how to implement it. In many cases, the built-in features meet your requirements. And best of all, Helix is a customizable framework, so you can plug in your own behavior, while retaining the automation provided by Helix. + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/Tutorial.md b/site-releases/0.7.0-incubating/src/site/markdown/Tutorial.md new file mode 100644 index 0000000000..081bc7aff4 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/Tutorial.md @@ -0,0 +1,290 @@ + + + + Tutorial + + +# Helix Tutorial + +In this tutorial, we will cover the roles of a Helix-managed cluster, and show the code you need to write to integrate with it. In many cases, there is a simple default behavior that is often appropriate, but you can also customize the behavior. + +Convention: we first cover the _basic_ approach, which is the easiest to implement. Then, we'll describe _advanced_ options, which give you more control over the system behavior, but require you to write more code. + + +### Prerequisites + +1. Read [Concepts/Terminology](./Concepts.html) and [Architecture](./Architecture.html) +2. Read the [Quickstart guide](./Quickstart.html) to learn how Helix models and manages a cluster +3. Install Helix source. See: [Quickstart](./Quickstart.html) for the steps. + +### Tutorial Outline + +1. [Participant](./tutorial_participant.html) +2. [Spectator](./tutorial_spectator.html) +3. [Controller](./tutorial_controller.html) +4. [Rebalancing Algorithms](./tutorial_rebalance.html) +5. [User-Defined Rebalancing](./tutorial_user_def_rebalancer.html) +6. [State Machines](./tutorial_state.html) +7. [Messaging](./tutorial_messaging.html) +8. [Customized health check](./tutorial_health.html) +9. [Throttling](./tutorial_throttling.html) +10. [Application Property Store](./tutorial_propstore.html) +11. [Logical Accessors](./tutorial_accessors.html) +12. [Admin Interface](./tutorial_admin.html) +13. [YAML Cluster Setup](./tutorial_yaml.html) + +### Preliminaries + +First, we need to set up the system. Let\'s walk through the steps in building a distributed system using Helix. We will show how to do this using both the Java admin interface, as well as the [cluster accessor](./tutorial_accessors.html) interface. You can choose either interface depending on which most closely matches your needs. + +### Start Zookeeper + +This starts a zookeeper in standalone mode. For production deployment, see [Apache Zookeeper](http://zookeeper.apache.org) for instructions. + +``` +./start-standalone-zookeeper.sh 2199 & +``` + +### Create a cluster + +Creating a cluster will define the cluster in appropriate znodes on zookeeper. + +Using the Java accessor API: + +``` +// Note: ZK_ADDRESS is the host:port of Zookeeper +String ZK_ADDRESS = "localhost:2199"; +HelixConnection connection = new ZKHelixConnection(ZK_ADDRESS); + +ClusterId clusterId = ClusterId.from("helix-demo"); +ClusterAccessor clusterAccessor = connection.createClusterAccessor(clusterId); +ClusterConfig clusterConfig = new ClusterConfig.Builder(clusterId).build(); +clusterAccessor.createCluster(clusterConfig); +``` + +OR + +Using the HelixAdmin Java interface: + +``` +// Create setup tool instance +// Note: ZK_ADDRESS is the host:port of Zookeeper +String ZK_ADDRESS = "localhost:2199"; +HelixAdmin admin = new ZKHelixAdmin(ZK_ADDRESS); + +String CLUSTER_NAME = "helix-demo"; +//Create cluster namespace in zookeeper +admin.addCluster(CLUSTER_NAME); +``` + +OR + +Using the command-line interface: + +``` +./helix-admin.sh --zkSvr localhost:2199 --addCluster helix-demo +``` + + +### Configure the nodes of the cluster + +First we\'ll add new nodes to the cluster, then configure the nodes in the cluster. Each node in the cluster must be uniquely identifiable. +The most commonly used convention is hostname_port. + +``` +int NUM_NODES = 2; +String hosts[] = new String[]{"localhost","localhost"}; +int ports[] = new int[]{7000,7001}; +for (int i = 0; i < NUM_NODES; i++) +{ + ParticipantId participantId = ParticipantId.from(hosts[i] + "_" + ports[i]); + + // set additional configuration for the participant; these can be accessed during node start up + UserConfig userConfig = new UserConfig(Scope.participant(participantId)); + userConfig.setSimpleField("key", "value"); + + // configure and add the participant + ParticipantConfig participantConfig = new ParticipantConfig.Builder(participantId) + .hostName(hosts[i]).port(ports[i]).enabled(true).userConfig(userConfig).build(); + clusterAccessor.addParticipantToCluster(participantConfig); +} +``` + +OR + +Using the HelixAdmin Java interface: + +``` +String CLUSTER_NAME = "helix-demo"; +int NUM_NODES = 2; +String hosts[] = new String[]{"localhost","localhost"}; +String ports[] = new String[]{7000,7001}; +for (int i = 0; i < NUM_NODES; i++) +{ + InstanceConfig instanceConfig = new InstanceConfig(hosts[i] + "_" + ports[i]); + instanceConfig.setHostName(hosts[i]); + instanceConfig.setPort(ports[i]); + instanceConfig.setInstanceEnabled(true); + + //Add additional system specific configuration if needed. These can be accessed during the node start up. + instanceConfig.getRecord().setSimpleField("key", "value"); + admin.addInstance(CLUSTER_NAME, instanceConfig); +} +``` + +### Configure the resource + +A _resource_ represents the actual task performed by the nodes. It can be a database, index, topic, queue or any other processing entity. +A _resource_ can be divided into many sub-parts known as _partitions_. + + +#### Define the _state model_ and _constraints_ + +For scalability and fault tolerance, each partition can have one or more replicas. +The _state model_ allows one to declare the system behavior by first enumerating the various STATES, and the TRANSITIONS between them. +A simple model is ONLINE-OFFLINE where ONLINE means the task is active and OFFLINE means it\'s not active. +You can also specify how many replicas must be in each state, these are known as _constraints_. +For example, in a search system, one might need more than one node serving the same index to handle the load. + +The allowed states: + +* MASTER +* SLAVE +* OFFLINE + +The allowed transitions: + +* OFFLINE to SLAVE +* SLAVE to OFFLINE +* SLAVE to MASTER +* MASTER to SLAVE + +The constraints: + +* no more than 1 MASTER per partition +* the rest of the replicas should be slaves + +The following snippet shows how to declare the _state model_ and _constraints_ for the MASTER-SLAVE model. + +``` +StateModelDefinition.Builder builder = new StateModelDefinition.Builder(STATE_MODEL_NAME); + +// Add states and their rank to indicate priority. A lower rank corresponds to a higher priority +builder.addState(MASTER, 1); +builder.addState(SLAVE, 2); +builder.addState(OFFLINE); + +// Set the initial state when the node starts +builder.initialState(OFFLINE); + +// Add transitions between the states. +builder.addTransition(OFFLINE, SLAVE); +builder.addTransition(SLAVE, OFFLINE); +builder.addTransition(SLAVE, MASTER); +builder.addTransition(MASTER, SLAVE); + +// set constraints on states. + +// static constraint: upper bound of 1 MASTER +builder.upperBound(MASTER, 1); + +// dynamic constraint: R means it should be derived based on the replication factor for the cluster +// this allows a different replication factor for each resource without +// having to define a new state model +// +builder.dynamicUpperBound(SLAVE, "R"); +StateModelDefinition statemodelDefinition = builder.build(); +``` + +Then, add the state model definition: + +``` +clusterAccessor.addStateModelDefinitionToCluster(stateModelDefinition); +``` + +OR + +``` +admin.addStateModelDef(CLUSTER_NAME, STATE_MODEL_NAME, stateModelDefinition); +``` + +#### Assigning partitions to nodes + +The final goal of Helix is to ensure that the constraints on the state model are satisfied. +Helix does this by assigning a STATE to a partition (such as MASTER, SLAVE), and placing it on a particular node. + +There are 3 assignment modes Helix can operate on + +* FULL_AUTO: Helix decides the placement and state of a partition. +* SEMI_AUTO: Application decides the placement but Helix decides the state of a partition. +* CUSTOMIZED: Application controls the placement and state of a partition. + +For more info on the assignment modes, see [Rebalancing Algorithms](./tutorial_rebalance.html) section of the tutorial. + +Here is an example of adding the resource in SEMI_AUTO mode (i.e. locations of partitions are specified a priori): + +``` +int NUM_PARTITIONS = 6; +int NUM_REPLICAS = 2; +ResourceId resourceId = resourceId.from("MyDB"); + +SemiAutoRebalancerContext context = new SemiAutoRebalancerContext.Builder(resourceId) + .replicaCount(NUM_REPLICAS).addPartitions(NUM_PARTITIONS) + .stateModelDefId(stateModelDefinition.getStateModelDefId()) + .addPreferenceList(partition1Id, preferenceList) // preferred locations of each partition + // add other preference lists per partition + .build(); + +// or add all preference lists at once if desired (map of PartitionId to List of ParticipantId) +context.setPreferenceLists(preferenceLists); + +// or generate a default set of preference lists given the set of all participants +context.generateDefaultConfiguration(stateModelDefinition, participantIdSet); + +// add the resource to the cluster +ResourceConfig resourceConfig = new ResourceConfig.Builder(resourceId) + .rebalancerContext(context) + .build(); +clusterAccessor.addResourceToCluster(resourceConfig); +``` + +OR + +``` +String RESOURCE_NAME = "MyDB"; +int NUM_PARTITIONS = 6; +String MODE = "SEMI_AUTO"; +int NUM_REPLICAS = 2; + +admin.addResource(CLUSTER_NAME, RESOURCE_NAME, NUM_PARTITIONS, STATE_MODEL_NAME, MODE); + +// specify the preference lists yourself +IdealState idealState = admin.getResourceIdealState(CLUSTER_NAME, RESOURCE_NAME); +idealState.setPreferenceList(partitionId, preferenceList); // preferred locations of each partition +// add other preference lists per partition + +// or add all preference lists at once if desired +idealState.getRecord().setListFields(preferenceLists); +admin.setResourceIdealState(CLUSTER_NAME, RESOURCE_NAME, idealState); + +// or generate a default set of preference lists +admin.rebalance(CLUSTER_NAME, RESOURCE_NAME, NUM_REPLICAS); +``` + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/UseCases.md b/site-releases/0.7.0-incubating/src/site/markdown/UseCases.md new file mode 100644 index 0000000000..001b01247a --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/UseCases.md @@ -0,0 +1,113 @@ + + + + Use Cases + + + +# Use cases at LinkedIn + +At LinkedIn Helix framework is used to manage 3 distributed data systems which are quite different from each other. + +* Espresso +* Databus +* Search As A Service + +## Espresso + +Espresso is a distributed, timeline consistent, scal- able, document store that supports local secondary indexing and local transactions. +Espresso databases are horizontally partitioned into a number of partitions, with each partition having a certain number of replicas +distributed across the storage nodes. +Espresso designates one replica of each partition as master and the rest as slaves; only one master may exist for each partition at any time. +Espresso enforces timeline consistency where only the master of a partition can accept writes to its records, and all slaves receive and +apply the same writes through a replication stream. +For load balancing, both master and slave partitions are assigned evenly across all storage nodes. +For fault tolerance, it adds the constraint that no two replicas of the same partition may be located on the same node. + +### State model +Espresso follows a Master-Slave state model. A replica can be in Offline,Slave or Master state. +The state machine table describes the next state given the Current State, Final State + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +### Constraints +* Max number of replicas in Master state:1 +* Execution mode AUTO. i.e on node failure no new replicas will be created. Only the State of remaining replicas will be changed. +* Number of mastered partitions on each node must be approximately same. +* The above constraint must be satisfied when a node fails or a new node is added. +* When new nodes are added the number of partitions moved must be minimized. +* When new nodes are added the max number of OFFLINE-SLAVE transitions that can happen concurrently on new node is X. + +## Databus + +Databus is a change data capture (CDC) system that provides a common pipeline for transporting events +from LinkedIn primary databases to caches within various applications. +Databus deploys a cluster of relays that pull the change log from multiple databases and +let consumers subscribe to the change log stream. Each Databus relay connects to one or more database servers and +hosts a certain subset of databases (and partitions) from those database servers. + +For a large partitioned database (e.g. Espresso), the change log is consumed by a bank of consumers. +Each databus partition is assigned to a consumer such that partitions are evenly distributed across consumers and each partition is +assigned to exactly one consumer at a time. The set of consumers may grow over time, and consumers may leave the group due to planned or unplanned +outages. In these cases, partitions must be reassigned, while maintaining balance and the single consumer-per-partition invariant. + +### State model +Databus consumers follow a simple Offline-Online state model. +The state machine table describes the next state given the Current State, Final State + +

+          OFFLINE  | ONLINE |   
+         ___________________|
+        |          |        |
+OFFLINE |   N/A    | ONLINE |
+        |__________|________|
+        |          |        |
+ONLINE  |  OFFLINE |   N/A  |
+        |__________|________|
+
+
+
+ + +## Search As A Service + +LinkedIn�s Search-as-a-service lets internal customers define custom indexes on a chosen dataset +and then makes those indexes searchable via a service API. The index service runs on a cluster of machines. +The index is broken into partitions and each partition has a configured number of replicas. +Each cluster server runs an instance of the Sensei system (an online index store) and hosts index partitions. +Each new indexing service gets assigned to a set of servers, and the partition replicas must be evenly distributed across those servers. + +### State model +![Helix Design](images/bootstrap_statemodel.gif) + + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/index.md b/site-releases/0.7.0-incubating/src/site/markdown/index.md new file mode 100644 index 0000000000..f983273709 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/index.md @@ -0,0 +1,60 @@ + + + + Home + + +Navigating the Documentation +---------------------------- + +### Conceptual Understanding + +[Concepts / Terminology](./Concepts.html) + +[Architecture](./Architecture.html) + +### Hands-on Helix + +[Getting Helix](./Building.html) + +[Quickstart](./Quickstart.html) + +[Tutorial](./Tutorial.html) + +[Javadocs](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/index.html) + +### Recipes + +[Distributed lock manager](./recipes/lock_manager.html) + +[Rabbit MQ consumer group](./recipes/rabbitmq_consumer_group.html) + +[Rsync replicated file store](./recipes/rsync_replicated_file_store.html) + +[Service discovery](./recipes/service_discovery.html) + +[Distributed Task DAG Execution](./recipes/task_dag_execution.html) + +[User-Defined Rebalancer Example](./recipes/user_def_rebalancer.html) + +### Download + +[0.7.0-incubating](./download.html) + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/recipes/lock_manager.md b/site-releases/0.7.0-incubating/src/site/markdown/recipes/lock_manager.md new file mode 100644 index 0000000000..252ace7eab --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/recipes/lock_manager.md @@ -0,0 +1,253 @@ + +Distributed lock manager +------------------------ +Distributed locks are used to synchronize accesses shared resources. Most applications use Zookeeper to model the distributed locks. + +The simplest way to model a lock using zookeeper is (See Zookeeper leader recipe for an exact and more advanced solution) + +* Each process tries to create an emphemeral node. +* If can successfully create it then, it acquires the lock +* Else it will watch on the znode and try to acquire the lock again if the current lock holder disappears + +This is good enough if there is only one lock. But in practice, an application will need many such locks. Distributing and managing the locks among difference process becomes challenging. Extending such a solution to many locks will result in + +* Uneven distribution of locks among nodes, the node that starts first will acquire all the lock. Nodes that start later will be idle. +* When a node fails, how the locks will be distributed among remaining nodes is not predicable. +* When new nodes are added the current nodes dont relinquish the locks so that new nodes can acquire some locks + +In other words we want a system to satisfy the following requirements. + +* Distribute locks evenly among all nodes to get better hardware utilization +* If a node fails, the locks that were acquired by that node should be evenly distributed among other nodes +* If nodes are added, locks must be evenly re-distributed among nodes. + +Helix provides a simple and elegant solution to this problem. Simply specify the number of locks and Helix will ensure that above constraints are satisfied. + +To quickly see this working run the lock-manager-demo script where 12 locks are evenly distributed among three nodes, and when a node fails, the locks get re-distributed among remaining two nodes. Note that Helix does not re-shuffle the locks completely, instead it simply distributes the locks relinquished by dead node among 2 remaining nodes evenly. + +---------------------------------------------------------------------------------------- + +#### Short version + This version starts multiple threads with in same process to simulate a multi node deployment. Try the long version to get a better idea of how it works. + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/distributed-lock-manager/target/distributed-lock-manager-pkg/bin +chmod +x * +./lock-manager-demo +``` + +##### Output + +``` +./lock-manager-demo +STARTING localhost_12000 +STARTING localhost_12002 +STARTING localhost_12001 +STARTED localhost_12000 +STARTED localhost_12002 +STARTED localhost_12001 +localhost_12001 acquired lock:lock-group_3 +localhost_12000 acquired lock:lock-group_8 +localhost_12001 acquired lock:lock-group_2 +localhost_12001 acquired lock:lock-group_4 +localhost_12002 acquired lock:lock-group_1 +localhost_12002 acquired lock:lock-group_10 +localhost_12000 acquired lock:lock-group_7 +localhost_12001 acquired lock:lock-group_5 +localhost_12002 acquired lock:lock-group_11 +localhost_12000 acquired lock:lock-group_6 +localhost_12002 acquired lock:lock-group_0 +localhost_12000 acquired lock:lock-group_9 +lockName acquired By +====================================== +lock-group_0 localhost_12002 +lock-group_1 localhost_12002 +lock-group_10 localhost_12002 +lock-group_11 localhost_12002 +lock-group_2 localhost_12001 +lock-group_3 localhost_12001 +lock-group_4 localhost_12001 +lock-group_5 localhost_12001 +lock-group_6 localhost_12000 +lock-group_7 localhost_12000 +lock-group_8 localhost_12000 +lock-group_9 localhost_12000 +Stopping localhost_12000 +localhost_12000 Interrupted +localhost_12001 acquired lock:lock-group_9 +localhost_12001 acquired lock:lock-group_8 +localhost_12002 acquired lock:lock-group_6 +localhost_12002 acquired lock:lock-group_7 +lockName acquired By +====================================== +lock-group_0 localhost_12002 +lock-group_1 localhost_12002 +lock-group_10 localhost_12002 +lock-group_11 localhost_12002 +lock-group_2 localhost_12001 +lock-group_3 localhost_12001 +lock-group_4 localhost_12001 +lock-group_5 localhost_12001 +lock-group_6 localhost_12002 +lock-group_7 localhost_12002 +lock-group_8 localhost_12001 +lock-group_9 localhost_12001 + +``` + +---------------------------------------------------------------------------------------- + +#### Long version +This provides more details on how to setup the cluster and where to plugin application code. + +##### start zookeeper + +``` +./start-standalone-zookeeper 2199 +``` + +##### Create a cluster + +``` +./helix-admin --zkSvr localhost:2199 --addCluster lock-manager-demo +``` + +##### Create a lock group + +Create a lock group and specify the number of locks in the lock group. + +``` +./helix-admin --zkSvr localhost:2199 --addResource lock-manager-demo lock-group 6 OnlineOffline FULL_AUTO +``` + +##### Start the nodes + +Create a Lock class that handles the callbacks. + +``` + +public class Lock extends StateModel +{ + private String lockName; + + public Lock(String lockName) + { + this.lockName = lockName; + } + + public void lock(Message m, NotificationContext context) + { + System.out.println(" acquired lock:"+ lockName ); + } + + public void release(Message m, NotificationContext context) + { + System.out.println(" releasing lock:"+ lockName ); + } + +} + +``` + +LockFactory that creates the lock + +``` +public class LockFactory extends StateModelFactory{ + + /* Instantiates the lock handler, one per lockName*/ + public Lock create(String lockName) + { + return new Lock(lockName); + } +} +``` + +At node start up, simply join the cluster and helix will invoke the appropriate callbacks on Lock instance. One can start any number of nodes and Helix detects that a new node has joined the cluster and re-distributes the locks automatically. + +``` +public class LockProcess{ + + public static void main(String args){ + String zkAddress= "localhost:2199"; + String clusterName = "lock-manager-demo"; + //Give a unique id to each process, most commonly used format hostname_port + String instanceName ="localhost_12000"; + ZKHelixAdmin helixAdmin = new ZKHelixAdmin(zkAddress); + //configure the instance and provide some metadata + InstanceConfig config = new InstanceConfig(instanceName); + config.setHostName("localhost"); + config.setPort("12000"); + admin.addInstance(clusterName, config); + //join the cluster + HelixManager manager; + manager = HelixManagerFactory.getHelixManager(clusterName, + instanceName, + InstanceType.PARTICIPANT, + zkAddress); + manager.getStateMachineEngine().registerStateModelFactory("OnlineOffline", modelFactory); + manager.connect(); + Thread.currentThread.join(); + } + +} +``` + +##### Start the controller + +Controller can be started either as a separate process or can be embedded within each node process + +###### Separate process +This is recommended when number of nodes in the cluster >100. For fault tolerance, you can run multiple controllers on different boxes. + +``` +./run-helix-controller --zkSvr localhost:2199 --cluster lock-manager-demo 2>&1 > /tmp/controller.log & +``` + +###### Embedded within the node process +This is recommended when the number of nodes in the cluster is less than 100. To start a controller from each process, simply add the following lines to MyClass + +``` +public class LockProcess{ + + public static void main(String args){ + String zkAddress= "localhost:2199"; + String clusterName = "lock-manager-demo"; + . + . + manager.connect(); + HelixManager controller; + controller = HelixControllerMain.startHelixController(zkAddress, + clusterName, + "controller", + HelixControllerMain.STANDALONE); + Thread.currentThread.join(); + } +} +``` + +---------------------------------------------------------------------------------------- + + + + + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/recipes/rabbitmq_consumer_group.md b/site-releases/0.7.0-incubating/src/site/markdown/recipes/rabbitmq_consumer_group.md new file mode 100644 index 0000000000..9edc2cb242 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/recipes/rabbitmq_consumer_group.md @@ -0,0 +1,227 @@ + + + +RabbitMQ Consumer Group +======================= + +[RabbitMQ](http://www.rabbitmq.com/) is a well known Open source software the provides robust messaging for applications. + +One of the commonly implemented recipes using this software is a work queue. http://www.rabbitmq.com/tutorials/tutorial-four-java.html describes the use case where + +* A producer sends a message with a routing key. +* The message is routed to the queue whose binding key exactly matches the routing key of the message. +* There are multiple consumers and each consumer is interested in processing only a subset of the messages by binding to the interested keys + +The example provided [here](http://www.rabbitmq.com/tutorials/tutorial-four-java.html) describes how multiple consumers can be started to process all the messages. + +While this works, in production systems one needs the following + +* Ability to handle failures: when a consumers fails another consumer must be started or the other consumers must start processing these messages that should have been processed by the failed consumer. +* When the existing consumers cannot keep up with the task generation rate, new consumers will be added. The tasks must be redistributed among all the consumers. + +In this recipe, we demonstrate handling of consumer failures and new consumer additions using Helix. + +Mapping this usecase to Helix is pretty easy as the binding key/routing key is equivalent to a partition. + +Let's take an example. Lets say the queue has 6 partitions, and we have 2 consumers to process all the queues. +What we want is all 6 queues to be evenly divided among 2 consumers. +Eventually when the system scales, we add more consumers to keep up. This will make each consumer process tasks from 2 queues. +Now let's say that a consumer failed which reduces the number of active consumers to 2. This means each consumer must process 3 queues. + +We showcase how such a dynamic App can be developed using Helix. Even though we use rabbitmq as the pub/sub system one can extend this solution to other pub/sub systems. + +Try it +====== + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/rabbitmq-consumer-group/bin +chmod +x * +export HELIX_PKG_ROOT=`pwd`/helix-core/target/helix-core-pkg +export HELIX_RABBITMQ_ROOT=`pwd`/recipes/rabbitmq-consumer-group/ +chmod +x $HELIX_PKG_ROOT/bin/* +chmod +x $HELIX_RABBITMQ_ROOT/bin/* +``` + + +Install Rabbit MQ +---------------- + +Setting up RabbitMQ on a local box is straightforward. You can find the instructions here +http://www.rabbitmq.com/download.html + +Start ZK +-------- +Start zookeeper at port 2199 + +``` +$HELIX_PKG_ROOT/bin/start-standalone-zookeeper 2199 +``` + +Setup the consumer group cluster +-------------------------------- +This will setup the cluster by creating a "rabbitmq-consumer-group" cluster and adds a "topic" with "6" queues. + +``` +$HELIX_RABBITMQ_ROOT/bin/setup-cluster.sh localhost:2199 +``` + +Add consumers +------------- +Start 2 consumers in 2 different terminals. Each consumer is given a unique id. + +``` +//start-consumer.sh zookeeperAddress (e.g. localhost:2181) consumerId , rabbitmqServer (e.g. localhost) +$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 0 localhost +$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 1 localhost + +``` + +Start HelixController +-------------------- +Now start a Helix controller that starts managing the "rabbitmq-consumer-group" cluster. + +``` +$HELIX_RABBITMQ_ROOT/bin/start-cluster-manager.sh localhost:2199 +``` + +Send messages to the Topic +-------------------------- + +Start sending messages to the topic. This script randomly selects a routing key (1-6) and sends the message to topic. +Based on the key, messages gets routed to the appropriate queue. + +``` +$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 20 +``` + +After running this, you should see all 20 messages being processed by 2 consumers. + +Add another consumer +-------------------- +Once a new consumer is started, helix detects it. In order to balance the load between 3 consumers, it deallocates 1 partition from the existing consumers and allocates it to the new consumer. We see that +each consumer is now processing only 2 queues. +Helix makes sure that old nodes are asked to stop consuming before the new consumer is asked to start consuming for a given partition. But the transitions for each partition can happen in parallel. + +``` +$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 2 localhost +``` + +Send messages again to the topic. + +``` +$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 100 +``` + +You should see that messages are now received by all 3 consumers. + +Stop a consumer +--------------- +In any terminal press CTRL^C and notice that Helix detects the consumer failure and distributes the 2 partitions that were processed by failed consumer to the remaining 2 active consumers. + + +How does it work +================ + +Find the entire code [here](https://git-wip-us.apache.org/repos/asf?p=incubator-helix.git;a=tree;f=recipes/rabbitmq-consumer-group/src/main/java/org/apache/helix/recipes/rabbitmq). + +Cluster setup +------------- +This step creates znode on zookeeper for the cluster and adds the state model. We use online offline state model since there is no need for other states. The consumer is either processing a queue or it is not. + +It creates a resource called "rabbitmq-consumer-group" with 6 partitions. The execution mode is set to FULL_AUTO. This means that the Helix controls the assignment of partition to consumers and automatically distributes the partitions evenly among the active consumers. When a consumer is added or removed, it ensures that a minimum number of partitions are shuffled. + +``` + zkclient = new ZkClient(zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT, + ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer()); + ZKHelixAdmin admin = new ZKHelixAdmin(zkclient); + + // add cluster + admin.addCluster(clusterName, true); + + // add state model definition + StateModelConfigGenerator generator = new StateModelConfigGenerator(); + admin.addStateModelDef(clusterName, "OnlineOffline", + new StateModelDefinition(generator.generateConfigForOnlineOffline())); + + // add resource "topic" which has 6 partitions + String resourceName = "rabbitmq-consumer-group"; + admin.addResource(clusterName, resourceName, 6, "OnlineOffline", "FULL_AUTO"); +``` + +Starting the consumers +---------------------- +The only thing consumers need to know is the zkaddress, cluster name and consumer id. It does not need to know anything else. + +``` + _manager = + HelixManagerFactory.getZKHelixManager(_clusterName, + _consumerId, + InstanceType.PARTICIPANT, + _zkAddr); + + StateMachineEngine stateMach = _manager.getStateMachineEngine(); + ConsumerStateModelFactory modelFactory = + new ConsumerStateModelFactory(_consumerId, _mqServer); + stateMach.registerStateModelFactory("OnlineOffline", modelFactory); + + _manager.connect(); + +``` + +Once the consumer has registered the statemodel and the controller is started, the consumer starts getting callbacks (onBecomeOnlineFromOffline) for the partition it needs to host. All it needs to do as part of the callback is to start consuming messages from the appropriate queue. Similarly, when the controller deallocates a partitions from a consumer, it fires onBecomeOfflineFromOnline for the same partition. +As a part of this transition, the consumer will stop consuming from a that queue. + +``` + @Transition(to = "ONLINE", from = "OFFLINE") + public void onBecomeOnlineFromOffline(Message message, NotificationContext context) + { + LOG.debug(_consumerId + " becomes ONLINE from OFFLINE for " + _partition); + + if (_thread == null) + { + LOG.debug("Starting ConsumerThread for " + _partition + "..."); + _thread = new ConsumerThread(_partition, _mqServer, _consumerId); + _thread.start(); + LOG.debug("Starting ConsumerThread for " + _partition + " done"); + + } + } + + @Transition(to = "OFFLINE", from = "ONLINE") + public void onBecomeOfflineFromOnline(Message message, NotificationContext context) + throws InterruptedException + { + LOG.debug(_consumerId + " becomes OFFLINE from ONLINE for " + _partition); + + if (_thread != null) + { + LOG.debug("Stopping " + _consumerId + " for " + _partition + "..."); + + _thread.interrupt(); + _thread.join(2000); + _thread = null; + LOG.debug("Stopping " + _consumerId + " for " + _partition + " done"); + + } + } +``` \ No newline at end of file diff --git a/site-releases/0.7.0-incubating/src/site/markdown/recipes/rsync_replicated_file_store.md b/site-releases/0.7.0-incubating/src/site/markdown/recipes/rsync_replicated_file_store.md new file mode 100644 index 0000000000..f8a74a0abc --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/recipes/rsync_replicated_file_store.md @@ -0,0 +1,165 @@ + + +Near real time rsync replicated file system +=========================================== + +Quickdemo +--------- + +* This demo starts 3 instances with id's as ```localhost_12001, localhost_12002, localhost_12003``` +* Each instance stores its files under ```/tmp//filestore``` +* ``` localhost_12001 ``` is designated as the master and ``` localhost_12002 and localhost_12003``` are the slaves. +* Files written to master are replicated to the slaves automatically. In this demo, a.txt and b.txt are written to ```/tmp/localhost_12001/filestore``` and it gets replicated to other folders. +* When the master is stopped, ```localhost_12002``` is promoted to master. +* The other slave ```localhost_12003``` stops replicating from ```localhost_12001``` and starts replicating from new master ```localhost_12002``` +* Files written to new master ```localhost_12002``` are replicated to ```localhost_12003``` +* In the end state of this quick demo, ```localhost_12002``` is the master and ```localhost_12003``` is the slave. Manually create files under ```/tmp/localhost_12002/filestore``` and see that appears in ```/tmp/localhost_12003/filestore``` +* Ignore the interrupted exceptions on the console :-). + + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd recipes/rsync-replicated-file-system/ +mvn clean install package -DskipTests +cd target/rsync-replicated-file-system-pkg/bin +chmod +x * +./quickdemo + +``` + +Overview +-------- + +There are many applications that require storage for storing large number of relatively small data files. Examples include media stores to store small videos, images, mail attachments etc. Each of these objects is typically kilobytes, often no larger than a few megabytes. An additional distinguishing feature of these usecases is also that files are typically only added or deleted, rarely updated. When there are updates, they are rare and do not have any concurrency requirements. + +These are much simpler requirements than what general purpose distributed file system have to satisfy including concurrent access to files, random access for reads and updates, posix compliance etc. To satisfy those requirements, general DFSs are also pretty complex that are expensive to build and maintain. + +A different implementation of a distributed file system includes HDFS which is inspired by Google's GFS. This is one of the most widely used distributed file system that forms the main data storage platform for Hadoop. HDFS is primary aimed at processing very large data sets and distributes files across a cluster of commodity servers by splitting up files in fixed size chunks. HDFS is not particularly well suited for storing a very large number of relatively tiny files. + +### File Store + +It's possible to build a vastly simpler system for the class of applications that have simpler requirements as we have pointed out. + +* Large number of files but each file is relatively small. +* Access is limited to create, delete and get entire files. +* No updates to files that are already created (or it's feasible to delete the old file and create a new one). + + +We call this system a Partitioned File Store (PFS) to distinguish it from other distributed file systems. This system needs to provide the following features: + +* CRD access to large number of small files +* Scalability: Files should be distributed across a large number of commodity servers based on the storage requirement. +* Fault-tolerance: Each file should be replicated on multiple servers so that individual server failures do not reduce availability. +* Elasticity: It should be possible to add capacity to the cluster easily. + + +Apache Helix is a generic cluster management framework that makes it very easy to provide the scalability, fault-tolerance and elasticity features. +Rsync can be easily used as a replication channel between servers so that each file gets replicated on multiple servers. + +Design +------ + +High level + +* Partition the file system based on the file name. +* At any time a single writer can write, we call this a master. +* For redundancy, we need to have additional replicas called slave. Slaves can optionally serve reads. +* Slave replicates data from the master. +* When a master fails, slave gets promoted to master. + +### Transaction log + +Every write on the master will result in creation/deletion of one or more files. In order to maintain timeline consistency slaves need to apply the changes in the same order. +To facilitate this, the master logs each transaction in a file and each transaction is associated with an 64 bit id in which the 32 LSB represents a sequence number and MSB represents the generation number. +Sequence gets incremented on every transaction and and generation is increment when a new master is elected. + +### Replication + +Replication is required to slave to keep up with the changes on the master. Every time the slave applies a change it checkpoints the last applied transaction id. +During restarts, this allows the slave to pull changes from the last checkpointed id. Similar to master, the slave logs each transaction to the transaction logs but instead of generating new transaction id, it uses the same id generated by the master. + + +### Fail over + +When a master fails, a new slave will be promoted to master. If the prev master node is reachable, then the new master will flush all the +changes from previous master before taking up mastership. The new master will record the end transaction id of the current generation and then starts new generation +with sequence starting from 1. After this the master will begin accepting writes. + + +![Partitioned File Store](../images/PFS-Generic.png) + + + +Rsync based solution +------------------- + +![Rsync based File Store](../images/RSYNC_BASED_PFS.png) + + +This application demonstrate a file store that uses rsync as the replication mechanism. One can envision a similar system where instead of using rsync, +can implement a custom solution to notify the slave of the changes and also provide an api to pull the change files. +#### Concept +* file_store_dir: Root directory for the actual data files +* change_log_dir: The transaction logs are generated under this folder. +* check_point_dir: The slave stores the check points ( last processed transaction) here. + +#### Master +* File server: This component support file uploads and downloads and writes the files to ```file_store_dir```. This is not included in this application. Idea is that most applications have different ways of implementing this component and has some business logic associated with it. It is not hard to come up with such a component if needed. +* File store watcher: This component watches the ```file_store_dir``` directory on the local file system for any changes and notifies the registered listeners of the changes. +* Change Log Generator: This registers as a listener of File System Watcher and on each notification logs the changes into a file under ```change_log_dir```. + +####Slave +* File server: This component on the slave will only support reads. +* Cluster state observer: Slave observes the cluster state and is able to know who is the current master. +* Replicator: This has two subcomponents + - Periodic rsync of change log: This is a background process that periodically rsyncs the ```change_log_dir``` of the master to its local directory + - Change Log Watcher: This watches the ```change_log_dir``` for changes and notifies the registered listeners of the change + - On demand rsync invoker: This is registered as a listener to change log watcher and on every change invokes rsync to sync only the changed file. + + +#### Coordination + +The coordination between nodes is done by Helix. Helix does the partition management and assigns the partition to multiple nodes based on the replication factor. It elects one the nodes as master and designates others as slaves. +It provides notifications to each node in the form of state transitions ( Offline to Slave, Slave to Master). It also provides notification when there is change is cluster state. +This allows the slave to stop replicating from current master and start replicating from new master. + +In this application, we have only one partition but its very easy to extend it to support multiple partitions. By partitioning the file store, one can add new nodes and Helix will automatically +re-distribute partitions among the nodes. To summarize, Helix provides partition management, fault tolerance and facilitates automated cluster expansion. + + + + + + + + + + + + + + + + + + + + + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/recipes/service_discovery.md b/site-releases/0.7.0-incubating/src/site/markdown/recipes/service_discovery.md new file mode 100644 index 0000000000..8e06ead472 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/recipes/service_discovery.md @@ -0,0 +1,191 @@ + +Service Discovery +----------------- + +One of the common usage of zookeeper is enable service discovery. +The basic idea is that when a server starts up it advertises its configuration/metadata such as host name port etc on zookeeper. +This allows clients to dynamically discover the servers that are currently active. One can think of this like a service registry to which a server registers when it starts and +is automatically deregistered when it shutdowns or crashes. In many cases it serves as an alternative to vips. + +The core idea behind this is to use zookeeper ephemeral nodes. The ephemeral nodes are created when the server registers and all its metadata is put into a znode. +When the server shutdowns, zookeeper automatically removes this znode. + +There are two ways the clients can dynamically discover the active servers + +#### ZOOKEEPER WATCH + +Clients can set a child watch under specific path on zookeeper. +When a new service is registered/deregistered, zookeeper notifies the client via watchevent and the client can read the list of services. Even though this looks trivial, +there are lot of things one needs to keep in mind like ensuring that you first set the watch back on zookeeper before reading data from zookeeper. + + +#### POLL + +Another approach is for the client to periodically read the zookeeper path and get the list of services. + + +Both approaches have pros and cons, for example setting a watch might trigger herd effect if there are large number of clients. This is worst especially when servers are starting up. +But good thing about setting watch is that clients are immediately notified of a change which is not true in case of polling. +In some cases, having both WATCH and POLL makes sense, WATCH allows one to get notifications as soon as possible while POLL provides a safety net if a watch event is missed because of code bug or zookeeper fails to notify. + +##### Other important scenarios to take care of +* What happens when zookeeper session expires. All the watches/ephemeral nodes previously added/created by this server are lost. +One needs to add the watches again , recreate the ephemeral nodes etc. +* Due to network issues or java GC pauses session expiry might happen again and again also known as flapping. Its important for the server to detect this and deregister itself. + +##### Other operational things to consider +* What if the node is behaving badly, one might kill the server but will lose the ability to debug. +It would be nice to have the ability to mark a server as disabled and clients know that a node is disabled and will not contact that node. + +#### Configuration ownership + +This is an important aspect that is often ignored in the initial stages of your development. In common, service discovery pattern means that servers start up with some configuration and then simply puts its configuration/metadata in zookeeper. While this works well in the beginning, +configuration management becomes very difficult since the servers themselves are statically configured. Any change in server configuration implies restarting of the server. Ideally, it will be nice to have the ability to change configuration dynamically without having to restart a server. + +Ideally you want a hybrid solution, a node starts with minimal configuration and gets the rest of configuration from zookeeper. + +h3. How to use Helix to achieve this + +Even though Helix has higher level abstraction in terms of statemachine, constraints and objectives, +service discovery is one of things that existed since we started. +The controller uses the exact mechanism we described above to discover when new servers join the cluster. +We create these znodes under /CLUSTERNAME/LIVEINSTANCES. +Since at any time there is only one controller, we use ZK watch to track the liveness of a server. + +This recipe, simply demonstrate how one can re-use that part for implementing service discovery. This demonstrates multiple MODE's of service discovery + +* POLL: The client reads from zookeeper at regular intervals 30 seconds. Use this if you have 100's of clients +* WATCH: The client sets up watcher and gets notified of the changes. Use this if you have 10's of clients. +* NONE: This does neither of the above, but reads directly from zookeeper when ever needed. + +Helix provides these additional features compared to other implementations available else where + +* It has the concept of disabling a node which means that a badly behaving node, can be disabled using helix admin api. +* It automatically detects if a node connects/disconnects from zookeeper repeatedly and disables the node. +* Configuration management + * Allows one to set configuration via admin api at various granulaties like cluster, instance, resource, partition + * Configuration can be dynamically changed. + * Notifies the server when configuration changes. + + +##### checkout and build + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/service-discovery/target/service-discovery-pkg/bin +chmod +x * +``` + +##### start zookeeper + +``` +./start-standalone-zookeeper 2199 +``` + +#### Run the demo + +``` +./service-discovery-demo.sh +``` + +#### Output + +``` +START:Service discovery demo mode:WATCH + Registering service + host.x.y.z_12000 + host.x.y.z_12001 + host.x.y.z_12002 + host.x.y.z_12003 + host.x.y.z_12004 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Deregistering service: + host.x.y.z_12002 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Registering service:host.x.y.z_12002 +END:Service discovery demo mode:WATCH +============================================= +START:Service discovery demo mode:POLL + Registering service + host.x.y.z_12000 + host.x.y.z_12001 + host.x.y.z_12002 + host.x.y.z_12003 + host.x.y.z_12004 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Deregistering service: + host.x.y.z_12002 + Sleeping for poll interval:30000 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Registering service:host.x.y.z_12002 +END:Service discovery demo mode:POLL +============================================= +START:Service discovery demo mode:NONE + Registering service + host.x.y.z_12000 + host.x.y.z_12001 + host.x.y.z_12002 + host.x.y.z_12003 + host.x.y.z_12004 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Deregistering service: + host.x.y.z_12000 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Registering service:host.x.y.z_12000 +END:Service discovery demo mode:NONE +============================================= + +``` + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/recipes/task_dag_execution.md b/site-releases/0.7.0-incubating/src/site/markdown/recipes/task_dag_execution.md new file mode 100644 index 0000000000..f0474e445e --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/recipes/task_dag_execution.md @@ -0,0 +1,204 @@ + + +# Distributed task execution + + +This recipe is intended to demonstrate how task dependencies can be modeled using primitives provided by Helix. A given task can be run with desired parallelism and will start only when up-stream dependencies are met. The demo executes the task DAG described below using 10 workers. Although the demo starts the workers as threads, there is no requirement that all the workers need to run in the same process. In reality, these workers run on many different boxes on a cluster. When worker fails, Helix takes care of +re-assigning a failed task partition to a new worker. + +Redis is used as a result store. Any other suitable implementation for TaskResultStore can be plugged in. + +### Workflow + + +#### Input + +10000 impression events and around 100 click events are pre-populated in task result store (redis). + +* **ImpEvent**: format: id,isFraudulent,country,gender + +* **ClickEvent**: format: id,isFraudulent,impEventId + +#### Stages + ++ **FilterImps**: Filters impression where isFraudulent=true. + ++ **FilterClicks**: Filters clicks where isFraudulent=true + ++ **impCountsByGender**: Generates impression counts grouped by gender. It does this by incrementing the count for 'impression_gender_counts:' in the task result store (redis hash). Depends on: **FilterImps** + ++ **impCountsByCountry**: Generates impression counts grouped by country. It does this by incrementing the count for 'impression_country_counts:' in the task result store (redis hash). Depends on: **FilterClicks** + ++ **impClickJoin**: Joins clicks with corresponding impression event using impEventId as the join key. Join is needed to pull dimensions not present in click event. Depends on: **FilterImps, FilterClicks** + ++ **clickCountsByGender**: Generates click counts grouped by gender. It does this by incrementing the count for click_gender_counts: in the task result store (redis hash). Depends on: **impClickJoin** + ++ **clickCountsByGender**: Generates click counts grouped by country. It does this by incrementing the count for click_country_counts: in the task result store (redis hash). Depends on: **impClickJoin** + ++ **report**: Reads from all aggregates generated by previous stages and prints them. Depends on: **impCountsByGender, impCountsByCountry, clickCountsByGender,clickCountsByGender** + + +### Creating DAG + +Each stage is represented as a Node along with the upstream dependency and desired parallelism. Each stage is modelled as a resource in Helix using OnlineOffline state model. As part of Offline to Online transition, we watch the external view of upstream resources and wait for them to transition to online state. See Task.java for additional info. + +``` + + Dag dag = new Dag(); + dag.addNode(new Node("filterImps", 10, "")); + dag.addNode(new Node("filterClicks", 5, "")); + dag.addNode(new Node("impClickJoin", 10, "filterImps,filterClicks")); + dag.addNode(new Node("impCountsByGender", 10, "filterImps")); + dag.addNode(new Node("impCountsByCountry", 10, "filterImps")); + dag.addNode(new Node("clickCountsByGender", 5, "impClickJoin")); + dag.addNode(new Node("clickCountsByCountry", 5, "impClickJoin")); + dag.addNode(new Node("report",1,"impCountsByGender,impCountsByCountry,clickCountsByGender,clickCountsByCountry")); + + +``` + +### DEMO + +In order to run the demo, use the following steps + +See http://redis.io/topics/quickstart on how to install redis server + +``` + +Start redis e.g: +./redis-server --port 6379 + +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd recipes/task-execution +mvn clean install package -DskipTests +cd target/task-execution-pkg/bin +chmod +x task-execution-demo.sh +./task-execution-demo.sh 2181 localhost 6379 + +``` + +``` + + + + + + +-----------------+ +----------------+ + | filterImps | | filterClicks | + | (parallelism=10)| | (parallelism=5)| + +----------+-----++ +-------+--------+ + | | | | + | | | | + | | | | + | | +------->--------v------------+ + +--------------<-+ +------v-------+ | impClickJoin | + |impCountsByGender |impCountsByCountry | (parallelism=10) | + |(parallelism=10) |(parallelism=10) ++-------------------+-+ + +-----------+--+ +---+----------+ | | + | | | | + | | | | + | | +--------v---------+ +-v-------------------+ + | | |clickCountsByGender |clickCountsByCountry | + | | |(parallelism=5) | |(parallelism=5) | + | | +----+-------------+ +---------------------+ + | | | | + | | | | + | | | | + +----->+-----+>-----------v----+<---------------+ + | report | + |(parallelism=1) | + +-----------------------+ + +``` + +(credit for above ascii art: http://www.asciiflow.com) + +### OUTPUT + +``` +Done populating dummy data +Executing filter task for filterImps_3 for impressions_demo +Executing filter task for filterImps_2 for impressions_demo +Executing filter task for filterImps_0 for impressions_demo +Executing filter task for filterImps_1 for impressions_demo +Executing filter task for filterImps_4 for impressions_demo +Executing filter task for filterClicks_3 for clicks_demo +Executing filter task for filterClicks_1 for clicks_demo +Executing filter task for filterImps_8 for impressions_demo +Executing filter task for filterImps_6 for impressions_demo +Executing filter task for filterClicks_2 for clicks_demo +Executing filter task for filterClicks_0 for clicks_demo +Executing filter task for filterImps_7 for impressions_demo +Executing filter task for filterImps_5 for impressions_demo +Executing filter task for filterClicks_4 for clicks_demo +Executing filter task for filterImps_9 for impressions_demo +Running AggTask for impCountsByGender_3 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_2 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_0 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_9 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_1 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_4 for filtered_impressions_demo gender +Running AggTask for impCountsByCountry_4 for filtered_impressions_demo country +Running AggTask for impCountsByGender_5 for filtered_impressions_demo gender +Executing JoinTask for impClickJoin_2 +Running AggTask for impCountsByCountry_3 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_1 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_0 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_2 for filtered_impressions_demo country +Running AggTask for impCountsByGender_6 for filtered_impressions_demo gender +Executing JoinTask for impClickJoin_1 +Executing JoinTask for impClickJoin_0 +Executing JoinTask for impClickJoin_3 +Running AggTask for impCountsByGender_8 for filtered_impressions_demo gender +Executing JoinTask for impClickJoin_4 +Running AggTask for impCountsByGender_7 for filtered_impressions_demo gender +Running AggTask for impCountsByCountry_5 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_6 for filtered_impressions_demo country +Executing JoinTask for impClickJoin_9 +Running AggTask for impCountsByCountry_8 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_7 for filtered_impressions_demo country +Executing JoinTask for impClickJoin_5 +Executing JoinTask for impClickJoin_6 +Running AggTask for impCountsByCountry_9 for filtered_impressions_demo country +Executing JoinTask for impClickJoin_8 +Executing JoinTask for impClickJoin_7 +Running AggTask for clickCountsByCountry_1 for joined_clicks_demo country +Running AggTask for clickCountsByCountry_0 for joined_clicks_demo country +Running AggTask for clickCountsByCountry_2 for joined_clicks_demo country +Running AggTask for clickCountsByCountry_3 for joined_clicks_demo country +Running AggTask for clickCountsByGender_1 for joined_clicks_demo gender +Running AggTask for clickCountsByCountry_4 for joined_clicks_demo country +Running AggTask for clickCountsByGender_3 for joined_clicks_demo gender +Running AggTask for clickCountsByGender_2 for joined_clicks_demo gender +Running AggTask for clickCountsByGender_4 for joined_clicks_demo gender +Running AggTask for clickCountsByGender_0 for joined_clicks_demo gender +Running reports task +Impression counts per country +{CANADA=1940, US=1958, CHINA=2014, UNKNOWN=2022, UK=1946} +Click counts per country +{US=24, CANADA=14, CHINA=26, UNKNOWN=14, UK=22} +Impression counts per gender +{F=3325, UNKNOWN=3259, M=3296} +Click counts per gender +{F=33, UNKNOWN=32, M=35} + + +``` + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/recipes/user_def_rebalancer.md b/site-releases/0.7.0-incubating/src/site/markdown/recipes/user_def_rebalancer.md new file mode 100644 index 0000000000..68fd9547b7 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/recipes/user_def_rebalancer.md @@ -0,0 +1,285 @@ + +Lock Manager with a User-Defined Rebalancer +------------------------------------------- +Helix is able to compute node preferences and state assignments automatically using general-purpose algorithms. In many cases, a distributed system implementer may choose to instead define a customized approach to computing the location of replicas, the state mapping, or both in response to the addition or removal of participants. The following is an implementation of the [Distributed Lock Manager](./lock_manager.html) that includes a user-defined rebalancer. + +### Define the cluster and locks + +The YAML file below fully defines the cluster and the locks. A lock can be in one of two states: locked and unlocked. Transitions can happen in either direction, and the locked is preferred. A resource in this example is the entire collection of locks to distribute. A partition is mapped to a lock; in this case that means there are 12 locks. These 12 locks will be distributed across 3 nodes. The constraints indicate that only one replica of a lock can be in the locked state at any given time. These locks can each only have a single holder, defined by a replica count of 1. + +Notice the rebalancer section of the definition. The mode is set to USER_DEFINED and the class name refers to the plugged-in rebalancer implementation that inherits from [HelixRebalancer](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/HelixRebalancer.html). This implementation is called whenever the state of the cluster changes, as is the case when participants are added or removed from the system. + +Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/resources/lock-manager-config.yaml + +``` +clusterName: lock-manager-custom-rebalancer # unique name for the cluster +resources: + - name: lock-group # unique resource name + rebalancer: # we will provide our own rebalancer + mode: USER_DEFINED + class: org.apache.helix.userdefinedrebalancer.LockManagerRebalancer + partitions: + count: 12 # number of locks + replicas: 1 # number of simultaneous holders for each lock + stateModel: + name: lock-unlock # unique model name + states: [LOCKED, RELEASED, DROPPED] # the list of possible states + transitions: # the list of possible transitions + - name: Unlock + from: LOCKED + to: RELEASED + - name: Lock + from: RELEASED + to: LOCKED + - name: DropLock + from: LOCKED + to: DROPPED + - name: DropUnlock + from: RELEASED + to: DROPPED + - name: Undrop + from: DROPPED + to: RELEASED + initialState: RELEASED + constraints: + state: + counts: # maximum number of replicas of a partition that can be in each state + - name: LOCKED + count: "1" + - name: RELEASED + count: "-1" + - name: DROPPED + count: "-1" + priorityList: [LOCKED, RELEASED, DROPPED] # states in order of priority + transition: # transitions priority to enforce order that transitions occur + priorityList: [Unlock, Lock, Undrop, DropUnlock, DropLock] +participants: # list of nodes that can acquire locks + - name: localhost_12001 + host: localhost + port: 12001 + - name: localhost_12002 + host: localhost + port: 12002 + - name: localhost_12003 + host: localhost + port: 12003 +``` + +Then, Helix\'s YAMLClusterSetup tool can read in the configuration and bootstrap the cluster immediately: + +``` +YAMLClusterSetup setup = new YAMLClusterSetup(zkAddress); +InputStream input = + Thread.currentThread().getContextClassLoader() + .getResourceAsStream("lock-manager-config.yaml"); +YAMLClusterSetup.YAMLClusterConfig config = setup.setupCluster(input); +``` + +### Write a rebalancer +Below is a full implementation of a rebalancer that extends [HelixRebalancer](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/HelixRebalancer.html). In this case, it simply throws out the previous resource assignment, computes the target node for as many partition replicas as can hold a lock in the LOCKED state (in this example, one), and assigns them the LOCKED state (which is at the head of the state preference list). Clearly a more robust implementation would likely examine the current ideal state to maintain current assignments, and the full state list to handle models more complicated than this one. However, for a simple lock holder implementation, this is sufficient. + +Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/java/org/apache/helix/userdefinedrebalancer/LockManagerRebalancer.java + +``` +@Override +public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, Cluster cluster, + ResourceCurrentState currentState) { + // Get the rebalcancer context (a basic partitioned one) + PartitionedRebalancerContext context = rebalancerConfig.getRebalancerContext( + PartitionedRebalancerContext.class); + + // Initialize an empty mapping of locks to participants + ResourceAssignment assignment = new ResourceAssignment(context.getResourceId()); + + // Get the list of live participants in the cluster + List liveParticipants = new ArrayList( + cluster.getLiveParticipantMap().keySet()); + + // Get the state model (should be a simple lock/unlock model) and the highest-priority state + StateModelDefId stateModelDefId = context.getStateModelDefId(); + StateModelDefinition stateModelDef = cluster.getStateModelMap().get(stateModelDefId); + if (stateModelDef.getStatesPriorityList().size() < 1) { + LOG.error("Invalid state model definition. There should be at least one state."); + return assignment; + } + State lockState = stateModelDef.getTypedStatesPriorityList().get(0); + + // Count the number of participants allowed to lock each lock + String stateCount = stateModelDef.getNumParticipantsPerState(lockState); + int lockHolders = 0; + try { + // a numeric value is a custom-specified number of participants allowed to lock the lock + lockHolders = Integer.parseInt(stateCount); + } catch (NumberFormatException e) { + LOG.error("Invalid state model definition. The lock state does not have a valid count"); + return assignment; + } + + // Fairly assign the lock state to the participants using a simple mod-based sequential + // assignment. For instance, if each lock can be held by 3 participants, lock 0 would be held + // by participants (0, 1, 2), lock 1 would be held by (1, 2, 3), and so on, wrapping around the + // number of participants as necessary. + // This assumes a simple lock-unlock model where the only state of interest is which nodes have + // acquired each lock. + int i = 0; + for (PartitionId partition : context.getPartitionSet()) { + Map replicaMap = new HashMap(); + for (int j = i; j < i + lockHolders; j++) { + int participantIndex = j % liveParticipants.size(); + ParticipantId participant = liveParticipants.get(participantIndex); + // enforce that a participant can only have one instance of a given lock + if (!replicaMap.containsKey(participant)) { + replicaMap.put(participant, lockState); + } + } + assignment.addReplicaMap(partition, replicaMap); + i++; + } + return assignment; +} +``` + +### Start up the participants +Here is a lock class based on the newly defined lock-unlock state model so that the participant can receive callbacks on state transitions. + +Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java + +``` +public class Lock extends StateModel { + private String lockName; + + public Lock(String lockName) { + this.lockName = lockName; + } + + @Transition(from = "RELEASED", to = "LOCKED") + public void lock(Message m, NotificationContext context) { + System.out.println(context.getManager().getInstanceName() + " acquired lock:" + lockName); + } + + @Transition(from = "LOCKED", to = "RELEASED") + public void release(Message m, NotificationContext context) { + System.out.println(context.getManager().getInstanceName() + " releasing lock:" + lockName); + } +} +``` + +Here is the factory to make the Lock class accessible. + +Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/java/org/apache/helix/userdefinedrebalancer/LockFactory.java + +``` +public class LockFactory extends StateModelFactory { + @Override + public Lock createNewStateModel(String lockName) { + return new Lock(lockName); + } +} +``` + +Finally, here is the factory registration and the start of the participant: + +``` +participantManager = + HelixManagerFactory.getZKHelixManager(clusterName, participantName, InstanceType.PARTICIPANT, + zkAddress); +participantManager.getStateMachineEngine().registerStateModelFactory(stateModelName, + new LockFactory()); +participantManager.connect(); +``` + +### Start up the controller + +``` +controllerManager = + HelixControllerMain.startHelixController(zkAddress, config.clusterName, "controller", + HelixControllerMain.STANDALONE); +``` + +### Try it out +#### Building +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/user-rebalanced-lock-manager/target/user-rebalanced-lock-manager-pkg/bin +chmod +x * +./lock-manager-demo.sh +``` + +#### Output + +``` +./lock-manager-demo +STARTING localhost_12002 +STARTING localhost_12001 +STARTING localhost_12003 +STARTED localhost_12001 +STARTED localhost_12003 +STARTED localhost_12002 +localhost_12003 acquired lock:lock-group_4 +localhost_12002 acquired lock:lock-group_8 +localhost_12001 acquired lock:lock-group_10 +localhost_12001 acquired lock:lock-group_3 +localhost_12001 acquired lock:lock-group_6 +localhost_12003 acquired lock:lock-group_0 +localhost_12002 acquired lock:lock-group_5 +localhost_12001 acquired lock:lock-group_9 +localhost_12002 acquired lock:lock-group_2 +localhost_12003 acquired lock:lock-group_7 +localhost_12003 acquired lock:lock-group_11 +localhost_12002 acquired lock:lock-group_1 +lockName acquired By +====================================== +lock-group_0 localhost_12003 +lock-group_1 localhost_12002 +lock-group_10 localhost_12001 +lock-group_11 localhost_12003 +lock-group_2 localhost_12002 +lock-group_3 localhost_12001 +lock-group_4 localhost_12003 +lock-group_5 localhost_12002 +lock-group_6 localhost_12001 +lock-group_7 localhost_12003 +lock-group_8 localhost_12002 +lock-group_9 localhost_12001 +Stopping the first participant +localhost_12001 Interrupted +localhost_12002 acquired lock:lock-group_3 +localhost_12003 acquired lock:lock-group_6 +localhost_12003 acquired lock:lock-group_10 +localhost_12002 acquired lock:lock-group_9 +lockName acquired By +====================================== +lock-group_0 localhost_12003 +lock-group_1 localhost_12002 +lock-group_10 localhost_12003 +lock-group_11 localhost_12003 +lock-group_2 localhost_12002 +lock-group_3 localhost_12002 +lock-group_4 localhost_12003 +lock-group_5 localhost_12002 +lock-group_6 localhost_12003 +lock-group_7 localhost_12003 +lock-group_8 localhost_12002 +lock-group_9 localhost_12002 +``` + +Notice that the lock assignment directly follows the assignment generated by the user-defined rebalancer both initially and after a participant is removed from the system. \ No newline at end of file diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_accessors.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_accessors.md new file mode 100644 index 0000000000..b4317109d0 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_accessors.md @@ -0,0 +1,125 @@ + + + + Tutorial - Logical Accessors + + +# [Helix Tutorial](./Tutorial.html): Logical Accessors + +Helix constructs follow a logical hierarchy. A cluster contains participants, and serve logical resources. Each resource can be divided into partitions, which themselves can be replicated. Helix now supports configuring and modifying clusters programmatically in a hierarchical way using logical accessors. + +[Click here](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/api/accessor/package-summary.html) for the Javadocs of the accessors. + +### An Example + +#### Configure a Participant + +A participant is a combination of a host, port, and a UserConfig. A UserConfig is an arbitrary set of properties a Helix user can attach to any participant. + +``` +ParticipantId participantId = ParticipantId.from("localhost_12345"); +ParticipantConfig participantConfig = new ParticipantConfig.Builder(participantId) + .hostName("localhost").port(12345).build(); +``` + +#### Configure a Resource + +##### RebalancerContext +A Resource is essentially a combination of a RebalancerContext and a UserConfig. A [RebalancerContext](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/context/RebalancerContext.html) consists of all the key properties required to rebalance a resource, including how it is partitioned and replicated, and what state model it follows. Most Helix resources will make use of a [PartitionedRebalancerContext](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.html), which is a RebalancerContext for resources that are partitioned. + +Recall that there are four [rebalancing modes](./tutorial_rebalance.html) that Helix provides, and so Helix also provides the following subclasses for PartitionedRebalancerContext: + +* [FullAutoRebalancerContext](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.html) for FULL_AUTO mode. +* [SemiAutoRebalancerContext](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.html) for SEMI_AUTO mode. This class allows a user to specify "preference lists" to indicate where each partition should ideally be served +* [CustomRebalancerContext](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.html) for CUSTOMIZED mode. This class allows a user tp specify "preference maps" to indicate the location and state for each partition replica. + +Helix also supports arbitrary subclasses of PartitionedRebalancerContext and even arbitrary implementations of RebalancerContext for applications that need a user-defined approach for rebalancing. For more, see [User-Defined Rebalancing](./tutorial_user_def_rebalancer.html) + +##### In Action + +Here is an example of a configured resource with a rebalancer context for FULL_AUTO mode and two partitions: + +``` +ResourceId resourceId = ResourceId.from("sampleResource"); +StateModelDefinition stateModelDef = getStateModelDef(); +Partition partition1 = new Partition(PartitionId.from(resourceId, "1")); +Partition partition2 = new Partition(PartitionId.from(resourceId, "2")); +FullAutoRebalancerContext rebalanceContext = + new FullAutoRebalancerContext.Builder(resourceId).replicaCount(1).addPartition(partition1) + .addPartition(partition2).stateModelDefId(stateModelDef.getStateModelDefId()).build(); +ResourceConfig resourceConfig = + new ResourceConfig.Builder(resourceId).rebalancerContext(rebalanceContext).build(); +``` + +#### Add the Cluster + +Now we can take the participant and resource configured above, add them to a cluster configuration, and then persist the entire cluster at once using a ClusterAccessor: + +``` +// configure the cluster +ClusterId clusterId = ClusterId.from("sampleCluster"); +ClusterConfig clusterConfig = new ClusterConfig.Builder(clusterId).addParticipant(participantConfig) + .addResource(resourceConfig).addStateModelDefinition(stateModelDef).build(); + +// create the cluster using a ClusterAccessor +HelixConnection connection = new ZkHelixConnection(zkAddr); +connection.connect(); +ClusterAccessor clusterAccessor = connection.createClusterAccessor(clusterId); +clusterAccessor.createCluster(clusterConfig); +``` + +### Create, Read, Update, and Delete + +Note that you don't have to specify the entire cluster beforehand! Helix provides a ClusterAccessor, ParticipantAccessor, and ResourceAccessor to allow changing as much or as little of the cluster as needed on the fly. You can add a resource or participant to a cluster, reconfigure a resource, participant, or cluster, remove components from the cluster, and more. See the [Javadocs](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/api/accessor/package-summary.html) to see all that the accessor classes can do. + +#### Delta Classes + +Updating a cluster, participant, or resource should involve selecting the element to change, and then letting Helix change only that component. To do this, Helix has included Delta classes for ClusterConfig, ParticipantConfig, and ResourceConfig. + +#### Example: Updating a Participant + +Tags are used for Helix depolyments where only certain participants can be allowed to serve certain resources. To do this, Helix only assigns resource replicas to participants who have a tag that the resource specifies. In this example, we will use ParticipantConfig.Delta to remove a participant tag and add another as part of a reconfiguration. + +``` +// specify the change to the participant +ParticipantConfig.Delta delta = new ParticipantConfig.Delta(participantId).addTag("newTag").removeTag("oldTag"); + +// update the participant configuration +ParticipantAccessor participantAccessor = connection.createParticipantAccessor(clusterId); +participantAccessor.updateParticipant(participantId, delta); +``` + +#### Example: Dropping a Resource +Removing a resource from the cluster is quite simple: + +``` +clusterAccessor.dropResourceFromCluster(resourceId); +``` + +#### Example: Reading the Cluster +Reading a full snapshot of the cluster is also a one-liner: + +``` +Cluster cluster = clusterAccessor.readCluster(); +``` + +### Atomic Accessors + +Helix also includes versions of ClusterAccessor, ParticipantAccessor, and ResourceAccessor that can complete operations atomically relative to one another. The specific semantics of the atomic operations are included in the Javadocs. These atomic classes should be used sparingly and only in cases where contention can adversely affect the correctness of a Helix-based cluster. For most deployments, this is not the case, and using these classes will cause a degradation in performance. However, the interface for all atomic accessors mirrors that of the non-atomic accessors. \ No newline at end of file diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_admin.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_admin.md new file mode 100644 index 0000000000..3285ad9914 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_admin.md @@ -0,0 +1,407 @@ + + + + Tutorial - Admin Operations + + +# [Helix Tutorial](./Tutorial.html): Admin Operations + +Helix provides a set of admin api for cluster management operations. They are supported via: + +* _Java API_ +* _Commandline interface_ +* _REST interface via helix-admin-webapp_ + +### Java API +See interface [_org.apache.helix.HelixAdmin_](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/HelixAdmin.html) + +### Command-line interface +The command-line tool comes with helix-core package: + +Get the command-line tool: + +``` + - git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git + - cd incubator-helix + - ./build + - cd helix-core/target/helix-core-pkg/bin + - chmod +x *.sh +``` + +Get help: + +``` + - ./helix-admin.sh --help +``` + +All other commands have this form: + +``` + ./helix-admin.sh --zkSvr +``` + +Admin commands and brief description: + +| Command syntax | Description | +| -------------- | ----------- | +| _\-\-activateCluster \_ | Enable/disable a cluster in distributed controller mode | +| _\-\-addCluster \_ | Add a new cluster | +| _\-\-addIdealState \_ | Add an ideal state to a cluster | +| _\-\-addInstanceTag \_ | Add a tag to an instance | +| _\-\-addNode \_ | Add an instance to a cluster | +| _\-\-addResource \_ | Add a new resource to a cluster | +| _\-\-addResourceProperty \_ | Add a resource property | +| _\-\-addStateModelDef \_ | Add a State model definition to a cluster | +| _\-\-dropCluster \_ | Delete a cluster | +| _\-\-dropNode \_ | Remove a node from a cluster | +| _\-\-dropResource \_ | Remove an existing resource from a cluster | +| _\-\-enableCluster \_ | Enable/disable a cluster | +| _\-\-enableInstance \_ | Enable/disable an instance | +| _\-\-enablePartition \_ | Enable/disable a partition | +| _\-\-getConfig \_ | Get user configs | +| _\-\-getConstraints \_ | Get constraints | +| _\-\-help_ | print help information | +| _\-\-instanceGroupTag \_ | Specify instance group tag, used with rebalance command | +| _\-\-listClusterInfo \_ | Show information of a cluster | +| _\-\-listClusters_ | List all clusters | +| _\-\-listInstanceInfo \_ | Show information of an instance | +| _\-\-listInstances \_ | List all instances in a cluster | +| _\-\-listPartitionInfo \_ | Show information of a partition | +| _\-\-listResourceInfo \_ | Show information of a resource | +| _\-\-listResources \_ | List all resources in a cluster | +| _\-\-listStateModel \_ | Show information of a state model | +| _\-\-listStateModels \_ | List all state models in a cluster | +| _\-\-maxPartitionsPerNode \_ | Specify the max partitions per instance, used with addResourceGroup command | +| _\-\-rebalance \_ | Rebalance a resource | +| _\-\-removeConfig \_ | Remove user configs | +| _\-\-removeConstraint \_ | Remove a constraint | +| _\-\-removeInstanceTag \_ | Remove a tag from an instance | +| _\-\-removeResourceProperty \_ | Remove a resource property | +| _\-\-resetInstance \_ | Reset all erroneous partitions on an instance | +| _\-\-resetPartition \_ | Reset an erroneous partition | +| _\-\-resetResource \_ | Reset all erroneous partitions of a resource | +| _\-\-setConfig \_ | Set user configs | +| _\-\-setConstraint \_ | Set a constraint | +| _\-\-swapInstance \_ | Swap an old instance with a new instance | +| _\-\-zkSvr \_ | Provide zookeeper address | + +### REST interface + +The REST interface comes wit helix-admin-webapp package: + +``` + - git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git + - cd incubator-helix + - ./build + - cd helix-admin-webapp/target/helix-admin-webapp-pkg/bin + - chmod +x *.sh + - ./run-rest-admin.sh --zkSvr --port // make sure zookeeper is running +``` + +#### URL and support methods + +* _/clusters_ + * List all clusters + + ``` + curl http://localhost:8100/clusters + ``` + + * Add a cluster + + ``` + curl -d 'jsonParameters={"command":"addCluster","clusterName":"MyCluster"}' -H "Content-Type: application/json" http://localhost:8100/clusters + ``` + +* _/clusters/{clusterName}_ + * List cluster information + + ``` + curl http://localhost:8100/clusters/MyCluster + ``` + + * Enable/disable a cluster in distributed controller mode + + ``` + curl -d 'jsonParameters={"command":"activateCluster","grandCluster":"MyControllerCluster","enabled":"true"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster + ``` + + * Remove a cluster + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster + ``` + +* _/clusters/{clusterName}/resourceGroups_ + * List all resources in a cluster + + ``` + curl http://localhost:8100/clusters/MyCluster/resourceGroups + ``` + + * Add a resource to cluster + + ``` + curl -d 'jsonParameters={"command":"addResource","resourceGroupName":"MyDB","partitions":"8","stateModelDefRef":"MasterSlave" }' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups + ``` + +* _/clusters/{clusterName}/resourceGroups/{resourceName}_ + * List resource information + + ``` + curl http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB + ``` + + * Drop a resource + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB + ``` + + * Reset all erroneous partitions of a resource + + ``` + curl -d 'jsonParameters={"command":"resetResource"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB + ``` + +* _/clusters/{clusterName}/resourceGroups/{resourceName}/idealState_ + * Rebalance a resource + + ``` + curl -d 'jsonParameters={"command":"rebalance","replicas":"3"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState + ``` + + * Add an ideal state + + ``` + echo jsonParameters={ + "command":"addIdealState" + }&newIdealState={ + "id" : "MyDB", + "simpleFields" : { + "IDEAL_STATE_MODE" : "AUTO", + "NUM_PARTITIONS" : "8", + "REBALANCE_MODE" : "SEMI_AUTO", + "REPLICAS" : "0", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + }, + "listFields" : { + }, + "mapFields" : { + "MyDB_0" : { + "localhost_1001" : "MASTER", + "localhost_1002" : "SLAVE" + } + } + } + > newIdealState.json + curl -d @'./newIdealState.json' -H 'Content-Type: application/json' http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState + ``` + + * Add resource property + + ``` + curl -d 'jsonParameters={"command":"addResourceProperty","REBALANCE_TIMER_PERIOD":"500"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState + ``` + +* _/clusters/{clusterName}/resourceGroups/{resourceName}/externalView_ + * Show resource external view + + ``` + curl http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/externalView + ``` +* _/clusters/{clusterName}/instances_ + * List all instances + + ``` + curl http://localhost:8100/clusters/MyCluster/instances + ``` + + * Add an instance + + ``` + curl -d 'jsonParameters={"command":"addInstance","instanceNames":"localhost_1001"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances + ``` + + * Swap an instance + + ``` + curl -d 'jsonParameters={"command":"swapInstance","oldInstance":"localhost_1001", "newInstance":"localhost_1002"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances + ``` +* _/clusters/{clusterName}/instances/{instanceName}_ + * Show instance information + + ``` + curl http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Enable/disable an instance + + ``` + curl -d 'jsonParameters={"command":"enableInstance","enabled":"false"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Drop an instance + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Disable/enable partitions on an instance + + ``` + curl -d 'jsonParameters={"command":"enablePartition","resource": "MyDB","partition":"MyDB_0", "enabled" : "false"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Reset an erroneous partition on an instance + + ``` + curl -d 'jsonParameters={"command":"resetPartition","resource": "MyDB","partition":"MyDB_0"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Reset all erroneous partitions on an instance + + ``` + curl -d 'jsonParameters={"command":"resetInstance"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + +* _/clusters/{clusterName}/configs_ + * Get user cluster level config + + ``` + curl http://localhost:8100/clusters/MyCluster/configs/cluster + ``` + + * Set user cluster level config + + ``` + curl -d 'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/cluster + ``` + + * Remove user cluster level config + + ``` + curl -d 'jsonParameters={"command":"removeConfig","configs":"key1,key2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/cluster + ``` + + * Get/set/remove user participant level config + + ``` + curl -d 'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/participant/localhost_1001 + ``` + + * Get/set/remove resource level config + + ``` + curl -d 'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/resource/MyDB + ``` + +* _/clusters/{clusterName}/controller_ + * Show controller information + + ``` + curl http://localhost:8100/clusters/MyCluster/Controller + ``` + + * Enable/disable cluster + + ``` + curl -d 'jsonParameters={"command":"enableCluster","enabled":"false"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/Controller + ``` + +* _/zkPath/{path}_ + * Get information for zookeeper path + + ``` + curl http://localhost:8100/zkPath/MyCluster + ``` + +* _/clusters/{clusterName}/StateModelDefs_ + * Show all state model definitions + + ``` + curl http://localhost:8100/clusters/MyCluster/StateModelDefs + ``` + + * Add a state mdoel definition + + ``` + echo jsonParameters={ + "command":"addStateModelDef" + }&newStateModelDef={ + "id" : "OnlineOffline", + "simpleFields" : { + "INITIAL_STATE" : "OFFLINE" + }, + "listFields" : { + "STATE_PRIORITY_LIST" : [ "ONLINE", "OFFLINE", "DROPPED" ], + "STATE_TRANSITION_PRIORITYLIST" : [ "OFFLINE-ONLINE", "ONLINE-OFFLINE", "OFFLINE-DROPPED" ] + }, + "mapFields" : { + "DROPPED.meta" : { + "count" : "-1" + }, + "OFFLINE.meta" : { + "count" : "-1" + }, + "OFFLINE.next" : { + "DROPPED" : "DROPPED", + "ONLINE" : "ONLINE" + }, + "ONLINE.meta" : { + "count" : "R" + }, + "ONLINE.next" : { + "DROPPED" : "OFFLINE", + "OFFLINE" : "OFFLINE" + } + } + } + > newStateModelDef.json + curl -d @'./untitled.txt' -H 'Content-Type: application/json' http://localhost:8100/clusters/MyCluster/StateModelDefs + ``` + +* _/clusters/{clusterName}/StateModelDefs/{stateModelDefName}_ + * Show a state model definition + + ``` + curl http://localhost:8100/clusters/MyCluster/StateModelDefs/OnlineOffline + ``` + +* _/clusters/{clusterName}/constraints/{constraintType}_ + * Show all contraints + + ``` + curl http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT + ``` + + * Set a contraint + + ``` + curl -d 'jsonParameters={"constraintAttributes":"RESOURCE=MyDB,CONSTRAINT_VALUE=1"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT/MyConstraint + ``` + + * Remove a constraint + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT/MyConstraint + ``` + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_controller.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_controller.md new file mode 100644 index 0000000000..1a4cc45d7d --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_controller.md @@ -0,0 +1,79 @@ + + + + Tutorial - Controller + + +# [Helix Tutorial](./Tutorial.html): Controller + +Next, let\'s implement the controller. This is the brain of the cluster. Helix makes sure there is exactly one active controller running the cluster. + +### Start the Helix Agent + + +It requires the following parameters: + +* clusterId: A logical ID to represent the group of nodes +* controllerId: A logical ID of the process creating the controller instance. Generally this is host:port. +* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. + +``` +HelixConnection connection = new ZKHelixConnection(zkConnectString); +HelixController controller = connection.createController(clusterId, controllerId); +``` + +### Controller Code + +The Controller needs to know about all changes in the cluster. Helix takes care of this with the default implementation. +If you need additional functionality, see GenericHelixController and ZKHelixController for how to configure the pipeline. + +``` +HelixConnection connection = new ZKHelixConnection(zkConnectString); +HelixController controller = connection.createController(clusterId, controllerId); +controller.startAsync(); +``` +The snippet above shows how the controller is started. You can also start the controller using command line interface. + +``` +cd helix/helix-core/target/helix-core-pkg/bin +./run-helix-controller.sh --zkSvr --cluster +``` + +### Controller deployment modes + +Helix provides multiple options to deploy the controller. + +#### STANDALONE + +The Controller can be started as a separate process to manage a cluster. This is the recommended approach. However, since one controller can be a single point of failure, multiple controller processes are required for reliability. Even if multiple controllers are running, only one will be actively managing the cluster at any time and is decided by a leader-election process. If the leader fails, another leader will take over managing the cluster. + +Even though we recommend this method of deployment, it has the drawback of having to manage an additional service for each cluster. See Controller As a Service option. + +#### EMBEDDED + +If setting up a separate controller process is not viable, then it is possible to embed the controller as a library in each of the participants. + +#### CONTROLLER AS A SERVICE + +One of the cool features we added in Helix is to use a set of controllers to manage a large number of clusters. + +For example if you have X clusters to be managed, instead of deploying X*3 (3 controllers for fault tolerance) controllers for each cluster, one can deploy just 3 controllers. Each controller can manage X/3 clusters. If any controller fails, the remaining two will manage X/2 clusters. + + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_health.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_health.md new file mode 100644 index 0000000000..e1a7f3c2d3 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_health.md @@ -0,0 +1,46 @@ + + + + Tutorial - Customizing Heath Checks + + +# [Helix Tutorial](./Tutorial.html): Customizing Health Checks + +In this chapter, we\'ll learn how to customize the health check, based on metrics of your distributed system. + +### Health Checks + +Note: _this in currently in development mode, not yet ready for production._ + +Helix provides the ability for each node in the system to report health metrics on a periodic basis. + +Helix supports multiple ways to aggregate these metrics: + +* SUM +* AVG +* EXPONENTIAL DECAY +* WINDOW + +Helix persists the aggregated value only. + +Applications can define a threshold on the aggregate values according to the SLAs, and when the SLA is violated Helix will fire an alert. +Currently Helix only fires an alert, but in a future release we plan to use these metrics to either mark the node dead or load balance the partitions. +This feature will be valuable for distributed systems that support multi-tenancy and have a large variation in work load patterns. In addition, this can be used to detect skewed partitions (hotspots) and rebalance the cluster. + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_messaging.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_messaging.md new file mode 100644 index 0000000000..f65ce7c889 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_messaging.md @@ -0,0 +1,71 @@ + + + + Tutorial - Messaging + + +# [Helix Tutorial](./Tutorial.html): Messaging + +In this chapter, we\'ll learn about messaging, a convenient feature in Helix for sending messages between nodes of a cluster. This is an interesting feature which is quite useful in practice. It is common that nodes in a distributed system require a mechanism to interact with each other. + +### Example: Bootstrapping a Replica + +Consider a search system where the index replica starts up and it does not have an index. A typical solution is to get the index from a common location, or to copy the index from another replica. + +Helix provides a messaging API for intra-cluster communication between nodes in the system. Helix provides a mechanism to specify the message recipient in terms of resource, partition, and state rather than specifying hostnames. Helix ensures that the message is delivered to all of the required recipients. In this particular use case, the instance can specify the recipient criteria as all replicas of the desired partition to bootstrap. +Since Helix is aware of the global state of the system, it can send the message to appropriate nodes. Once the nodes respond, Helix provides the bootstrapping replica with all the responses. + +This is a very generic API and can also be used to schedule various periodic tasks in the cluster, such as data backups, log cleanup, etc. +System Admins can also perform ad-hoc tasks, such as on-demand backups or a system command (such as rm -rf ;) across all nodes of the cluster + +``` + ClusterMessagingService messagingService = manager.getMessagingService(); + + // Construct the Message + Message requestBackupUriRequest = new Message( + MessageType.USER_DEFINE_MSG, UUID.randomUUID().toString()); + requestBackupUriRequest + .setMsgSubType(BootstrapProcess.REQUEST_BOOTSTRAP_URL); + requestBackupUriRequest.setMsgState(MessageState.NEW); + + // Set the Recipient criteria: all nodes that satisfy the criteria will receive the message + Criteria recipientCriteria = new Criteria(); + recipientCriteria.setInstanceName("%"); + recipientCriteria.setRecipientInstanceType(InstanceType.PARTICIPANT); + recipientCriteria.setResource("MyDB"); + recipientCriteria.setPartition(""); + + // Should be processed only by process(es) that are active at the time of sending the message + // This means if the recipient is restarted after message is sent, it will not be processe. + recipientCriteria.setSessionSpecific(true); + + // wait for 30 seconds + int timeout = 30000; + + // the handler that will be invoked when any recipient responds to the message. + BootstrapReplyHandler responseHandler = new BootstrapReplyHandler(); + + // this will return only after all recipients respond or after timeout + int sentMessageCount = messagingService.sendAndWait(recipientCriteria, + requestBackupUriRequest, responseHandler, timeout); +``` + +See HelixManager.DefaultMessagingService in [Javadocs](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/messaging/DefaultMessagingService.html) for more info. + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_participant.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_participant.md new file mode 100644 index 0000000000..da55cbdead --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_participant.md @@ -0,0 +1,97 @@ + + + + Tutorial - Participant + + +# [Helix Tutorial](./Tutorial.html): Participant + +In this chapter, we\'ll learn how to implement a Participant, which is a primary functional component of a distributed system. + + +### Start the Helix Agent + +The Helix agent is a common component that connects each system component with the controller. + +It requires the following parameters: + +* clusterId: A logical ID to represent the group of nodes +* participantId: A logical ID of the process creating the manager instance. Generally this is host:port. +* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. + +After the Helix participant instance is created, only thing that needs to be registered is the state model factory. +The methods of the State Model will be called when controller sends transitions to the Participant. In this example, we'll use the OnlineOffline factory. Other options include: + +* MasterSlaveStateModelFactory +* LeaderStandbyStateModelFactory +* BootstrapHandler +* _An application defined state model factory_ + + +``` +HelixConnection connection = new ZKHelixConnection(zkConnectString); +HelixParticipant participant = connection.createParticipant(clusterId, participantId); +StateMachineEngine stateMach = participant.getStateMachineEngine(); + +// create a stateModelFactory that returns a statemodel object for each partition. +StateModelFactory stateModelFactory = new OnlineOfflineStateModelFactory(); +stateMach.registerStateModelFactory(stateModelType, stateModelFactory); +participant.startAsync(); +``` + +Helix doesn\'t know what it means to change from OFFLINE\-\-\>ONLINE or ONLINE\-\-\>OFFLINE. The following code snippet shows where you insert your system logic for these two state transitions. + +``` +public class OnlineOfflineStateModelFactory extends StateModelFactory { + @Override + public StateModel createNewStateModel(String stateUnitKey) { + OnlineOfflineStateModel stateModel = new OnlineOfflineStateModel(); + return stateModel; + } + @StateModelInfo(states = "{'OFFLINE','ONLINE'}", initialState = "OFFINE") + public static class OnlineOfflineStateModel extends StateModel { + + @Transition(from = "OFFLINE", to = "ONLINE") + public void onBecomeOnlineFromOffline(Message message, + NotificationContext context) { + + System.out.println("OnlineOfflineStateModel.onBecomeOnlineFromOffline()"); + + //////////////////////////////////////////////////////////////////////////////////////////////// + // Application logic to handle transition // + // For example, you might start a service, run initialization, etc // + //////////////////////////////////////////////////////////////////////////////////////////////// + } + + @Transition(from = "ONLINE", to = "OFFLINE") + public void onBecomeOfflineFromOnline(Message message, + NotificationContext context) { + + System.out.println("OnlineOfflineStateModel.onBecomeOfflineFromOnline()"); + + //////////////////////////////////////////////////////////////////////////////////////////////// + // Application logic to handle transition // + // For example, you might shutdown a service, log this event, or change monitoring settings // + //////////////////////////////////////////////////////////////////////////////////////////////// + } + } +} +``` + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_propstore.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_propstore.md new file mode 100644 index 0000000000..41bcc69b65 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_propstore.md @@ -0,0 +1,34 @@ + + + + Tutorial - Application Property Store + + +# [Helix Tutorial](./Tutorial.html): Application Property Store + +In this chapter, we\'ll learn how to use the application property store. + +### Property Store + +It is common that an application needs support for distributed, shared data structures. Helix uses Zookeeper to store the application data and hence provides notifications when the data changes. + +While you could use Zookeeper directly, Helix supports caching the data and a write-through cache. This is far more efficient than reading from Zookeeper for every access. + +See [HelixManager.getHelixPropertyStore](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/store/package-summary.html) for details. diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_rebalance.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_rebalance.md new file mode 100644 index 0000000000..8f42a5ab52 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_rebalance.md @@ -0,0 +1,181 @@ + + + + Tutorial - Rebalancing Algorithms + + +# [Helix Tutorial](./Tutorial.html): Rebalancing Algorithms + +The placement of partitions in a distributed system is essential for the reliability and scalability of the system. For example, when a node fails, it is important that the partitions hosted on that node are reallocated evenly among the remaining nodes. Consistent hashing is one such algorithm that can satisfy this guarantee. Helix provides a variant of consistent hashing based on the RUSH algorithm, among others. + +This means given a number of partitions, replicas and number of nodes, Helix does the automatic assignment of partition to nodes such that: + +* Each node has the same number of partitions +* Replicas of the same partition do not stay on the same node +* When a node fails, the partitions will be equally distributed among the remaining nodes +* When new nodes are added, the number of partitions moved will be minimized along with satisfying the above criteria + +Helix employs a rebalancing algorithm to compute the _ideal state_ of the system. When the _current state_ differs from the _ideal state_, Helix uses it as the target state of the system and computes the appropriate transitions needed to bring it to the _ideal state_. + +Helix makes it easy to perform this operation, while giving you control over the algorithm. In this section, we\'ll see how to implement the desired behavior. + +Helix has four options for rebalancing, in increasing order of customization by the system builder: + +* FULL_AUTO +* SEMI_AUTO +* CUSTOMIZED +* USER_DEFINED + +``` + |FULL_AUTO | SEMI_AUTO | CUSTOMIZED| USER_DEFINED | + ---------------------------------------------------------| + LOCATION | HELIX | APP | APP | APP | + ---------------------------------------------------------| + STATE | HELIX | HELIX | APP | APP | + ---------------------------------------------------------- +``` + + +### FULL_AUTO + +When the rebalance mode is set to FULL_AUTO, Helix controls both the location of the replica along with the state. This option is useful for applications where creation of a replica is not expensive. + +For example, consider this system that uses a MasterSlave state model, with 3 partitions and 2 replicas in the ideal state. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "REBALANCE_MODE" : "FULL_AUTO", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [], + "MyResource_1" : [], + "MyResource_2" : [] + }, + "mapFields" : { + } +} +``` + +If there are 3 nodes in the cluster, then Helix will balance the masters and slaves equally. The ideal state is therefore: + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +Another typical example is evenly distributing a group of tasks among the currently healthy processes. For example, if there are 60 tasks and 4 nodes, Helix assigns 15 tasks to each node. +When one node fails, Helix redistributes its 15 tasks to the remaining 3 nodes, resulting in a balanced 20 tasks per node. Similarly, if a node is added, Helix re-allocates 3 tasks from each of the 4 nodes to the 5th node, resulting in a balanced distribution of 12 tasks per node.. + +#### SEMI_AUTO + +When the application needs to control the placement of the replicas, use the SEMI_AUTO rebalance mode. + +Example: In the ideal state below, the partition \'MyResource_0\' is constrained to be placed only on node1 or node2. The choice of _state_ is still controlled by Helix. That means MyResource_0.MASTER could be on node1 and MyResource_0.SLAVE on node2, or vice-versa but neither would be placed on node3. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [node1, node2], + "MyResource_1" : [node2, node3], + "MyResource_2" : [node3, node1] + }, + "mapFields" : { + } +} +``` + +The MasterSlave state model requires that a partition has exactly one MASTER at all times, and the other replicas should be SLAVEs. In this simple example with 2 replicas per partition, there would be one MASTER and one SLAVE. Upon failover, a SLAVE has to assume mastership, and a new SLAVE will be generated. + +In this mode when node1 fails, unlike in FULL_AUTO mode the partition is _not_ moved from node1 to node3. Instead, Helix will decide to change the state of MyResource_0 on node2 from SLAVE to MASTER, based on the system constraints. + +#### CUSTOMIZED + +Helix offers a third mode called CUSTOMIZED, in which the application controls the placement _and_ state of each replica. The application needs to implement a callback interface that Helix invokes when the cluster state changes. +Within this callback, the application can recompute the idealstate. Helix will then issue appropriate transitions such that _Idealstate_ and _Currentstate_ converges. + +Here\'s an example, again with 3 partitions, 2 replicas per partition, and the MasterSlave state model: + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "REBALANCE_MODE" : "CUSTOMIZED", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +Suppose the current state of the system is 'MyResource_0' -> {N1:MASTER, N2:SLAVE} and the application changes the ideal state to 'MyResource_0' -> {N1:SLAVE,N2:MASTER}. While the application decides which node is MASTER and which is SLAVE, Helix will not blindly issue MASTER-->SLAVE to N1 and SLAVE-->MASTER to N2 in parallel, since that might result in a transient state where both N1 and N2 are masters, which violates the MasterSlave constraint that there is exactly one MASTER at a time. Helix will first issue MASTER-->SLAVE to N1 and after it is completed, it will issue SLAVE-->MASTER to N2. + +#### USER_DEFINED + +For maximum flexibility, Helix exposes an interface that can allow applications to plug in custom rebalancing logic. By providing the name of a class that implements the Rebalancer interface, Helix will automatically call the contained method whenever there is a change to the live participants in the cluster. For more, see [User-Defined Rebalancer](./tutorial_user_def_rebalancer.html). + +#### Backwards Compatibility + +In previous versions, FULL_AUTO was called AUTO_REBALANCE and SEMI_AUTO was called AUTO. Furthermore, they were presented as the IDEAL_STATE_MODE. Helix supports both IDEAL_STATE_MODE and REBALANCE_MODE, but IDEAL_STATE_MODE is now deprecated and may be phased out in future versions. diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_spectator.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_spectator.md new file mode 100644 index 0000000000..24c1cf484e --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_spectator.md @@ -0,0 +1,76 @@ + + + + Tutorial - Spectator + + +# [Helix Tutorial](./Tutorial.html): Spectator + +Next, we\'ll learn how to implement a Spectator. Typically, a spectator needs to react to changes within the distributed system. Examples: a client that needs to know where to send a request, a topic consumer in a consumer group. The spectator is automatically informed of changes in the _external state_ of the cluster, but it does not have to add any code to keep track of other components in the system. + +### Start the Helix agent + +Same as for a Participant, The Helix agent is the common component that connects each system component with the controller. + +It requires the following parameters: + +* clusterName: A logical name to represent the group of nodes +* instanceName: A logical name of the process creating the manager instance. Generally this is host:port. +* instanceType: Type of the process. This can be one of the following types, in this case, use SPECTATOR: + * CONTROLLER: Process that controls the cluster, any number of controllers can be started but only one will be active at any given time. + * PARTICIPANT: Process that performs the actual task in the distributed system. + * SPECTATOR: Process that observes the changes in the cluster. + * ADMIN: To carry out system admin actions. +* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. + +After the Helix manager instance is created, only thing that needs to be registered is the listener. When the ExternalView changes, the listener is notified. + +### Spectator Code + +A spectator observes the cluster and is notified when the state of the system changes. Helix consolidates the state of entire cluster in one Znode called ExternalView. +Helix provides a default implementation RoutingTableProvider that caches the cluster state and updates it when there is a change in the cluster. + +``` +manager = HelixManagerFactory.getZKHelixManager(clusterName, + instanceName, + InstanceType.PARTICIPANT, + zkConnectString); +manager.connect(); +RoutingTableProvider routingTableProvider = new RoutingTableProvider(); +manager.addExternalViewChangeListener(routingTableProvider); +``` + +In the following code snippet, the application sends the request to a valid instance by interrogating the external view. Suppose the desired resource for this request is in the partition myDB_1. + +``` +## instances = routingTableProvider.getInstances(, "PARTITION_NAME", "PARTITION_STATE"); +instances = routingTableProvider.getInstances("myDB", "myDB_1", "ONLINE"); + +//////////////////////////////////////////////////////////////////////////////////////////////// +// Application-specific code to send a request to one of the instances // +//////////////////////////////////////////////////////////////////////////////////////////////// + +theInstance = instances.get(0); // should choose an instance and throw an exception if none are available +result = theInstance.sendRequest(yourApplicationRequest, responseObject); + +``` + +When the external view changes, the application needs to react by sending requests to a different instance. + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_state.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_state.md new file mode 100644 index 0000000000..4f7b1b587a --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_state.md @@ -0,0 +1,131 @@ + + + + Tutorial - State Machine Configuration + + +# [Helix Tutorial](./Tutorial.html): State Machine Configuration + +In this chapter, we\'ll learn about the state models provided by Helix, and how to create your own custom state model. + +## State Models + +Helix comes with 3 default state models that are commonly used. It is possible to have multiple state models in a cluster. +Every resource that is added should be configured to use a state model that govern its _ideal state_. + +### MASTER-SLAVE + +* 3 states: OFFLINE, SLAVE, MASTER +* Maximum number of masters: 1 +* Slaves are based on the replication factor. The replication factor can be specified while adding the resource. + + +### ONLINE-OFFLINE + +* Has 2 states: OFFLINE and ONLINE. This simple state model is a good starting point for most applications. + +### LEADER-STANDBY + +* 1 Leader and multiple stand-bys. The idea is that exactly one leader accomplishes a designated task, the stand-bys are ready to take over if the leader fails. + +## Constraints + +In addition to the state machine configuration, one can specify the constraints of states and transitions. + +For example, one can say: + +* MASTER:1 +
Maximum number of replicas in MASTER state at any time is 1 + +* OFFLINE-SLAVE:5 +
Maximum number of OFFLINE-SLAVE transitions that can happen concurrently in the system is 5 in this example. + +### Dynamic State Constraints + +We also support two dynamic upper bounds for the number of replicas in each state: + +* N: The number of replicas in the state is at most the number of live participants in the cluster +* R: The number of replicas in the state is at most the specified replica count for the partition + +### State Priority + +Helix uses a greedy approach to satisfy the state constraints. For example, if the state machine configuration says it needs 1 MASTER and 2 SLAVES, but only 1 node is active, Helix must promote it to MASTER. This behavior is achieved by providing the state priority list as \[MASTER, SLAVE\]. + +### State Transition Priority + +Helix tries to fire as many transitions as possible in parallel to reach the stable state without violating constraints. By default, Helix simply sorts the transitions alphabetically and fires as many as it can without violating the constraints. You can control this by overriding the priority order. + +## Special States + +### DROPPED + +The DROPPED state is used to signify a replica that was served by a given participant, but is no longer served. This allows Helix and its participants to effectively clean up. There are two requirements that every new state model should follow with respect to the DROPPED state: + +* The DROPPED state must be defined +* There must be a path to DROPPED for every state in the model + +### ERROR + +The ERROR state is used whenever the participant serving a partition encountered an error and cannot continue to serve the partition. HelixAdmin has \"reset\" functionality to allow for participants to recover from the ERROR state. + +## Annotated Example + +Below is a complete definition of a Master-Slave state model. Notice the fields marked REQUIRED; these are essential for any state model definition. + +``` +StateModelDefinition stateModel = new StateModelDefinition.Builder("MasterSlave") + // OFFLINE is the state that the system starts in (initial state is REQUIRED) + .initialState("OFFLINE") + + // Lowest number here indicates highest priority, no value indicates lowest priority + .addState("MASTER", 1) + .addState("SLAVE", 2) + .addState("OFFLINE") + + // Note the special inclusion of the DROPPED state (REQUIRED) + .addState(HelixDefinedState.DROPPED.toString()) + + // No more than one master allowed + .upperBound("MASTER", 1) + + // R indicates an upper bound of number of replicas for each partition + .dynamicUpperBound("SLAVE", "R") + + // Add some high-priority transitions + .addTransition("SLAVE", "MASTER", 1) + .addTransition("OFFLINE", "SLAVE", 2) + + // Using the same priority value indicates that these transitions can fire in any order + .addTransition("MASTER", "SLAVE", 3) + .addTransition("SLAVE", "OFFLINE", 3) + + // Not specifying a value defaults to lowest priority + // Notice the inclusion of the OFFLINE to DROPPED transition + // Since every state has a path to OFFLINE, they each now have a path to DROPPED (REQUIRED) + .addTransition("OFFLINE", HelixDefinedState.DROPPED.toString()) + + // Create the StateModelDefinition instance + .build(); + + // Use the isValid() function to make sure the StateModelDefinition will work without issues + Assert.assertTrue(stateModel.isValid()); +``` + + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_throttling.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_throttling.md new file mode 100644 index 0000000000..7417979c7e --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_throttling.md @@ -0,0 +1,38 @@ + + + + Tutorial - Throttling + + +# [Helix Tutorial](./Tutorial.html): Throttling + +In this chapter, we\'ll learn how to control the parallel execution of cluster tasks. Only a centralized cluster manager with global knowledge is capable of coordinating this decision. + +### Throttling + +Since all state changes in the system are triggered through transitions, Helix can control the number of transitions that can happen in parallel. Some of the transitions may be light weight, but some might involve moving data, which is quite expensive from a network and IOPS perspective. + +Helix allows applications to set a threshold on transitions. The threshold can be set at multiple scopes: + +* MessageType e.g STATE_TRANSITION +* TransitionType e.g SLAVE-MASTER +* Resource e.g database +* Node i.e per-node maximum transitions in parallel + diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_user_def_rebalancer.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_user_def_rebalancer.md new file mode 100644 index 0000000000..f30aafca08 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_user_def_rebalancer.md @@ -0,0 +1,227 @@ + + + + Tutorial - User-Defined Rebalancing + + +# [Helix Tutorial](./Tutorial.html): User-Defined Rebalancing + +Even though Helix can compute both the location and the state of replicas internally using a default fully-automatic rebalancer, specific applications may require rebalancing strategies that optimize for different requirements. Thus, Helix allows applications to plug in arbitrary rebalancer algorithms that implement a provided interface. One of the main design goals of Helix is to provide maximum flexibility to any distributed application. Thus, it allows applications to fully implement the rebalancer, which is the core constraint solver in the system, if the application developer so chooses. + +Whenever the state of the cluster changes, as is the case when participants join or leave the cluster, Helix automatically calls the rebalancer to compute a new mapping of all the replicas in the resource. When using a pluggable rebalancer, the only required step is to register it with Helix. Subsequently, no additional bootstrapping steps are necessary. Helix uses reflection to look up and load the class dynamically at runtime. As a result, it is also technically possible to change the rebalancing strategy used at any time. + +The [HelixRebalancer](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/HelixRebalancer.html) interface is as follows: + +``` +public void init(HelixManager helixManager); + +public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, Cluster cluster, + ResourceCurrentState currentState); +``` +The first parameter is a configuration of the resource to rebalance, the second is a full cache of all of the cluster data available to Helix, and the third is a snapshot of the actual current placements and state assignments. From the cluster variable, it is also possible to access the ResourceAssignment last generated by this rebalancer. Internally, Helix implements the same interface for its own rebalancing routines, so a user-defined rebalancer will be cognizant of the same information about the cluster as an internal implementation. Helix strives to provide applications the ability to implement algorithms that may require a large portion of the entire state of the cluster to make the best placement and state assignment decisions possible. + +A ResourceAssignment is a full representation of the location and the state of each replica of each partition of a given resource. This is a simple representation of the placement that the algorithm believes is the best possible. If the placement meets all defined constraints, this is what will become the actual state of the distributed system. + +### Rebalancer Context + +Helix provides an interface called [RebalancerContext](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/context/RebalancerContext.html). For each of the four main [rebalancing modes](./tutorial_rebalance.html), there is a base class called [PartitionedRebalancerContext](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.html), which contains all of the basic properties required for a partitioned resource. Helix provides three derived classes for PartitionedRebalancerContext: FullAutoRebalancerContext, SemiAutoRebalancerContext, and CustomizedRebalancerContext. If none of these work for your application, you can create your own class that derives PartiitonedRebalancerContext (or even only implements RebalancerContext). + +### Specifying a Rebalancer + +#### Using Logical Accessors +To specify the rebalancer, one can use ```PartitionedRebalancerContext#setRebalancerRef(RebalancerRef)``` to specify the specific implementation of the rebalancerClass. For example, here's a base constructed PartitionedRebalancerContext with a user-specified class: + +``` +RebalancerRef rebalancerRef = RebalancerRef.from(className); +PartitionedRebalancerContext rebalanceContext = + new PartitionedRebalancerContext.Builder(resourceId).replicaCount(1).addPartition(partition1) + .addPartition(partition2).stateModelDefId(stateModelDef.getStateModelDefId()) + .rebalancerRef(rebalancerRef).build(); +``` + +The class name is a fully-qualified class name consisting of its package and its name, and the class should implement the Rebalancer interface. Now, the context can be added to a ResourceConfig through ```ResourceConfig.Builder#rebalancerContext(RebalancerContext)``` and the context will automatically be made available to the rebalancer for all subsequent executions. + +#### Using HelixAdmin +For implementations that set up the cluster through existing code, the following HelixAdmin calls will update the Rebalancer class: + +``` +IdealState idealState = helixAdmin.getResourceIdealState(clusterName, resourceName); +idealState.setRebalanceMode(RebalanceMode.USER_DEFINED); +idealState.setRebalancerClassName(className); +helixAdmin.setResourceIdealState(clusterName, resourceName, idealState); +``` +There are two key fields to set to specify that a pluggable rebalancer should be used. First, the rebalance mode should be set to USER_DEFINED, and second the rebalancer class name should be set to a class that implements Rebalancer and is within the scope of the project. The class name is a fully-qualified class name consisting of its package and its name. + +#### Using YAML +Alternatively, the rebalancer class name can be specified in a YAML file representing the cluster configuration. The requirements are the same, but the representation is more compact. Below are the first few lines of an example YAML file. To see a full YAML specification, see the [YAML tutorial](./tutorial_yaml.html). + +``` +clusterName: lock-manager-custom-rebalancer # unique name for the cluster +resources: + - name: lock-group # unique resource name + rebalancer: # we will provide our own rebalancer + mode: USER_DEFINED + class: domain.project.helix.rebalancer.UserDefinedRebalancerClass +... +``` + +### Example +We demonstrate plugging in a simple user-defined rebalancer as part of a revisit of the [distributed lock manager](./recipes/user_def_rebalancer.html) example. It includes a functional Rebalancer implementation, as well as the entire YAML file used to define the cluster. + +Consider the case where partitions are locks in a lock manager and 6 locks are to be distributed evenly to a set of participants, and only one participant can hold each lock. We can define a rebalancing algorithm that simply takes the modulus of the lock number and the number of participants to evenly distribute the locks across participants. Helix allows capping the number of partitions a participant can accept, but since locks are lightweight, we do not need to define a restriction in this case. The following is a succinct implementation of this algorithm. + +``` +@Override +public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, Cluster cluster, + ResourceCurrentState currentState) { + // Get the rebalcancer context (a basic partitioned one) + PartitionedRebalancerContext context = rebalancerConfig.getRebalancerContext( + PartitionedRebalancerContext.class); + + // Initialize an empty mapping of locks to participants + ResourceAssignment assignment = new ResourceAssignment(context.getResourceId()); + + // Get the list of live participants in the cluster + List liveParticipants = new ArrayList( + cluster.getLiveParticipantMap().keySet()); + + // Get the state model (should be a simple lock/unlock model) and the highest-priority state + StateModelDefId stateModelDefId = context.getStateModelDefId(); + StateModelDefinition stateModelDef = cluster.getStateModelMap().get(stateModelDefId); + if (stateModelDef.getStatesPriorityList().size() < 1) { + LOG.error("Invalid state model definition. There should be at least one state."); + return assignment; + } + State lockState = stateModelDef.getTypedStatesPriorityList().get(0); + + // Count the number of participants allowed to lock each lock + String stateCount = stateModelDef.getNumParticipantsPerState(lockState); + int lockHolders = 0; + try { + // a numeric value is a custom-specified number of participants allowed to lock the lock + lockHolders = Integer.parseInt(stateCount); + } catch (NumberFormatException e) { + LOG.error("Invalid state model definition. The lock state does not have a valid count"); + return assignment; + } + + // Fairly assign the lock state to the participants using a simple mod-based sequential + // assignment. For instance, if each lock can be held by 3 participants, lock 0 would be held + // by participants (0, 1, 2), lock 1 would be held by (1, 2, 3), and so on, wrapping around the + // number of participants as necessary. + // This assumes a simple lock-unlock model where the only state of interest is which nodes have + // acquired each lock. + int i = 0; + for (PartitionId partition : context.getPartitionSet()) { + Map replicaMap = new HashMap(); + for (int j = i; j < i + lockHolders; j++) { + int participantIndex = j % liveParticipants.size(); + ParticipantId participant = liveParticipants.get(participantIndex); + // enforce that a participant can only have one instance of a given lock + if (!replicaMap.containsKey(participant)) { + replicaMap.put(participant, lockState); + } + } + assignment.addReplicaMap(partition, replicaMap); + i++; + } + return assignment; +} +``` + +Here is the ResourceAssignment emitted by the user-defined rebalancer for a 3-participant system whenever there is a change to the set of participants. + +* Participant_A joins + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_A": "LOCKED"}, + "lock_2": { "Participant_A": "LOCKED"}, + "lock_3": { "Participant_A": "LOCKED"}, + "lock_4": { "Participant_A": "LOCKED"}, + "lock_5": { "Participant_A": "LOCKED"}, +} +``` + +A ResourceAssignment is a mapping for each resource of partition to the participant serving each replica and the state of each replica. The state model is a simple LOCKED/RELEASED model, so participant A holds all lock partitions in the LOCKED state. + +* Participant_B joins + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_B": "LOCKED"}, + "lock_2": { "Participant_A": "LOCKED"}, + "lock_3": { "Participant_B": "LOCKED"}, + "lock_4": { "Participant_A": "LOCKED"}, + "lock_5": { "Participant_B": "LOCKED"}, +} +``` + +Now that there are two participants, the simple mod-based function assigns every other lock to the second participant. On any system change, the rebalancer is invoked so that the application can define how to redistribute its resources. + +* Participant_C joins (steady state) + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_B": "LOCKED"}, + "lock_2": { "Participant_C": "LOCKED"}, + "lock_3": { "Participant_A": "LOCKED"}, + "lock_4": { "Participant_B": "LOCKED"}, + "lock_5": { "Participant_C": "LOCKED"}, +} +``` + +This is the steady state of the system. Notice that four of the six locks now have a different owner. That is because of the naïve modulus-based assignmemt approach used by the user-defined rebalancer. However, the interface is flexible enough to allow you to employ consistent hashing or any other scheme if minimal movement is a system requirement. + +* Participant_B fails + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_C": "LOCKED"}, + "lock_2": { "Participant_A": "LOCKED"}, + "lock_3": { "Participant_C": "LOCKED"}, + "lock_4": { "Participant_A": "LOCKED"}, + "lock_5": { "Participant_C": "LOCKED"}, +} +``` + +On any node failure, as in the case of node addition, the rebalancer is invoked automatically so that it can generate a new mapping as a response to the change. Helix ensures that the Rebalancer has the opportunity to reassign locks as required by the application. + +* Participant_B (or the replacement for the original Participant_B) rejoins + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_B": "LOCKED"}, + "lock_2": { "Participant_C": "LOCKED"}, + "lock_3": { "Participant_A": "LOCKED"}, + "lock_4": { "Participant_B": "LOCKED"}, + "lock_5": { "Participant_C": "LOCKED"}, +} +``` + +The rebalancer was invoked once again and the resulting ResourceAssignment reflects the steady state. + +### Caveats +- The rebalancer class must be available at runtime, or else Helix will not attempt to rebalance at all \ No newline at end of file diff --git a/site-releases/0.7.0-incubating/src/site/markdown/tutorial_yaml.md b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_yaml.md new file mode 100644 index 0000000000..0f8e0cce11 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/markdown/tutorial_yaml.md @@ -0,0 +1,102 @@ + + + + Tutorial - YAML Cluster Setup + + +# [Helix Tutorial](./Tutorial.html): YAML Cluster Setup + +As an alternative to using Helix Admin to set up the cluster, its resources, constraints, and the state model, Helix supports bootstrapping a cluster configuration based on a YAML file. Below is an annotated example of such a file for a simple distributed lock manager where a lock can only be LOCKED or RELEASED, and each lock only allows a single participant to hold it in the LOCKED state. + +``` +clusterName: lock-manager-custom-rebalancer # unique name for the cluster (required) +resources: + - name: lock-group # unique resource name (required) + rebalancer: # required + mode: USER_DEFINED # required - USER_DEFINED means we will provide our own rebalancer + class: org.apache.helix.userdefinedrebalancer.LockManagerRebalancer # required for USER_DEFINED + partitions: + count: 12 # number of partitions for the resource (default is 1) + replicas: 1 # number of replicas per partition (default is 1) + stateModel: + name: lock-unlock # model name (required) + states: [LOCKED, RELEASED, DROPPED] # the list of possible states (required if model not built-in) + transitions: # the list of possible transitions (required if model not built-in) + - name: Unlock + from: LOCKED + to: RELEASED + - name: Lock + from: RELEASED + to: LOCKED + - name: DropLock + from: LOCKED + to: DROPPED + - name: DropUnlock + from: RELEASED + to: DROPPED + - name: Undrop + from: DROPPED + to: RELEASED + initialState: RELEASED # (required if model not built-in) + constraints: + state: + counts: # maximum number of replicas of a partition that can be in each state (required if model not built-in) + - name: LOCKED + count: "1" + - name: RELEASED + count: "-1" + - name: DROPPED + count: "-1" + priorityList: [LOCKED, RELEASED, DROPPED] # states in order of priority (all priorities equal if not specified) + transition: # transitions priority to enforce order that transitions occur + priorityList: [Unlock, Lock, Undrop, DropUnlock, DropLock] # all priorities equal if not specified +participants: # list of nodes that can serve replicas (optional if dynamic joining is active, required otherwise) + - name: localhost_12001 + host: localhost + port: 12001 + - name: localhost_12002 + host: localhost + port: 12002 + - name: localhost_12003 + host: localhost + port: 12003 +``` + +Using a file like the one above, the cluster can be set up either with the command line: + +``` +incubator-helix/helix-core/target/helix-core/pkg/bin/YAMLClusterSetup.sh localhost:2199 lock-manager-config.yaml +``` + +or with code: + +``` +YAMLClusterSetup setup = new YAMLClusterSetup(zkAddress); +InputStream input = + Thread.currentThread().getContextClassLoader() + .getResourceAsStream("lock-manager-config.yaml"); +YAMLClusterSetup.YAMLClusterConfig config = setup.setupCluster(input); +``` + +Some notes: + +- A rebalancer class is only required for the USER_DEFINED mode. It is ignored otherwise. + +- Built-in state models, like OnlineOffline, LeaderStandby, and MasterSlave, or state models that have already been added only require a name for stateModel. If partition and/or replica counts are not provided, a value of 1 is assumed. \ No newline at end of file diff --git a/site-releases/0.7.0-incubating/src/site/resources/.htaccess b/site-releases/0.7.0-incubating/src/site/resources/.htaccess new file mode 100644 index 0000000000..d5c7bf3f94 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/resources/.htaccess @@ -0,0 +1,20 @@ +# +# 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. +# + +Redirect /download.html /download.cgi diff --git a/site-releases/0.7.0-incubating/src/site/resources/download.cgi b/site-releases/0.7.0-incubating/src/site/resources/download.cgi new file mode 100644 index 0000000000..f9a0e3007c --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/resources/download.cgi @@ -0,0 +1,22 @@ +#!/bin/sh +# Just call the standard mirrors.cgi script. It will use download.html +# as the input template. +# +# 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. +# +exec /www/www.apache.org/dyn/mirrors/mirrors.cgi $* diff --git a/site-releases/0.7.0-incubating/src/site/resources/images/HELIX-components.png b/site-releases/0.7.0-incubating/src/site/resources/images/HELIX-components.png new file mode 100644 index 0000000000..c0c35aee2c Binary files /dev/null and b/site-releases/0.7.0-incubating/src/site/resources/images/HELIX-components.png differ diff --git a/site-releases/0.7.0-incubating/src/site/resources/images/PFS-Generic.png b/site-releases/0.7.0-incubating/src/site/resources/images/PFS-Generic.png new file mode 100644 index 0000000000..7eea3a0497 Binary files /dev/null and b/site-releases/0.7.0-incubating/src/site/resources/images/PFS-Generic.png differ diff --git a/site-releases/0.7.0-incubating/src/site/resources/images/RSYNC_BASED_PFS.png b/site-releases/0.7.0-incubating/src/site/resources/images/RSYNC_BASED_PFS.png new file mode 100644 index 0000000000..0cc55ae100 Binary files /dev/null and b/site-releases/0.7.0-incubating/src/site/resources/images/RSYNC_BASED_PFS.png differ diff --git a/site-releases/0.7.0-incubating/src/site/resources/images/bootstrap_statemodel.gif b/site-releases/0.7.0-incubating/src/site/resources/images/bootstrap_statemodel.gif new file mode 100644 index 0000000000..b8f8a42589 Binary files /dev/null and b/site-releases/0.7.0-incubating/src/site/resources/images/bootstrap_statemodel.gif differ diff --git a/site-releases/0.7.0-incubating/src/site/resources/images/helix-architecture.png b/site-releases/0.7.0-incubating/src/site/resources/images/helix-architecture.png new file mode 100644 index 0000000000..6f69a2db34 Binary files /dev/null and b/site-releases/0.7.0-incubating/src/site/resources/images/helix-architecture.png differ diff --git a/site-releases/0.7.0-incubating/src/site/resources/images/helix-logo.jpg b/site-releases/0.7.0-incubating/src/site/resources/images/helix-logo.jpg new file mode 100644 index 0000000000..d6428f600f Binary files /dev/null and b/site-releases/0.7.0-incubating/src/site/resources/images/helix-logo.jpg differ diff --git a/site-releases/0.7.0-incubating/src/site/resources/images/helix-znode-layout.png b/site-releases/0.7.0-incubating/src/site/resources/images/helix-znode-layout.png new file mode 100644 index 0000000000..5bafc45f21 Binary files /dev/null and b/site-releases/0.7.0-incubating/src/site/resources/images/helix-znode-layout.png differ diff --git a/site-releases/0.7.0-incubating/src/site/resources/images/statemachine.png b/site-releases/0.7.0-incubating/src/site/resources/images/statemachine.png new file mode 100644 index 0000000000..43d27ecfbc Binary files /dev/null and b/site-releases/0.7.0-incubating/src/site/resources/images/statemachine.png differ diff --git a/site-releases/0.7.0-incubating/src/site/resources/images/system.png b/site-releases/0.7.0-incubating/src/site/resources/images/system.png new file mode 100644 index 0000000000..f8a05c8764 Binary files /dev/null and b/site-releases/0.7.0-incubating/src/site/resources/images/system.png differ diff --git a/site-releases/0.7.0-incubating/src/site/site.xml b/site-releases/0.7.0-incubating/src/site/site.xml new file mode 100644 index 0000000000..babbe1c523 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/site.xml @@ -0,0 +1,120 @@ + + + + + images/helix-logo.jpg + http://helix.incubator.apache.org/site-releases/0.7.0-incubating-site + + + http://incubator.apache.org/images/egg-logo.png + http://incubator.apache.org/ + + + + + + + org.apache.maven.skins + maven-fluido-skin + 1.3.0 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+
Apache Helix, Apache, the Apache feather logo, and the Apache Helix project logos are trademarks of The Apache Software Foundation. + All other marks mentioned may be trademarks or registered trademarks of their respective owners.
+ Privacy Policy +
+
+ + + + + + + true + + true + + + ApacheHelix + true + false + + + + +
diff --git a/site-releases/0.7.0-incubating/src/site/xdoc/download.xml.vm b/site-releases/0.7.0-incubating/src/site/xdoc/download.xml.vm new file mode 100644 index 0000000000..14e22c5ec0 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/site/xdoc/download.xml.vm @@ -0,0 +1,213 @@ + + +#set( $releaseName = "0.7.0-incubating" ) +#set( $releaseDate = "11/22/2013" ) + + + + Apache Incubator Helix Downloads + Apache Helix Documentation Team + + + +
+ + + +
+ +
+

Apache Helix artifacts are distributed in source and binary form under the terms of the + Apache License, Version 2.0. + See the included LICENSE and NOTICE files included in each artifact for additional license + information. +

+

Use the links below to download a source distribution of Apache Helix. + It is good practice to verify the integrity of the distribution files.

+
+ +
+

Release date: ${releaseDate}

+

${releaseName} Release notes

+ + + +

+ [if-any logo] + + logo + + [end] + The currently selected mirror is + [preferred]. + If you encounter a problem with this mirror, + please select another mirror. + If all mirrors are failing, there are + backup + mirrors + (at the end of the mirrors list) that should be available. +

+ +
+ Other mirrors: + + +
+ +

+ You may also consult the + complete list of mirrors. +

+ + + + + + + + + + + + + + + + +
ArtifactSignatures
+ helix-${releaseName}-src.zip + + asc + md5 + sha1 +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + +
ArtifactSignatures
+ helix-core-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+ helix-admin-webapp-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+ helix-agent-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+ helix-examples-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+
+
+ + + +
+

We strongly recommend you verify the integrity of the downloaded files with both PGP and MD5.

+ +

The PGP signatures can be verified using PGP or + GPG. + First download the KEYS as well as the + *.asc signature file for the particular distribution. Make sure you get these files from the main + distribution directory, rather than from a mirror. Then verify the signatures using one of the following sets of + commands: + + $ pgp -ka KEYS +$ pgp helix-*.zip.asc + + $ gpg --import KEYS +$ gpg --verify helix-*.zip.asc +

+

Alternatively, you can verify the MD5 signature on the files. A Unix/Linux program called + md5 or + md5sum is included in most distributions. It is also available as part of + GNU Textutils. + Windows users can get binary md5 programs from these (and likely other) places: +

+

+
+ +
diff --git a/site-releases/0.7.0-incubating/src/test/conf/testng.xml b/site-releases/0.7.0-incubating/src/test/conf/testng.xml new file mode 100644 index 0000000000..58f0803678 --- /dev/null +++ b/site-releases/0.7.0-incubating/src/test/conf/testng.xml @@ -0,0 +1,27 @@ + + + + + + + + + + diff --git a/site-releases/pom.xml b/site-releases/pom.xml new file mode 100644 index 0000000000..bfdb1f495c --- /dev/null +++ b/site-releases/pom.xml @@ -0,0 +1,50 @@ + + + + + org.apache.helix + helix + 0.7.1-incubating-SNAPSHOT + + 4.0.0 + pom + + site-releases + Apache Helix :: Site Releases + + + 0.6.1-incubating + 0.6.2-incubating + 0.7.0-incubating + trunk + + + + + + + + + + + + + + diff --git a/site-releases/trunk/pom.xml b/site-releases/trunk/pom.xml new file mode 100644 index 0000000000..1ccdf0d0b0 --- /dev/null +++ b/site-releases/trunk/pom.xml @@ -0,0 +1,51 @@ + + + + 4.0.0 + + + org.apache.helix + site-releases + 0.7.1-incubating-SNAPSHOT + + + trunk-site + bundle + Apache Helix :: Site :: trunk + + + + + + + org.testng + testng + 6.0.1 + + + + + + + + + + + diff --git a/site-releases/trunk/src/site/apt/privacy-policy.apt b/site-releases/trunk/src/site/apt/privacy-policy.apt new file mode 100644 index 0000000000..ada93631d1 --- /dev/null +++ b/site-releases/trunk/src/site/apt/privacy-policy.apt @@ -0,0 +1,52 @@ + ---- + Privacy Policy + ----- + Olivier Lamy + ----- + 2013-02-04 + ----- + +~~ 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. + +Privacy Policy + + Information about your use of this website is collected using server access logs and a tracking cookie. The + collected information consists of the following: + + [[1]] The IP address from which you access the website; + + [[2]] The type of browser and operating system you use to access our site; + + [[3]] The date and time you access our site; + + [[4]] The pages you visit; and + + [[5]] The addresses of pages from where you followed a link to our site. + + [] + + Part of this information is gathered using a tracking cookie set by the + {{{http://www.google.com/analytics/}Google Analytics}} service and handled by Google as described in their + {{{http://www.google.com/privacy.html}privacy policy}}. See your browser documentation for instructions on how to + disable the cookie if you prefer not to share this data with Google. + + We use the gathered information to help us make our site more useful to visitors and to better understand how and + when our site is used. We do not track or collect personally identifiable information or associate gathered data + with any personally identifying information from other sources. + + By using this website, you consent to the collection of this data in the manner and for the purpose described above. diff --git a/site-releases/trunk/src/site/apt/releasing.apt b/site-releases/trunk/src/site/apt/releasing.apt new file mode 100644 index 0000000000..11d0cd92aa --- /dev/null +++ b/site-releases/trunk/src/site/apt/releasing.apt @@ -0,0 +1,107 @@ + ----- + Helix release process + ----- + ----- + 2012-12-15 + ----- + +~~ 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. + +~~ NOTE: For help with the syntax of this file, see: +~~ http://maven.apache.org/guides/mini/guide-apt-format.html + +Helix release process + + [[1]] Post to the dev list a few days before you plan to do an Helix release + + [[2]] Your maven setting must contains the entry to be able to deploy. + + ~/.m2/settings.xml + ++------------- + + apache.releases.https + + + ++------------- + + [[3]] Apache DAV passwords + ++------------- + Add the following info into your ~/.netrc + machine git-wip-us.apache.org login + ++------------- + [[4]] Release Helix + You should have a GPG agent running in the session you will run the maven release commands(preferred), and confirm it works by running "gpg -ab" (type some text and press Ctrl-D). + If you do not have a GPG agent running, make sure that you have the "apache-release" profile set in your settings.xml as shown below. + + Run the release + ++------------- +mvn release:prepare release:perform -B ++------------- + + GPG configuration in maven settings xml: + ++------------- + + apache-release + + [GPG_PASSWORD] + + ++------------- + + [[4]] go to https://repository.apache.org and close your staged repository. Note the repository url (format https://repository.apache.org/content/repositories/orgapachehelix-019/org/apache/helix/helix/0.6-incubating/) + ++------------- +svn co https://dist.apache.org/repos/dist/dev/incubator/helix helix-dev-release +cd helix-dev-release +sh ./release-script-svn.sh version stagingRepoUrl +then svn add +then svn ci ++------------- + + [[5]] Validating the release + ++------------- + * Download sources, extract, build and run tests - mvn clean package + * Verify license headers - mvn -Prat -DskipTests + * Download binaries and .asc files + * Download release manager's public key - From the KEYS file, get the release manager's public key finger print and run gpg --keyserver pgpkeys.mit.edu --recv-key + * Validate authenticity of key - run gpg --fingerprint + * Check signatures of all the binaries using gpg ++------------- + + [[6]] Call for a vote in the dev list and wait for 72 hrs. for the vote results. 3 binding votes are necessary for the release to be finalized. example + After the vote has passed, move the files from dist dev to dist release: svn mv https://dist.apache.org/repos/dist/dev/incubator/helix/version to https://dist.apache.org/repos/dist/release/incubator/helix/ + + [[7]] Prepare release note. Add a page in src/site/apt/releasenotes/ and change value of \ in parent pom. + + + [[8]] Send out an announcement of the release to: + + * users@helix.incubator.apache.org + + * dev@helix.incubator.apache.org + + [[9]] Celebrate ! + + diff --git a/site-releases/trunk/src/site/markdown/Architecture.md b/site-releases/trunk/src/site/markdown/Architecture.md new file mode 100644 index 0000000000..933e917c7e --- /dev/null +++ b/site-releases/trunk/src/site/markdown/Architecture.md @@ -0,0 +1,252 @@ + + + + Architecture + + +Architecture +---------------------------- +Helix aims to provide the following abilities to a distributed system: + +* Automatic management of a cluster hosting partitioned, replicated resources. +* Soft and hard failure detection and handling. +* Automatic load balancing via smart placement of resources on servers(nodes) based on server capacity and resource profile (size of partition, access patterns, etc). +* Centralized config management and self discovery. Eliminates the need to modify config on each node. +* Fault tolerance and optimized rebalancing during cluster expansion. +* Manages entire operational lifecycle of a node. Addition, start, stop, enable/disable without downtime. +* Monitor cluster health and provide alerts on SLA violation. +* Service discovery mechanism to route requests. + +To build such a system, we need a mechanism to co-ordinate between different nodes and other components in the system. This mechanism can be achieved with software that reacts to any change in the cluster and comes up with a set of tasks needed to bring the cluster to a stable state. The set of tasks will be assigned to one or more nodes in the cluster. Helix serves this purpose of managing the various components in the cluster. + +![Helix Design](images/system.png) + +Distributed System Components + +In general any distributed system cluster will have the following components and properties: + +* A set of nodes also referred to as instances. +* A set of resources which can be databases, lucene indexes or tasks. +* Each resource is also partitioned into one or more Partitions. +* Each partition may have one or more copies called replicas. +* Each replica can have a state associated with it. For example Master, Slave, Leader, Standby, Online, Offline etc + +Roles +----- + +![Helix Design](images/HELIX-components.png) + +Not all nodes in a distributed system will perform similar functionalities. For example, a few nodes might be serving requests and a few nodes might be sending requests, and some nodes might be controlling the nodes in the cluster. Thus, Helix categorizes nodes by their specific roles in the system. + +We have divided Helix nodes into 3 logical components based on their responsibilities: + +1. Participant: The nodes that actually host the distributed resources. +2. Spectator: The nodes that simply observe the Participant state and route the request accordingly. Routers, for example, need to know the instance on which a partition is hosted and its state in order to route the request to the appropriate end point. +3. Controller: The controller observes and controls the Participant nodes. It is responsible for coordinating all transitions in the cluster and ensuring that state constraints are satisfied and cluster stability is maintained. + +These are simply logical components and can be deployed as per the system requirements. For example: + +1. The controller can be deployed as a separate service +2. The controller can be deployed along with a Participant but only one Controller will be active at any given time. + +Both have pros and cons, which will be discussed later and one can chose the mode of deployment as per system needs. + + +## Cluster state metadata store + +We need a distributed store to maintain the state of the cluster and a notification system to notify if there is any change in the cluster state. Helix uses Zookeeper to achieve this functionality. + +Zookeeper provides: + +* A way to represent PERSISTENT state which basically remains until its deleted. +* A way to represent TRANSIENT/EPHEMERAL state which vanishes when the process that created the state dies. +* Notification mechanism when there is a change in PERSISTENT and EPHEMERAL state + +The namespace provided by ZooKeeper is much like that of a standard file system. A name is a sequence of path elements separated by a slash (/). Every node[ZNode] in ZooKeeper\'s namespace is identified by a path. + +More info on Zookeeper can be found at http://zookeeper.apache.org + +## State machine and constraints + +Even though the concepts of Resources, Partitions, and Replicas are common to most distributed systems, one thing that differentiates one distributed system from another is the way each partition is assigned a state and the constraints on each state. + +For example: + +1. If a system is serving read-only data then all partition\'s replicas are equal and they can either be ONLINE or OFFLINE. +2. If a system takes _both_ reads and writes but ensure that writes go through only one partition, the states will be MASTER, SLAVE, and OFFLINE. Writes go through the MASTER and replicate to the SLAVEs. Optionally, reads can go through SLAVES. + +Apart from defining state for each partition, the transition path to each state can be application specific. For example, in order to become MASTER it might be a requirement to first become a SLAVE. This ensures that if the SLAVE does not have the data as part of OFFLINE-SLAVE transition it can bootstrap data from other nodes in the system. + +Helix provides a way to configure an application specific state machine along with constraints on each state. Along with constraints on STATE, Helix also provides a way to specify constraints on transitions. (More on this later.) + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +![Helix Design](images/statemachine.png) + +## Concepts + +The following terminologies are used in Helix to model a state machine. + +* IdealState: The state in which we need the cluster to be in if all nodes are up and running. In other words, all state constraints are satisfied. +* CurrentState: Represents the actual current state of each node in the cluster +* ExternalView: Represents the combined view of CurrentState of all nodes. + +The goal of Helix is always to make the CurrentState of the system same as the IdealState. Some scenarios where this may not be true are: + +* When all nodes are down +* When one or more nodes fail +* New nodes are added and the partitions need to be reassigned + +### IdealState + +Helix lets the application define the IdealState on a resource basis which basically consists of: + +* List of partitions. Example: 64 +* Number of replicas for each partition. Example: 3 +* Node and State for each replica. + +Example: + +* Partition-1, replica-1, Master, Node-1 +* Partition-1, replica-2, Slave, Node-2 +* Partition-1, replica-3, Slave, Node-3 +* ..... +* ..... +* Partition-p, replica-3, Slave, Node-n + +Helix comes with various algorithms to automatically assign the partitions to nodes. The default algorithm minimizes the number of shuffles that happen when new nodes are added to the system. + +### CurrentState + +Every instance in the cluster hosts one or more partitions of a resource. Each of the partitions has a state associated with it. + +Example Node-1 + +* Partition-1, Master +* Partition-2, Slave +* .... +* .... +* Partition-p, Slave + +### ExternalView + +External clients needs to know the state of each partition in the cluster and the Node hosting that partition. Helix provides one view of the system to Spectators as _ExternalView_. ExternalView is simply an aggregate of all node CurrentStates. + +* Partition-1, replica-1, Master, Node-1 +* Partition-1, replica-2, Slave, Node-2 +* Partition-1, replica-3, Slave, Node-3 +* ..... +* ..... +* Partition-p, replica-3, Slave, Node-n + +## Process Workflow + +Mode of operation in a cluster + +A node process can be one of the following: + +* Participant: The process registers itself in the cluster and acts on the messages received in its queue and updates the current state. Example: a storage node in a distributed database +* Spectator: The process is simply interested in the changes in the Externalview. +* Controller: This process actively controls the cluster by reacting to changes in cluster state and sending messages to Participants. + + +### Participant Node Process + +* When Node starts up, it registers itself under _LiveInstances_ +* After registering, it waits for new _Messages_ in the message queue +* When it receives a message, it will perform the required task as indicated in the message +* After the task is completed, depending on the task outcome it updates the CurrentState + +### Controller Process + +* Watches IdealState +* Notified when a node goes down/comes up or node is added/removed. Watches LiveInstances and CurrentState of each node in the cluster +* Triggers appropriate state transitions by sending message to Participants + +### Spectator Process + +* When the process starts, it asks the Helix agent to be notified of changes in ExternalView +* Whenever it receives a notification, it reads the Externalview and performs required duties. + +#### Interaction between controller, participant and spectator + +The following picture shows how controllers, participants and spectators interact with each other. + +![Helix Architecture](images/helix-architecture.png) + +## Core algorithm + +* Controller gets the IdealState and the CurrentState of active storage nodes from Zookeeper +* Compute the delta between IdealState and CurrentState for each partition across all participant nodes +* For each partition compute tasks based on the State Machine Table. It\'s possible to configure priority on the state Transition. For example, in case of Master-Slave: + * Attempt mastership transfer if possible without violating constraint. + * Partition Addition + * Drop Partition +* Add the tasks in parallel if possible to the respective queue for each storage node (if the tasks added are mutually independent) +* If a task is dependent on another task being completed, do not add that task +* After any task is completed by a Participant, Controllers gets notified of the change and the State Transition algorithm is re-run until the CurrentState is same as IdealState. + +## Helix ZNode layout + +Helix organizes znodes under clusterName in multiple levels. + +The top level (under the cluster name) ZNodes are all Helix-defined and in upper case: + +* PROPERTYSTORE: application property store +* STATEMODELDEFES: state model definitions +* INSTANCES: instance runtime information including current state and messages +* CONFIGS: configurations +* IDEALSTATES: ideal states +* EXTERNALVIEW: external views +* LIVEINSTANCES: live instances +* CONTROLLER: cluster controller runtime information + +Under INSTANCES, there are runtime ZNodes for each instance. An instance organizes ZNodes as follows: + +* CURRENTSTATES + * sessionId + * resourceName +* ERRORS +* STATUSUPDATES +* MESSAGES +* HEALTHREPORT + +Under CONFIGS, there are different scopes of configurations: + +* RESOURCE: contains resource scope configurations +* CLUSTER: contains cluster scope configurations +* PARTICIPANT: contains participant scope configurations + +The following image shows an example of Helix znodes layout for a cluster named "test-cluster": + +![Helix znode layout](images/helix-znode-layout.png) diff --git a/site-releases/trunk/src/site/markdown/Building.md b/site-releases/trunk/src/site/markdown/Building.md new file mode 100644 index 0000000000..2d8a51b403 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/Building.md @@ -0,0 +1,29 @@ + + +Build Instructions +------------------ + +Requirements: JDK 1.6+, Maven 2.0.8+ + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn install package -DskipTests +``` diff --git a/site-releases/trunk/src/site/markdown/Concepts.md b/site-releases/trunk/src/site/markdown/Concepts.md new file mode 100644 index 0000000000..fa5d0ba75f --- /dev/null +++ b/site-releases/trunk/src/site/markdown/Concepts.md @@ -0,0 +1,275 @@ + + + + Concepts + + +Concepts +---------------------------- + +Helix is based on the idea that a given task has the following attributes associated with it: + +* _Location of the task_. For example it runs on Node N1 +* _State_. For example, it is running, stopped etc. + +In Helix terminology, a task is referred to as a _resource_. + +### IdealState + +IdealState simply allows one to map tasks to location and state. A standard way of expressing this in Helix: + +``` + "TASK_NAME" : { + "LOCATION" : "STATE" + } + +``` +Consider a simple case where you want to launch a task \'myTask\' on node \'N1\'. The IdealState for this can be expressed as follows: + +``` +{ + "id" : "MyTask", + "mapFields" : { + "myTask" : { + "N1" : "ONLINE", + } + } +} +``` +### Partition + +If this task get too big to fit on one box, you might want to divide it into subtasks. Each subtask is referred to as a _partition_ in Helix. Let\'s say you want to divide the task into 3 subtasks/partitions, the IdealState can be changed as shown below. + +\'myTask_0\', \'myTask_1\', \'myTask_2\' are logical names representing the partitions of myTask. Each tasks runs on N1, N2 and N3 respectively. + +``` +{ + "id" : "myTask", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + } + "mapFields" : { + "myTask_0" : { + "N1" : "ONLINE", + }, + "myTask_1" : { + "N2" : "ONLINE", + }, + "myTask_2" : { + "N3" : "ONLINE", + } + } +} +``` + +### Replica + +Partitioning allows one to split the data/task into multiple subparts. But let\'s say the request rate for each partition increases. The common solution is to have multiple copies for each partition. Helix refers to the copy of a partition as a _replica_. Adding a replica also increases the availability of the system during failures. One can see this methodology employed often in search systems. The index is divided into shards, and each shard has multiple copies. + +Let\'s say you want to add one additional replica for each task. The IdealState can simply be changed as shown below. + +For increasing the availability of the system, it\'s better to place the replica of a given partition on different nodes. + +``` +{ + "id" : "myIndex", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + }, + "mapFields" : { + "myIndex_0" : { + "N1" : "ONLINE", + "N2" : "ONLINE" + }, + "myIndex_1" : { + "N2" : "ONLINE", + "N3" : "ONLINE" + }, + "myIndex_2" : { + "N3" : "ONLINE", + "N1" : "ONLINE" + } + } +} +``` + +### State + +Now let\'s take a slightly more complicated scenario where a task represents a database. Unlike an index which is in general read-only, a database supports both reads and writes. Keeping the data consistent among the replicas is crucial in distributed data stores. One commonly applied technique is to assign one replica as the MASTER and remaining replicas as SLAVEs. All writes go to the MASTER and are then replicated to the SLAVE replicas. + +Helix allows one to assign different states to each replica. Let\'s say you have two MySQL instances N1 and N2, where one will serve as MASTER and another as SLAVE. The IdealState can be changed to: + +``` +{ + "id" : "myDB", + "simpleFields" : { + "NUM_PARTITIONS" : "1", + "REPLICAS" : "2", + }, + "mapFields" : { + "myDB" : { + "N1" : "MASTER", + "N2" : "SLAVE", + } + } +} + +``` + + +### State Machine and Transitions + +IdealState allows one to exactly specify the desired state of the cluster. Given an IdealState, Helix takes up the responsibility of ensuring that the cluster reaches the IdealState. The Helix _controller_ reads the IdealState and then commands each Participant to take appropriate actions to move from one state to another until it matches the IdealState. These actions are referred to as _transitions_ in Helix. + +The next logical question is: how does the _controller_ compute the transitions required to get to IdealState? This is where the finite state machine concept comes in. Helix allows applications to plug in a finite state machine. A state machine consists of the following: + +* State: Describes the role of a replica +* Transition: An action that allows a replica to move from one state to another, thus changing its role. + +Here is an example of MasterSlave state machine: + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +Helix allows each resource to be associated with one state machine. This means you can have one resource as an index and another as a database in the same cluster. One can associate each resource with a state machine as follows: + +``` +{ + "id" : "myDB", + "simpleFields" : { + "NUM_PARTITIONS" : "1", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "myDB" : { + "N1" : "MASTER", + "N2" : "SLAVE", + } + } +} + +``` + +### Current State + +CurrentState of a resource simply represents its actual state at a Participant. In the below example: + +* INSTANCE_NAME: Unique name representing the process +* SESSION_ID: ID that is automatically assigned every time a process joins the cluster + +``` +{ + "id":"MyResource" + ,"simpleFields":{ + ,"SESSION_ID":"13d0e34675e0002" + ,"INSTANCE_NAME":"node1" + ,"STATE_MODEL_DEF":"MasterSlave" + } + ,"mapFields":{ + "MyResource_0":{ + "CURRENT_STATE":"SLAVE" + } + ,"MyResource_1":{ + "CURRENT_STATE":"MASTER" + } + ,"MyResource_2":{ + "CURRENT_STATE":"MASTER" + } + } +} +``` +Each node in the cluster has its own CurrentState. + +### External View + +In order to communicate with the Participants, external clients need to know the current state of each of the Participants. The external clients are referred to as Spectators. In order to make the life of Spectator simple, Helix provides an ExternalView that is an aggregated view of the current state across all nodes. The ExternalView has a similar format as IdealState. + +``` +{ + "id":"MyResource", + "mapFields":{ + "MyResource_0":{ + "N1":"SLAVE", + "N2":"MASTER", + "N3":"OFFLINE" + }, + "MyResource_1":{ + "N1":"MASTER", + "N2":"SLAVE", + "N3":"ERROR" + }, + "MyResource_2":{ + "N1":"MASTER", + "N2":"SLAVE", + "N3":"SLAVE" + } + } +} +``` + +### Rebalancer + +The core component of Helix is the Controller which runs the Rebalancer algorithm on every cluster event. Cluster events can be one of the following: + +* Nodes start/stop and soft/hard failures +* New nodes are added/removed +* Ideal state changes + +There are few more examples such as configuration changes, etc. The key takeaway: there are many ways to trigger the rebalancer. + +When a rebalancer is run it simply does the following: + +* Compares the IdealState and current state +* Computes the transitions required to reach the IdealState +* Issues the transitions to each Participant + +The above steps happen for every change in the system. Once the current state matches the IdealState, the system is considered stable which implies \'IdealState = CurrentState = ExternalView\' + +### Dynamic IdealState + +One of the things that makes Helix powerful is that IdealState can be changed dynamically. This means one can listen to cluster events like node failures and dynamically change the ideal state. Helix will then take care of triggering the respective transitions in the system. + +Helix comes with a few algorithms to automatically compute the IdealState based on the constraints. For example, if you have a resource of 3 partitions and 2 replicas, Helix can automatically compute the IdealState based on the nodes that are currently active. See the [tutorial](./tutorial_rebalance.html) to find out more about various execution modes of Helix like FULL_AUTO, SEMI_AUTO and CUSTOMIZED. + + + + + + + + + + + + diff --git a/site-releases/trunk/src/site/markdown/Features.md b/site-releases/trunk/src/site/markdown/Features.md new file mode 100644 index 0000000000..ba9d0e72d0 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/Features.md @@ -0,0 +1,313 @@ + + + + Features + + +Features +---------------------------- + + +### CONFIGURING IDEALSTATE + + +Read concepts page for definition of Idealstate. + +The placement of partitions in a DDS is very critical for reliability and scalability of the system. +For example, when a node fails, it is important that the partitions hosted on that node are reallocated evenly among the remaining nodes. Consistent hashing is one such algorithm that can guarantee this. +Helix by default comes with a variant of consistent hashing based of the RUSH algorithm. + +This means given a number of partitions, replicas and number of nodes Helix does the automatic assignment of partition to nodes such that + +* Each node has the same number of partitions and replicas of the same partition do not stay on the same node. +* When a node fails, the partitions will be equally distributed among the remaining nodes +* When new nodes are added, the number of partitions moved will be minimized along with satisfying the above two criteria. + + +Helix provides multiple ways to control the placement and state of a replica. + +``` + + |AUTO REBALANCE| AUTO | CUSTOM | + ----------------------------------------- + LOCATION | HELIX | APP | APP | + ----------------------------------------- + STATE | HELIX | HELIX | APP | + ----------------------------------------- +``` + +#### HELIX EXECUTION MODE + + +Idealstate is defined as the state of the DDS when all nodes are up and running and healthy. +Helix uses this as the target state of the system and computes the appropriate transitions needed in the system to bring it to a stable state. + +Helix supports 3 different execution modes which allows application to explicitly control the placement and state of the replica. + +##### AUTO_REBALANCE + +When the idealstate mode is set to AUTO_REBALANCE, Helix controls both the location of the replica along with the state. This option is useful for applications where creation of a replica is not expensive. Example + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "IDEAL_STATE_MODE" : "AUTO_REBALANCE", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [], + "MyResource_1" : [], + "MyResource_2" : [] + }, + "mapFields" : { + } +} +``` + +If there are 3 nodes in the cluster, then Helix will internally compute the ideal state as + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +Another typical example is evenly distributing a group of tasks among the currently alive processes. For example, if there are 60 tasks and 4 nodes, Helix assigns 15 tasks to each node. +When one node fails Helix redistributes its 15 tasks to the remaining 3 nodes. Similarly, if a node is added, Helix re-allocates 3 tasks from each of the 4 nodes to the 5th node. + +#### AUTO + +When the idealstate mode is set to AUTO, Helix only controls STATE of the replicas where as the location of the partition is controlled by application. Example: The below idealstate indicates thats 'MyResource_0' must be only on node1 and node2. But gives the control of assigning the STATE to Helix. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "IDEAL_STATE_MODE" : "AUTO", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [node1, node2], + "MyResource_1" : [node2, node3], + "MyResource_2" : [node3, node1] + }, + "mapFields" : { + } +} +``` +In this mode when node1 fails, unlike in AUTO-REBALANCE mode the partition is not moved from node1 to others nodes in the cluster. Instead, Helix will decide to change the state of MyResource_0 in N2 based on the system constraints. For example, if a system constraint specified that there should be 1 Master and if the Master failed, then node2 will be made the new master. + +#### CUSTOM + +Helix offers a third mode called CUSTOM, in which application can completely control the placement and state of each replica. Applications will have to implement an interface that Helix will invoke when the cluster state changes. +Within this callback, the application can recompute the idealstate. Helix will then issue appropriate transitions such that Idealstate and Currentstate converges. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "IDEAL_STATE_MODE" : "CUSTOM", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +For example, the current state of the system might be 'MyResource_0' -> {N1:MASTER,N2:SLAVE} and the application changes the ideal state to 'MyResource_0' -> {N1:SLAVE,N2:MASTER}. Helix will not blindly issue MASTER-->SLAVE to N1 and SLAVE-->MASTER to N2 in parallel since it might result in a transient state where both N1 and N2 are masters. +Helix will first issue MASTER-->SLAVE to N1 and after its completed it will issue SLAVE-->MASTER to N2. + + +### State Machine Configuration + +Helix comes with 3 default state models that are most commonly used. Its possible to have multiple state models in a cluster. +Every resource that is added should have a reference to the state model. + +* MASTER-SLAVE: Has 3 states OFFLINE,SLAVE,MASTER. Max masters is 1. Slaves will be based on the replication factor. Replication factor can be specified while adding the resource +* ONLINE-OFFLINE: Has 2 states OFFLINE and ONLINE. Very simple state model and most applications start off with this state model. +* LEADER-STANDBY:1 Leader and many stand bys. In general the standby's are idle. + +Apart from providing the state machine configuration, one can specify the constraints of states and transitions. + +For example one can say +Master:1. Max number of replicas in Master state at any time is 1. +OFFLINE-SLAVE:5 Max number of Offline-Slave transitions that can happen concurrently in the system + +STATE PRIORITY +Helix uses greedy approach to satisfy the state constraints. For example if the state machine configuration says it needs 1 master and 2 slaves but only 1 node is active, Helix must promote it to master. This behavior is achieved by providing the state priority list as MASTER,SLAVE. + +STATE TRANSITION PRIORITY +Helix tries to fire as many transitions as possible in parallel to reach the stable state without violating constraints. By default Helix simply sorts the transitions alphabetically and fires as many as it can without violating the constraints. +One can control this by overriding the priority order. + +### Config management + +Helix allows applications to store application specific properties. The configuration can have different scopes. + +* Cluster +* Node specific +* Resource specific +* Partition specific + +Helix also provides notifications when any configs are changed. This allows applications to support dynamic configuration changes. + +See HelixManager.getConfigAccessor for more info + +### Intra cluster messaging api + +This is an interesting feature which is quite useful in practice. Often times, nodes in DDS requires a mechanism to interact with each other. One such requirement is a process of bootstrapping a replica. + +Consider a search system use case where the index replica starts up and it does not have an index. One of the commonly used solutions is to get the index from a common location or to copy the index from another replica. +Helix provides a messaging api, that can be used to talk to other nodes in the system. The value added that Helix provides here is, message recipient can be specified in terms of resource, +partition, state and Helix ensures that the message is delivered to all of the required recipients. In this particular use case, the instance can specify the recipient criteria as all replicas of P1. +Since Helix is aware of the global state of the system, it can send the message to appropriate nodes. Once the nodes respond Helix provides the bootstrapping replica with all the responses. + +This is a very generic api and can also be used to schedule various periodic tasks in the cluster like data backups etc. +System Admins can also perform adhoc tasks like on demand backup or execute a system command(like rm -rf ;-)) across all nodes. + +``` + ClusterMessagingService messagingService = manager.getMessagingService(); + //CONSTRUCT THE MESSAGE + Message requestBackupUriRequest = new Message( + MessageType.USER_DEFINE_MSG, UUID.randomUUID().toString()); + requestBackupUriRequest + .setMsgSubType(BootstrapProcess.REQUEST_BOOTSTRAP_URL); + requestBackupUriRequest.setMsgState(MessageState.NEW); + //SET THE RECIPIENT CRITERIA, All nodes that satisfy the criteria will receive the message + Criteria recipientCriteria = new Criteria(); + recipientCriteria.setInstanceName("%"); + recipientCriteria.setRecipientInstanceType(InstanceType.PARTICIPANT); + recipientCriteria.setResource("MyDB"); + recipientCriteria.setPartition(""); + //Should be processed only the process that is active at the time of sending the message. + //This means if the recipient is restarted after message is sent, it will not be processed. + recipientCriteria.setSessionSpecific(true); + // wait for 30 seconds + int timeout = 30000; + //The handler that will be invoked when any recipient responds to the message. + BootstrapReplyHandler responseHandler = new BootstrapReplyHandler(); + //This will return only after all recipients respond or after timeout. + int sentMessageCount = messagingService.sendAndWait(recipientCriteria, + requestBackupUriRequest, responseHandler, timeout); +``` + +See HelixManager.getMessagingService for more info. + + +### Application specific property storage + +There are several usecases where applications needs support for distributed data structures. Helix uses Zookeeper to store the application data and hence provides notifications when the data changes. +One value add Helix provides is the ability to specify cache the data and also write through cache. This is more efficient than reading from ZK every time. + +See HelixManager.getHelixPropertyStore + +### Throttling + +Since all state changes in the system are triggered through transitions, Helix can control the number of transitions that can happen in parallel. Some of the transitions may be light weight but some might involve moving data around which is quite expensive. +Helix allows applications to set threshold on transitions. The threshold can be set at the multiple scopes. + +* MessageType e.g STATE_TRANSITION +* TransitionType e.g SLAVE-MASTER +* Resource e.g database +* Node i.e per node max transitions in parallel. + +See HelixManager.getHelixAdmin.addMessageConstraint() + +### Health monitoring and alerting + +This in currently in development mode, not yet productionized. + +Helix provides ability for each node in the system to report health metrics on a periodic basis. +Helix supports multiple ways to aggregate these metrics like simple SUM, AVG, EXPONENTIAL DECAY, WINDOW. Helix will only persist the aggregated value. +Applications can define threshold on the aggregate values according to the SLA's and when the SLA is violated Helix will fire an alert. +Currently Helix only fires an alert but eventually we plan to use this metrics to either mark the node dead or load balance the partitions. +This feature will be valuable in for distributed systems that support multi-tenancy and have huge variation in work load patterns. Another place this can be used is to detect skewed partitions and rebalance the cluster. + +This feature is not yet stable and do not recommend to be used in production. + + +### Controller deployment modes + +Read Architecture wiki for more details on the Role of a controller. In simple words, it basically controls the participants in the cluster by issuing transitions. + +Helix provides multiple options to deploy the controller. + +#### STANDALONE + +Controller can be started as a separate process to manage a cluster. This is the recommended approach. How ever since one controller can be a single point of failure, multiple controller processes are required for reliability. +Even if multiple controllers are running only one will be actively managing the cluster at any time and is decided by a leader election process. If the leader fails, another leader will resume managing the cluster. + +Even though we recommend this method of deployment, it has the drawback of having to manage an additional service for each cluster. See Controller As a Service option. + +#### EMBEDDED + +If setting up a separate controller process is not viable, then it is possible to embed the controller as a library in each of the participant. + +#### CONTROLLER AS A SERVICE + +One of the cool feature we added in helix was use a set of controllers to manage a large number of clusters. +For example if you have X clusters to be managed, instead of deploying X*3(3 controllers for fault tolerance) controllers for each cluster, one can deploy only 3 controllers. Each controller can manage X/3 clusters. +If any controller fails the remaining two will manage X/2 clusters. At LinkedIn, we always deploy controllers in this mode. + + + + + + + + diff --git a/site-releases/trunk/src/site/markdown/Quickstart.md b/site-releases/trunk/src/site/markdown/Quickstart.md new file mode 100644 index 0000000000..348f58ac94 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/Quickstart.md @@ -0,0 +1,621 @@ + + + + Quickstart + + +Get Helix +--------- + +First, let\'s get Helix, either build it, or download. + +### Build + + git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git + cd incubator-helix + ./build + cd helix-core/target/helix-core-pkg/bin //This folder contains all the scripts used in following sections + chmod +x * + +Overview +-------- + +In this Quickstart, we\'ll set up a master-slave replicated, partitioned system. Then we\'ll demonstrate how to add a node, rebalance the partitions, and show how Helix manages failover. + + +Let\'s Do It +------------ + +Helix provides command line interfaces to set up the cluster and view the cluster state. The best way to understand how Helix views a cluster is to build a cluster. + +#### First, get to the tools directory + +If you built the code + +``` +cd incubator-helix/helix-core/target/helix-core-pkg/bin +``` + +If you downloaded the release package, extract it. + + +Short Version +------------- +You can observe the components working together in this demo, which does the following: + +* Create a cluster +* Add 2 nodes (participants) to the cluster +* Set up a resource with 6 partitions and 2 replicas: 1 Master, and 1 Slave per partition +* Show the cluster state after Helix balances the partitions +* Add a third node +* Show the cluster state. Note that the third node has taken mastership of 2 partitions. +* Kill the third node (Helix takes care of failover) +* Show the cluster state. Note that the two surviving nodes take over mastership of the partitions from the failed node + +##### Run the demo + +``` +cd incubator-helix/helix-core/target/helix-core-pkg/bin +./quickstart.sh +``` + +##### 2 nodes are set up and the partitions rebalanced + +The cluster state is as follows: + +``` +CLUSTER STATE: After starting 2 nodes + localhost_12000 localhost_12001 + MyResource_0 M S + MyResource_1 S M + MyResource_2 M S + MyResource_3 M S + MyResource_4 S M + MyResource_5 S M +``` + +Note there is one master and one slave per partition. + +##### A third node is added and the cluster rebalanced + +The cluster state changes to: + +``` +CLUSTER STATE: After adding a third node + localhost_12000 localhost_12001 localhost_12002 + MyResource_0 S M S + MyResource_1 S S M + MyResource_2 M S S + MyResource_3 S S M + MyResource_4 M S S + MyResource_5 S M S +``` + +Note there is one master and _two_ slaves per partition. This is expected because there are three nodes. + +##### Finally, a node is killed to simulate a failure + +Helix makes sure each partition has a master. The cluster state changes to: + +``` +CLUSTER STATE: After the 3rd node stops/crashes + localhost_12000 localhost_12001 localhost_12002 + MyResource_0 S M - + MyResource_1 S M - + MyResource_2 M S - + MyResource_3 M S - + MyResource_4 M S - + MyResource_5 S M - +``` + + +Long Version +------------ +Now you can run the same steps by hand. In the detailed version, we\'ll do the following: + +* Define a cluster +* Add two nodes to the cluster +* Add a 6-partition resource with 1 master and 2 slave replicas per partition +* Verify that the cluster is healthy and inspect the Helix view +* Expand the cluster: add a few nodes and rebalance the partitions +* Failover: stop a node and verify the mastership transfer + +### Install and Start Zookeeper + +Zookeeper can be started in standalone mode or replicated mode. + +More info is available at + +* http://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html +* http://zookeeper.apache.org/doc/trunk/zookeeperAdmin.html#sc_zkMulitServerSetup + +In this example, let\'s start zookeeper in local mode. + +##### start zookeeper locally on port 2199 + + ./start-standalone-zookeeper.sh 2199 & + +### Define the Cluster + +The helix-admin tool is used for cluster administration tasks. In the Quickstart, we\'ll use the command line interface. Helix supports a REST interface as well. + +zookeeper_address is of the format host:port e.g localhost:2199 for standalone or host1:port,host2:port for multi-node. + +Next, we\'ll set up a cluster MYCLUSTER cluster with these attributes: + +* 3 instances running on localhost at ports 12913,12914,12915 +* One database named myDB with 6 partitions +* Each partition will have 3 replicas with 1 master, 2 slaves +* zookeeper running locally at localhost:2199 + +##### Create the cluster MYCLUSTER + ## helix-admin.sh --zkSvr --addCluster + ./helix-admin.sh --zkSvr localhost:2199 --addCluster MYCLUSTER + +##### Add nodes to the cluster + +In this case we\'ll add three nodes: localhost:12913, localhost:12914, localhost:12915 + + ## helix-admin.sh --zkSvr --addNode + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12913 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12914 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12915 + +#### Define the resource and partitioning + +In this example, the resource is a database, partitioned 6 ways. (In a production system, it\'s common to over-partition for better load balancing. Helix has been used in production to manage hundreds of databases each with 10s or 100s of partitions running on 10s of physical nodes.) + +##### Create a database with 6 partitions using the MasterSlave state model. + +Helix ensures there will be exactly one master for each partition. + + ## helix-admin.sh --zkSvr --addResource + ./helix-admin.sh --zkSvr localhost:2199 --addResource MYCLUSTER myDB 6 MasterSlave + +##### Now we can let Helix assign partitions to nodes. + +This command will distribute the partitions amongst all the nodes in the cluster. In this example, each partition has 3 replicas. + + ## helix-admin.sh --zkSvr --rebalance + ./helix-admin.sh --zkSvr localhost:2199 --rebalance MYCLUSTER myDB 3 + +Now the cluster is defined in Zookeeper. The nodes (localhost:12913, localhost:12914, localhost:12915) and resource (myDB, with 6 partitions using the MasterSlave model). And the _ideal state_ has been calculated, assuming a replication factor of 3. + +##### Start the Helix Controller + +Now that the cluster is defined in Zookeeper, the Helix controller can manage the cluster. + + ## Start the cluster manager, which will manage MYCLUSTER + ./run-helix-controller.sh --zkSvr localhost:2199 --cluster MYCLUSTER 2>&1 > /tmp/controller.log & + +##### Start up the cluster to be managed + +We\'ve started up Zookeeper, defined the cluster, the resources, the partitioning, and started up the Helix controller. Next, we\'ll start up the nodes of the system to be managed. Each node is a Participant, which is an instance of the system component to be managed. Helix assigns work to Participants, keeps track of their roles and health, and takes action when a node fails. + + # start up each instance. These are mock implementations that are actively managed by Helix + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12913 --stateModelType MasterSlave 2>&1 > /tmp/participant_12913.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12914 --stateModelType MasterSlave 2>&1 > /tmp/participant_12914.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12915 --stateModelType MasterSlave 2>&1 > /tmp/participant_12915.log + + +#### Inspect the Cluster + +Now, let\'s see the Helix view of our cluster. We\'ll work our way down as follows: + +``` +Clusters -> MYCLUSTER -> instances -> instance detail + -> resources -> resource detail + -> partitions +``` + +A single Helix controller can manage multiple clusters, though so far, we\'ve only defined one cluster. Let\'s see: + +``` +## List existing clusters +./helix-admin.sh --zkSvr localhost:2199 --listClusters + +Existing clusters: +MYCLUSTER +``` + +Now, let\'s see the Helix view of MYCLUSTER + +``` +## helix-admin.sh --zkSvr --listClusterInfo +./helix-admin.sh --zkSvr localhost:2199 --listClusterInfo MYCLUSTER + +Existing resources in cluster MYCLUSTER: +myDB +Instances in cluster MYCLUSTER: +localhost_12915 +localhost_12914 +localhost_12913 +``` + + +Let\'s look at the details of an instance + +``` +## ./helix-admin.sh --zkSvr --listInstanceInfo +./helix-admin.sh --zkSvr localhost:2199 --listInstanceInfo MYCLUSTER localhost_12913 + +InstanceConfig: { + "id" : "localhost_12913", + "mapFields" : { + }, + "listFields" : { + }, + "simpleFields" : { + "HELIX_ENABLED" : "true", + "HELIX_HOST" : "localhost", + "HELIX_PORT" : "12913" + } +} +``` + + +##### Query info of a resource + +``` +## helix-admin.sh --zkSvr --listResourceInfo +./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB + +IdealState for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + }, + "myDB_1" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_3" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_4" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + "myDB_0" : [ "localhost_12914", "localhost_12913", "localhost_12915" ], + "myDB_1" : [ "localhost_12915", "localhost_12913", "localhost_12914" ], + "myDB_2" : [ "localhost_12913", "localhost_12915", "localhost_12914" ], + "myDB_3" : [ "localhost_12915", "localhost_12913", "localhost_12914" ], + "myDB_4" : [ "localhost_12913", "localhost_12914", "localhost_12915" ], + "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ] + }, + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "6", + "REPLICAS" : "3", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + } +} + +ExternalView for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + }, + "myDB_1" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_3" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12915" : "MASTER" + }, + "myDB_4" : { + "localhost_12913" : "MASTER", + "localhost_12914" : "SLAVE", + "localhost_12915" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + }, + "simpleFields" : { + "BUCKET_SIZE" : "0" + } +} +``` + +Now, let\'s look at one of the partitions: + + ## helix-admin.sh --zkSvr --listPartitionInfo + ./helix-admin.sh --zkSvr localhost:2199 --listPartitionInfo MYCLUSTER myDB myDB_0 + +#### Expand the Cluster + +Next, we\'ll show how Helix does the work that you\'d otherwise have to build into your system. When you add capacity to your cluster, you want the work to be evenly distributed. In this example, we started with 3 nodes, with 6 partitions. The partitions were evenly balanced, 2 masters and 4 slaves per node. Let\'s add 3 more nodes: localhost:12916, localhost:12917, localhost:12918 + + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12916 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12917 + ./helix-admin.sh --zkSvr localhost:2199 --addNode MYCLUSTER localhost:12918 + +And start up these instances: + + # start up each instance. These are mock implementations that are actively managed by Helix + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12916 --stateModelType MasterSlave 2>&1 > /tmp/participant_12916.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12917 --stateModelType MasterSlave 2>&1 > /tmp/participant_12917.log + ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12918 --stateModelType MasterSlave 2>&1 > /tmp/participant_12918.log + + +And now, let Helix do the work for you. To shift the work, simply rebalance. After the rebalance, each node will have one master and two slaves. + + ./helix-admin.sh --zkSvr localhost:2199 --rebalance MYCLUSTER myDB 3 + +#### View the cluster + +OK, let\'s see how it looks: + + +``` +./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB + +IdealState for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "SLAVE", + "localhost_12918" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + "myDB_0" : [ "localhost_12917", "localhost_12913", "localhost_12914" ], + "myDB_1" : [ "localhost_12918", "localhost_12917", "localhost_12916" ], + "myDB_2" : [ "localhost_12913", "localhost_12917", "localhost_12918" ], + "myDB_3" : [ "localhost_12915", "localhost_12917", "localhost_12918" ], + "myDB_4" : [ "localhost_12916", "localhost_12917", "localhost_12918" ], + "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ] + }, + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "6", + "REPLICAS" : "3", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + } +} + +ExternalView for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "SLAVE", + "localhost_12918" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + }, + "simpleFields" : { + "BUCKET_SIZE" : "0" + } +} +``` + +Mission accomplished. The partitions are nicely balanced. + +#### How about Failover? + +Building a fault tolerant system isn\'t trivial, but with Helix, it\'s easy. Helix detects a failed instance, and triggers mastership transfer automatically. + +First, let's fail an instance. In this example, we\'ll kill localhost:12918 to simulate a failure. + +We lost localhost:12918, so myDB_1 lost its MASTER. Helix can fix that, it will transfer mastership to a healthy node that is currently a SLAVE, say localhost:12197. Helix balances the load as best as it can, given there are 6 partitions on 5 nodes. Let\'s see: + + +``` +./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB + +IdealState for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "SLAVE", + "localhost_12918" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE", + "localhost_12918" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + "myDB_0" : [ "localhost_12917", "localhost_12913", "localhost_12914" ], + "myDB_1" : [ "localhost_12918", "localhost_12917", "localhost_12916" ], + "myDB_2" : [ "localhost_12913", "localhost_12918", "localhost_12917" ], + "myDB_3" : [ "localhost_12915", "localhost_12918", "localhost_12917" ], + "myDB_4" : [ "localhost_12916", "localhost_12917", "localhost_12918" ], + "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ] + }, + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "6", + "REPLICAS" : "3", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + } +} + +ExternalView for myDB: +{ + "id" : "myDB", + "mapFields" : { + "myDB_0" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_1" : { + "localhost_12916" : "SLAVE", + "localhost_12917" : "MASTER" + }, + "myDB_2" : { + "localhost_12913" : "MASTER", + "localhost_12917" : "SLAVE" + }, + "myDB_3" : { + "localhost_12915" : "MASTER", + "localhost_12917" : "SLAVE" + }, + "myDB_4" : { + "localhost_12916" : "MASTER", + "localhost_12917" : "SLAVE" + }, + "myDB_5" : { + "localhost_12913" : "SLAVE", + "localhost_12914" : "MASTER", + "localhost_12915" : "SLAVE" + } + }, + "listFields" : { + }, + "simpleFields" : { + "BUCKET_SIZE" : "0" + } +} +``` + +As we\'ve seen in this Quickstart, Helix takes care of partitioning, load balancing, elasticity, failure detection and recovery. + +##### ZooInspector + +You can view all of the underlying data by going direct to zookeeper. Use ZooInspector that comes with zookeeper to browse the data. This is a java applet (make sure you have X windows) + +To start zooinspector run the following command from /contrib/ZooInspector + + java -cp zookeeper-3.3.3-ZooInspector.jar:lib/jtoaster-1.0.4.jar:../../lib/log4j-1.2.15.jar:../../zookeeper-3.3.3.jar org.apache.zookeeper.inspector.ZooInspector + +#### Next + +Now that you understand the idea of Helix, read the [tutorial](./tutorial.html) to learn how to choose the right state model and constraints for your system, and how to implement it. In many cases, the built-in features meet your requirements. And best of all, Helix is a customizable framework, so you can plug in your own behavior, while retaining the automation provided by Helix. + diff --git a/site-releases/trunk/src/site/markdown/Tutorial.md b/site-releases/trunk/src/site/markdown/Tutorial.md new file mode 100644 index 0000000000..ee5a3938ef --- /dev/null +++ b/site-releases/trunk/src/site/markdown/Tutorial.md @@ -0,0 +1,284 @@ + + + + Tutorial + + +# Helix Tutorial + +In this tutorial, we will cover the roles of a Helix-managed cluster, and show the code you need to write to integrate with it. In many cases, there is a simple default behavior that is often appropriate, but you can also customize the behavior. + +Convention: we first cover the _basic_ approach, which is the easiest to implement. Then, we'll describe _advanced_ options, which give you more control over the system behavior, but require you to write more code. + + +### Prerequisites + +1. Read [Concepts/Terminology](./Concepts.html) and [Architecture](./Architecture.html) +2. Read the [Quickstart guide](./Quickstart.html) to learn how Helix models and manages a cluster +3. Install Helix source. See: [Quickstart](./Quickstart.html) for the steps. + +### Tutorial Outline + +1. [Participant](./tutorial_participant.html) +2. [Spectator](./tutorial_spectator.html) +3. [Controller](./tutorial_controller.html) +4. [Rebalancing Algorithms](./tutorial_rebalance.html) +5. [User-Defined Rebalancing](./tutorial_user_def_rebalancer.html) +6. [State Machines](./tutorial_state.html) +7. [Messaging](./tutorial_messaging.html) +8. [Customized health check](./tutorial_health.html) +9. [Throttling](./tutorial_throttling.html) +10. [Application Property Store](./tutorial_propstore.html) +11. [Logical Accessors](./tutorial_accessors.html) +12. [Admin Interface](./tutorial_admin.html) +13. [YAML Cluster Setup](./tutorial_yaml.html) + +### Preliminaries + +First, we need to set up the system. Let\'s walk through the steps in building a distributed system using Helix. We will show how to do this using both the Java admin interface, as well as the [cluster accessor](./tutorial_accessors.html) interface. You can choose either interface depending on which most closely matches your needs. + +### Start Zookeeper + +This starts a zookeeper in standalone mode. For production deployment, see [Apache Zookeeper](http://zookeeper.apache.org) for instructions. + +``` + ./start-standalone-zookeeper.sh 2199 & +``` + +### Create a cluster + +Creating a cluster will define the cluster in appropriate znodes on zookeeper. + +Using the Java accessor API: + +``` +// Note: ZK_ADDRESS is the host:port of Zookeeper +String ZK_ADDRESS = "localhost:2199"; +HelixConnection connection = new ZKHelixConnection(ZK_ADDRESS); + +ClusterId clusterId = ClusterId.from("helix-demo"); +ClusterAccessor clusterAccessor = connection.createClusterAccessor(clusterId); +ClusterConfig clusterConfig = new ClusterConfig.Builder(clusterId).build(); +clusterAccessor.createCluster(clusterConfig); +``` + +OR + +Using the HelixAdmin Java interface: + +``` +// Create setup tool instance +// Note: ZK_ADDRESS is the host:port of Zookeeper +String ZK_ADDRESS = "localhost:2199"; +HelixAdmin admin = new ZKHelixAdmin(ZK_ADDRESS); + +String CLUSTER_NAME = "helix-demo"; +//Create cluster namespace in zookeeper +admin.addCluster(CLUSTER_NAME); +``` + +OR + +Using the command-line interface: + +``` + ./helix-admin.sh --zkSvr localhost:2199 --addCluster helix-demo +``` + + +### Configure the nodes of the cluster + +First we\'ll add new nodes to the cluster, then configure the nodes in the cluster. Each node in the cluster must be uniquely identifiable. +The most commonly used convention is hostname_port. + +``` +int NUM_NODES = 2; +String hosts[] = new String[]{"localhost","localhost"}; +int ports[] = new int[]{7000,7001}; +for (int i = 0; i < NUM_NODES; i++) +{ + ParticipantId participantId = ParticipantId.from(hosts[i] + "_" + ports[i]); + + // set additional configuration for the participant; these can be accessed during node start up + UserConfig userConfig = new UserConfig(Scope.participant(participantId)); + userConfig.setSimpleField("key", "value"); + + // configure and add the participant + ParticipantConfig participantConfig = new ParticipantConfig.Builder(participantId) + .hostName(hosts[i]).port(ports[i]).enabled(true).userConfig(userConfig).build(); + clusterAccessor.addParticipantToCluster(participantConfig); +} +``` + +OR + +Using the HelixAdmin Java interface: + +``` +String CLUSTER_NAME = "helix-demo"; +int NUM_NODES = 2; +String hosts[] = new String[]{"localhost","localhost"}; +String ports[] = new String[]{7000,7001}; +for (int i = 0; i < NUM_NODES; i++) +{ + InstanceConfig instanceConfig = new InstanceConfig(hosts[i] + "_" + ports[i]); + instanceConfig.setHostName(hosts[i]); + instanceConfig.setPort(ports[i]); + instanceConfig.setInstanceEnabled(true); + + //Add additional system specific configuration if needed. These can be accessed during the node start up. + instanceConfig.getRecord().setSimpleField("key", "value"); + admin.addInstance(CLUSTER_NAME, instanceConfig); +} +``` + +### Configure the resource + +A _resource_ represents the actual task performed by the nodes. It can be a database, index, topic, queue or any other processing entity. +A _resource_ can be divided into many sub-parts known as _partitions_. + + +#### Define the _state model_ and _constraints_ + +For scalability and fault tolerance, each partition can have one or more replicas. +The _state model_ allows one to declare the system behavior by first enumerating the various STATES, and the TRANSITIONS between them. +A simple model is ONLINE-OFFLINE where ONLINE means the task is active and OFFLINE means it\'s not active. +You can also specify how many replicas must be in each state, these are known as _constraints_. +For example, in a search system, one might need more than one node serving the same index to handle the load. + +The allowed states: + +* MASTER +* SLAVE +* OFFLINE + +The allowed transitions: + +* OFFLINE to SLAVE +* SLAVE to OFFLINE +* SLAVE to MASTER +* MASTER to SLAVE + +The constraints: + +* no more than 1 MASTER per partition +* the rest of the replicas should be slaves + +The following snippet shows how to declare the _state model_ and _constraints_ for the MASTER-SLAVE model. + +``` +StateModelDefinition.Builder builder = new StateModelDefinition.Builder(STATE_MODEL_NAME); + +// Add states and their rank to indicate priority. A lower rank corresponds to a higher priority +builder.addState(MASTER, 1); +builder.addState(SLAVE, 2); +builder.addState(OFFLINE); + +// Set the initial state when the node starts +builder.initialState(OFFLINE); + +// Add transitions between the states. +builder.addTransition(OFFLINE, SLAVE); +builder.addTransition(SLAVE, OFFLINE); +builder.addTransition(SLAVE, MASTER); +builder.addTransition(MASTER, SLAVE); + +// set constraints on states. + +// static constraint: upper bound of 1 MASTER +builder.upperBound(MASTER, 1); + +// dynamic constraint: R means it should be derived based on the replication factor for the cluster +// this allows a different replication factor for each resource without +// having to define a new state model +// +builder.dynamicUpperBound(SLAVE, "R"); +StateModelDefinition statemodelDefinition = builder.build(); +``` + +Then, add the state model definition: + +``` +clusterAccessor.addStateModelDefinitionToCluster(stateModelDefinition); +``` + +OR + +``` +admin.addStateModelDef(CLUSTER_NAME, STATE_MODEL_NAME, stateModelDefinition); +``` + +#### Assigning partitions to nodes + +The final goal of Helix is to ensure that the constraints on the state model are satisfied. +Helix does this by assigning a STATE to a partition (such as MASTER, SLAVE), and placing it on a particular node. + +There are 3 assignment modes Helix can operate on + +* FULL_AUTO: Helix decides the placement and state of a partition. +* SEMI_AUTO: Application decides the placement but Helix decides the state of a partition. +* CUSTOMIZED: Application controls the placement and state of a partition. + +For more info on the assignment modes, see [Rebalancing Algorithms](./tutorial_rebalance.html) section of the tutorial. + +Here is an example of adding the resource in SEMI_AUTO mode (i.e. locations of partitions are specified a priori): + +``` +int NUM_PARTITIONS = 6; +int NUM_REPLICAS = 2; +ResourceId resourceId = resourceId.from("MyDB"); + +SemiAutoRebalancerContext context = new SemiAutoRebalancerContext.Builder(resourceId) + .replicaCount(NUM_REPLICAS).addPartitions(NUM_PARTITIONS) + .stateModelDefId(stateModelDefinition.getStateModelDefId()) + .addPreferenceList(partition1Id, preferenceList) // preferred locations of each partition + // add other preference lists per partition + .build(); + +// or add all preference lists at once if desired (map of PartitionId to List of ParticipantId) +context.setPreferenceLists(preferenceLists); + +// or generate a default set of preference lists given the set of all participants +context.generateDefaultConfiguration(stateModelDefinition, participantIdSet); +``` + +OR + +``` +String RESOURCE_NAME = "MyDB"; +int NUM_PARTITIONS = 6; +String MODE = "SEMI_AUTO"; +int NUM_REPLICAS = 2; + +admin.addResource(CLUSTER_NAME, RESOURCE_NAME, NUM_PARTITIONS, STATE_MODEL_NAME, MODE); + +// specify the preference lists yourself +IdealState idealState = admin.getResourceIdealState(CLUSTER_NAME, RESOURCE_NAME); +idealState.setPreferenceList(partitionId, preferenceList); // preferred locations of each partition +// add other preference lists per partition + +// or add all preference lists at once if desired +idealState.getRecord().setListFields(preferenceLists); +admin.setResourceIdealState(CLUSTER_NAME, RESOURCE_NAME, idealState); + +// or generate a default set of preference lists +admin.rebalance(CLUSTER_NAME, RESOURCE_NAME, NUM_REPLICAS); +``` + diff --git a/site-releases/trunk/src/site/markdown/UseCases.md b/site-releases/trunk/src/site/markdown/UseCases.md new file mode 100644 index 0000000000..001b01247a --- /dev/null +++ b/site-releases/trunk/src/site/markdown/UseCases.md @@ -0,0 +1,113 @@ + + + + Use Cases + + + +# Use cases at LinkedIn + +At LinkedIn Helix framework is used to manage 3 distributed data systems which are quite different from each other. + +* Espresso +* Databus +* Search As A Service + +## Espresso + +Espresso is a distributed, timeline consistent, scal- able, document store that supports local secondary indexing and local transactions. +Espresso databases are horizontally partitioned into a number of partitions, with each partition having a certain number of replicas +distributed across the storage nodes. +Espresso designates one replica of each partition as master and the rest as slaves; only one master may exist for each partition at any time. +Espresso enforces timeline consistency where only the master of a partition can accept writes to its records, and all slaves receive and +apply the same writes through a replication stream. +For load balancing, both master and slave partitions are assigned evenly across all storage nodes. +For fault tolerance, it adds the constraint that no two replicas of the same partition may be located on the same node. + +### State model +Espresso follows a Master-Slave state model. A replica can be in Offline,Slave or Master state. +The state machine table describes the next state given the Current State, Final State + +``` + OFFLINE | SLAVE | MASTER + _____________________________ + | | | | +OFFLINE | N/A | SLAVE | SLAVE | + |__________|________|_________| + | | | | +SLAVE | OFFLINE | N/A | MASTER | + |__________|________|_________| + | | | | +MASTER | SLAVE | SLAVE | N/A | + |__________|________|_________| + +``` + +### Constraints +* Max number of replicas in Master state:1 +* Execution mode AUTO. i.e on node failure no new replicas will be created. Only the State of remaining replicas will be changed. +* Number of mastered partitions on each node must be approximately same. +* The above constraint must be satisfied when a node fails or a new node is added. +* When new nodes are added the number of partitions moved must be minimized. +* When new nodes are added the max number of OFFLINE-SLAVE transitions that can happen concurrently on new node is X. + +## Databus + +Databus is a change data capture (CDC) system that provides a common pipeline for transporting events +from LinkedIn primary databases to caches within various applications. +Databus deploys a cluster of relays that pull the change log from multiple databases and +let consumers subscribe to the change log stream. Each Databus relay connects to one or more database servers and +hosts a certain subset of databases (and partitions) from those database servers. + +For a large partitioned database (e.g. Espresso), the change log is consumed by a bank of consumers. +Each databus partition is assigned to a consumer such that partitions are evenly distributed across consumers and each partition is +assigned to exactly one consumer at a time. The set of consumers may grow over time, and consumers may leave the group due to planned or unplanned +outages. In these cases, partitions must be reassigned, while maintaining balance and the single consumer-per-partition invariant. + +### State model +Databus consumers follow a simple Offline-Online state model. +The state machine table describes the next state given the Current State, Final State + +

+          OFFLINE  | ONLINE |   
+         ___________________|
+        |          |        |
+OFFLINE |   N/A    | ONLINE |
+        |__________|________|
+        |          |        |
+ONLINE  |  OFFLINE |   N/A  |
+        |__________|________|
+
+
+
+ + +## Search As A Service + +LinkedIn�s Search-as-a-service lets internal customers define custom indexes on a chosen dataset +and then makes those indexes searchable via a service API. The index service runs on a cluster of machines. +The index is broken into partitions and each partition has a configured number of replicas. +Each cluster server runs an instance of the Sensei system (an online index store) and hosts index partitions. +Each new indexing service gets assigned to a set of servers, and the partition replicas must be evenly distributed across those servers. + +### State model +![Helix Design](images/bootstrap_statemodel.gif) + + diff --git a/site-releases/trunk/src/site/markdown/index.md b/site-releases/trunk/src/site/markdown/index.md new file mode 100644 index 0000000000..2eae3742e2 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/index.md @@ -0,0 +1,56 @@ + + + + Home + + +Navigating the Documentation +---------------------------- + +### Conceptual Understanding + +[Concepts / Terminology](./Concepts.html) + +[Architecture](./Architecture.html) + +### Hands-on Helix + +[Getting Helix](./Building.html) + +[Quickstart](./Quickstart.html) + +[Tutorial](./Tutorial.html) + +[Javadocs](http://helix.incubator.apache.org/apidocs) + +### Recipes + +[Distributed lock manager](./recipes/lock_manager.html) + +[Rabbit MQ consumer group](./recipes/rabbitmq_consumer_group.html) + +[Rsync replicated file store](./recipes/rsync_replicated_file_store.html) + +[Service discovery](./recipes/service_discovery.html) + +[Distributed Task DAG Execution](./recipes/task_dag_execution.html) + +[User-Defined Rebalancer Example](./recipes/user_def_rebalancer.html) + diff --git a/site-releases/trunk/src/site/markdown/recipes/lock_manager.md b/site-releases/trunk/src/site/markdown/recipes/lock_manager.md new file mode 100644 index 0000000000..252ace7eab --- /dev/null +++ b/site-releases/trunk/src/site/markdown/recipes/lock_manager.md @@ -0,0 +1,253 @@ + +Distributed lock manager +------------------------ +Distributed locks are used to synchronize accesses shared resources. Most applications use Zookeeper to model the distributed locks. + +The simplest way to model a lock using zookeeper is (See Zookeeper leader recipe for an exact and more advanced solution) + +* Each process tries to create an emphemeral node. +* If can successfully create it then, it acquires the lock +* Else it will watch on the znode and try to acquire the lock again if the current lock holder disappears + +This is good enough if there is only one lock. But in practice, an application will need many such locks. Distributing and managing the locks among difference process becomes challenging. Extending such a solution to many locks will result in + +* Uneven distribution of locks among nodes, the node that starts first will acquire all the lock. Nodes that start later will be idle. +* When a node fails, how the locks will be distributed among remaining nodes is not predicable. +* When new nodes are added the current nodes dont relinquish the locks so that new nodes can acquire some locks + +In other words we want a system to satisfy the following requirements. + +* Distribute locks evenly among all nodes to get better hardware utilization +* If a node fails, the locks that were acquired by that node should be evenly distributed among other nodes +* If nodes are added, locks must be evenly re-distributed among nodes. + +Helix provides a simple and elegant solution to this problem. Simply specify the number of locks and Helix will ensure that above constraints are satisfied. + +To quickly see this working run the lock-manager-demo script where 12 locks are evenly distributed among three nodes, and when a node fails, the locks get re-distributed among remaining two nodes. Note that Helix does not re-shuffle the locks completely, instead it simply distributes the locks relinquished by dead node among 2 remaining nodes evenly. + +---------------------------------------------------------------------------------------- + +#### Short version + This version starts multiple threads with in same process to simulate a multi node deployment. Try the long version to get a better idea of how it works. + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/distributed-lock-manager/target/distributed-lock-manager-pkg/bin +chmod +x * +./lock-manager-demo +``` + +##### Output + +``` +./lock-manager-demo +STARTING localhost_12000 +STARTING localhost_12002 +STARTING localhost_12001 +STARTED localhost_12000 +STARTED localhost_12002 +STARTED localhost_12001 +localhost_12001 acquired lock:lock-group_3 +localhost_12000 acquired lock:lock-group_8 +localhost_12001 acquired lock:lock-group_2 +localhost_12001 acquired lock:lock-group_4 +localhost_12002 acquired lock:lock-group_1 +localhost_12002 acquired lock:lock-group_10 +localhost_12000 acquired lock:lock-group_7 +localhost_12001 acquired lock:lock-group_5 +localhost_12002 acquired lock:lock-group_11 +localhost_12000 acquired lock:lock-group_6 +localhost_12002 acquired lock:lock-group_0 +localhost_12000 acquired lock:lock-group_9 +lockName acquired By +====================================== +lock-group_0 localhost_12002 +lock-group_1 localhost_12002 +lock-group_10 localhost_12002 +lock-group_11 localhost_12002 +lock-group_2 localhost_12001 +lock-group_3 localhost_12001 +lock-group_4 localhost_12001 +lock-group_5 localhost_12001 +lock-group_6 localhost_12000 +lock-group_7 localhost_12000 +lock-group_8 localhost_12000 +lock-group_9 localhost_12000 +Stopping localhost_12000 +localhost_12000 Interrupted +localhost_12001 acquired lock:lock-group_9 +localhost_12001 acquired lock:lock-group_8 +localhost_12002 acquired lock:lock-group_6 +localhost_12002 acquired lock:lock-group_7 +lockName acquired By +====================================== +lock-group_0 localhost_12002 +lock-group_1 localhost_12002 +lock-group_10 localhost_12002 +lock-group_11 localhost_12002 +lock-group_2 localhost_12001 +lock-group_3 localhost_12001 +lock-group_4 localhost_12001 +lock-group_5 localhost_12001 +lock-group_6 localhost_12002 +lock-group_7 localhost_12002 +lock-group_8 localhost_12001 +lock-group_9 localhost_12001 + +``` + +---------------------------------------------------------------------------------------- + +#### Long version +This provides more details on how to setup the cluster and where to plugin application code. + +##### start zookeeper + +``` +./start-standalone-zookeeper 2199 +``` + +##### Create a cluster + +``` +./helix-admin --zkSvr localhost:2199 --addCluster lock-manager-demo +``` + +##### Create a lock group + +Create a lock group and specify the number of locks in the lock group. + +``` +./helix-admin --zkSvr localhost:2199 --addResource lock-manager-demo lock-group 6 OnlineOffline FULL_AUTO +``` + +##### Start the nodes + +Create a Lock class that handles the callbacks. + +``` + +public class Lock extends StateModel +{ + private String lockName; + + public Lock(String lockName) + { + this.lockName = lockName; + } + + public void lock(Message m, NotificationContext context) + { + System.out.println(" acquired lock:"+ lockName ); + } + + public void release(Message m, NotificationContext context) + { + System.out.println(" releasing lock:"+ lockName ); + } + +} + +``` + +LockFactory that creates the lock + +``` +public class LockFactory extends StateModelFactory{ + + /* Instantiates the lock handler, one per lockName*/ + public Lock create(String lockName) + { + return new Lock(lockName); + } +} +``` + +At node start up, simply join the cluster and helix will invoke the appropriate callbacks on Lock instance. One can start any number of nodes and Helix detects that a new node has joined the cluster and re-distributes the locks automatically. + +``` +public class LockProcess{ + + public static void main(String args){ + String zkAddress= "localhost:2199"; + String clusterName = "lock-manager-demo"; + //Give a unique id to each process, most commonly used format hostname_port + String instanceName ="localhost_12000"; + ZKHelixAdmin helixAdmin = new ZKHelixAdmin(zkAddress); + //configure the instance and provide some metadata + InstanceConfig config = new InstanceConfig(instanceName); + config.setHostName("localhost"); + config.setPort("12000"); + admin.addInstance(clusterName, config); + //join the cluster + HelixManager manager; + manager = HelixManagerFactory.getHelixManager(clusterName, + instanceName, + InstanceType.PARTICIPANT, + zkAddress); + manager.getStateMachineEngine().registerStateModelFactory("OnlineOffline", modelFactory); + manager.connect(); + Thread.currentThread.join(); + } + +} +``` + +##### Start the controller + +Controller can be started either as a separate process or can be embedded within each node process + +###### Separate process +This is recommended when number of nodes in the cluster >100. For fault tolerance, you can run multiple controllers on different boxes. + +``` +./run-helix-controller --zkSvr localhost:2199 --cluster lock-manager-demo 2>&1 > /tmp/controller.log & +``` + +###### Embedded within the node process +This is recommended when the number of nodes in the cluster is less than 100. To start a controller from each process, simply add the following lines to MyClass + +``` +public class LockProcess{ + + public static void main(String args){ + String zkAddress= "localhost:2199"; + String clusterName = "lock-manager-demo"; + . + . + manager.connect(); + HelixManager controller; + controller = HelixControllerMain.startHelixController(zkAddress, + clusterName, + "controller", + HelixControllerMain.STANDALONE); + Thread.currentThread.join(); + } +} +``` + +---------------------------------------------------------------------------------------- + + + + + diff --git a/site-releases/trunk/src/site/markdown/recipes/rabbitmq_consumer_group.md b/site-releases/trunk/src/site/markdown/recipes/rabbitmq_consumer_group.md new file mode 100644 index 0000000000..9edc2cb242 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/recipes/rabbitmq_consumer_group.md @@ -0,0 +1,227 @@ + + + +RabbitMQ Consumer Group +======================= + +[RabbitMQ](http://www.rabbitmq.com/) is a well known Open source software the provides robust messaging for applications. + +One of the commonly implemented recipes using this software is a work queue. http://www.rabbitmq.com/tutorials/tutorial-four-java.html describes the use case where + +* A producer sends a message with a routing key. +* The message is routed to the queue whose binding key exactly matches the routing key of the message. +* There are multiple consumers and each consumer is interested in processing only a subset of the messages by binding to the interested keys + +The example provided [here](http://www.rabbitmq.com/tutorials/tutorial-four-java.html) describes how multiple consumers can be started to process all the messages. + +While this works, in production systems one needs the following + +* Ability to handle failures: when a consumers fails another consumer must be started or the other consumers must start processing these messages that should have been processed by the failed consumer. +* When the existing consumers cannot keep up with the task generation rate, new consumers will be added. The tasks must be redistributed among all the consumers. + +In this recipe, we demonstrate handling of consumer failures and new consumer additions using Helix. + +Mapping this usecase to Helix is pretty easy as the binding key/routing key is equivalent to a partition. + +Let's take an example. Lets say the queue has 6 partitions, and we have 2 consumers to process all the queues. +What we want is all 6 queues to be evenly divided among 2 consumers. +Eventually when the system scales, we add more consumers to keep up. This will make each consumer process tasks from 2 queues. +Now let's say that a consumer failed which reduces the number of active consumers to 2. This means each consumer must process 3 queues. + +We showcase how such a dynamic App can be developed using Helix. Even though we use rabbitmq as the pub/sub system one can extend this solution to other pub/sub systems. + +Try it +====== + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/rabbitmq-consumer-group/bin +chmod +x * +export HELIX_PKG_ROOT=`pwd`/helix-core/target/helix-core-pkg +export HELIX_RABBITMQ_ROOT=`pwd`/recipes/rabbitmq-consumer-group/ +chmod +x $HELIX_PKG_ROOT/bin/* +chmod +x $HELIX_RABBITMQ_ROOT/bin/* +``` + + +Install Rabbit MQ +---------------- + +Setting up RabbitMQ on a local box is straightforward. You can find the instructions here +http://www.rabbitmq.com/download.html + +Start ZK +-------- +Start zookeeper at port 2199 + +``` +$HELIX_PKG_ROOT/bin/start-standalone-zookeeper 2199 +``` + +Setup the consumer group cluster +-------------------------------- +This will setup the cluster by creating a "rabbitmq-consumer-group" cluster and adds a "topic" with "6" queues. + +``` +$HELIX_RABBITMQ_ROOT/bin/setup-cluster.sh localhost:2199 +``` + +Add consumers +------------- +Start 2 consumers in 2 different terminals. Each consumer is given a unique id. + +``` +//start-consumer.sh zookeeperAddress (e.g. localhost:2181) consumerId , rabbitmqServer (e.g. localhost) +$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 0 localhost +$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 1 localhost + +``` + +Start HelixController +-------------------- +Now start a Helix controller that starts managing the "rabbitmq-consumer-group" cluster. + +``` +$HELIX_RABBITMQ_ROOT/bin/start-cluster-manager.sh localhost:2199 +``` + +Send messages to the Topic +-------------------------- + +Start sending messages to the topic. This script randomly selects a routing key (1-6) and sends the message to topic. +Based on the key, messages gets routed to the appropriate queue. + +``` +$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 20 +``` + +After running this, you should see all 20 messages being processed by 2 consumers. + +Add another consumer +-------------------- +Once a new consumer is started, helix detects it. In order to balance the load between 3 consumers, it deallocates 1 partition from the existing consumers and allocates it to the new consumer. We see that +each consumer is now processing only 2 queues. +Helix makes sure that old nodes are asked to stop consuming before the new consumer is asked to start consuming for a given partition. But the transitions for each partition can happen in parallel. + +``` +$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 2 localhost +``` + +Send messages again to the topic. + +``` +$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 100 +``` + +You should see that messages are now received by all 3 consumers. + +Stop a consumer +--------------- +In any terminal press CTRL^C and notice that Helix detects the consumer failure and distributes the 2 partitions that were processed by failed consumer to the remaining 2 active consumers. + + +How does it work +================ + +Find the entire code [here](https://git-wip-us.apache.org/repos/asf?p=incubator-helix.git;a=tree;f=recipes/rabbitmq-consumer-group/src/main/java/org/apache/helix/recipes/rabbitmq). + +Cluster setup +------------- +This step creates znode on zookeeper for the cluster and adds the state model. We use online offline state model since there is no need for other states. The consumer is either processing a queue or it is not. + +It creates a resource called "rabbitmq-consumer-group" with 6 partitions. The execution mode is set to FULL_AUTO. This means that the Helix controls the assignment of partition to consumers and automatically distributes the partitions evenly among the active consumers. When a consumer is added or removed, it ensures that a minimum number of partitions are shuffled. + +``` + zkclient = new ZkClient(zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT, + ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer()); + ZKHelixAdmin admin = new ZKHelixAdmin(zkclient); + + // add cluster + admin.addCluster(clusterName, true); + + // add state model definition + StateModelConfigGenerator generator = new StateModelConfigGenerator(); + admin.addStateModelDef(clusterName, "OnlineOffline", + new StateModelDefinition(generator.generateConfigForOnlineOffline())); + + // add resource "topic" which has 6 partitions + String resourceName = "rabbitmq-consumer-group"; + admin.addResource(clusterName, resourceName, 6, "OnlineOffline", "FULL_AUTO"); +``` + +Starting the consumers +---------------------- +The only thing consumers need to know is the zkaddress, cluster name and consumer id. It does not need to know anything else. + +``` + _manager = + HelixManagerFactory.getZKHelixManager(_clusterName, + _consumerId, + InstanceType.PARTICIPANT, + _zkAddr); + + StateMachineEngine stateMach = _manager.getStateMachineEngine(); + ConsumerStateModelFactory modelFactory = + new ConsumerStateModelFactory(_consumerId, _mqServer); + stateMach.registerStateModelFactory("OnlineOffline", modelFactory); + + _manager.connect(); + +``` + +Once the consumer has registered the statemodel and the controller is started, the consumer starts getting callbacks (onBecomeOnlineFromOffline) for the partition it needs to host. All it needs to do as part of the callback is to start consuming messages from the appropriate queue. Similarly, when the controller deallocates a partitions from a consumer, it fires onBecomeOfflineFromOnline for the same partition. +As a part of this transition, the consumer will stop consuming from a that queue. + +``` + @Transition(to = "ONLINE", from = "OFFLINE") + public void onBecomeOnlineFromOffline(Message message, NotificationContext context) + { + LOG.debug(_consumerId + " becomes ONLINE from OFFLINE for " + _partition); + + if (_thread == null) + { + LOG.debug("Starting ConsumerThread for " + _partition + "..."); + _thread = new ConsumerThread(_partition, _mqServer, _consumerId); + _thread.start(); + LOG.debug("Starting ConsumerThread for " + _partition + " done"); + + } + } + + @Transition(to = "OFFLINE", from = "ONLINE") + public void onBecomeOfflineFromOnline(Message message, NotificationContext context) + throws InterruptedException + { + LOG.debug(_consumerId + " becomes OFFLINE from ONLINE for " + _partition); + + if (_thread != null) + { + LOG.debug("Stopping " + _consumerId + " for " + _partition + "..."); + + _thread.interrupt(); + _thread.join(2000); + _thread = null; + LOG.debug("Stopping " + _consumerId + " for " + _partition + " done"); + + } + } +``` \ No newline at end of file diff --git a/site-releases/trunk/src/site/markdown/recipes/rsync_replicated_file_store.md b/site-releases/trunk/src/site/markdown/recipes/rsync_replicated_file_store.md new file mode 100644 index 0000000000..f8a74a0abc --- /dev/null +++ b/site-releases/trunk/src/site/markdown/recipes/rsync_replicated_file_store.md @@ -0,0 +1,165 @@ + + +Near real time rsync replicated file system +=========================================== + +Quickdemo +--------- + +* This demo starts 3 instances with id's as ```localhost_12001, localhost_12002, localhost_12003``` +* Each instance stores its files under ```/tmp//filestore``` +* ``` localhost_12001 ``` is designated as the master and ``` localhost_12002 and localhost_12003``` are the slaves. +* Files written to master are replicated to the slaves automatically. In this demo, a.txt and b.txt are written to ```/tmp/localhost_12001/filestore``` and it gets replicated to other folders. +* When the master is stopped, ```localhost_12002``` is promoted to master. +* The other slave ```localhost_12003``` stops replicating from ```localhost_12001``` and starts replicating from new master ```localhost_12002``` +* Files written to new master ```localhost_12002``` are replicated to ```localhost_12003``` +* In the end state of this quick demo, ```localhost_12002``` is the master and ```localhost_12003``` is the slave. Manually create files under ```/tmp/localhost_12002/filestore``` and see that appears in ```/tmp/localhost_12003/filestore``` +* Ignore the interrupted exceptions on the console :-). + + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd recipes/rsync-replicated-file-system/ +mvn clean install package -DskipTests +cd target/rsync-replicated-file-system-pkg/bin +chmod +x * +./quickdemo + +``` + +Overview +-------- + +There are many applications that require storage for storing large number of relatively small data files. Examples include media stores to store small videos, images, mail attachments etc. Each of these objects is typically kilobytes, often no larger than a few megabytes. An additional distinguishing feature of these usecases is also that files are typically only added or deleted, rarely updated. When there are updates, they are rare and do not have any concurrency requirements. + +These are much simpler requirements than what general purpose distributed file system have to satisfy including concurrent access to files, random access for reads and updates, posix compliance etc. To satisfy those requirements, general DFSs are also pretty complex that are expensive to build and maintain. + +A different implementation of a distributed file system includes HDFS which is inspired by Google's GFS. This is one of the most widely used distributed file system that forms the main data storage platform for Hadoop. HDFS is primary aimed at processing very large data sets and distributes files across a cluster of commodity servers by splitting up files in fixed size chunks. HDFS is not particularly well suited for storing a very large number of relatively tiny files. + +### File Store + +It's possible to build a vastly simpler system for the class of applications that have simpler requirements as we have pointed out. + +* Large number of files but each file is relatively small. +* Access is limited to create, delete and get entire files. +* No updates to files that are already created (or it's feasible to delete the old file and create a new one). + + +We call this system a Partitioned File Store (PFS) to distinguish it from other distributed file systems. This system needs to provide the following features: + +* CRD access to large number of small files +* Scalability: Files should be distributed across a large number of commodity servers based on the storage requirement. +* Fault-tolerance: Each file should be replicated on multiple servers so that individual server failures do not reduce availability. +* Elasticity: It should be possible to add capacity to the cluster easily. + + +Apache Helix is a generic cluster management framework that makes it very easy to provide the scalability, fault-tolerance and elasticity features. +Rsync can be easily used as a replication channel between servers so that each file gets replicated on multiple servers. + +Design +------ + +High level + +* Partition the file system based on the file name. +* At any time a single writer can write, we call this a master. +* For redundancy, we need to have additional replicas called slave. Slaves can optionally serve reads. +* Slave replicates data from the master. +* When a master fails, slave gets promoted to master. + +### Transaction log + +Every write on the master will result in creation/deletion of one or more files. In order to maintain timeline consistency slaves need to apply the changes in the same order. +To facilitate this, the master logs each transaction in a file and each transaction is associated with an 64 bit id in which the 32 LSB represents a sequence number and MSB represents the generation number. +Sequence gets incremented on every transaction and and generation is increment when a new master is elected. + +### Replication + +Replication is required to slave to keep up with the changes on the master. Every time the slave applies a change it checkpoints the last applied transaction id. +During restarts, this allows the slave to pull changes from the last checkpointed id. Similar to master, the slave logs each transaction to the transaction logs but instead of generating new transaction id, it uses the same id generated by the master. + + +### Fail over + +When a master fails, a new slave will be promoted to master. If the prev master node is reachable, then the new master will flush all the +changes from previous master before taking up mastership. The new master will record the end transaction id of the current generation and then starts new generation +with sequence starting from 1. After this the master will begin accepting writes. + + +![Partitioned File Store](../images/PFS-Generic.png) + + + +Rsync based solution +------------------- + +![Rsync based File Store](../images/RSYNC_BASED_PFS.png) + + +This application demonstrate a file store that uses rsync as the replication mechanism. One can envision a similar system where instead of using rsync, +can implement a custom solution to notify the slave of the changes and also provide an api to pull the change files. +#### Concept +* file_store_dir: Root directory for the actual data files +* change_log_dir: The transaction logs are generated under this folder. +* check_point_dir: The slave stores the check points ( last processed transaction) here. + +#### Master +* File server: This component support file uploads and downloads and writes the files to ```file_store_dir```. This is not included in this application. Idea is that most applications have different ways of implementing this component and has some business logic associated with it. It is not hard to come up with such a component if needed. +* File store watcher: This component watches the ```file_store_dir``` directory on the local file system for any changes and notifies the registered listeners of the changes. +* Change Log Generator: This registers as a listener of File System Watcher and on each notification logs the changes into a file under ```change_log_dir```. + +####Slave +* File server: This component on the slave will only support reads. +* Cluster state observer: Slave observes the cluster state and is able to know who is the current master. +* Replicator: This has two subcomponents + - Periodic rsync of change log: This is a background process that periodically rsyncs the ```change_log_dir``` of the master to its local directory + - Change Log Watcher: This watches the ```change_log_dir``` for changes and notifies the registered listeners of the change + - On demand rsync invoker: This is registered as a listener to change log watcher and on every change invokes rsync to sync only the changed file. + + +#### Coordination + +The coordination between nodes is done by Helix. Helix does the partition management and assigns the partition to multiple nodes based on the replication factor. It elects one the nodes as master and designates others as slaves. +It provides notifications to each node in the form of state transitions ( Offline to Slave, Slave to Master). It also provides notification when there is change is cluster state. +This allows the slave to stop replicating from current master and start replicating from new master. + +In this application, we have only one partition but its very easy to extend it to support multiple partitions. By partitioning the file store, one can add new nodes and Helix will automatically +re-distribute partitions among the nodes. To summarize, Helix provides partition management, fault tolerance and facilitates automated cluster expansion. + + + + + + + + + + + + + + + + + + + + + diff --git a/site-releases/trunk/src/site/markdown/recipes/service_discovery.md b/site-releases/trunk/src/site/markdown/recipes/service_discovery.md new file mode 100644 index 0000000000..8e06ead472 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/recipes/service_discovery.md @@ -0,0 +1,191 @@ + +Service Discovery +----------------- + +One of the common usage of zookeeper is enable service discovery. +The basic idea is that when a server starts up it advertises its configuration/metadata such as host name port etc on zookeeper. +This allows clients to dynamically discover the servers that are currently active. One can think of this like a service registry to which a server registers when it starts and +is automatically deregistered when it shutdowns or crashes. In many cases it serves as an alternative to vips. + +The core idea behind this is to use zookeeper ephemeral nodes. The ephemeral nodes are created when the server registers and all its metadata is put into a znode. +When the server shutdowns, zookeeper automatically removes this znode. + +There are two ways the clients can dynamically discover the active servers + +#### ZOOKEEPER WATCH + +Clients can set a child watch under specific path on zookeeper. +When a new service is registered/deregistered, zookeeper notifies the client via watchevent and the client can read the list of services. Even though this looks trivial, +there are lot of things one needs to keep in mind like ensuring that you first set the watch back on zookeeper before reading data from zookeeper. + + +#### POLL + +Another approach is for the client to periodically read the zookeeper path and get the list of services. + + +Both approaches have pros and cons, for example setting a watch might trigger herd effect if there are large number of clients. This is worst especially when servers are starting up. +But good thing about setting watch is that clients are immediately notified of a change which is not true in case of polling. +In some cases, having both WATCH and POLL makes sense, WATCH allows one to get notifications as soon as possible while POLL provides a safety net if a watch event is missed because of code bug or zookeeper fails to notify. + +##### Other important scenarios to take care of +* What happens when zookeeper session expires. All the watches/ephemeral nodes previously added/created by this server are lost. +One needs to add the watches again , recreate the ephemeral nodes etc. +* Due to network issues or java GC pauses session expiry might happen again and again also known as flapping. Its important for the server to detect this and deregister itself. + +##### Other operational things to consider +* What if the node is behaving badly, one might kill the server but will lose the ability to debug. +It would be nice to have the ability to mark a server as disabled and clients know that a node is disabled and will not contact that node. + +#### Configuration ownership + +This is an important aspect that is often ignored in the initial stages of your development. In common, service discovery pattern means that servers start up with some configuration and then simply puts its configuration/metadata in zookeeper. While this works well in the beginning, +configuration management becomes very difficult since the servers themselves are statically configured. Any change in server configuration implies restarting of the server. Ideally, it will be nice to have the ability to change configuration dynamically without having to restart a server. + +Ideally you want a hybrid solution, a node starts with minimal configuration and gets the rest of configuration from zookeeper. + +h3. How to use Helix to achieve this + +Even though Helix has higher level abstraction in terms of statemachine, constraints and objectives, +service discovery is one of things that existed since we started. +The controller uses the exact mechanism we described above to discover when new servers join the cluster. +We create these znodes under /CLUSTERNAME/LIVEINSTANCES. +Since at any time there is only one controller, we use ZK watch to track the liveness of a server. + +This recipe, simply demonstrate how one can re-use that part for implementing service discovery. This demonstrates multiple MODE's of service discovery + +* POLL: The client reads from zookeeper at regular intervals 30 seconds. Use this if you have 100's of clients +* WATCH: The client sets up watcher and gets notified of the changes. Use this if you have 10's of clients. +* NONE: This does neither of the above, but reads directly from zookeeper when ever needed. + +Helix provides these additional features compared to other implementations available else where + +* It has the concept of disabling a node which means that a badly behaving node, can be disabled using helix admin api. +* It automatically detects if a node connects/disconnects from zookeeper repeatedly and disables the node. +* Configuration management + * Allows one to set configuration via admin api at various granulaties like cluster, instance, resource, partition + * Configuration can be dynamically changed. + * Notifies the server when configuration changes. + + +##### checkout and build + +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/service-discovery/target/service-discovery-pkg/bin +chmod +x * +``` + +##### start zookeeper + +``` +./start-standalone-zookeeper 2199 +``` + +#### Run the demo + +``` +./service-discovery-demo.sh +``` + +#### Output + +``` +START:Service discovery demo mode:WATCH + Registering service + host.x.y.z_12000 + host.x.y.z_12001 + host.x.y.z_12002 + host.x.y.z_12003 + host.x.y.z_12004 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Deregistering service: + host.x.y.z_12002 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Registering service:host.x.y.z_12002 +END:Service discovery demo mode:WATCH +============================================= +START:Service discovery demo mode:POLL + Registering service + host.x.y.z_12000 + host.x.y.z_12001 + host.x.y.z_12002 + host.x.y.z_12003 + host.x.y.z_12004 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Deregistering service: + host.x.y.z_12002 + Sleeping for poll interval:30000 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Registering service:host.x.y.z_12002 +END:Service discovery demo mode:POLL +============================================= +START:Service discovery demo mode:NONE + Registering service + host.x.y.z_12000 + host.x.y.z_12001 + host.x.y.z_12002 + host.x.y.z_12003 + host.x.y.z_12004 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12000 + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Deregistering service: + host.x.y.z_12000 + SERVICES AVAILABLE + SERVICENAME HOST PORT + myServiceName host.x.y.z 12001 + myServiceName host.x.y.z 12002 + myServiceName host.x.y.z 12003 + myServiceName host.x.y.z 12004 + Registering service:host.x.y.z_12000 +END:Service discovery demo mode:NONE +============================================= + +``` + diff --git a/site-releases/trunk/src/site/markdown/recipes/task_dag_execution.md b/site-releases/trunk/src/site/markdown/recipes/task_dag_execution.md new file mode 100644 index 0000000000..f0474e445e --- /dev/null +++ b/site-releases/trunk/src/site/markdown/recipes/task_dag_execution.md @@ -0,0 +1,204 @@ + + +# Distributed task execution + + +This recipe is intended to demonstrate how task dependencies can be modeled using primitives provided by Helix. A given task can be run with desired parallelism and will start only when up-stream dependencies are met. The demo executes the task DAG described below using 10 workers. Although the demo starts the workers as threads, there is no requirement that all the workers need to run in the same process. In reality, these workers run on many different boxes on a cluster. When worker fails, Helix takes care of +re-assigning a failed task partition to a new worker. + +Redis is used as a result store. Any other suitable implementation for TaskResultStore can be plugged in. + +### Workflow + + +#### Input + +10000 impression events and around 100 click events are pre-populated in task result store (redis). + +* **ImpEvent**: format: id,isFraudulent,country,gender + +* **ClickEvent**: format: id,isFraudulent,impEventId + +#### Stages + ++ **FilterImps**: Filters impression where isFraudulent=true. + ++ **FilterClicks**: Filters clicks where isFraudulent=true + ++ **impCountsByGender**: Generates impression counts grouped by gender. It does this by incrementing the count for 'impression_gender_counts:' in the task result store (redis hash). Depends on: **FilterImps** + ++ **impCountsByCountry**: Generates impression counts grouped by country. It does this by incrementing the count for 'impression_country_counts:' in the task result store (redis hash). Depends on: **FilterClicks** + ++ **impClickJoin**: Joins clicks with corresponding impression event using impEventId as the join key. Join is needed to pull dimensions not present in click event. Depends on: **FilterImps, FilterClicks** + ++ **clickCountsByGender**: Generates click counts grouped by gender. It does this by incrementing the count for click_gender_counts: in the task result store (redis hash). Depends on: **impClickJoin** + ++ **clickCountsByGender**: Generates click counts grouped by country. It does this by incrementing the count for click_country_counts: in the task result store (redis hash). Depends on: **impClickJoin** + ++ **report**: Reads from all aggregates generated by previous stages and prints them. Depends on: **impCountsByGender, impCountsByCountry, clickCountsByGender,clickCountsByGender** + + +### Creating DAG + +Each stage is represented as a Node along with the upstream dependency and desired parallelism. Each stage is modelled as a resource in Helix using OnlineOffline state model. As part of Offline to Online transition, we watch the external view of upstream resources and wait for them to transition to online state. See Task.java for additional info. + +``` + + Dag dag = new Dag(); + dag.addNode(new Node("filterImps", 10, "")); + dag.addNode(new Node("filterClicks", 5, "")); + dag.addNode(new Node("impClickJoin", 10, "filterImps,filterClicks")); + dag.addNode(new Node("impCountsByGender", 10, "filterImps")); + dag.addNode(new Node("impCountsByCountry", 10, "filterImps")); + dag.addNode(new Node("clickCountsByGender", 5, "impClickJoin")); + dag.addNode(new Node("clickCountsByCountry", 5, "impClickJoin")); + dag.addNode(new Node("report",1,"impCountsByGender,impCountsByCountry,clickCountsByGender,clickCountsByCountry")); + + +``` + +### DEMO + +In order to run the demo, use the following steps + +See http://redis.io/topics/quickstart on how to install redis server + +``` + +Start redis e.g: +./redis-server --port 6379 + +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd recipes/task-execution +mvn clean install package -DskipTests +cd target/task-execution-pkg/bin +chmod +x task-execution-demo.sh +./task-execution-demo.sh 2181 localhost 6379 + +``` + +``` + + + + + + +-----------------+ +----------------+ + | filterImps | | filterClicks | + | (parallelism=10)| | (parallelism=5)| + +----------+-----++ +-------+--------+ + | | | | + | | | | + | | | | + | | +------->--------v------------+ + +--------------<-+ +------v-------+ | impClickJoin | + |impCountsByGender |impCountsByCountry | (parallelism=10) | + |(parallelism=10) |(parallelism=10) ++-------------------+-+ + +-----------+--+ +---+----------+ | | + | | | | + | | | | + | | +--------v---------+ +-v-------------------+ + | | |clickCountsByGender |clickCountsByCountry | + | | |(parallelism=5) | |(parallelism=5) | + | | +----+-------------+ +---------------------+ + | | | | + | | | | + | | | | + +----->+-----+>-----------v----+<---------------+ + | report | + |(parallelism=1) | + +-----------------------+ + +``` + +(credit for above ascii art: http://www.asciiflow.com) + +### OUTPUT + +``` +Done populating dummy data +Executing filter task for filterImps_3 for impressions_demo +Executing filter task for filterImps_2 for impressions_demo +Executing filter task for filterImps_0 for impressions_demo +Executing filter task for filterImps_1 for impressions_demo +Executing filter task for filterImps_4 for impressions_demo +Executing filter task for filterClicks_3 for clicks_demo +Executing filter task for filterClicks_1 for clicks_demo +Executing filter task for filterImps_8 for impressions_demo +Executing filter task for filterImps_6 for impressions_demo +Executing filter task for filterClicks_2 for clicks_demo +Executing filter task for filterClicks_0 for clicks_demo +Executing filter task for filterImps_7 for impressions_demo +Executing filter task for filterImps_5 for impressions_demo +Executing filter task for filterClicks_4 for clicks_demo +Executing filter task for filterImps_9 for impressions_demo +Running AggTask for impCountsByGender_3 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_2 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_0 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_9 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_1 for filtered_impressions_demo gender +Running AggTask for impCountsByGender_4 for filtered_impressions_demo gender +Running AggTask for impCountsByCountry_4 for filtered_impressions_demo country +Running AggTask for impCountsByGender_5 for filtered_impressions_demo gender +Executing JoinTask for impClickJoin_2 +Running AggTask for impCountsByCountry_3 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_1 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_0 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_2 for filtered_impressions_demo country +Running AggTask for impCountsByGender_6 for filtered_impressions_demo gender +Executing JoinTask for impClickJoin_1 +Executing JoinTask for impClickJoin_0 +Executing JoinTask for impClickJoin_3 +Running AggTask for impCountsByGender_8 for filtered_impressions_demo gender +Executing JoinTask for impClickJoin_4 +Running AggTask for impCountsByGender_7 for filtered_impressions_demo gender +Running AggTask for impCountsByCountry_5 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_6 for filtered_impressions_demo country +Executing JoinTask for impClickJoin_9 +Running AggTask for impCountsByCountry_8 for filtered_impressions_demo country +Running AggTask for impCountsByCountry_7 for filtered_impressions_demo country +Executing JoinTask for impClickJoin_5 +Executing JoinTask for impClickJoin_6 +Running AggTask for impCountsByCountry_9 for filtered_impressions_demo country +Executing JoinTask for impClickJoin_8 +Executing JoinTask for impClickJoin_7 +Running AggTask for clickCountsByCountry_1 for joined_clicks_demo country +Running AggTask for clickCountsByCountry_0 for joined_clicks_demo country +Running AggTask for clickCountsByCountry_2 for joined_clicks_demo country +Running AggTask for clickCountsByCountry_3 for joined_clicks_demo country +Running AggTask for clickCountsByGender_1 for joined_clicks_demo gender +Running AggTask for clickCountsByCountry_4 for joined_clicks_demo country +Running AggTask for clickCountsByGender_3 for joined_clicks_demo gender +Running AggTask for clickCountsByGender_2 for joined_clicks_demo gender +Running AggTask for clickCountsByGender_4 for joined_clicks_demo gender +Running AggTask for clickCountsByGender_0 for joined_clicks_demo gender +Running reports task +Impression counts per country +{CANADA=1940, US=1958, CHINA=2014, UNKNOWN=2022, UK=1946} +Click counts per country +{US=24, CANADA=14, CHINA=26, UNKNOWN=14, UK=22} +Impression counts per gender +{F=3325, UNKNOWN=3259, M=3296} +Click counts per gender +{F=33, UNKNOWN=32, M=35} + + +``` + diff --git a/site-releases/trunk/src/site/markdown/recipes/user_def_rebalancer.md b/site-releases/trunk/src/site/markdown/recipes/user_def_rebalancer.md new file mode 100644 index 0000000000..68fd9547b7 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/recipes/user_def_rebalancer.md @@ -0,0 +1,285 @@ + +Lock Manager with a User-Defined Rebalancer +------------------------------------------- +Helix is able to compute node preferences and state assignments automatically using general-purpose algorithms. In many cases, a distributed system implementer may choose to instead define a customized approach to computing the location of replicas, the state mapping, or both in response to the addition or removal of participants. The following is an implementation of the [Distributed Lock Manager](./lock_manager.html) that includes a user-defined rebalancer. + +### Define the cluster and locks + +The YAML file below fully defines the cluster and the locks. A lock can be in one of two states: locked and unlocked. Transitions can happen in either direction, and the locked is preferred. A resource in this example is the entire collection of locks to distribute. A partition is mapped to a lock; in this case that means there are 12 locks. These 12 locks will be distributed across 3 nodes. The constraints indicate that only one replica of a lock can be in the locked state at any given time. These locks can each only have a single holder, defined by a replica count of 1. + +Notice the rebalancer section of the definition. The mode is set to USER_DEFINED and the class name refers to the plugged-in rebalancer implementation that inherits from [HelixRebalancer](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/HelixRebalancer.html). This implementation is called whenever the state of the cluster changes, as is the case when participants are added or removed from the system. + +Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/resources/lock-manager-config.yaml + +``` +clusterName: lock-manager-custom-rebalancer # unique name for the cluster +resources: + - name: lock-group # unique resource name + rebalancer: # we will provide our own rebalancer + mode: USER_DEFINED + class: org.apache.helix.userdefinedrebalancer.LockManagerRebalancer + partitions: + count: 12 # number of locks + replicas: 1 # number of simultaneous holders for each lock + stateModel: + name: lock-unlock # unique model name + states: [LOCKED, RELEASED, DROPPED] # the list of possible states + transitions: # the list of possible transitions + - name: Unlock + from: LOCKED + to: RELEASED + - name: Lock + from: RELEASED + to: LOCKED + - name: DropLock + from: LOCKED + to: DROPPED + - name: DropUnlock + from: RELEASED + to: DROPPED + - name: Undrop + from: DROPPED + to: RELEASED + initialState: RELEASED + constraints: + state: + counts: # maximum number of replicas of a partition that can be in each state + - name: LOCKED + count: "1" + - name: RELEASED + count: "-1" + - name: DROPPED + count: "-1" + priorityList: [LOCKED, RELEASED, DROPPED] # states in order of priority + transition: # transitions priority to enforce order that transitions occur + priorityList: [Unlock, Lock, Undrop, DropUnlock, DropLock] +participants: # list of nodes that can acquire locks + - name: localhost_12001 + host: localhost + port: 12001 + - name: localhost_12002 + host: localhost + port: 12002 + - name: localhost_12003 + host: localhost + port: 12003 +``` + +Then, Helix\'s YAMLClusterSetup tool can read in the configuration and bootstrap the cluster immediately: + +``` +YAMLClusterSetup setup = new YAMLClusterSetup(zkAddress); +InputStream input = + Thread.currentThread().getContextClassLoader() + .getResourceAsStream("lock-manager-config.yaml"); +YAMLClusterSetup.YAMLClusterConfig config = setup.setupCluster(input); +``` + +### Write a rebalancer +Below is a full implementation of a rebalancer that extends [HelixRebalancer](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/HelixRebalancer.html). In this case, it simply throws out the previous resource assignment, computes the target node for as many partition replicas as can hold a lock in the LOCKED state (in this example, one), and assigns them the LOCKED state (which is at the head of the state preference list). Clearly a more robust implementation would likely examine the current ideal state to maintain current assignments, and the full state list to handle models more complicated than this one. However, for a simple lock holder implementation, this is sufficient. + +Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/java/org/apache/helix/userdefinedrebalancer/LockManagerRebalancer.java + +``` +@Override +public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, Cluster cluster, + ResourceCurrentState currentState) { + // Get the rebalcancer context (a basic partitioned one) + PartitionedRebalancerContext context = rebalancerConfig.getRebalancerContext( + PartitionedRebalancerContext.class); + + // Initialize an empty mapping of locks to participants + ResourceAssignment assignment = new ResourceAssignment(context.getResourceId()); + + // Get the list of live participants in the cluster + List liveParticipants = new ArrayList( + cluster.getLiveParticipantMap().keySet()); + + // Get the state model (should be a simple lock/unlock model) and the highest-priority state + StateModelDefId stateModelDefId = context.getStateModelDefId(); + StateModelDefinition stateModelDef = cluster.getStateModelMap().get(stateModelDefId); + if (stateModelDef.getStatesPriorityList().size() < 1) { + LOG.error("Invalid state model definition. There should be at least one state."); + return assignment; + } + State lockState = stateModelDef.getTypedStatesPriorityList().get(0); + + // Count the number of participants allowed to lock each lock + String stateCount = stateModelDef.getNumParticipantsPerState(lockState); + int lockHolders = 0; + try { + // a numeric value is a custom-specified number of participants allowed to lock the lock + lockHolders = Integer.parseInt(stateCount); + } catch (NumberFormatException e) { + LOG.error("Invalid state model definition. The lock state does not have a valid count"); + return assignment; + } + + // Fairly assign the lock state to the participants using a simple mod-based sequential + // assignment. For instance, if each lock can be held by 3 participants, lock 0 would be held + // by participants (0, 1, 2), lock 1 would be held by (1, 2, 3), and so on, wrapping around the + // number of participants as necessary. + // This assumes a simple lock-unlock model where the only state of interest is which nodes have + // acquired each lock. + int i = 0; + for (PartitionId partition : context.getPartitionSet()) { + Map replicaMap = new HashMap(); + for (int j = i; j < i + lockHolders; j++) { + int participantIndex = j % liveParticipants.size(); + ParticipantId participant = liveParticipants.get(participantIndex); + // enforce that a participant can only have one instance of a given lock + if (!replicaMap.containsKey(participant)) { + replicaMap.put(participant, lockState); + } + } + assignment.addReplicaMap(partition, replicaMap); + i++; + } + return assignment; +} +``` + +### Start up the participants +Here is a lock class based on the newly defined lock-unlock state model so that the participant can receive callbacks on state transitions. + +Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java + +``` +public class Lock extends StateModel { + private String lockName; + + public Lock(String lockName) { + this.lockName = lockName; + } + + @Transition(from = "RELEASED", to = "LOCKED") + public void lock(Message m, NotificationContext context) { + System.out.println(context.getManager().getInstanceName() + " acquired lock:" + lockName); + } + + @Transition(from = "LOCKED", to = "RELEASED") + public void release(Message m, NotificationContext context) { + System.out.println(context.getManager().getInstanceName() + " releasing lock:" + lockName); + } +} +``` + +Here is the factory to make the Lock class accessible. + +Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/java/org/apache/helix/userdefinedrebalancer/LockFactory.java + +``` +public class LockFactory extends StateModelFactory { + @Override + public Lock createNewStateModel(String lockName) { + return new Lock(lockName); + } +} +``` + +Finally, here is the factory registration and the start of the participant: + +``` +participantManager = + HelixManagerFactory.getZKHelixManager(clusterName, participantName, InstanceType.PARTICIPANT, + zkAddress); +participantManager.getStateMachineEngine().registerStateModelFactory(stateModelName, + new LockFactory()); +participantManager.connect(); +``` + +### Start up the controller + +``` +controllerManager = + HelixControllerMain.startHelixController(zkAddress, config.clusterName, "controller", + HelixControllerMain.STANDALONE); +``` + +### Try it out +#### Building +``` +git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git +cd incubator-helix +mvn clean install package -DskipTests +cd recipes/user-rebalanced-lock-manager/target/user-rebalanced-lock-manager-pkg/bin +chmod +x * +./lock-manager-demo.sh +``` + +#### Output + +``` +./lock-manager-demo +STARTING localhost_12002 +STARTING localhost_12001 +STARTING localhost_12003 +STARTED localhost_12001 +STARTED localhost_12003 +STARTED localhost_12002 +localhost_12003 acquired lock:lock-group_4 +localhost_12002 acquired lock:lock-group_8 +localhost_12001 acquired lock:lock-group_10 +localhost_12001 acquired lock:lock-group_3 +localhost_12001 acquired lock:lock-group_6 +localhost_12003 acquired lock:lock-group_0 +localhost_12002 acquired lock:lock-group_5 +localhost_12001 acquired lock:lock-group_9 +localhost_12002 acquired lock:lock-group_2 +localhost_12003 acquired lock:lock-group_7 +localhost_12003 acquired lock:lock-group_11 +localhost_12002 acquired lock:lock-group_1 +lockName acquired By +====================================== +lock-group_0 localhost_12003 +lock-group_1 localhost_12002 +lock-group_10 localhost_12001 +lock-group_11 localhost_12003 +lock-group_2 localhost_12002 +lock-group_3 localhost_12001 +lock-group_4 localhost_12003 +lock-group_5 localhost_12002 +lock-group_6 localhost_12001 +lock-group_7 localhost_12003 +lock-group_8 localhost_12002 +lock-group_9 localhost_12001 +Stopping the first participant +localhost_12001 Interrupted +localhost_12002 acquired lock:lock-group_3 +localhost_12003 acquired lock:lock-group_6 +localhost_12003 acquired lock:lock-group_10 +localhost_12002 acquired lock:lock-group_9 +lockName acquired By +====================================== +lock-group_0 localhost_12003 +lock-group_1 localhost_12002 +lock-group_10 localhost_12003 +lock-group_11 localhost_12003 +lock-group_2 localhost_12002 +lock-group_3 localhost_12002 +lock-group_4 localhost_12003 +lock-group_5 localhost_12002 +lock-group_6 localhost_12003 +lock-group_7 localhost_12003 +lock-group_8 localhost_12002 +lock-group_9 localhost_12002 +``` + +Notice that the lock assignment directly follows the assignment generated by the user-defined rebalancer both initially and after a participant is removed from the system. \ No newline at end of file diff --git a/site-releases/trunk/src/site/markdown/tutorial_accessors.md b/site-releases/trunk/src/site/markdown/tutorial_accessors.md new file mode 100644 index 0000000000..bde50d2ccf --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_accessors.md @@ -0,0 +1,125 @@ + + + + Tutorial - Logical Accessors + + +# [Helix Tutorial](./Tutorial.html): Logical Accessors + +Helix constructs follow a logical hierarchy. A cluster contains participants, and serve logical resources. Each resource can be divided into partitions, which themselves can be replicated. Helix now supports configuring and modifying clusters programmatically in a hierarchical way using logical accessors. + +[Click here](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/api/accessor/package-summary.html) for the Javadocs of the accessors. + +### An Example + +#### Configure a Participant + +A participant is a combination of a host, port, and a UserConfig. A UserConfig is an arbitrary set of properties a Helix user can attach to any participant. + +``` +ParticipantId participantId = ParticipantId.from("localhost_12345"); +ParticipantConfig participantConfig = new ParticipantConfig.Builder(participantId) + .hostName("localhost").port(12345).build(); +``` + +#### Configure a Resource + +##### RebalancerContext +A Resource is essentially a combination of a RebalancerContext and a UserConfig. A [RebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/RebalancerContext.html) consists of all the key properties required to rebalance a resource, including how it is partitioned and replicated, and what state model it follows. Most Helix resources will make use of a [PartitionedRebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.html), which is a RebalancerContext for resources that are partitioned. + +Recall that there are four [rebalancing modes](./tutorial_rebalance.html) that Helix provides, and so Helix also provides the following subclasses for PartitionedRebalancerContext: + +* [FullAutoRebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.html) for FULL_AUTO mode. +* [SemiAutoRebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.html) for SEMI_AUTO mode. This class allows a user to specify "preference lists" to indicate where each partition should ideally be served +* [CustomRebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.html) for CUSTOMIZED mode. This class allows a user tp specify "preference maps" to indicate the location and state for each partition replica. + +Helix also supports arbitrary subclasses of PartitionedRebalancerContext and even arbitrary implementations of RebalancerContext for applications that need a user-defined approach for rebalancing. For more, see [User-Defined Rebalancing](./tutorial_user_def_rebalancer.html) + +##### In Action + +Here is an example of a configured resource with a rebalancer context for FULL_AUTO mode and two partitions: + +``` +ResourceId resourceId = ResourceId.from("sampleResource"); +StateModelDefinition stateModelDef = getStateModelDef(); +Partition partition1 = new Partition(PartitionId.from(resourceId, "1")); +Partition partition2 = new Partition(PartitionId.from(resourceId, "2")); +FullAutoRebalancerContext rebalanceContext = + new FullAutoRebalancerContext.Builder(resourceId).replicaCount(1).addPartition(partition1) + .addPartition(partition2).stateModelDefId(stateModelDef.getStateModelDefId()).build(); +ResourceConfig resourceConfig = + new ResourceConfig.Builder(resourceId).rebalancerContext(rebalanceContext).build(); +``` + +#### Add the Cluster + +Now we can take the participant and resource configured above, add them to a cluster configuration, and then persist the entire cluster at once using a ClusterAccessor: + +``` +// configure the cluster +ClusterId clusterId = ClusterId.from("sampleCluster"); +ClusterConfig clusterConfig = new ClusterConfig.Builder(clusterId).addParticipant(participantConfig) + .addResource(resourceConfig).addStateModelDefinition(stateModelDef).build(); + +// create the cluster using a ClusterAccessor +HelixConnection connection = new ZkHelixConnection(zkAddr); +connection.connect(); +ClusterAccessor clusterAccessor = connection.createClusterAccessor(clusterId); +clusterAccessor.createCluster(clusterConfig); +``` + +### Create, Read, Update, and Delete + +Note that you don't have to specify the entire cluster beforehand! Helix provides a ClusterAccessor, ParticipantAccessor, and ResourceAccessor to allow changing as much or as little of the cluster as needed on the fly. You can add a resource or participant to a cluster, reconfigure a resource, participant, or cluster, remove components from the cluster, and more. See the [Javadocs](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/api/accessor/package-summary.html) to see all that the accessor classes can do. + +#### Delta Classes + +Updating a cluster, participant, or resource should involve selecting the element to change, and then letting Helix change only that component. To do this, Helix has included Delta classes for ClusterConfig, ParticipantConfig, and ResourceConfig. + +#### Example: Updating a Participant + +Tags are used for Helix depolyments where only certain participants can be allowed to serve certain resources. To do this, Helix only assigns resource replicas to participants who have a tag that the resource specifies. In this example, we will use ParticipantConfig.Delta to remove a participant tag and add another as part of a reconfiguration. + +``` +// specify the change to the participant +ParticipantConfig.Delta delta = new ParticipantConfig.Delta(participantId).addTag("newTag").removeTag("oldTag"); + +// update the participant configuration +ParticipantAccessor participantAccessor = connection.createParticipantAccessor(clusterId); +participantAccessor.updateParticipant(participantId, delta); +``` + +#### Example: Dropping a Resource +Removing a resource from the cluster is quite simple: + +``` +clusterAccessor.dropResourceFromCluster(resourceId); +``` + +#### Example: Reading the Cluster +Reading a full snapshot of the cluster is also a one-liner: + +``` +Cluster cluster = clusterAccessor.readCluster(); +``` + +### Atomic Accessors + +Helix also includes versions of ClusterAccessor, ParticipantAccessor, and ResourceAccessor that can complete operations atomically relative to one another. The specific semantics of the atomic operations are included in the Javadocs. These atomic classes should be used sparingly and only in cases where contention can adversely affect the correctness of a Helix-based cluster. For most deployments, this is not the case, and using these classes will cause a degradation in performance. However, the interface for all atomic accessors mirrors that of the non-atomic accessors. \ No newline at end of file diff --git a/site-releases/trunk/src/site/markdown/tutorial_admin.md b/site-releases/trunk/src/site/markdown/tutorial_admin.md new file mode 100644 index 0000000000..f269a4a76e --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_admin.md @@ -0,0 +1,407 @@ + + + + Tutorial - Admin Operations + + +# [Helix Tutorial](./Tutorial.html): Admin Operations + +Helix provides a set of admin api for cluster management operations. They are supported via: + +* _Java API_ +* _Commandline interface_ +* _REST interface via helix-admin-webapp_ + +### Java API +See interface [_org.apache.helix.HelixAdmin_](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/HelixAdmin.html) + +### Command-line interface +The command-line tool comes with helix-core package: + +Get the command-line tool: + +``` + - git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git + - cd incubator-helix + - ./build + - cd helix-core/target/helix-core-pkg/bin + - chmod +x *.sh +``` + +Get help: + +``` + - ./helix-admin.sh --help +``` + +All other commands have this form: + +``` + ./helix-admin.sh --zkSvr +``` + +Admin commands and brief description: + +| Command syntax | Description | +| -------------- | ----------- | +| _\-\-activateCluster \_ | Enable/disable a cluster in distributed controller mode | +| _\-\-addCluster \_ | Add a new cluster | +| _\-\-addIdealState \_ | Add an ideal state to a cluster | +| _\-\-addInstanceTag \_ | Add a tag to an instance | +| _\-\-addNode \_ | Add an instance to a cluster | +| _\-\-addResource \_ | Add a new resource to a cluster | +| _\-\-addResourceProperty \_ | Add a resource property | +| _\-\-addStateModelDef \_ | Add a State model definition to a cluster | +| _\-\-dropCluster \_ | Delete a cluster | +| _\-\-dropNode \_ | Remove a node from a cluster | +| _\-\-dropResource \_ | Remove an existing resource from a cluster | +| _\-\-enableCluster \_ | Enable/disable a cluster | +| _\-\-enableInstance \_ | Enable/disable an instance | +| _\-\-enablePartition \_ | Enable/disable a partition | +| _\-\-getConfig \_ | Get user configs | +| _\-\-getConstraints \_ | Get constraints | +| _\-\-help_ | print help information | +| _\-\-instanceGroupTag \_ | Specify instance group tag, used with rebalance command | +| _\-\-listClusterInfo \_ | Show information of a cluster | +| _\-\-listClusters_ | List all clusters | +| _\-\-listInstanceInfo \_ | Show information of an instance | +| _\-\-listInstances \_ | List all instances in a cluster | +| _\-\-listPartitionInfo \_ | Show information of a partition | +| _\-\-listResourceInfo \_ | Show information of a resource | +| _\-\-listResources \_ | List all resources in a cluster | +| _\-\-listStateModel \_ | Show information of a state model | +| _\-\-listStateModels \_ | List all state models in a cluster | +| _\-\-maxPartitionsPerNode \_ | Specify the max partitions per instance, used with addResourceGroup command | +| _\-\-rebalance \_ | Rebalance a resource | +| _\-\-removeConfig \_ | Remove user configs | +| _\-\-removeConstraint \_ | Remove a constraint | +| _\-\-removeInstanceTag \_ | Remove a tag from an instance | +| _\-\-removeResourceProperty \_ | Remove a resource property | +| _\-\-resetInstance \_ | Reset all erroneous partitions on an instance | +| _\-\-resetPartition \_ | Reset an erroneous partition | +| _\-\-resetResource \_ | Reset all erroneous partitions of a resource | +| _\-\-setConfig \_ | Set user configs | +| _\-\-setConstraint \_ | Set a constraint | +| _\-\-swapInstance \_ | Swap an old instance with a new instance | +| _\-\-zkSvr \_ | Provide zookeeper address | + +### REST interface + +The REST interface comes wit helix-admin-webapp package: + +``` + - git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git + - cd incubator-helix + - ./build + - cd helix-admin-webapp/target/helix-admin-webapp-pkg/bin + - chmod +x *.sh + - ./run-rest-admin.sh --zkSvr --port // make sure zookeeper is running +``` + +#### URL and support methods + +* _/clusters_ + * List all clusters + + ``` + curl http://localhost:8100/clusters + ``` + + * Add a cluster + + ``` + curl -d 'jsonParameters={"command":"addCluster","clusterName":"MyCluster"}' -H "Content-Type: application/json" http://localhost:8100/clusters + ``` + +* _/clusters/{clusterName}_ + * List cluster information + + ``` + curl http://localhost:8100/clusters/MyCluster + ``` + + * Enable/disable a cluster in distributed controller mode + + ``` + curl -d 'jsonParameters={"command":"activateCluster","grandCluster":"MyControllerCluster","enabled":"true"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster + ``` + + * Remove a cluster + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster + ``` + +* _/clusters/{clusterName}/resourceGroups_ + * List all resources in a cluster + + ``` + curl http://localhost:8100/clusters/MyCluster/resourceGroups + ``` + + * Add a resource to cluster + + ``` + curl -d 'jsonParameters={"command":"addResource","resourceGroupName":"MyDB","partitions":"8","stateModelDefRef":"MasterSlave" }' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups + ``` + +* _/clusters/{clusterName}/resourceGroups/{resourceName}_ + * List resource information + + ``` + curl http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB + ``` + + * Drop a resource + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB + ``` + + * Reset all erroneous partitions of a resource + + ``` + curl -d 'jsonParameters={"command":"resetResource"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB + ``` + +* _/clusters/{clusterName}/resourceGroups/{resourceName}/idealState_ + * Rebalance a resource + + ``` + curl -d 'jsonParameters={"command":"rebalance","replicas":"3"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState + ``` + + * Add an ideal state + + ``` + echo jsonParameters={ + "command":"addIdealState" + }&newIdealState={ + "id" : "MyDB", + "simpleFields" : { + "IDEAL_STATE_MODE" : "AUTO", + "NUM_PARTITIONS" : "8", + "REBALANCE_MODE" : "SEMI_AUTO", + "REPLICAS" : "0", + "STATE_MODEL_DEF_REF" : "MasterSlave", + "STATE_MODEL_FACTORY_NAME" : "DEFAULT" + }, + "listFields" : { + }, + "mapFields" : { + "MyDB_0" : { + "localhost_1001" : "MASTER", + "localhost_1002" : "SLAVE" + } + } + } + > newIdealState.json + curl -d @'./newIdealState.json' -H 'Content-Type: application/json' http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState + ``` + + * Add resource property + + ``` + curl -d 'jsonParameters={"command":"addResourceProperty","REBALANCE_TIMER_PERIOD":"500"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState + ``` + +* _/clusters/{clusterName}/resourceGroups/{resourceName}/externalView_ + * Show resource external view + + ``` + curl http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/externalView + ``` +* _/clusters/{clusterName}/instances_ + * List all instances + + ``` + curl http://localhost:8100/clusters/MyCluster/instances + ``` + + * Add an instance + + ``` + curl -d 'jsonParameters={"command":"addInstance","instanceNames":"localhost_1001"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances + ``` + + * Swap an instance + + ``` + curl -d 'jsonParameters={"command":"swapInstance","oldInstance":"localhost_1001", "newInstance":"localhost_1002"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances + ``` +* _/clusters/{clusterName}/instances/{instanceName}_ + * Show instance information + + ``` + curl http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Enable/disable an instance + + ``` + curl -d 'jsonParameters={"command":"enableInstance","enabled":"false"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Drop an instance + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Disable/enable partitions on an instance + + ``` + curl -d 'jsonParameters={"command":"enablePartition","resource": "MyDB","partition":"MyDB_0", "enabled" : "false"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Reset an erroneous partition on an instance + + ``` + curl -d 'jsonParameters={"command":"resetPartition","resource": "MyDB","partition":"MyDB_0"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + + * Reset all erroneous partitions on an instance + + ``` + curl -d 'jsonParameters={"command":"resetInstance"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/instances/localhost_1001 + ``` + +* _/clusters/{clusterName}/configs_ + * Get user cluster level config + + ``` + curl http://localhost:8100/clusters/MyCluster/configs/cluster + ``` + + * Set user cluster level config + + ``` + curl -d 'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/cluster + ``` + + * Remove user cluster level config + + ``` + curl -d 'jsonParameters={"command":"removeConfig","configs":"key1,key2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/cluster + ``` + + * Get/set/remove user participant level config + + ``` + curl -d 'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/participant/localhost_1001 + ``` + + * Get/set/remove resource level config + + ``` + curl -d 'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/configs/resource/MyDB + ``` + +* _/clusters/{clusterName}/controller_ + * Show controller information + + ``` + curl http://localhost:8100/clusters/MyCluster/Controller + ``` + + * Enable/disable cluster + + ``` + curl -d 'jsonParameters={"command":"enableCluster","enabled":"false"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/Controller + ``` + +* _/zkPath/{path}_ + * Get information for zookeeper path + + ``` + curl http://localhost:8100/zkPath/MyCluster + ``` + +* _/clusters/{clusterName}/StateModelDefs_ + * Show all state model definitions + + ``` + curl http://localhost:8100/clusters/MyCluster/StateModelDefs + ``` + + * Add a state mdoel definition + + ``` + echo jsonParameters={ + "command":"addStateModelDef" + }&newStateModelDef={ + "id" : "OnlineOffline", + "simpleFields" : { + "INITIAL_STATE" : "OFFLINE" + }, + "listFields" : { + "STATE_PRIORITY_LIST" : [ "ONLINE", "OFFLINE", "DROPPED" ], + "STATE_TRANSITION_PRIORITYLIST" : [ "OFFLINE-ONLINE", "ONLINE-OFFLINE", "OFFLINE-DROPPED" ] + }, + "mapFields" : { + "DROPPED.meta" : { + "count" : "-1" + }, + "OFFLINE.meta" : { + "count" : "-1" + }, + "OFFLINE.next" : { + "DROPPED" : "DROPPED", + "ONLINE" : "ONLINE" + }, + "ONLINE.meta" : { + "count" : "R" + }, + "ONLINE.next" : { + "DROPPED" : "OFFLINE", + "OFFLINE" : "OFFLINE" + } + } + } + > newStateModelDef.json + curl -d @'./untitled.txt' -H 'Content-Type: application/json' http://localhost:8100/clusters/MyCluster/StateModelDefs + ``` + +* _/clusters/{clusterName}/StateModelDefs/{stateModelDefName}_ + * Show a state model definition + + ``` + curl http://localhost:8100/clusters/MyCluster/StateModelDefs/OnlineOffline + ``` + +* _/clusters/{clusterName}/constraints/{constraintType}_ + * Show all contraints + + ``` + curl http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT + ``` + + * Set a contraint + + ``` + curl -d 'jsonParameters={"constraintAttributes":"RESOURCE=MyDB,CONSTRAINT_VALUE=1"}' -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT/MyConstraint + ``` + + * Remove a constraint + + ``` + curl -X DELETE http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT/MyConstraint + ``` + diff --git a/site-releases/trunk/src/site/markdown/tutorial_controller.md b/site-releases/trunk/src/site/markdown/tutorial_controller.md new file mode 100644 index 0000000000..1a4cc45d7d --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_controller.md @@ -0,0 +1,79 @@ + + + + Tutorial - Controller + + +# [Helix Tutorial](./Tutorial.html): Controller + +Next, let\'s implement the controller. This is the brain of the cluster. Helix makes sure there is exactly one active controller running the cluster. + +### Start the Helix Agent + + +It requires the following parameters: + +* clusterId: A logical ID to represent the group of nodes +* controllerId: A logical ID of the process creating the controller instance. Generally this is host:port. +* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. + +``` +HelixConnection connection = new ZKHelixConnection(zkConnectString); +HelixController controller = connection.createController(clusterId, controllerId); +``` + +### Controller Code + +The Controller needs to know about all changes in the cluster. Helix takes care of this with the default implementation. +If you need additional functionality, see GenericHelixController and ZKHelixController for how to configure the pipeline. + +``` +HelixConnection connection = new ZKHelixConnection(zkConnectString); +HelixController controller = connection.createController(clusterId, controllerId); +controller.startAsync(); +``` +The snippet above shows how the controller is started. You can also start the controller using command line interface. + +``` +cd helix/helix-core/target/helix-core-pkg/bin +./run-helix-controller.sh --zkSvr --cluster +``` + +### Controller deployment modes + +Helix provides multiple options to deploy the controller. + +#### STANDALONE + +The Controller can be started as a separate process to manage a cluster. This is the recommended approach. However, since one controller can be a single point of failure, multiple controller processes are required for reliability. Even if multiple controllers are running, only one will be actively managing the cluster at any time and is decided by a leader-election process. If the leader fails, another leader will take over managing the cluster. + +Even though we recommend this method of deployment, it has the drawback of having to manage an additional service for each cluster. See Controller As a Service option. + +#### EMBEDDED + +If setting up a separate controller process is not viable, then it is possible to embed the controller as a library in each of the participants. + +#### CONTROLLER AS A SERVICE + +One of the cool features we added in Helix is to use a set of controllers to manage a large number of clusters. + +For example if you have X clusters to be managed, instead of deploying X*3 (3 controllers for fault tolerance) controllers for each cluster, one can deploy just 3 controllers. Each controller can manage X/3 clusters. If any controller fails, the remaining two will manage X/2 clusters. + + diff --git a/site-releases/trunk/src/site/markdown/tutorial_health.md b/site-releases/trunk/src/site/markdown/tutorial_health.md new file mode 100644 index 0000000000..e1a7f3c2d3 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_health.md @@ -0,0 +1,46 @@ + + + + Tutorial - Customizing Heath Checks + + +# [Helix Tutorial](./Tutorial.html): Customizing Health Checks + +In this chapter, we\'ll learn how to customize the health check, based on metrics of your distributed system. + +### Health Checks + +Note: _this in currently in development mode, not yet ready for production._ + +Helix provides the ability for each node in the system to report health metrics on a periodic basis. + +Helix supports multiple ways to aggregate these metrics: + +* SUM +* AVG +* EXPONENTIAL DECAY +* WINDOW + +Helix persists the aggregated value only. + +Applications can define a threshold on the aggregate values according to the SLAs, and when the SLA is violated Helix will fire an alert. +Currently Helix only fires an alert, but in a future release we plan to use these metrics to either mark the node dead or load balance the partitions. +This feature will be valuable for distributed systems that support multi-tenancy and have a large variation in work load patterns. In addition, this can be used to detect skewed partitions (hotspots) and rebalance the cluster. + diff --git a/site-releases/trunk/src/site/markdown/tutorial_messaging.md b/site-releases/trunk/src/site/markdown/tutorial_messaging.md new file mode 100644 index 0000000000..4bdce0eca5 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_messaging.md @@ -0,0 +1,71 @@ + + + + Tutorial - Messaging + + +# [Helix Tutorial](./Tutorial.html): Messaging + +In this chapter, we\'ll learn about messaging, a convenient feature in Helix for sending messages between nodes of a cluster. This is an interesting feature which is quite useful in practice. It is common that nodes in a distributed system require a mechanism to interact with each other. + +### Example: Bootstrapping a Replica + +Consider a search system where the index replica starts up and it does not have an index. A typical solution is to get the index from a common location, or to copy the index from another replica. + +Helix provides a messaging API for intra-cluster communication between nodes in the system. Helix provides a mechanism to specify the message recipient in terms of resource, partition, and state rather than specifying hostnames. Helix ensures that the message is delivered to all of the required recipients. In this particular use case, the instance can specify the recipient criteria as all replicas of the desired partition to bootstrap. +Since Helix is aware of the global state of the system, it can send the message to appropriate nodes. Once the nodes respond, Helix provides the bootstrapping replica with all the responses. + +This is a very generic API and can also be used to schedule various periodic tasks in the cluster, such as data backups, log cleanup, etc. +System Admins can also perform ad-hoc tasks, such as on-demand backups or a system command (such as rm -rf ;) across all nodes of the cluster + +``` + ClusterMessagingService messagingService = manager.getMessagingService(); + + // Construct the Message + Message requestBackupUriRequest = new Message( + MessageType.USER_DEFINE_MSG, UUID.randomUUID().toString()); + requestBackupUriRequest + .setMsgSubType(BootstrapProcess.REQUEST_BOOTSTRAP_URL); + requestBackupUriRequest.setMsgState(MessageState.NEW); + + // Set the Recipient criteria: all nodes that satisfy the criteria will receive the message + Criteria recipientCriteria = new Criteria(); + recipientCriteria.setInstanceName("%"); + recipientCriteria.setRecipientInstanceType(InstanceType.PARTICIPANT); + recipientCriteria.setResource("MyDB"); + recipientCriteria.setPartition(""); + + // Should be processed only by process(es) that are active at the time of sending the message + // This means if the recipient is restarted after message is sent, it will not be processe. + recipientCriteria.setSessionSpecific(true); + + // wait for 30 seconds + int timeout = 30000; + + // the handler that will be invoked when any recipient responds to the message. + BootstrapReplyHandler responseHandler = new BootstrapReplyHandler(); + + // this will return only after all recipients respond or after timeout + int sentMessageCount = messagingService.sendAndWait(recipientCriteria, + requestBackupUriRequest, responseHandler, timeout); +``` + +See HelixManager.DefaultMessagingService in [Javadocs](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/messaging/DefaultMessagingService.html) for more info. + diff --git a/site-releases/trunk/src/site/markdown/tutorial_participant.md b/site-releases/trunk/src/site/markdown/tutorial_participant.md new file mode 100644 index 0000000000..da55cbdead --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_participant.md @@ -0,0 +1,97 @@ + + + + Tutorial - Participant + + +# [Helix Tutorial](./Tutorial.html): Participant + +In this chapter, we\'ll learn how to implement a Participant, which is a primary functional component of a distributed system. + + +### Start the Helix Agent + +The Helix agent is a common component that connects each system component with the controller. + +It requires the following parameters: + +* clusterId: A logical ID to represent the group of nodes +* participantId: A logical ID of the process creating the manager instance. Generally this is host:port. +* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. + +After the Helix participant instance is created, only thing that needs to be registered is the state model factory. +The methods of the State Model will be called when controller sends transitions to the Participant. In this example, we'll use the OnlineOffline factory. Other options include: + +* MasterSlaveStateModelFactory +* LeaderStandbyStateModelFactory +* BootstrapHandler +* _An application defined state model factory_ + + +``` +HelixConnection connection = new ZKHelixConnection(zkConnectString); +HelixParticipant participant = connection.createParticipant(clusterId, participantId); +StateMachineEngine stateMach = participant.getStateMachineEngine(); + +// create a stateModelFactory that returns a statemodel object for each partition. +StateModelFactory stateModelFactory = new OnlineOfflineStateModelFactory(); +stateMach.registerStateModelFactory(stateModelType, stateModelFactory); +participant.startAsync(); +``` + +Helix doesn\'t know what it means to change from OFFLINE\-\-\>ONLINE or ONLINE\-\-\>OFFLINE. The following code snippet shows where you insert your system logic for these two state transitions. + +``` +public class OnlineOfflineStateModelFactory extends StateModelFactory { + @Override + public StateModel createNewStateModel(String stateUnitKey) { + OnlineOfflineStateModel stateModel = new OnlineOfflineStateModel(); + return stateModel; + } + @StateModelInfo(states = "{'OFFLINE','ONLINE'}", initialState = "OFFINE") + public static class OnlineOfflineStateModel extends StateModel { + + @Transition(from = "OFFLINE", to = "ONLINE") + public void onBecomeOnlineFromOffline(Message message, + NotificationContext context) { + + System.out.println("OnlineOfflineStateModel.onBecomeOnlineFromOffline()"); + + //////////////////////////////////////////////////////////////////////////////////////////////// + // Application logic to handle transition // + // For example, you might start a service, run initialization, etc // + //////////////////////////////////////////////////////////////////////////////////////////////// + } + + @Transition(from = "ONLINE", to = "OFFLINE") + public void onBecomeOfflineFromOnline(Message message, + NotificationContext context) { + + System.out.println("OnlineOfflineStateModel.onBecomeOfflineFromOnline()"); + + //////////////////////////////////////////////////////////////////////////////////////////////// + // Application logic to handle transition // + // For example, you might shutdown a service, log this event, or change monitoring settings // + //////////////////////////////////////////////////////////////////////////////////////////////// + } + } +} +``` + diff --git a/site-releases/trunk/src/site/markdown/tutorial_propstore.md b/site-releases/trunk/src/site/markdown/tutorial_propstore.md new file mode 100644 index 0000000000..ec0d71bb37 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_propstore.md @@ -0,0 +1,34 @@ + + + + Tutorial - Application Property Store + + +# [Helix Tutorial](./Tutorial.html): Application Property Store + +In this chapter, we\'ll learn how to use the application property store. + +### Property Store + +It is common that an application needs support for distributed, shared data structures. Helix uses Zookeeper to store the application data and hence provides notifications when the data changes. + +While you could use Zookeeper directly, Helix supports caching the data and a write-through cache. This is far more efficient than reading from Zookeeper for every access. + +See [HelixManager.getHelixPropertyStore](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/store/package-summary.html) for details. diff --git a/site-releases/trunk/src/site/markdown/tutorial_rebalance.md b/site-releases/trunk/src/site/markdown/tutorial_rebalance.md new file mode 100644 index 0000000000..8f42a5ab52 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_rebalance.md @@ -0,0 +1,181 @@ + + + + Tutorial - Rebalancing Algorithms + + +# [Helix Tutorial](./Tutorial.html): Rebalancing Algorithms + +The placement of partitions in a distributed system is essential for the reliability and scalability of the system. For example, when a node fails, it is important that the partitions hosted on that node are reallocated evenly among the remaining nodes. Consistent hashing is one such algorithm that can satisfy this guarantee. Helix provides a variant of consistent hashing based on the RUSH algorithm, among others. + +This means given a number of partitions, replicas and number of nodes, Helix does the automatic assignment of partition to nodes such that: + +* Each node has the same number of partitions +* Replicas of the same partition do not stay on the same node +* When a node fails, the partitions will be equally distributed among the remaining nodes +* When new nodes are added, the number of partitions moved will be minimized along with satisfying the above criteria + +Helix employs a rebalancing algorithm to compute the _ideal state_ of the system. When the _current state_ differs from the _ideal state_, Helix uses it as the target state of the system and computes the appropriate transitions needed to bring it to the _ideal state_. + +Helix makes it easy to perform this operation, while giving you control over the algorithm. In this section, we\'ll see how to implement the desired behavior. + +Helix has four options for rebalancing, in increasing order of customization by the system builder: + +* FULL_AUTO +* SEMI_AUTO +* CUSTOMIZED +* USER_DEFINED + +``` + |FULL_AUTO | SEMI_AUTO | CUSTOMIZED| USER_DEFINED | + ---------------------------------------------------------| + LOCATION | HELIX | APP | APP | APP | + ---------------------------------------------------------| + STATE | HELIX | HELIX | APP | APP | + ---------------------------------------------------------- +``` + + +### FULL_AUTO + +When the rebalance mode is set to FULL_AUTO, Helix controls both the location of the replica along with the state. This option is useful for applications where creation of a replica is not expensive. + +For example, consider this system that uses a MasterSlave state model, with 3 partitions and 2 replicas in the ideal state. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "REBALANCE_MODE" : "FULL_AUTO", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [], + "MyResource_1" : [], + "MyResource_2" : [] + }, + "mapFields" : { + } +} +``` + +If there are 3 nodes in the cluster, then Helix will balance the masters and slaves equally. The ideal state is therefore: + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +Another typical example is evenly distributing a group of tasks among the currently healthy processes. For example, if there are 60 tasks and 4 nodes, Helix assigns 15 tasks to each node. +When one node fails, Helix redistributes its 15 tasks to the remaining 3 nodes, resulting in a balanced 20 tasks per node. Similarly, if a node is added, Helix re-allocates 3 tasks from each of the 4 nodes to the 5th node, resulting in a balanced distribution of 12 tasks per node.. + +#### SEMI_AUTO + +When the application needs to control the placement of the replicas, use the SEMI_AUTO rebalance mode. + +Example: In the ideal state below, the partition \'MyResource_0\' is constrained to be placed only on node1 or node2. The choice of _state_ is still controlled by Helix. That means MyResource_0.MASTER could be on node1 and MyResource_0.SLAVE on node2, or vice-versa but neither would be placed on node3. + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "REBALANCE_MODE" : "SEMI_AUTO", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + } + "listFields" : { + "MyResource_0" : [node1, node2], + "MyResource_1" : [node2, node3], + "MyResource_2" : [node3, node1] + }, + "mapFields" : { + } +} +``` + +The MasterSlave state model requires that a partition has exactly one MASTER at all times, and the other replicas should be SLAVEs. In this simple example with 2 replicas per partition, there would be one MASTER and one SLAVE. Upon failover, a SLAVE has to assume mastership, and a new SLAVE will be generated. + +In this mode when node1 fails, unlike in FULL_AUTO mode the partition is _not_ moved from node1 to node3. Instead, Helix will decide to change the state of MyResource_0 on node2 from SLAVE to MASTER, based on the system constraints. + +#### CUSTOMIZED + +Helix offers a third mode called CUSTOMIZED, in which the application controls the placement _and_ state of each replica. The application needs to implement a callback interface that Helix invokes when the cluster state changes. +Within this callback, the application can recompute the idealstate. Helix will then issue appropriate transitions such that _Idealstate_ and _Currentstate_ converges. + +Here\'s an example, again with 3 partitions, 2 replicas per partition, and the MasterSlave state model: + +``` +{ + "id" : "MyResource", + "simpleFields" : { + "REBALANCE_MODE" : "CUSTOMIZED", + "NUM_PARTITIONS" : "3", + "REPLICAS" : "2", + "STATE_MODEL_DEF_REF" : "MasterSlave", + }, + "mapFields" : { + "MyResource_0" : { + "N1" : "MASTER", + "N2" : "SLAVE", + }, + "MyResource_1" : { + "N2" : "MASTER", + "N3" : "SLAVE", + }, + "MyResource_2" : { + "N3" : "MASTER", + "N1" : "SLAVE", + } + } +} +``` + +Suppose the current state of the system is 'MyResource_0' -> {N1:MASTER, N2:SLAVE} and the application changes the ideal state to 'MyResource_0' -> {N1:SLAVE,N2:MASTER}. While the application decides which node is MASTER and which is SLAVE, Helix will not blindly issue MASTER-->SLAVE to N1 and SLAVE-->MASTER to N2 in parallel, since that might result in a transient state where both N1 and N2 are masters, which violates the MasterSlave constraint that there is exactly one MASTER at a time. Helix will first issue MASTER-->SLAVE to N1 and after it is completed, it will issue SLAVE-->MASTER to N2. + +#### USER_DEFINED + +For maximum flexibility, Helix exposes an interface that can allow applications to plug in custom rebalancing logic. By providing the name of a class that implements the Rebalancer interface, Helix will automatically call the contained method whenever there is a change to the live participants in the cluster. For more, see [User-Defined Rebalancer](./tutorial_user_def_rebalancer.html). + +#### Backwards Compatibility + +In previous versions, FULL_AUTO was called AUTO_REBALANCE and SEMI_AUTO was called AUTO. Furthermore, they were presented as the IDEAL_STATE_MODE. Helix supports both IDEAL_STATE_MODE and REBALANCE_MODE, but IDEAL_STATE_MODE is now deprecated and may be phased out in future versions. diff --git a/site-releases/trunk/src/site/markdown/tutorial_spectator.md b/site-releases/trunk/src/site/markdown/tutorial_spectator.md new file mode 100644 index 0000000000..24c1cf484e --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_spectator.md @@ -0,0 +1,76 @@ + + + + Tutorial - Spectator + + +# [Helix Tutorial](./Tutorial.html): Spectator + +Next, we\'ll learn how to implement a Spectator. Typically, a spectator needs to react to changes within the distributed system. Examples: a client that needs to know where to send a request, a topic consumer in a consumer group. The spectator is automatically informed of changes in the _external state_ of the cluster, but it does not have to add any code to keep track of other components in the system. + +### Start the Helix agent + +Same as for a Participant, The Helix agent is the common component that connects each system component with the controller. + +It requires the following parameters: + +* clusterName: A logical name to represent the group of nodes +* instanceName: A logical name of the process creating the manager instance. Generally this is host:port. +* instanceType: Type of the process. This can be one of the following types, in this case, use SPECTATOR: + * CONTROLLER: Process that controls the cluster, any number of controllers can be started but only one will be active at any given time. + * PARTICIPANT: Process that performs the actual task in the distributed system. + * SPECTATOR: Process that observes the changes in the cluster. + * ADMIN: To carry out system admin actions. +* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. + +After the Helix manager instance is created, only thing that needs to be registered is the listener. When the ExternalView changes, the listener is notified. + +### Spectator Code + +A spectator observes the cluster and is notified when the state of the system changes. Helix consolidates the state of entire cluster in one Znode called ExternalView. +Helix provides a default implementation RoutingTableProvider that caches the cluster state and updates it when there is a change in the cluster. + +``` +manager = HelixManagerFactory.getZKHelixManager(clusterName, + instanceName, + InstanceType.PARTICIPANT, + zkConnectString); +manager.connect(); +RoutingTableProvider routingTableProvider = new RoutingTableProvider(); +manager.addExternalViewChangeListener(routingTableProvider); +``` + +In the following code snippet, the application sends the request to a valid instance by interrogating the external view. Suppose the desired resource for this request is in the partition myDB_1. + +``` +## instances = routingTableProvider.getInstances(, "PARTITION_NAME", "PARTITION_STATE"); +instances = routingTableProvider.getInstances("myDB", "myDB_1", "ONLINE"); + +//////////////////////////////////////////////////////////////////////////////////////////////// +// Application-specific code to send a request to one of the instances // +//////////////////////////////////////////////////////////////////////////////////////////////// + +theInstance = instances.get(0); // should choose an instance and throw an exception if none are available +result = theInstance.sendRequest(yourApplicationRequest, responseObject); + +``` + +When the external view changes, the application needs to react by sending requests to a different instance. + diff --git a/site-releases/trunk/src/site/markdown/tutorial_state.md b/site-releases/trunk/src/site/markdown/tutorial_state.md new file mode 100644 index 0000000000..4f7b1b587a --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_state.md @@ -0,0 +1,131 @@ + + + + Tutorial - State Machine Configuration + + +# [Helix Tutorial](./Tutorial.html): State Machine Configuration + +In this chapter, we\'ll learn about the state models provided by Helix, and how to create your own custom state model. + +## State Models + +Helix comes with 3 default state models that are commonly used. It is possible to have multiple state models in a cluster. +Every resource that is added should be configured to use a state model that govern its _ideal state_. + +### MASTER-SLAVE + +* 3 states: OFFLINE, SLAVE, MASTER +* Maximum number of masters: 1 +* Slaves are based on the replication factor. The replication factor can be specified while adding the resource. + + +### ONLINE-OFFLINE + +* Has 2 states: OFFLINE and ONLINE. This simple state model is a good starting point for most applications. + +### LEADER-STANDBY + +* 1 Leader and multiple stand-bys. The idea is that exactly one leader accomplishes a designated task, the stand-bys are ready to take over if the leader fails. + +## Constraints + +In addition to the state machine configuration, one can specify the constraints of states and transitions. + +For example, one can say: + +* MASTER:1 +
Maximum number of replicas in MASTER state at any time is 1 + +* OFFLINE-SLAVE:5 +
Maximum number of OFFLINE-SLAVE transitions that can happen concurrently in the system is 5 in this example. + +### Dynamic State Constraints + +We also support two dynamic upper bounds for the number of replicas in each state: + +* N: The number of replicas in the state is at most the number of live participants in the cluster +* R: The number of replicas in the state is at most the specified replica count for the partition + +### State Priority + +Helix uses a greedy approach to satisfy the state constraints. For example, if the state machine configuration says it needs 1 MASTER and 2 SLAVES, but only 1 node is active, Helix must promote it to MASTER. This behavior is achieved by providing the state priority list as \[MASTER, SLAVE\]. + +### State Transition Priority + +Helix tries to fire as many transitions as possible in parallel to reach the stable state without violating constraints. By default, Helix simply sorts the transitions alphabetically and fires as many as it can without violating the constraints. You can control this by overriding the priority order. + +## Special States + +### DROPPED + +The DROPPED state is used to signify a replica that was served by a given participant, but is no longer served. This allows Helix and its participants to effectively clean up. There are two requirements that every new state model should follow with respect to the DROPPED state: + +* The DROPPED state must be defined +* There must be a path to DROPPED for every state in the model + +### ERROR + +The ERROR state is used whenever the participant serving a partition encountered an error and cannot continue to serve the partition. HelixAdmin has \"reset\" functionality to allow for participants to recover from the ERROR state. + +## Annotated Example + +Below is a complete definition of a Master-Slave state model. Notice the fields marked REQUIRED; these are essential for any state model definition. + +``` +StateModelDefinition stateModel = new StateModelDefinition.Builder("MasterSlave") + // OFFLINE is the state that the system starts in (initial state is REQUIRED) + .initialState("OFFLINE") + + // Lowest number here indicates highest priority, no value indicates lowest priority + .addState("MASTER", 1) + .addState("SLAVE", 2) + .addState("OFFLINE") + + // Note the special inclusion of the DROPPED state (REQUIRED) + .addState(HelixDefinedState.DROPPED.toString()) + + // No more than one master allowed + .upperBound("MASTER", 1) + + // R indicates an upper bound of number of replicas for each partition + .dynamicUpperBound("SLAVE", "R") + + // Add some high-priority transitions + .addTransition("SLAVE", "MASTER", 1) + .addTransition("OFFLINE", "SLAVE", 2) + + // Using the same priority value indicates that these transitions can fire in any order + .addTransition("MASTER", "SLAVE", 3) + .addTransition("SLAVE", "OFFLINE", 3) + + // Not specifying a value defaults to lowest priority + // Notice the inclusion of the OFFLINE to DROPPED transition + // Since every state has a path to OFFLINE, they each now have a path to DROPPED (REQUIRED) + .addTransition("OFFLINE", HelixDefinedState.DROPPED.toString()) + + // Create the StateModelDefinition instance + .build(); + + // Use the isValid() function to make sure the StateModelDefinition will work without issues + Assert.assertTrue(stateModel.isValid()); +``` + + diff --git a/site-releases/trunk/src/site/markdown/tutorial_throttling.md b/site-releases/trunk/src/site/markdown/tutorial_throttling.md new file mode 100644 index 0000000000..7417979c7e --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_throttling.md @@ -0,0 +1,38 @@ + + + + Tutorial - Throttling + + +# [Helix Tutorial](./Tutorial.html): Throttling + +In this chapter, we\'ll learn how to control the parallel execution of cluster tasks. Only a centralized cluster manager with global knowledge is capable of coordinating this decision. + +### Throttling + +Since all state changes in the system are triggered through transitions, Helix can control the number of transitions that can happen in parallel. Some of the transitions may be light weight, but some might involve moving data, which is quite expensive from a network and IOPS perspective. + +Helix allows applications to set a threshold on transitions. The threshold can be set at multiple scopes: + +* MessageType e.g STATE_TRANSITION +* TransitionType e.g SLAVE-MASTER +* Resource e.g database +* Node i.e per-node maximum transitions in parallel + diff --git a/site-releases/trunk/src/site/markdown/tutorial_user_def_rebalancer.md b/site-releases/trunk/src/site/markdown/tutorial_user_def_rebalancer.md new file mode 100644 index 0000000000..6246f6899e --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_user_def_rebalancer.md @@ -0,0 +1,227 @@ + + + + Tutorial - User-Defined Rebalancing + + +# [Helix Tutorial](./Tutorial.html): User-Defined Rebalancing + +Even though Helix can compute both the location and the state of replicas internally using a default fully-automatic rebalancer, specific applications may require rebalancing strategies that optimize for different requirements. Thus, Helix allows applications to plug in arbitrary rebalancer algorithms that implement a provided interface. One of the main design goals of Helix is to provide maximum flexibility to any distributed application. Thus, it allows applications to fully implement the rebalancer, which is the core constraint solver in the system, if the application developer so chooses. + +Whenever the state of the cluster changes, as is the case when participants join or leave the cluster, Helix automatically calls the rebalancer to compute a new mapping of all the replicas in the resource. When using a pluggable rebalancer, the only required step is to register it with Helix. Subsequently, no additional bootstrapping steps are necessary. Helix uses reflection to look up and load the class dynamically at runtime. As a result, it is also technically possible to change the rebalancing strategy used at any time. + +The [HelixRebalancer](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/HelixRebalancer.html) interface is as follows: + +``` +public void init(HelixManager helixManager); + +public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, Cluster cluster, + ResourceCurrentState currentState); +``` +The first parameter is a configuration of the resource to rebalance, the second is a full cache of all of the cluster data available to Helix, and the third is a snapshot of the actual current placements and state assignments. From the cluster variable, it is also possible to access the ResourceAssignment last generated by this rebalancer. Internally, Helix implements the same interface for its own rebalancing routines, so a user-defined rebalancer will be cognizant of the same information about the cluster as an internal implementation. Helix strives to provide applications the ability to implement algorithms that may require a large portion of the entire state of the cluster to make the best placement and state assignment decisions possible. + +A ResourceAssignment is a full representation of the location and the state of each replica of each partition of a given resource. This is a simple representation of the placement that the algorithm believes is the best possible. If the placement meets all defined constraints, this is what will become the actual state of the distributed system. + +### Rebalancer Context + +Helix provides an interface called [RebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/RebalancerContext.html). For each of the four main [rebalancing modes](./tutorial_rebalance.html), there is a base class called [PartitionedRebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.html), which contains all of the basic properties required for a partitioned resource. Helix provides three derived classes for PartitionedRebalancerContext: FullAutoRebalancerContext, SemiAutoRebalancerContext, and CustomizedRebalancerContext. If none of these work for your application, you can create your own class that derives PartiitonedRebalancerContext (or even only implements RebalancerContext). + +### Specifying a Rebalancer + +#### Using Logical Accessors +To specify the rebalancer, one can use ```PartitionedRebalancerContext#setRebalancerRef(RebalancerRef)``` to specify the specific implementation of the rebalancerClass. For example, here's a base constructed PartitionedRebalancerContext with a user-specified class: + +``` +RebalancerRef rebalancerRef = RebalancerRef.from(className); +PartitionedRebalancerContext rebalanceContext = + new PartitionedRebalancerContext.Builder(resourceId).replicaCount(1).addPartition(partition1) + .addPartition(partition2).stateModelDefId(stateModelDef.getStateModelDefId()) + .rebalancerRef(rebalancerRef).build(); +``` + +The class name is a fully-qualified class name consisting of its package and its name, and the class should implement the Rebalancer interface. Now, the context can be added to a ResourceConfig through ```ResourceConfig.Builder#rebalancerContext(RebalancerContext)``` and the context will automatically be made available to the rebalancer for all subsequent executions. + +#### Using HelixAdmin +For implementations that set up the cluster through existing code, the following HelixAdmin calls will update the Rebalancer class: + +``` +IdealState idealState = helixAdmin.getResourceIdealState(clusterName, resourceName); +idealState.setRebalanceMode(RebalanceMode.USER_DEFINED); +idealState.setRebalancerClassName(className); +helixAdmin.setResourceIdealState(clusterName, resourceName, idealState); +``` +There are two key fields to set to specify that a pluggable rebalancer should be used. First, the rebalance mode should be set to USER_DEFINED, and second the rebalancer class name should be set to a class that implements Rebalancer and is within the scope of the project. The class name is a fully-qualified class name consisting of its package and its name. + +#### Using YAML +Alternatively, the rebalancer class name can be specified in a YAML file representing the cluster configuration. The requirements are the same, but the representation is more compact. Below are the first few lines of an example YAML file. To see a full YAML specification, see the [YAML tutorial](./tutorial_yaml.html). + +``` +clusterName: lock-manager-custom-rebalancer # unique name for the cluster +resources: + - name: lock-group # unique resource name + rebalancer: # we will provide our own rebalancer + mode: USER_DEFINED + class: domain.project.helix.rebalancer.UserDefinedRebalancerClass +... +``` + +### Example +We demonstrate plugging in a simple user-defined rebalancer as part of a revisit of the [distributed lock manager](./recipes/user_def_rebalancer.html) example. It includes a functional Rebalancer implementation, as well as the entire YAML file used to define the cluster. + +Consider the case where partitions are locks in a lock manager and 6 locks are to be distributed evenly to a set of participants, and only one participant can hold each lock. We can define a rebalancing algorithm that simply takes the modulus of the lock number and the number of participants to evenly distribute the locks across participants. Helix allows capping the number of partitions a participant can accept, but since locks are lightweight, we do not need to define a restriction in this case. The following is a succinct implementation of this algorithm. + +``` +@Override +public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, Cluster cluster, + ResourceCurrentState currentState) { + // Get the rebalcancer context (a basic partitioned one) + PartitionedRebalancerContext context = rebalancerConfig.getRebalancerContext( + PartitionedRebalancerContext.class); + + // Initialize an empty mapping of locks to participants + ResourceAssignment assignment = new ResourceAssignment(context.getResourceId()); + + // Get the list of live participants in the cluster + List liveParticipants = new ArrayList( + cluster.getLiveParticipantMap().keySet()); + + // Get the state model (should be a simple lock/unlock model) and the highest-priority state + StateModelDefId stateModelDefId = context.getStateModelDefId(); + StateModelDefinition stateModelDef = cluster.getStateModelMap().get(stateModelDefId); + if (stateModelDef.getStatesPriorityList().size() < 1) { + LOG.error("Invalid state model definition. There should be at least one state."); + return assignment; + } + State lockState = stateModelDef.getTypedStatesPriorityList().get(0); + + // Count the number of participants allowed to lock each lock + String stateCount = stateModelDef.getNumParticipantsPerState(lockState); + int lockHolders = 0; + try { + // a numeric value is a custom-specified number of participants allowed to lock the lock + lockHolders = Integer.parseInt(stateCount); + } catch (NumberFormatException e) { + LOG.error("Invalid state model definition. The lock state does not have a valid count"); + return assignment; + } + + // Fairly assign the lock state to the participants using a simple mod-based sequential + // assignment. For instance, if each lock can be held by 3 participants, lock 0 would be held + // by participants (0, 1, 2), lock 1 would be held by (1, 2, 3), and so on, wrapping around the + // number of participants as necessary. + // This assumes a simple lock-unlock model where the only state of interest is which nodes have + // acquired each lock. + int i = 0; + for (PartitionId partition : context.getPartitionSet()) { + Map replicaMap = new HashMap(); + for (int j = i; j < i + lockHolders; j++) { + int participantIndex = j % liveParticipants.size(); + ParticipantId participant = liveParticipants.get(participantIndex); + // enforce that a participant can only have one instance of a given lock + if (!replicaMap.containsKey(participant)) { + replicaMap.put(participant, lockState); + } + } + assignment.addReplicaMap(partition, replicaMap); + i++; + } + return assignment; +} +``` + +Here is the ResourceAssignment emitted by the user-defined rebalancer for a 3-participant system whenever there is a change to the set of participants. + +* Participant_A joins + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_A": "LOCKED"}, + "lock_2": { "Participant_A": "LOCKED"}, + "lock_3": { "Participant_A": "LOCKED"}, + "lock_4": { "Participant_A": "LOCKED"}, + "lock_5": { "Participant_A": "LOCKED"}, +} +``` + +A ResourceAssignment is a mapping for each resource of partition to the participant serving each replica and the state of each replica. The state model is a simple LOCKED/RELEASED model, so participant A holds all lock partitions in the LOCKED state. + +* Participant_B joins + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_B": "LOCKED"}, + "lock_2": { "Participant_A": "LOCKED"}, + "lock_3": { "Participant_B": "LOCKED"}, + "lock_4": { "Participant_A": "LOCKED"}, + "lock_5": { "Participant_B": "LOCKED"}, +} +``` + +Now that there are two participants, the simple mod-based function assigns every other lock to the second participant. On any system change, the rebalancer is invoked so that the application can define how to redistribute its resources. + +* Participant_C joins (steady state) + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_B": "LOCKED"}, + "lock_2": { "Participant_C": "LOCKED"}, + "lock_3": { "Participant_A": "LOCKED"}, + "lock_4": { "Participant_B": "LOCKED"}, + "lock_5": { "Participant_C": "LOCKED"}, +} +``` + +This is the steady state of the system. Notice that four of the six locks now have a different owner. That is because of the naïve modulus-based assignmemt approach used by the user-defined rebalancer. However, the interface is flexible enough to allow you to employ consistent hashing or any other scheme if minimal movement is a system requirement. + +* Participant_B fails + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_C": "LOCKED"}, + "lock_2": { "Participant_A": "LOCKED"}, + "lock_3": { "Participant_C": "LOCKED"}, + "lock_4": { "Participant_A": "LOCKED"}, + "lock_5": { "Participant_C": "LOCKED"}, +} +``` + +On any node failure, as in the case of node addition, the rebalancer is invoked automatically so that it can generate a new mapping as a response to the change. Helix ensures that the Rebalancer has the opportunity to reassign locks as required by the application. + +* Participant_B (or the replacement for the original Participant_B) rejoins + +``` +{ + "lock_0": { "Participant_A": "LOCKED"}, + "lock_1": { "Participant_B": "LOCKED"}, + "lock_2": { "Participant_C": "LOCKED"}, + "lock_3": { "Participant_A": "LOCKED"}, + "lock_4": { "Participant_B": "LOCKED"}, + "lock_5": { "Participant_C": "LOCKED"}, +} +``` + +The rebalancer was invoked once again and the resulting ResourceAssignment reflects the steady state. + +### Caveats +- The rebalancer class must be available at runtime, or else Helix will not attempt to rebalance at all \ No newline at end of file diff --git a/site-releases/trunk/src/site/markdown/tutorial_yaml.md b/site-releases/trunk/src/site/markdown/tutorial_yaml.md new file mode 100644 index 0000000000..0f8e0cce11 --- /dev/null +++ b/site-releases/trunk/src/site/markdown/tutorial_yaml.md @@ -0,0 +1,102 @@ + + + + Tutorial - YAML Cluster Setup + + +# [Helix Tutorial](./Tutorial.html): YAML Cluster Setup + +As an alternative to using Helix Admin to set up the cluster, its resources, constraints, and the state model, Helix supports bootstrapping a cluster configuration based on a YAML file. Below is an annotated example of such a file for a simple distributed lock manager where a lock can only be LOCKED or RELEASED, and each lock only allows a single participant to hold it in the LOCKED state. + +``` +clusterName: lock-manager-custom-rebalancer # unique name for the cluster (required) +resources: + - name: lock-group # unique resource name (required) + rebalancer: # required + mode: USER_DEFINED # required - USER_DEFINED means we will provide our own rebalancer + class: org.apache.helix.userdefinedrebalancer.LockManagerRebalancer # required for USER_DEFINED + partitions: + count: 12 # number of partitions for the resource (default is 1) + replicas: 1 # number of replicas per partition (default is 1) + stateModel: + name: lock-unlock # model name (required) + states: [LOCKED, RELEASED, DROPPED] # the list of possible states (required if model not built-in) + transitions: # the list of possible transitions (required if model not built-in) + - name: Unlock + from: LOCKED + to: RELEASED + - name: Lock + from: RELEASED + to: LOCKED + - name: DropLock + from: LOCKED + to: DROPPED + - name: DropUnlock + from: RELEASED + to: DROPPED + - name: Undrop + from: DROPPED + to: RELEASED + initialState: RELEASED # (required if model not built-in) + constraints: + state: + counts: # maximum number of replicas of a partition that can be in each state (required if model not built-in) + - name: LOCKED + count: "1" + - name: RELEASED + count: "-1" + - name: DROPPED + count: "-1" + priorityList: [LOCKED, RELEASED, DROPPED] # states in order of priority (all priorities equal if not specified) + transition: # transitions priority to enforce order that transitions occur + priorityList: [Unlock, Lock, Undrop, DropUnlock, DropLock] # all priorities equal if not specified +participants: # list of nodes that can serve replicas (optional if dynamic joining is active, required otherwise) + - name: localhost_12001 + host: localhost + port: 12001 + - name: localhost_12002 + host: localhost + port: 12002 + - name: localhost_12003 + host: localhost + port: 12003 +``` + +Using a file like the one above, the cluster can be set up either with the command line: + +``` +incubator-helix/helix-core/target/helix-core/pkg/bin/YAMLClusterSetup.sh localhost:2199 lock-manager-config.yaml +``` + +or with code: + +``` +YAMLClusterSetup setup = new YAMLClusterSetup(zkAddress); +InputStream input = + Thread.currentThread().getContextClassLoader() + .getResourceAsStream("lock-manager-config.yaml"); +YAMLClusterSetup.YAMLClusterConfig config = setup.setupCluster(input); +``` + +Some notes: + +- A rebalancer class is only required for the USER_DEFINED mode. It is ignored otherwise. + +- Built-in state models, like OnlineOffline, LeaderStandby, and MasterSlave, or state models that have already been added only require a name for stateModel. If partition and/or replica counts are not provided, a value of 1 is assumed. \ No newline at end of file diff --git a/site-releases/trunk/src/site/resources/.htaccess b/site-releases/trunk/src/site/resources/.htaccess new file mode 100644 index 0000000000..d5c7bf3f94 --- /dev/null +++ b/site-releases/trunk/src/site/resources/.htaccess @@ -0,0 +1,20 @@ +# +# 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. +# + +Redirect /download.html /download.cgi diff --git a/site-releases/trunk/src/site/resources/download.cgi b/site-releases/trunk/src/site/resources/download.cgi new file mode 100644 index 0000000000..f9a0e3007c --- /dev/null +++ b/site-releases/trunk/src/site/resources/download.cgi @@ -0,0 +1,22 @@ +#!/bin/sh +# Just call the standard mirrors.cgi script. It will use download.html +# as the input template. +# +# 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. +# +exec /www/www.apache.org/dyn/mirrors/mirrors.cgi $* diff --git a/site-releases/trunk/src/site/resources/images/HELIX-components.png b/site-releases/trunk/src/site/resources/images/HELIX-components.png new file mode 100644 index 0000000000..c0c35aee2c Binary files /dev/null and b/site-releases/trunk/src/site/resources/images/HELIX-components.png differ diff --git a/site-releases/trunk/src/site/resources/images/PFS-Generic.png b/site-releases/trunk/src/site/resources/images/PFS-Generic.png new file mode 100644 index 0000000000..7eea3a0497 Binary files /dev/null and b/site-releases/trunk/src/site/resources/images/PFS-Generic.png differ diff --git a/site-releases/trunk/src/site/resources/images/RSYNC_BASED_PFS.png b/site-releases/trunk/src/site/resources/images/RSYNC_BASED_PFS.png new file mode 100644 index 0000000000..0cc55ae100 Binary files /dev/null and b/site-releases/trunk/src/site/resources/images/RSYNC_BASED_PFS.png differ diff --git a/site-releases/trunk/src/site/resources/images/bootstrap_statemodel.gif b/site-releases/trunk/src/site/resources/images/bootstrap_statemodel.gif new file mode 100644 index 0000000000..b8f8a42589 Binary files /dev/null and b/site-releases/trunk/src/site/resources/images/bootstrap_statemodel.gif differ diff --git a/site-releases/trunk/src/site/resources/images/helix-architecture.png b/site-releases/trunk/src/site/resources/images/helix-architecture.png new file mode 100644 index 0000000000..6f69a2db34 Binary files /dev/null and b/site-releases/trunk/src/site/resources/images/helix-architecture.png differ diff --git a/site-releases/trunk/src/site/resources/images/helix-logo.jpg b/site-releases/trunk/src/site/resources/images/helix-logo.jpg new file mode 100644 index 0000000000..d6428f600f Binary files /dev/null and b/site-releases/trunk/src/site/resources/images/helix-logo.jpg differ diff --git a/site-releases/trunk/src/site/resources/images/helix-znode-layout.png b/site-releases/trunk/src/site/resources/images/helix-znode-layout.png new file mode 100644 index 0000000000..5bafc45f21 Binary files /dev/null and b/site-releases/trunk/src/site/resources/images/helix-znode-layout.png differ diff --git a/site-releases/trunk/src/site/resources/images/statemachine.png b/site-releases/trunk/src/site/resources/images/statemachine.png new file mode 100644 index 0000000000..43d27ecfbc Binary files /dev/null and b/site-releases/trunk/src/site/resources/images/statemachine.png differ diff --git a/site-releases/trunk/src/site/resources/images/system.png b/site-releases/trunk/src/site/resources/images/system.png new file mode 100644 index 0000000000..f8a05c8764 Binary files /dev/null and b/site-releases/trunk/src/site/resources/images/system.png differ diff --git a/site-releases/trunk/src/site/site.xml b/site-releases/trunk/src/site/site.xml new file mode 100644 index 0000000000..52b9f8a7d5 --- /dev/null +++ b/site-releases/trunk/src/site/site.xml @@ -0,0 +1,118 @@ + + + + + images/helix-logo.jpg + http://helix.incubator.apache.org/site-releases/0.7.0-incubating-site + + + http://incubator.apache.org/images/egg-logo.png + http://incubator.apache.org/ + + + + + + + org.apache.maven.skins + maven-fluido-skin + 1.3.0 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+
Apache Helix, Apache, the Apache feather logo, and the Apache Helix project logos are trademarks of The Apache Software Foundation. + All other marks mentioned may be trademarks or registered trademarks of their respective owners.
+ Privacy Policy +
+
+ + + + + + + true + + true + + + ApacheHelix + true + false + + + + +
diff --git a/site-releases/trunk/src/site/xdoc/download.xml.vm b/site-releases/trunk/src/site/xdoc/download.xml.vm new file mode 100644 index 0000000000..41355dbf89 --- /dev/null +++ b/site-releases/trunk/src/site/xdoc/download.xml.vm @@ -0,0 +1,193 @@ + + +#set( $releaseName = "0.7.0-incubating" ) +#set( $releaseDate = "10/31/2013" ) + + + + Apache Incubator Helix Downloads + Apache Helix Documentation Team + + + +
+ + + +
+ +
+

Apache Helix artifacts are distributed in source and binary form under the terms of the + Apache License, Version 2.0. + See the included LICENSE and NOTICE files included in each artifact for additional license + information. +

+

Use the links below to download a source distribution of Apache Helix. + It is good practice to verify the integrity of the distribution files.

+
+ +
+

Release date: ${releaseDate}

+

${releaseName} Release notes

+ + + +

+ [if-any logo] + + logo + + [end] + The currently selected mirror is + [preferred]. + If you encounter a problem with this mirror, + please select another mirror. + If all mirrors are failing, there are + backup + mirrors + (at the end of the mirrors list) that should be available. +

+ +
+ Other mirrors: + + +
+ +

+ You may also consult the + complete list of mirrors. +

+ + + + + + + + + + + + + + + + +
ArtifactSignatures
+ helix-${releaseName}-src.zip + + asc + md5 + sha1 +
+
+ + + + + + + + + + + + + + + + + + +
ArtifactSignatures
+ helix-core-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+ helix-admin-webapp-${releaseName}-pkg.tar + + asc + md5 + sha1 +
+
+
+ + + +
+

We strongly recommend you verify the integrity of the downloaded files with both PGP and MD5.

+ +

The PGP signatures can be verified using PGP or + GPG. + First download the KEYS as well as the + *.asc signature file for the particular distribution. Make sure you get these files from the main + distribution directory, rather than from a mirror. Then verify the signatures using one of the following sets of + commands: + + $ pgp -ka KEYS +$ pgp helix-*.zip.asc + + $ gpg --import KEYS +$ gpg --verify helix-*.zip.asc +

+

Alternatively, you can verify the MD5 signature on the files. A Unix/Linux program called + md5 or + md5sum is included in most distributions. It is also available as part of + GNU Textutils. + Windows users can get binary md5 programs from these (and likely other) places: +

+

+
+ +
diff --git a/site-releases/trunk/src/test/conf/testng.xml b/site-releases/trunk/src/test/conf/testng.xml new file mode 100644 index 0000000000..58f0803678 --- /dev/null +++ b/site-releases/trunk/src/test/conf/testng.xml @@ -0,0 +1,27 @@ + + + + + + + + + + diff --git a/src/site/apt/releasenotes/release-0.6.2-incubating.apt b/src/site/apt/releasenotes/release-0.6.2-incubating.apt new file mode 100644 index 0000000000..51afc62361 --- /dev/null +++ b/src/site/apt/releasenotes/release-0.6.2-incubating.apt @@ -0,0 +1,181 @@ + ----- + Release Notes for Apache Helix 0.6.2-incubating + ----- + +~~ 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. + +~~ NOTE: For help with the syntax of this file, see: +~~ http://maven.apache.org/guides/mini/guide-apt-format.html + +Release Notes for Apache Helix 0.6.2-incubating + + The Apache Helix team would like to announce the release of Apache Helix 0.6.2-incubating + + This is the third release under the Apache umbrella. + + Helix is a generic cluster management framework used for the automatic management of partitioned, replicated and distributed resources hosted on a cluster of nodes. Helix provides the following features: + + * Automatic assignment of resource/partition to nodes + + * Node failure detection and recovery + + * Dynamic addition of Resources + + * Dynamic addition of nodes to the cluster + + * Pluggable distributed state machine to manage the state of a resource via state transitions + + * Automatic load balancing and throttling of transitions + + [] + +* Changes + +** Sub-task + + * [HELIX-28] - ZkHelixManager.handleNewSession() can happen when a liveinstance already exists + + * [HELIX-85] - Remove mock service module + + * [HELIX-106] - Remove all string constants in the code + + * [HELIX-107] - Add support to set custom objects into ZNRecord + + * [HELIX-124] - race condition in ZkHelixManager.handleNewSession() + + * [HELIX-165] - Add dependency for Guava libraries + + * [HELIX-169] - Take care of consecutive handleNewSession() and session expiry during handleNewSession() + + * [HELIX-170] - HelixManager#isLeader() should compare both instanceName and sessionId + + * [HELIX-195] - Race condition between FINALIZE callbacks and Zk Callbacks + + * [HELIX-207] - Add javadocs to classes and public methods in the top-level package + + * [HELIX-208] - Add javadocs to classes and public methods in the model package + + * [HELIX-277] - FULL_AUTO rebalancer should not prefer nodes that are just coming up + +** Bug + + * [HELIX-7] - Tune test parameters to fix random test failures + + * [HELIX-87] - Bad repository links in website + + * [HELIX-117] - backward incompatibility problem in accessing zkPath vis HelixWebAdmin + + * [HELIX-118] - PropertyStore -> HelixPropertyStore backwards incompatible location + + * [HELIX-119] - HelixManager serializer no longer needs ByteArraySerializer for /PROPERTYSTORE + + * [HELIX-129] - ZKDumper should use byte[] instead of String to read/write file/zk + + * [HELIX-131] - Connection timeout not set while connecting to zookeeper via zkHelixAdmin + + * [HELIX-133] - Cluster-admin command parsing does not work with removeConfig + + * [HELIX-140] - In ClusterSetup.java, the removeConfig is wrong wired to getConfig + + * [HELIX-141] - Autorebalance does not work reliably and fails when replica>1 + + * [HELIX-144] - Need to validate StateModelDefinition when adding new StateModelDefinition to Cluster + + * [HELIX-147] - Fix typo in Idealstate property max_partitions_per_instance + + * [HELIX-148] - Current preferred placement for auto rebalace is suboptimal for n > p + + * [HELIX-150] - Auto rebalance might not evenly distribute states across nodes + + * [HELIX-151] - Auto rebalance doesn't assign some replicas when other nodes could make room + + * [HELIX-153] - Auto rebalance tester uses the returned map fields, but production uses only list fields + + * [HELIX-155] - PropertyKey.instances() is wrongly wired to CONFIG type instead of INSTANCES type + + * [HELIX-197] - state model leak + + * [HELIX-199] - ZNRecord should not publish rawPayload unless it exists + + * [HELIX-216] - Allow HelixAdmin addResource to accept the old rebalancing types + + * [HELIX-221] - Can't find default error->dropped transition method using name convention + + * [HELIX-257] - Upgrade Restlet to 2.1.4 - due security flaw + + * [HELIX-258] - Upgrade Apache Camel due to CVE-2013-4330 + + * [HELIX-264] - fix zkclient#close() bug + + * [HELIX-279] - Apply gc handling fixes to main ZKHelixManager class + + * [HELIX-280] - Full auto rebalancer should check for resource tag first + + * [HELIX-288] - helix-core uses an old version of guava + + * [HELIX-299] - Some files in 0.6.2 are missing license headers + +** Improvement + + * [HELIX-20] - AUTO-REBALANCE helix controller should re-assign disabled partitions on a node to other available nodes + + * [HELIX-70] - Make Helix OSGi ready + + * [HELIX-149] - Allow clients to pass in preferred placement strategies + + * [HELIX-198] - Unify helix code style + + * [HELIX-218] - Add a reviewboard submission script + + * [HELIX-284] - Support participant auto join in YAML cluster setup + +** New Feature + + * [HELIX-215] - Allow setting up the cluster with a YAML file + +** Task + + * [HELIX-95] - Tracker for 0.6.2 release + + * [HELIX-154] - Auto rebalance algorithm should not depend on state + + * [HELIX-166] - Rename modes to auto, semi-auto, and custom + + * [HELIX-173] - Move rebalancing strategies to separate classes that implement the Rebalancer interface + + * [HELIX-188] - Add admin command line / REST API documentations + + * [HELIX-194] - ZNRecord has too many constructors + + * [HELIX-205] - Have user-defined rebalancers use RebalanceMode.USER_DEFINED + + * [HELIX-210] - Add support to set data with expect version in BaseDataAccessor + + * [HELIX-217] - Remove mock service module + + * [HELIX-273] - Rebalancer interface should remain unchanged in 0.6.2 + + * [HELIX-274] - Verify FULL_AUTO tagged node behavior + + * [HELIX-285] - add integration test util's + + [] + + Cheers, + -- + The Apache Helix Team diff --git a/src/site/apt/releasenotes/release-0.7.0-incubating.apt b/src/site/apt/releasenotes/release-0.7.0-incubating.apt new file mode 100644 index 0000000000..7661df00d3 --- /dev/null +++ b/src/site/apt/releasenotes/release-0.7.0-incubating.apt @@ -0,0 +1,174 @@ + ----- + Release Notes for Apache Helix 0.7.0-incubating + ----- + +~~ 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. + +~~ NOTE: For help with the syntax of this file, see: +~~ http://maven.apache.org/guides/mini/guide-apt-format.html + +Release Notes for Apache Helix 0.7.0-incubating + + The Apache Helix team would like to announce the release of Apache Helix 0.7.0-incubating + + This is the fourth release and second major release under the Apache umbrella. + + Helix is a generic cluster management framework used for the automatic management of partitioned, replicated and distributed resources hosted on a cluster of nodes. Helix provides the following features: + + * Automatic assignment of resource/partition to nodes + + * Node failure detection and recovery + + * Dynamic addition of Resources + + * Dynamic addition of nodes to the cluster + + * Pluggable distributed state machine to manage the state of a resource via state transitions + + * Automatic load balancing and throttling of transitions + + * Configurable, pluggable rebalancing + + [] + +* Changes + +** Sub-task + + * [HELIX-18] - Unify cluster setup and helixadmin + + * [HELIX-79] - consecutive GC may mess up helix session ids + + * [HELIX-83] - Add typed classes to denote helix ids + + * [HELIX-90] - Clean up Api's + + * [HELIX-98] - clean up setting constraint api + + * [HELIX-100] - Improve the helix config api + + * [HELIX-102] - Add new wrapper classes for Participant, Controller, Spectator, Administrator + + * [HELIX-104] - Add support to reuse zkclient + + * [HELIX-123] - ZkHelixManager.isLeader() should check session id in addition to instance name + + * [HELIX-139] - Need to double check the logic to prevent 2 controllers to control the same cluster + + * [HELIX-168] - separate HelixManager implementation for participant, controller, and distributed controller + + * [HELIX-176] - Need a list of tests that must pass to certify a release + + * [HELIX-224] - Move helix examples to separate module + + * [HELIX-233] - Ensure that website and wiki fully capture the updated changes in 0.7.0 + + * [HELIX-234] - Create concrete id classes for constructs, replacing strings + + * [HELIX-235] - Create a hierarchical logical model for the cluster + + * [HELIX-236] - Create a hierarchical cluster snapshot to replace ClusterDataCache + + * [HELIX-237] - Create helix-internal config classes for the hierarchical model + + * [HELIX-238] - Create accessors for the logical model + + * [HELIX-239] - List use cases for the logical model + + * [HELIX-240] - Write an example of the key use cases for the logical model + + * [HELIX-241] - Write the controller pipeline with the logical model + + * [HELIX-242] - Re-integrate the scheduler rebalancing into the new controller pipeline + + * [HELIX-243] - Fix failing tests related to helix model overhaul + + * [HELIX-244] - Redesign rebalancers using rebalancer-specific configs + + * [HELIX-246] - Refactor scheduler task config to comply with new rebalancer config and fix related scheduler task tests + + * [HELIX-248] - Resource logical model should be general enough to handle various resource types + + * [HELIX-268] - Atomic API + + * [HELIX-297] - Make 0.7.0 backward compatible for user-defined rebalancing + + +** Bug + + * [HELIX-40] - fix zkclient subscribe path leaking and zk callback-handler leaking in case of session expiry + + * [HELIX-46] - Add REST/cli admin command for message selection constraints + + * [HELIX-47] - when drop resource, remove resource-level config also + + * [HELIX-48] - use resource instead of db in output messages + + * [HELIX-50] - Ensure num replicas and preference list size in idealstate matches + + * [HELIX-59] - controller not cleaning dead external view generated from old sessions + + * [HELIX-136] - Write IdealState back to ZK when computed by custom Rebalancer + + * [HELIX-200] - helix controller send ERROR->DROPPED transition infinitely + + * [HELIX-214] - User-defined rebalancer should never use SEMI_AUTO code paths + + * [HELIX-225] - fix helix-example package build error + + * [HELIX-271] - ZkHelixAdmin#addResource() backward compatible problem + + * [HELIX-292] - ZNRecordStreamingSerializer should not assume id comes first + + * [HELIX-296] - HelixConnection in 0.7.0 does not remove LiveInstance znode + + * [HELIX-300] - Some files in 0.7.0 are missing license headers + + * [HELIX-302] - fix helix version compare bug + +** Improvement + + * [HELIX-37] - Cleanup CallbackHandler + + * [HELIX-202] - Ideal state should be a full mapping, not just a set of instance preferences + +** Task + + * [HELIX-109] - Review Helix model package + + * [HELIX-174] - Clean up ideal state calculators, move them to the controller rebalancer package + + * [HELIX-212] - Rebalancer interface should have 1 function to compute the entire ideal state + + * [HELIX-232] - Validation of 0.7.0 + + * [HELIX-290] - Ensure 0.7.0 can respond correctly to ideal state changes + + * [HELIX-295] - Upgrade or remove xstream dependency + + * [HELIX-301] - Update integration test utils for 0.7.0 + +** Test + + * [HELIX-286] - add a test for redefine state model definition + + [] + + Cheers, + -- + The Apache Helix Team diff --git a/src/site/apt/releasing.apt b/src/site/apt/releasing.apt index 11d0cd92aa..62f6bb9f57 100644 --- a/src/site/apt/releasing.apt +++ b/src/site/apt/releasing.apt @@ -1,5 +1,5 @@ ----- - Helix release process + Helix Release Process ----- ----- 2012-12-15 @@ -27,18 +27,18 @@ Helix release process - [[1]] Post to the dev list a few days before you plan to do an Helix release + [[1]] Post to dev@helix.incubator.apache.org a few days before you plan to do a Helix release - [[2]] Your maven setting must contains the entry to be able to deploy. + [[2]] Your Maven settings must contain this entry to be able to deploy. ~/.m2/settings.xml +------------- - - apache.releases.https - - - + + apache.releases.https + [USERNAME] + [PASSWORD] + +------------- [[3]] Apache DAV passwords @@ -49,14 +49,10 @@ Helix release process +------------- [[4]] Release Helix - You should have a GPG agent running in the session you will run the maven release commands(preferred), and confirm it works by running "gpg -ab" (type some text and press Ctrl-D). - If you do not have a GPG agent running, make sure that you have the "apache-release" profile set in your settings.xml as shown below. - Run the release + You should have a GPG agent running in the session you will run the maven release commands (preferred), and confirm it works by running "gpg -ab" (type some text and press Ctrl-D). + If you do not have a GPG agent running, make sure that you have the "apache-release" profile set in your settings.xml as shown below. -+------------- -mvn release:prepare release:perform -B -+------------- GPG configuration in maven settings xml: @@ -69,17 +65,28 @@ mvn release:prepare release:perform -B +------------- - [[4]] go to https://repository.apache.org and close your staged repository. Note the repository url (format https://repository.apache.org/content/repositories/orgapachehelix-019/org/apache/helix/helix/0.6-incubating/) + Run the release + ++------------- +mvn release:prepare +mvn release:perform ++------------- + + [[5]] Go to https://repository.apache.org and close your staged repository. Log in, click on Staging Repositories, check your repository, and click Close. Note the repository url (format https://repository.apache.org/content/repositories/orgapachehelix-[NNN]/org/apache/helix/helix/[VERSION]-incubating/) + + [[6]] Stage the release (stagingRepoUrl format https://repository.apache.org/content/repositories/orgapachehelix-[NNN]/org/apache/helix/helix/[VERSION]-incubating/) +------------- svn co https://dist.apache.org/repos/dist/dev/incubator/helix helix-dev-release cd helix-dev-release sh ./release-script-svn.sh version stagingRepoUrl -then svn add -then svn ci +svn add +gpg -k email@domain.com >> KEYS +gpg --armor --export email@domain.com >> KEYS +svn ci +------------- - [[5]] Validating the release + [[7]] Validate the release +------------- * Download sources, extract, build and run tests - mvn clean package @@ -90,18 +97,188 @@ then svn ci * Check signatures of all the binaries using gpg +------------- - [[6]] Call for a vote in the dev list and wait for 72 hrs. for the vote results. 3 binding votes are necessary for the release to be finalized. example - After the vote has passed, move the files from dist dev to dist release: svn mv https://dist.apache.org/repos/dist/dev/incubator/helix/version to https://dist.apache.org/repos/dist/release/incubator/helix/ + [[8]] Call for a vote in the dev list and wait for 72 hours for the vote results. + ++------------- +Subject: [VOTE] Apache Helix [VERSION]-incubating Release +To: dev@helix.incubator.apache.org +--------------------------------------------------------- +Hi, I'd like to release Apache Helix [VERSION]-incubating. + +Release notes: http://helix.incubator.apache.org/releasenotes/release-[VERSION]-incubating.html + +Maven staged release repository: https://repository.apache.org/content/repositories/orgapachehelix-[NNN]/ + +Distribution: +* binaries: https://dist.apache.org/repos/dist/dev/incubator/helix/[VERSION]-incubating/binaries/ +* sources: https://dist.apache.org/repos/dist/dev/incubator/helix/[VERSION]-incubating/src/ + +KEYS file available here: https://dist.apache.org/repos/dist/release/incubator/helix/KEYS + +Vote open for 72H + +[+1] +[0] +[-1] ++------------- + + [[9]] If there are no objections, send the following email. Otherwise, repeat the previous step. + ++------------- +Subject: [RESULT][VOTE] Apache Helix [VERSION]-incubating Release +To: dev@helix.incubator.apache.org +----------------------------------------------------------------- +Thanks for voting on the 0.6.2-incubating release. It has now exceeded 72 hours so I am closing the vote. + +Binding +1s: + [Names of IPMC members (i.e. mentors) who +1'd this release] + +Nonbinding +1s: + [All other +1s] + +Binding 0s: + [Names of IPMC members (i.e. mentors) who 0'd this release] + +Nonbinding 0s: + [All other 0s] + +Binding -1s: + [Names of IPMC members (i.e. mentors) who -1'd this release] + +Nonbinding -1s: + [All other -1s] + +I will now start a vote on the general incubator list, as that is the next step in the release approval process. ++------------- + + [[10]] Open a vote on the general incubator mailing list. A total of 3 IPMC +1s are required for the release to be approved. This total can include the IPMC members who voted +1 in the previous vote. + ++------------- +Subject: [VOTE] Apache Helix [VERSION]-incubating Release +To: general@incubator.apache.org +--------------------------------------------------------- +Hi, + +This is to call for a vote on releasing the following candidate as Apache +Helix [VERSION]-incubating. This is the first release candidate of our third +release at Apache. + +Apache Helix is a generic cluster management framework that makes it easy +to build partitioned and replicated, fault tolerant and scalable +distributed systems. + +Release notes: +http://helix.incubator.apache.org/site-releases/[VERSION]-incubating-site/releasenotes/release-[VERSION]-incubating.html - [[7]] Prepare release note. Add a page in src/site/apt/releasenotes/ and change value of \ in parent pom. +Our vote thread on helix-dev: +http://markmail.org/message/[MESSAGE ID] +The following IPMC members have voted +1 +[IPMC members who voted +1 in the previous voting round] - [[8]] Send out an announcement of the release to: +Release artifacts: +https://repository.apache.org/content/repositories/orgapachehelix-[NNN] - * users@helix.incubator.apache.org +Distribution: +* binaries: +https://dist.apache.org/repos/dist/dev/incubator/helix/[VERSION]-incubating/binaries/ +* sources: +https://dist.apache.org/repos/dist/dev/incubator/helix/[VERSION]-incubating/src/ - * dev@helix.incubator.apache.org +The 0.6.2-incubating release tag +https://git-wip-us.apache.org/repos/asf?p=incubator-helix.git;a=tag;h=[TAG HASH] + +KEYS file available here: +https://dist.apache.org/repos/dist/dev/incubator/helix/KEYS + +Please vote on the release. The vote will be open for 72 hours. + +[+1] +[0] +[-1] + +Thanks, +The Apache Helix Team ++------------- + + [[11]] After 72 hours, if the sum of IPMC members who have voted +1 in the two rounds of voting is at least 3, close the vote with the following email. + ++------------- +Subject: [RESULT][VOTE] Apache Helix [VERSION]-incubating Release +To: general@incubator.apache.org +----------------------------------------------------------------- +Hi: + +Closing the vote since it has passed 72 hours. + +Here is the result: + ++1: [NNN] (binding) +[IPMC members who voted +1] + +0: [NNN] (binding) +[IPMC members who voted 0] + +-1: [NNN] (binding) +[IPMC members who voted -1] + +The vote has passed, thanks a lot to everyone for voting, Thanks to the +mentors for all the support! + +Cheers, +The Helix Team ++------------- + + [[12]] Move the keys, sources, and binaries from the dev tree to the release tree: + ++------------- +svn rm https://dist.apache.org/repos/dist/release/incubator/helix/KEYS +svn mv https://dist.apache.org/repos/dist/dev/incubator/helix/[VERSION]-incubating https://dist.apache.org/repos/dist/release/incubator/helix +svn mv https://dist.apache.org/repos/dist/dev/incubator/helix/KEYS https://dist.apache.org/repos/dist/release/incubator/helix ++------------- + + [[13]] Go to https://repository.apache.org and release your staged repository. Log in, click on Staging Repositories, check your repository, and click Release. + + [[14]] Prepare release notes. Add a page in src/site/apt/releasenotes/ and site-releases/[VERSION]-incubating/apt/releasenotes and change the value of \ in parent pom. + + + [[15]] Send out an announcement of the release to Helix developers and users: + ++------------- +Subject: [ANNOUNCE] Apache Helix [VERSION]-incubating Release +To: dev@helix.incubator.apache.org; user@helix.incubator.apache.org +------------------------------------------------------------------- +The Apache Helix Team is pleased to announce the [NTH] release, +[VERSION]-incubating, of the Apache Helix project. + +Apache Helix is a generic cluster management framework that makes it easy +to build partitioned, fault tolerant, and scalable distributed systems. + +The full release notes are available here: +http://helix.incubator.apache.org/releasenotes/release-[VERSION]-incubating.html + +You can declare a maven dependency to use it: + + + org.apache.helix + helix-core + [VERSION]-incubating + + +Or download the release sources: +http://helix.incubator.apache.org/site-releases/[VERSION]-incubating-site/download.cgi + +Additional info + +Website: http://helix.incubator.apache.org/ +Helix mailing lists: http://helix.incubator.apache.org/mail-lists.html + +We hope you will enjoy using the latest release of Apache Helix! + +Cheers, +Apache Helix Team ++------------- - [[9]] Celebrate ! + [[16]] Celebrate! diff --git a/src/site/markdown/Architecture.md b/src/site/markdown/Architecture.md index 7acf590fbf..933e917c7e 100644 --- a/src/site/markdown/Architecture.md +++ b/src/site/markdown/Architecture.md @@ -17,7 +17,12 @@ specific language governing permissions and limitations under the License. --> + + Architecture + +Architecture +---------------------------- Helix aims to provide the following abilities to a distributed system: * Automatic management of a cluster hosting partitioned, replicated resources. @@ -29,16 +34,16 @@ Helix aims to provide the following abilities to a distributed system: * Monitor cluster health and provide alerts on SLA violation. * Service discovery mechanism to route requests. -To build such a system, we need a mechanism to co-ordinate between different nodes/components in the system. This mechanism can be achieved with a software that reacts to any change in the cluster and comes up with a set of tasks needed to bring the cluster to a stable state. The set of tasks will be assigned to one or more nodes in the cluster. Helix serves this purpose of managing the various components in the cluster. +To build such a system, we need a mechanism to co-ordinate between different nodes and other components in the system. This mechanism can be achieved with software that reacts to any change in the cluster and comes up with a set of tasks needed to bring the cluster to a stable state. The set of tasks will be assigned to one or more nodes in the cluster. Helix serves this purpose of managing the various components in the cluster. ![Helix Design](images/system.png) Distributed System Components -In general any distributed system cluster will have the following +In general any distributed system cluster will have the following components and properties: -* Set of nodes also referred to as an instance. -* Set of resources which can be a database, lucene index or a task. +* A set of nodes also referred to as instances. +* A set of resources which can be databases, lucene indexes or tasks. * Each resource is also partitioned into one or more Partitions. * Each partition may have one or more copies called replicas. * Each replica can have a state associated with it. For example Master, Slave, Leader, Standby, Online, Offline etc @@ -48,47 +53,46 @@ Roles ![Helix Design](images/HELIX-components.png) -Not all nodes in a distributed system will perform similar functionality. For e.g, a few nodes might be serving requests, few nodes might be sending the request and some nodes might be controlling the nodes in the cluster. Based on functionality we have grouped them into +Not all nodes in a distributed system will perform similar functionalities. For example, a few nodes might be serving requests and a few nodes might be sending requests, and some nodes might be controlling the nodes in the cluster. Thus, Helix categorizes nodes by their specific roles in the system. -We have divided Helix in 3 logical components based on their responsibility - -1. PARTICIPANT: The nodes that actually host the distributed resources. -2. SPECTATOR: The nodes that simply observe the PARTICIPANT State and route the request accordingly. Routers, for example, need to know the Instance on which a partition is hosted and its state in order to route the request to the appropriate end point. -3. CONTROLLER: The controller observes and controls the PARTICIPANT nodes. It is responsible for coordinating all transitions in the cluster and ensuring that state constraints are satisfied and cluster stability is maintained. +We have divided Helix nodes into 3 logical components based on their responsibilities: +1. Participant: The nodes that actually host the distributed resources. +2. Spectator: The nodes that simply observe the Participant state and route the request accordingly. Routers, for example, need to know the instance on which a partition is hosted and its state in order to route the request to the appropriate end point. +3. Controller: The controller observes and controls the Participant nodes. It is responsible for coordinating all transitions in the cluster and ensuring that state constraints are satisfied and cluster stability is maintained. These are simply logical components and can be deployed as per the system requirements. For example: -1. Controller can be deployed as a separate service -2. Controller can be deployed along with a Participant but only one Controller will be active at any given time. +1. The controller can be deployed as a separate service +2. The controller can be deployed along with a Participant but only one Controller will be active at any given time. Both have pros and cons, which will be discussed later and one can chose the mode of deployment as per system needs. -## Cluster state/metadata store +## Cluster state metadata store We need a distributed store to maintain the state of the cluster and a notification system to notify if there is any change in the cluster state. Helix uses Zookeeper to achieve this functionality. Zookeeper provides: * A way to represent PERSISTENT state which basically remains until its deleted. -* A way to represent TRANSIENT/EPHEMERAL state which vanishes when the process that created the STATE dies. -* Notification mechanism when there is a change in PERSISTENT/EPHEMERAL STATE +* A way to represent TRANSIENT/EPHEMERAL state which vanishes when the process that created the state dies. +* Notification mechanism when there is a change in PERSISTENT and EPHEMERAL state -The namespace provided by ZooKeeper is much like that of a standard file system. A name is a sequence of path elements separated by a slash (/). Every node[ZNODE] in ZooKeeper\'s namespace is identified by a path. +The namespace provided by ZooKeeper is much like that of a standard file system. A name is a sequence of path elements separated by a slash (/). Every node[ZNode] in ZooKeeper\'s namespace is identified by a path. -More info on Zookeeper can be found here http://zookeeper.apache.org +More info on Zookeeper can be found at http://zookeeper.apache.org -## Statemachine and constraints +## State machine and constraints -Even though the concept of Resource, Partition, Replicas is common to most distributed systems, one thing that differentiates one distributed system from another is the way each partition is assigned a state and the constraints on each state. +Even though the concepts of Resources, Partitions, and Replicas are common to most distributed systems, one thing that differentiates one distributed system from another is the way each partition is assigned a state and the constraints on each state. For example: -1. If a system is serving READ ONLY data then all partition\'s replicas are equal and they can either be ONLINE or OFFLINE. -2. If a system takes BOTH READ and WRITES but ensure that WRITES go through only one partition then the states will be MASTER, SLAVE and OFFLINE. Writes go through the MASTER and is replicated to the SLAVES. Optionally, READS can go through SLAVES. +1. If a system is serving read-only data then all partition\'s replicas are equal and they can either be ONLINE or OFFLINE. +2. If a system takes _both_ reads and writes but ensure that writes go through only one partition, the states will be MASTER, SLAVE, and OFFLINE. Writes go through the MASTER and replicate to the SLAVEs. Optionally, reads can go through SLAVES. -Apart from defining STATE for each partition, the transition path to each STATE can be application specific. For example, in order to become MASTER it might be a requirement to first become a SLAVE. This ensures that if the SLAVE does not have the data as part of OFFLINE-SLAVE transition it can bootstrap data from other nodes in the system. +Apart from defining state for each partition, the transition path to each state can be application specific. For example, in order to become MASTER it might be a requirement to first become a SLAVE. This ensures that if the SLAVE does not have the data as part of OFFLINE-SLAVE transition it can bootstrap data from other nodes in the system. Helix provides a way to configure an application specific state machine along with constraints on each state. Along with constraints on STATE, Helix also provides a way to specify constraints on transitions. (More on this later.) @@ -113,17 +117,17 @@ MASTER | SLAVE | SLAVE | N/A | The following terminologies are used in Helix to model a state machine. -* IDEALSTATE: The state in which we need the cluster to be in if all nodes are up and running. In other words, all state constraints are satisfied. -* CURRENTSTATE: Represents the current state of each node in the cluster -* EXTERNALVIEW: Represents the combined view of CURRENTSTATE of all nodes. +* IdealState: The state in which we need the cluster to be in if all nodes are up and running. In other words, all state constraints are satisfied. +* CurrentState: Represents the actual current state of each node in the cluster +* ExternalView: Represents the combined view of CurrentState of all nodes. -The goal of Helix is always to make the CURRENTSTATE of the system same as the IDEALSTATE. Some scenarios where this may not be true are: +The goal of Helix is always to make the CurrentState of the system same as the IdealState. Some scenarios where this may not be true are: * When all nodes are down * When one or more nodes fail * New nodes are added and the partitions need to be reassigned -### IDEALSTATE +### IdealState Helix lets the application define the IdealState on a resource basis which basically consists of: @@ -140,11 +144,11 @@ Example: * ..... * Partition-p, replica-3, Slave, Node-n -Helix comes with various algorithms to automatically assign the partitions to nodes. The default algorithm minimizes the number of shuffles that happen when new nodes are added to the system +Helix comes with various algorithms to automatically assign the partitions to nodes. The default algorithm minimizes the number of shuffles that happen when new nodes are added to the system. -### CURRENTSTATE +### CurrentState -Every instance in the cluster hosts one or more partitions of a resource. Each of the partitions has a State associated with it. +Every instance in the cluster hosts one or more partitions of a resource. Each of the partitions has a state associated with it. Example Node-1 @@ -154,9 +158,9 @@ Example Node-1 * .... * Partition-p, Slave -### EXTERNALVIEW +### ExternalView -External clients needs to know the state of each partition in the cluster and the Node hosting that partition. Helix provides one view of the system to SPECTATORS as EXTERNAL VIEW. EXTERNAL VIEW is simply an aggregate of all CURRENTSTATE +External clients needs to know the state of each partition in the cluster and the Node hosting that partition. Helix provides one view of the system to Spectators as _ExternalView_. ExternalView is simply an aggregate of all node CurrentStates. * Partition-1, replica-1, Master, Node-1 * Partition-1, replica-2, Slave, Node-2 @@ -171,28 +175,28 @@ Mode of operation in a cluster A node process can be one of the following: -* PARTICIPANT: The process registers itself in the cluster and acts on the messages received in its queue and updates the current state. Example: Storage Node -* SPECTATOR: The process is simply interested in the changes in the Externalview. The Router is a spectator of the Storage cluster. -* CONTROLLER: This process actively controls the cluster by reacting to changes in Cluster State and sending messages to PARTICIPANTS. +* Participant: The process registers itself in the cluster and acts on the messages received in its queue and updates the current state. Example: a storage node in a distributed database +* Spectator: The process is simply interested in the changes in the Externalview. +* Controller: This process actively controls the cluster by reacting to changes in cluster state and sending messages to Participants. ### Participant Node Process -* When Node starts up, it registers itself under LIVEINSTANCES -* After registering, it waits for new Messages in the message queue +* When Node starts up, it registers itself under _LiveInstances_ +* After registering, it waits for new _Messages_ in the message queue * When it receives a message, it will perform the required task as indicated in the message -* After the task is completed, depending on the task outcome it updates the CURRENTSTATE +* After the task is completed, depending on the task outcome it updates the CurrentState ### Controller Process -* Watches IDEALSTATE -* Node goes down/comes up or Node is added/removed. Watches LIVEINSTANCES and CURRENTSTATE of each Node in the cluster -* Triggers appropriate state transition by sending message to PARTICIPANT +* Watches IdealState +* Notified when a node goes down/comes up or node is added/removed. Watches LiveInstances and CurrentState of each node in the cluster +* Triggers appropriate state transitions by sending message to Participants ### Spectator Process -* When the process starts, it asks cluster manager agent to be notified of changes in ExternalView -* Whenever it receives a notification, it reads the Externalview and performs required duties. For the Router, it updates its routing table. +* When the process starts, it asks the Helix agent to be notified of changes in ExternalView +* Whenever it receives a notification, it reads the Externalview and performs required duties. #### Interaction between controller, participant and spectator @@ -212,11 +216,11 @@ The following picture shows how controllers, participants and spectators interac * If a task is dependent on another task being completed, do not add that task * After any task is completed by a Participant, Controllers gets notified of the change and the State Transition algorithm is re-run until the CurrentState is same as IdealState. -## Helix znode layout +## Helix ZNode layout Helix organizes znodes under clusterName in multiple levels. -The top level (under clusterName) znodes are all Helix defined and in upper case: +The top level (under the cluster name) ZNodes are all Helix-defined and in upper case: * PROPERTYSTORE: application property store * STATEMODELDEFES: state model definitions @@ -227,7 +231,7 @@ The top level (under clusterName) znodes are all Helix defined and in upper case * LIVEINSTANCES: live instances * CONTROLLER: cluster controller runtime information -Under INSTANCES, there are runtime znodes for each instance. An instance organizes znodes as follows: +Under INSTANCES, there are runtime ZNodes for each instance. An instance organizes ZNodes as follows: * CURRENTSTATES * sessionId diff --git a/src/site/markdown/Concepts.md b/src/site/markdown/Concepts.md index 02d74065e9..5bf42ac22d 100644 --- a/src/site/markdown/Concepts.md +++ b/src/site/markdown/Concepts.md @@ -17,6 +17,13 @@ specific language governing permissions and limitations under the License. --> + + Concepts + + +Concepts +---------------------------- + Helix is based on the idea that a given task has the following attributes associated with it: * _Location of the task_. For example it runs on Node N1 @@ -48,7 +55,7 @@ Consider a simple case where you want to launch a task \'myTask\' on node \'N1\' ``` ### Partition -If this task get too big to fit on one box, you might want to divide it into subTasks. Each subTask is referred to as a _partition_ in Helix. Let\'s say you want to divide the task into 3 subTasks/partitions, the IdealState can be changed as shown below. +If this task get too big to fit on one box, you might want to divide it into subtasks. Each subtask is referred to as a _partition_ in Helix. Let\'s say you want to divide the task into 3 subtasks/partitions, the IdealState can be changed as shown below. \'myTask_0\', \'myTask_1\', \'myTask_2\' are logical names representing the partitions of myTask. Each tasks runs on N1, N2 and N3 respectively. @@ -74,7 +81,7 @@ If this task get too big to fit on one box, you might want to divide it into sub ### Replica -Partitioning allows one to split the data/task into multiple subparts. But let\'s say the request rate each partition increases. The common solution is to have multiple copies for each partition. Helix refers to the copy of a partition as a _replica_. Adding a replica also increases the availability of the system during failures. One can see this methodology employed often in Search systems. The index is divided into shards, and each shard has multiple copies. +Partitioning allows one to split the data/task into multiple subparts. But let\'s say the request rate for each partition increases. The common solution is to have multiple copies for each partition. Helix refers to the copy of a partition as a _replica_. Adding a replica also increases the availability of the system during failures. One can see this methodology employed often in search systems. The index is divided into shards, and each shard has multiple copies. Let\'s say you want to add one additional replica for each task. The IdealState can simply be changed as shown below. @@ -106,7 +113,7 @@ For increasing the availability of the system, it\'s better to place the replica ### State -Now let\'s take a slightly complicated scenario where a task represents a database. Unlike an index which is in general read-only, a database supports both reads and writes. Keeping the data consistent among the replicas is crucial in distributed data stores. One commonly applied technique is to assign one replica as MASTER and remaining replicas as SLAVE. All writes go to the MASTER and are then replicated to the SLAVE replicas. +Now let\'s take a slightly more complicated scenario where a task represents a database. Unlike an index which is in general read-only, a database supports both reads and writes. Keeping the data consistent among the replicas is crucial in distributed data stores. One commonly applied technique is to assign one replica as the MASTER and remaining replicas as SLAVEs. All writes go to the MASTER and are then replicated to the SLAVE replicas. Helix allows one to assign different states to each replica. Let\'s say you have two MySQL instances N1 and N2, where one will serve as MASTER and another as SLAVE. The IdealState can be changed to: @@ -130,14 +137,14 @@ Helix allows one to assign different states to each replica. Let\'s say you have ### State Machine and Transitions -IdealState allows one to exactly specify the desired state of the cluster. Given an IdealState, Helix takes up the responsibility of ensuring that the cluster reaches the IdealState. The Helix _controller_ reads the IdealState and then commands the Participant to take appropriate actions to move from one state to another until it matches the IdealState. These actions are referred to as _transitions_ in Helix. +IdealState allows one to exactly specify the desired state of the cluster. Given an IdealState, Helix takes up the responsibility of ensuring that the cluster reaches the IdealState. The Helix _controller_ reads the IdealState and then commands each Participant to take appropriate actions to move from one state to another until it matches the IdealState. These actions are referred to as _transitions_ in Helix. The next logical question is: how does the _controller_ compute the transitions required to get to IdealState? This is where the finite state machine concept comes in. Helix allows applications to plug in a finite state machine. A state machine consists of the following: * State: Describes the role of a replica -* Transition: An action that allows a replica to move from one State to another, thus changing its role. +* Transition: An action that allows a replica to move from one state to another, thus changing its role. -Here is an example of MASTERSLAVE state machine, +Here is an example of MasterSlave state machine: ``` OFFLINE | SLAVE | MASTER @@ -176,7 +183,7 @@ Helix allows each resource to be associated with one state machine. This means y ### Current State -CurrentState of a resource simply represents its actual state at a PARTICIPANT. In the below example: +CurrentState of a resource simply represents its actual state at a Participant. In the below example: * INSTANCE_NAME: Unique name representing the process * SESSION_ID: ID that is automatically assigned every time a process joins the cluster @@ -206,7 +213,7 @@ Each node in the cluster has its own CurrentState. ### External View -In order to communicate with the PARTICIPANTs, external clients need to know the current state of each of the PARTICIPANTs. The external clients are referred to as SPECTATORS. In order to make the life of SPECTATOR simple, Helix provides an EXTERNALVIEW that is an aggregated view of the current state across all nodes. The EXTERNALVIEW has a similar format as IDEALSTATE. +In order to communicate with the Participants, external clients need to know the current state of each of the Participants. The external clients are referred to as Spectators. In order to make the life of Spectator simple, Helix provides an ExternalView that is an aggregated view of the current state across all nodes. The ExternalView has a similar format as IdealState. ``` { @@ -233,27 +240,27 @@ In order to communicate with the PARTICIPANTs, external clients need to know the ### Rebalancer -The core component of Helix is the CONTROLLER which runs the REBALANCER algorithm on every cluster event. Cluster events can be one of the following: +The core component of Helix is the Controller which runs the Rebalancer algorithm on every cluster event. Cluster events can be one of the following: * Nodes start/stop and soft/hard failures * New nodes are added/removed * Ideal state changes -There are few more such as config changes, etc. The key takeaway: there are many ways to trigger the rebalancer. +There are few more examples such as configuration changes, etc. The key takeaway: there are many ways to trigger the rebalancer. When a rebalancer is run it simply does the following: * Compares the IdealState and current state * Computes the transitions required to reach the IdealState -* Issues the transitions to each PARTICIPANT +* Issues the transitions to each Participant -The above steps happen for every change in the system. Once the current state matches the IdealState, the system is considered stable which implies \'IDEALSTATE = CURRENTSTATE = EXTERNALVIEW\' +The above steps happen for every change in the system. Once the current state matches the IdealState, the system is considered stable which implies \'IdealState = CurrentState = ExternalView\' ### Dynamic IdealState One of the things that makes Helix powerful is that IdealState can be changed dynamically. This means one can listen to cluster events like node failures and dynamically change the ideal state. Helix will then take care of triggering the respective transitions in the system. -Helix comes with a few algorithms to automatically compute the IdealState based on the constraints. For example, if you have a resource of 3 partitions and 2 replicas, Helix can automatically compute the IdealState based on the nodes that are currently active. See the [tutorial](./tutorial_rebalance.html) to find out more about various execution modes of Helix like AUTO_REBALANCE, AUTO and CUSTOM. +Helix comes with a few algorithms to automatically compute the IdealState based on the constraints. For example, if you have a resource of 3 partitions and 2 replicas, Helix can automatically compute the IdealState based on the nodes that are currently active. See the [tutorial](./site-releases/0.7.0-incubating-site/tutorial_rebalance.html) to find out more about various execution modes of Helix like FULL_AUTO, SEMI_AUTO and CUSTOMIZED. diff --git a/src/site/markdown/IRC.md b/src/site/markdown/IRC.md new file mode 100644 index 0000000000..6dd450fad4 --- /dev/null +++ b/src/site/markdown/IRC.md @@ -0,0 +1,33 @@ + + + + IRC + + +IRC +--- + +We\'re often available on the ```#apachehelix``` channel on the ```chat.freenode.net``` server. You can use your favorite IRC client or the web client below to connect. + +### Web IRC Client + +To connect to our channel, just enter a username below. + +