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 2013/11/07 02:19:18 UTC

[10/53] [abbrv] [HELIX-234] Convert-all-Id.x-to-x.from-and-assorted-code-cleanup

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/util/TestZKClientPool.java
----------------------------------------------------------------------
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 14ad709..9afca37 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.I0Itec.zkclient.exception.ZkNoNodeException;
 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java
index cc7c1cb..001f599 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java
@@ -26,7 +26,7 @@ import org.apache.helix.Criteria;
 import org.apache.helix.HelixManager;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.Id;
+import org.apache.helix.api.MessageId;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
 import org.apache.helix.model.Message.MessageType;
@@ -63,7 +63,7 @@ public class BootstrapHandler extends StateModelFactory<StateModel> {
       HelixManager manager = context.getManager();
       ClusterMessagingService messagingService = manager.getMessagingService();
       Message requestBackupUriRequest =
-          new Message(MessageType.USER_DEFINE_MSG, Id.message(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();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-examples/src/main/java/org/apache/helix/examples/BootstrapProcess.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/BootstrapProcess.java b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapProcess.java
index e843d14..e67037a 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/BootstrapProcess.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapProcess.java
@@ -1,24 +1,5 @@
 package org.apache.helix.examples;
 
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT 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.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Date;
@@ -45,7 +26,6 @@ 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;
 
 /**
  * This process does little more than handling the state transition messages.
@@ -83,7 +63,6 @@ public class BootstrapProcess {
 
   // private StateMachineEngine genericStateMachineHandler;
 
-  private String _file = null;
   private StateModelFactory<StateModel> stateModelFactory;
   private final int delay;
 
@@ -155,7 +134,8 @@ public class BootstrapProcess {
           result.getTaskResultMap().put(
               "BOOTSTRAP_URL",
               "http://" + hostName + ":" + port + "/getFile?path=/data/bootstrap/"
-                  + _message.getResourceId().stringify() + "/" + _message.getPartitionId().stringify() + ".tar");
+                  + _message.getResourceId().stringify() + "/"
+                  + _message.getPartitionId().stringify() + ".tar");
 
           result.getTaskResultMap().put("BOOTSTRAP_TIME", "" + new Date().getTime());
         }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-examples/src/main/java/org/apache/helix/examples/ExampleProcess.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/ExampleProcess.java b/helix-examples/src/main/java/org/apache/helix/examples/ExampleProcess.java
index 34a13e5..57c61cd 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/ExampleProcess.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/ExampleProcess.java
@@ -37,7 +37,6 @@ 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;
 
 public class ExampleProcess {
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-examples/src/main/java/org/apache/helix/examples/IdealStateExample.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/IdealStateExample.java b/helix-examples/src/main/java/org/apache/helix/examples/IdealStateExample.java
index 7c5192d..a9183f5 100644
--- a/helix-examples/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 class IdealStateExample {
     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++) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java b/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
index affbea8..b278c39 100644
--- a/helix-examples/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.model.Message;
 import org.apache.helix.participant.statemachine.StateModel;
 import org.apache.helix.participant.statemachine.StateModelFactory;
 
-@SuppressWarnings("rawtypes")
 public class MasterSlaveStateModelFactory extends StateModelFactory<StateModel> {
   int _delay;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java b/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java
index e08394c..7be5ab1 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java
@@ -1,5 +1,6 @@
 package org.apache.helix.examples;
 
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.helix.BaseDataAccessor;
@@ -9,14 +10,16 @@ import org.apache.helix.api.ClusterAccessor;
 import org.apache.helix.api.ClusterConfig;
 import org.apache.helix.api.ClusterId;
 import org.apache.helix.api.FullAutoRebalancerConfig;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.ParticipantConfig;
 import org.apache.helix.api.ParticipantId;
+import org.apache.helix.api.Partition;
+import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.ResourceConfig;
 import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.Scope;
 import org.apache.helix.api.State;
 import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.StateModelDefinitionAccessor;
+import org.apache.helix.api.UserConfig;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
@@ -25,6 +28,8 @@ import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Lists;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -52,27 +57,40 @@ public class NewModelExample {
       LOG.error("USAGE: NewModelExample 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();
-    ClusterId clusterId = Id.cluster("exampleCluster");
+
+    // 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 cluster =
         new ClusterConfig.Builder(clusterId).addResource(resource).addParticipant(participant)
-            .build();
+            .addStateModelDefinition(lockUnlock).userConfig(userConfig).build();
+
+    // set up accessors to work with Zookeeper-persisted data
     int timeOutInSec = Integer.parseInt(System.getProperty(ZKHelixAdmin.CONNECTION_TIMEOUT, "30"));
     ZkClient zkClient = new ZkClient(args[0], timeOutInSec * 1000);
     zkClient.setZkSerializer(new ZNRecordSerializer());
     zkClient.waitUntilConnected(timeOutInSec, TimeUnit.SECONDS);
     BaseDataAccessor<ZNRecord> baseDataAccessor = new ZkBaseDataAccessor<ZNRecord>(zkClient);
     HelixDataAccessor accessor = new ZKHelixDataAccessor(clusterId.stringify(), baseDataAccessor);
-    persistStateModel(lockUnlock, accessor);
-    createCluster(cluster, accessor);
-  }
 
-  private static void persistStateModel(StateModelDefinition stateModelDef,
-      HelixDataAccessor helixAccessor) {
-    StateModelDefinitionAccessor accessor = new StateModelDefinitionAccessor(helixAccessor);
-    accessor.addStateModelDefinition(stateModelDef);
+    // create the cluster
+    createCluster(cluster, accessor);
   }
 
   private static void createCluster(ClusterConfig cluster, HelixDataAccessor helixAccessor) {
@@ -81,19 +99,49 @@ public class NewModelExample {
   }
 
   private static ParticipantConfig getParticipant() {
-    ParticipantId participantId = Id.participant("localhost_0");
+    // identify the participant
+    ParticipantId participantId = ParticipantId.from("localhost_0");
+
+    // create (optional) participant user config properties
+    UserConfig userConfig = new UserConfig(Scope.participant(participantId));
+    List<String> 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);
+        new ParticipantConfig.Builder(participantId).hostName("localhost").port(0)
+            .userConfig(userConfig);
     return participantBuilder.build();
   }
 
   private static ResourceConfig getResource(StateModelDefinition stateModelDef) {
-    ResourceId resourceId = Id.resource("exampleResource");
+    // identify the resource
+    ResourceId resourceId = ResourceId.from("exampleResource");
+
+    // create a partition with no user-defined configuration
+    Partition partition1 = new Partition(PartitionId.from("partition1"));
+
+    // create a partition with (optional) user-defined configuration
+    PartitionId partition2Id = PartitionId.from("partition2");
+    UserConfig partition2Config = new UserConfig(Scope.partition(partition2Id));
+    partition2Config.setSimpleField("sampleString", "partition config");
+    Partition partition2 = new Partition(partition2Id, partition2Config);
+
+    // specify the rebalancer configuration
+    // this resource will be rebalanced in FULL_AUTO mode, so use the FullAutoRebalancerConfig
+    // builder
     FullAutoRebalancerConfig.Builder rebalanceConfigBuilder =
-        new FullAutoRebalancerConfig.Builder(resourceId).replicaCount(3).addPartitions(5)
-            .stateModelDef(stateModelDef.getStateModelDefId());
+        new FullAutoRebalancerConfig.Builder(resourceId).replicaCount(3).addPartition(partition1)
+            .addPartition(partition2).stateModelDef(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).rebalancerConfig(rebalanceConfigBuilder.build());
+        new ResourceConfig.Builder(resourceId).rebalancerConfig(rebalanceConfigBuilder.build())
+            .userConfig(userConfig);
     return resourceBuilder.build();
   }
 
@@ -101,7 +149,7 @@ public class NewModelExample {
     final State LOCKED = State.from("LOCKED");
     final State RELEASED = State.from("RELEASED");
     final State DROPPED = State.from("DROPPED");
-    StateModelDefId stateModelId = Id.stateModelDef("LockUnlock");
+    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)

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java b/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java
index 2ff7c2c..f157aac 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java
@@ -32,7 +32,6 @@ import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.State;
 import org.apache.helix.api.StateModelDefId;
 import org.apache.helix.controller.HelixControllerMain;
@@ -51,7 +50,7 @@ public class Quickstart {
   private static final int NUM_PARTITIONS = 6;
   private static final int NUM_REPLICAS = 2;
 
-  private static final StateModelDefId STATE_MODEL_NAME = Id.stateModelDef("MyStateModel");
+  private static final StateModelDefId STATE_MODEL_NAME = StateModelDefId.from("MyStateModel");
 
   // states
   private static final State SLAVE = State.from("SLAVE");
@@ -96,7 +95,8 @@ public class Quickstart {
 
     // 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.stringify(), "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);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogGenerator.java
----------------------------------------------------------------------
diff --git a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogGenerator.java b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogGenerator.java
index f3ba86c..cdb4cfb 100644
--- a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogGenerator.java
+++ b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogGenerator.java
@@ -22,9 +22,7 @@ package org.apache.helix.filestore;
 import java.io.BufferedOutputStream;
 import java.io.DataOutputStream;
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
-import java.nio.charset.Charset;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogProcessor.java
----------------------------------------------------------------------
diff --git a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogProcessor.java b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogProcessor.java
index c5c169f..ce42ad4 100644
--- a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogProcessor.java
+++ b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogProcessor.java
@@ -19,17 +19,11 @@ package org.apache.helix.filestore;
  * under the License.
  */
 
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.Date;
 import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.commons.io.FileUtils;
-
 /**
  * Processes the change log and invokes rsync for every change on the remote
  * machine

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogReader.java
----------------------------------------------------------------------
diff --git a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogReader.java b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogReader.java
index 34a9d0a..7409f1b 100644
--- a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogReader.java
+++ b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/ChangeLogReader.java
@@ -23,9 +23,7 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
-import java.nio.charset.Charset;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStore.java
----------------------------------------------------------------------
diff --git a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStore.java b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStore.java
index 88ea7a2..6448411 100644
--- a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStore.java
+++ b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStore.java
@@ -19,15 +19,10 @@ package org.apache.helix.filestore;
  * 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.manager.zk.ZKHelixAdmin;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.participant.StateMachineEngine;
 
 public class FileStore {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/Test.java
----------------------------------------------------------------------
diff --git a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/Test.java b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/Test.java
index 7e4dddb..133bd92 100644
--- a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/Test.java
+++ b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/Test.java
@@ -21,13 +21,10 @@ package org.apache.helix.filestore;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Date;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicReference;
 
 public class Test {
   public static void main(String[] args) throws InterruptedException {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/recipes/service-discovery/src/main/java/org/apache/helix/servicediscovery/ServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/recipes/service-discovery/src/main/java/org/apache/helix/servicediscovery/ServiceDiscovery.java b/recipes/service-discovery/src/main/java/org/apache/helix/servicediscovery/ServiceDiscovery.java
index 4642079..670e563 100644
--- a/recipes/service-discovery/src/main/java/org/apache/helix/servicediscovery/ServiceDiscovery.java
+++ b/recipes/service-discovery/src/main/java/org/apache/helix/servicediscovery/ServiceDiscovery.java
@@ -21,7 +21,6 @@ package org.apache.helix.servicediscovery;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;