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

[29/50] [abbrv] git commit: Creating a more user-friendly service class for stateless services

Creating a more user-friendly service class for stateless services


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

Branch: refs/heads/master
Commit: 4ea6bcef711db208eaec78b6948ac2cae20291d1
Parents: 080a15f
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Tue Mar 4 11:37:43 2014 -0800
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Tue Mar 4 11:37:43 2014 -0800

----------------------------------------------------------------------
 .../manager/zk/AbstractParticipantService.java  | 142 -------------------
 .../participant/AbstractParticipantService.java | 142 +++++++++++++++++++
 .../helix/provisioning/ParticipantLauncher.java |   2 +-
 .../StatelessParticipantService.java            |  86 +++++++++++
 .../participant/StatelessServiceStateModel.java |  56 ++++++++
 .../StatelessServiceStateModelFactory.java      |  39 +++++
 .../yarn/example/HelloWorldService.java         |  34 ++---
 .../yarn/example/HelloWorldStateModel.java      |  33 -----
 .../example/HelloWorldStateModelFactory.java    |  13 --
 9 files changed, 339 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/4ea6bcef/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractParticipantService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractParticipantService.java b/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractParticipantService.java
deleted file mode 100644
index 49a7159..0000000
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractParticipantService.java
+++ /dev/null
@@ -1,142 +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 org.apache.helix.HelixConnection;
-import org.apache.helix.HelixParticipant;
-import org.apache.helix.PreConnectCallback;
-import org.apache.helix.api.id.ClusterId;
-import org.apache.helix.api.id.ParticipantId;
-
-import com.google.common.util.concurrent.AbstractService;
-
-/**
- * A modeling of a helix participant as a self-contained service.
- */
-public abstract class AbstractParticipantService extends AbstractService {
-  private final ClusterId _clusterId;
-  private final ParticipantId _participantId;
-  private HelixParticipant _participant;
-  private HelixConnection _connection;
-  boolean initialized;
-
-  /**
-   * Initialize the service.
-   * @param connection A live Helix connection
-   * @param clusterId the cluster to join
-   * @param participantId a unique identifier that this participant will join with
-   */
-  public AbstractParticipantService(HelixConnection connection, ClusterId clusterId,
-      ParticipantId participantId) {
-    _connection = connection;
-    _clusterId = clusterId;
-    _participantId = participantId;
-  }
-
-  @Override
-  protected final void doStart() {
-    _participant = _connection.createParticipant(_clusterId, _participantId);
-
-    // add a preconnect callback
-    _participant.addPreConnectCallback(new PreConnectCallback() {
-      @Override
-      public void onPreConnect() {
-        if (initialized) {
-          onReconnect();
-        } else {
-          init();
-          initialized = true;
-        }
-      }
-    });
-
-    // start and notify
-    if (!_connection.isConnected()) {
-      _connection.connect();
-    }
-    _participant.start();
-    notifyStarted();
-  }
-
-  @Override
-  protected final void doStop() {
-    _participant.stop();
-    notifyStopped();
-  }
-
-  /**
-   * Invoked when connection is re-established to zookeeper. Typical scenario this is invoked is
-   * when there is a long GC pause that causes the node to disconnect from the cluster and
-   * reconnects. NOTE: When the service disconnects all its states are reset to initial state.
-   */
-  protected void onReconnect() {
-    // default implementation does nothing.
-  }
-
-  /**
-   * Initialize the participant. For example, here is where you can
-   * <ul>
-   * <li>Read configuration of the cluster,resource, node</li>
-   * <li>Read configuration of the cluster,resource, node register a state machine: <br/>
-   * <br/>
-   * <code>
-   * HelixParticipant participant = getParticipant();
-   * participant.getStateMachineEngine().registerStateModelFactory(stateModelDefId, factory);
-   * </code><br/>
-   * <br/>
-   * </li>
-   * </ul>
-   * This code is called after connecting to zookeeper but before creating the liveinstance.
-   */
-  protected abstract void init();
-
-  /**
-   * Get an instantiated participant instance.
-   * @return HelixParticipant
-   */
-  public HelixParticipant getParticipant() {
-    return _participant;
-  }
-
-  /**
-   * @return ClusterId
-   * @see {@link ClusterId}
-   */
-  public ClusterId getClusterId() {
-    return _clusterId;
-  }
-
-  /**
-   * @see {@link ParticipantId}
-   * @return
-   */
-  public ParticipantId getParticipantId() {
-    return _participantId;
-  }
-
-  /**
-   * @see {@link HelixConnection}
-   * @return HelixConnection
-   */
-  public HelixConnection getConnection() {
-    return _connection;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/4ea6bcef/helix-core/src/main/java/org/apache/helix/participant/AbstractParticipantService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/participant/AbstractParticipantService.java b/helix-core/src/main/java/org/apache/helix/participant/AbstractParticipantService.java
new file mode 100644
index 0000000..cd22762
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/participant/AbstractParticipantService.java
@@ -0,0 +1,142 @@
+package org.apache.helix.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 org.apache.helix.HelixConnection;
+import org.apache.helix.HelixParticipant;
+import org.apache.helix.PreConnectCallback;
+import org.apache.helix.api.id.ClusterId;
+import org.apache.helix.api.id.ParticipantId;
+
+import com.google.common.util.concurrent.AbstractService;
+
+/**
+ * A modeling of a helix participant as a self-contained service.
+ */
+public abstract class AbstractParticipantService extends AbstractService {
+  private final ClusterId _clusterId;
+  private final ParticipantId _participantId;
+  private HelixParticipant _participant;
+  private HelixConnection _connection;
+  boolean initialized;
+
+  /**
+   * Initialize the service.
+   * @param connection A live Helix connection
+   * @param clusterId the cluster to join
+   * @param participantId a unique identifier that this participant will join with
+   */
+  public AbstractParticipantService(HelixConnection connection, ClusterId clusterId,
+      ParticipantId participantId) {
+    _connection = connection;
+    _clusterId = clusterId;
+    _participantId = participantId;
+  }
+
+  @Override
+  protected final void doStart() {
+    _participant = _connection.createParticipant(_clusterId, _participantId);
+
+    // add a preconnect callback
+    _participant.addPreConnectCallback(new PreConnectCallback() {
+      @Override
+      public void onPreConnect() {
+        if (initialized) {
+          onReconnect();
+        } else {
+          init();
+          initialized = true;
+        }
+      }
+    });
+
+    // start and notify
+    if (!_connection.isConnected()) {
+      _connection.connect();
+    }
+    _participant.start();
+    notifyStarted();
+  }
+
+  @Override
+  protected final void doStop() {
+    _participant.stop();
+    notifyStopped();
+  }
+
+  /**
+   * Invoked when connection is re-established to zookeeper. Typical scenario this is invoked is
+   * when there is a long GC pause that causes the node to disconnect from the cluster and
+   * reconnects. NOTE: When the service disconnects all its states are reset to initial state.
+   */
+  protected void onReconnect() {
+    // default implementation does nothing.
+  }
+
+  /**
+   * Initialize the participant. For example, here is where you can
+   * <ul>
+   * <li>Read configuration of the cluster,resource, node</li>
+   * <li>Read configuration of the cluster,resource, node register a state machine: <br/>
+   * <br/>
+   * <code>
+   * HelixParticipant participant = getParticipant();
+   * participant.getStateMachineEngine().registerStateModelFactory(stateModelDefId, factory);
+   * </code><br/>
+   * <br/>
+   * </li>
+   * </ul>
+   * This code is called after connecting to zookeeper but before creating the liveinstance.
+   */
+  protected abstract void init();
+
+  /**
+   * Get an instantiated participant instance.
+   * @return HelixParticipant
+   */
+  public HelixParticipant getParticipant() {
+    return _participant;
+  }
+
+  /**
+   * @return ClusterId
+   * @see {@link ClusterId}
+   */
+  public ClusterId getClusterId() {
+    return _clusterId;
+  }
+
+  /**
+   * @see {@link ParticipantId}
+   * @return
+   */
+  public ParticipantId getParticipantId() {
+    return _participantId;
+  }
+
+  /**
+   * @see {@link HelixConnection}
+   * @return HelixConnection
+   */
+  public HelixConnection getConnection() {
+    return _connection;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/4ea6bcef/helix-provisioning/src/main/java/org/apache/helix/provisioning/ParticipantLauncher.java
----------------------------------------------------------------------
diff --git a/helix-provisioning/src/main/java/org/apache/helix/provisioning/ParticipantLauncher.java b/helix-provisioning/src/main/java/org/apache/helix/provisioning/ParticipantLauncher.java
index 55bb618..60231fb 100644
--- a/helix-provisioning/src/main/java/org/apache/helix/provisioning/ParticipantLauncher.java
+++ b/helix-provisioning/src/main/java/org/apache/helix/provisioning/ParticipantLauncher.java
@@ -10,13 +10,13 @@ import org.apache.helix.HelixConnection;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.ParticipantId;
-import org.apache.helix.manager.zk.AbstractParticipantService;
 import org.apache.helix.manager.zk.ZkHelixConnection;
 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.Message;
 import org.apache.helix.model.Message.MessageType;
+import org.apache.helix.participant.AbstractParticipantService;
 import org.apache.log4j.Logger;
 
 /**

http://git-wip-us.apache.org/repos/asf/helix/blob/4ea6bcef/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessParticipantService.java
----------------------------------------------------------------------
diff --git a/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessParticipantService.java b/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessParticipantService.java
new file mode 100644
index 0000000..d937c5c
--- /dev/null
+++ b/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessParticipantService.java
@@ -0,0 +1,86 @@
+package org.apache.helix.provisioning.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 org.apache.helix.HelixConnection;
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.accessor.ResourceAccessor;
+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.api.id.StateModelDefId;
+import org.apache.helix.participant.AbstractParticipantService;
+import org.apache.helix.provisioning.ServiceConfig;
+import org.apache.log4j.Logger;
+
+public abstract class StatelessParticipantService extends AbstractParticipantService {
+  private static final Logger LOG = Logger.getLogger(StatelessParticipantService.class);
+
+  private final String _serviceName;
+
+  public StatelessParticipantService(HelixConnection connection, ClusterId clusterId,
+      ParticipantId participantId, String serviceName) {
+    super(connection, clusterId, participantId);
+    _serviceName = serviceName;
+  }
+
+  @Override
+  protected void init() {
+    ClusterId clusterId = getClusterId();
+    ResourceAccessor resourceAccessor = getConnection().createResourceAccessor(clusterId);
+    ResourceId resourceId = ResourceId.from(_serviceName);
+    UserConfig userConfig = resourceAccessor.readUserConfig(resourceId);
+    ServiceConfig serviceConfig = new ServiceConfig(Scope.resource(resourceId));
+    serviceConfig.setSimpleFields(userConfig.getSimpleFields());
+    serviceConfig.setListFields(userConfig.getListFields());
+    serviceConfig.setMapFields(userConfig.getMapFields());
+    LOG.info("Starting service:" + _serviceName + " with configuration:" + serviceConfig);
+    StatelessServiceStateModelFactory stateModelFactory =
+        new StatelessServiceStateModelFactory(this);
+    getParticipant().getStateMachineEngine().registerStateModelFactory(
+        StateModelDefId.from("StatelessService"), stateModelFactory);
+    init(serviceConfig);
+  }
+
+  /**
+   * Get the name of this stateless service
+   * @return service name
+   */
+  public String getName() {
+    return _serviceName;
+  }
+
+  /**
+   * Initialize the service with a configuration
+   */
+  protected abstract void init(ServiceConfig serviceConfig);
+
+  /**
+   * Invoked when this service is instructed to go online
+   */
+  protected abstract void goOnline();
+
+  /**
+   * Invoked when this service is instructed to go offline
+   */
+  protected abstract void goOffine();
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/4ea6bcef/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessServiceStateModel.java
----------------------------------------------------------------------
diff --git a/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessServiceStateModel.java b/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessServiceStateModel.java
new file mode 100644
index 0000000..f653de8
--- /dev/null
+++ b/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessServiceStateModel.java
@@ -0,0 +1,56 @@
+package org.apache.helix.provisioning.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 org.apache.helix.NotificationContext;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.model.Message;
+import org.apache.helix.participant.statemachine.StateModel;
+import org.apache.helix.participant.statemachine.StateModelInfo;
+import org.apache.helix.participant.statemachine.Transition;
+import org.apache.log4j.Logger;
+
+@StateModelInfo(initialState = "OFFLINE", states = {
+    "OFFLINE", "ONLINE", "ERROR"
+})
+public class StatelessServiceStateModel extends StateModel {
+  private static final Logger LOG = Logger.getLogger(StatelessServiceStateModel.class);
+
+  private final StatelessParticipantService _service;
+
+  public StatelessServiceStateModel(PartitionId partitionId, StatelessParticipantService service) {
+    _service = service;
+    // ignore partition
+  }
+
+  @Transition(to = "ONLINE", from = "OFFLINE")
+  public void onBecomeOnlineFromOffline(Message message, NotificationContext context)
+      throws Exception {
+    LOG.info("Started " + _service.getName() + " service");
+    _service.goOnline();
+  }
+
+  @Transition(to = "OFFLINE", from = "ONLINE")
+  public void onBecomeOfflineFromOnline(Message message, NotificationContext context)
+      throws InterruptedException {
+    LOG.info("Stopped " + _service.getName() + " service");
+    _service.goOffine();
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/4ea6bcef/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessServiceStateModelFactory.java
----------------------------------------------------------------------
diff --git a/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessServiceStateModelFactory.java b/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessServiceStateModelFactory.java
new file mode 100644
index 0000000..19c1488
--- /dev/null
+++ b/helix-provisioning/src/main/java/org/apache/helix/provisioning/participant/StatelessServiceStateModelFactory.java
@@ -0,0 +1,39 @@
+package org.apache.helix.provisioning.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 org.apache.helix.api.id.PartitionId;
+import org.apache.helix.participant.statemachine.HelixStateModelFactory;
+
+public class StatelessServiceStateModelFactory extends
+    HelixStateModelFactory<StatelessServiceStateModel> {
+
+  private final StatelessParticipantService _service;
+
+  public StatelessServiceStateModelFactory(StatelessParticipantService service) {
+    _service = service;
+  }
+
+  @Override
+  public StatelessServiceStateModel createNewStateModel(PartitionId partitionId) {
+    return new StatelessServiceStateModel(partitionId, _service);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/4ea6bcef/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldService.java
----------------------------------------------------------------------
diff --git a/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldService.java b/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldService.java
index 8999817..269ae0c 100644
--- a/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldService.java
+++ b/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldService.java
@@ -1,17 +1,14 @@
 package org.apache.helix.provisioning.yarn.example;
 
 import org.apache.helix.HelixConnection;
-import org.apache.helix.api.accessor.ResourceAccessor;
-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.api.id.StateModelDefId;
-import org.apache.helix.manager.zk.AbstractParticipantService;
-import org.apache.helix.provisioning.yarn.example.HelloWorldStateModelFactory;
+import org.apache.helix.participant.AbstractParticipantService;
+import org.apache.helix.provisioning.ServiceConfig;
+import org.apache.helix.provisioning.participant.StatelessParticipantService;
 import org.apache.log4j.Logger;
 
-public class HelloWorldService extends AbstractParticipantService {
+public class HelloWorldService extends StatelessParticipantService {
 
   private static Logger LOG = Logger.getLogger(AbstractParticipantService.class);
 
@@ -19,23 +16,22 @@ public class HelloWorldService extends AbstractParticipantService {
 
   public HelloWorldService(HelixConnection connection, ClusterId clusterId,
       ParticipantId participantId) {
-    super(connection, clusterId, participantId);
+    super(connection, clusterId, participantId, SERVICE_NAME);
   }
 
-  /**
-   * init method to setup appropriate call back handlers.
-   */
   @Override
-  public void init() {
-    ClusterId clusterId = getClusterId();
-    ResourceAccessor resourceAccessor = getConnection().createResourceAccessor(clusterId);
-    UserConfig serviceConfig = resourceAccessor.readUserConfig(ResourceId.from(SERVICE_NAME));
-    LOG.info("Starting service:" + SERVICE_NAME + " with configuration:" + serviceConfig);
+  protected void init(ServiceConfig serviceConfig) {
+    LOG.info("Initialized service with config " + serviceConfig);
+  }
 
-    HelloWorldStateModelFactory stateModelFactory = new HelloWorldStateModelFactory();
-    getParticipant().getStateMachineEngine().registerStateModelFactory(
-        StateModelDefId.from("StatelessService"), stateModelFactory);
+  @Override
+  protected void goOnline() {
+    LOG.info("HelloWorld service is told to go online");
+  }
 
+  @Override
+  protected void goOffine() {
+    LOG.info("HelloWorld service is told to go offline");
   }
 
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/4ea6bcef/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldStateModel.java
----------------------------------------------------------------------
diff --git a/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldStateModel.java b/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldStateModel.java
deleted file mode 100644
index 078d847..0000000
--- a/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldStateModel.java
+++ /dev/null
@@ -1,33 +0,0 @@
-package org.apache.helix.provisioning.yarn.example;
-
-import org.apache.helix.NotificationContext;
-import org.apache.helix.api.id.PartitionId;
-import org.apache.helix.model.Message;
-import org.apache.helix.participant.statemachine.StateModel;
-import org.apache.helix.participant.statemachine.StateModelInfo;
-import org.apache.helix.participant.statemachine.Transition;
-import org.apache.log4j.Logger;
-
-@StateModelInfo(initialState = "OFFLINE", states = {
-    "OFFLINE", "ONLINE", "ERROR"
-})
-public class HelloWorldStateModel extends StateModel {
-
-  private static Logger LOG = Logger.getLogger(HelloWorldStateModel.class);
-
-  public HelloWorldStateModel(PartitionId partitionId) {
-    // ignore the partitionId
-  }
-
-  @Transition(to = "ONLINE", from = "OFFLINE")
-  public void onBecomeOnlineFromOffline(Message message, NotificationContext context)
-      throws Exception {
-    LOG.info("Started HelloWorld service");
-  }
-
-  @Transition(to = "OFFLINE", from = "ONLINE")
-  public void onBecomeOfflineFromOnline(Message message, NotificationContext context)
-      throws InterruptedException {
-    LOG.info("Stopped HelloWorld service");
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/4ea6bcef/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldStateModelFactory.java
----------------------------------------------------------------------
diff --git a/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldStateModelFactory.java b/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldStateModelFactory.java
deleted file mode 100644
index 2766f6d..0000000
--- a/recipes/helloworld-provisioning-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/HelloWorldStateModelFactory.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.helix.provisioning.yarn.example;
-
-import org.apache.helix.api.id.PartitionId;
-import org.apache.helix.participant.statemachine.HelixStateModelFactory;
-import org.apache.helix.participant.statemachine.StateModel;
-import org.apache.helix.provisioning.yarn.example.HelloWorldStateModel;
-
-public class HelloWorldStateModelFactory extends HelixStateModelFactory<StateModel> {
-	@Override
-	public StateModel createNewStateModel(PartitionId partitionId) {
-		return new HelloWorldStateModel(partitionId);
-	}
-}