flume-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From esam...@apache.org
Subject svn commit: r1165705 - in /incubator/flume/branches/flume-728: flume-ng-core/src/main/java/org/apache/flume/ flume-ng-core/src/main/java/org/apache/flume/lifecycle/ flume-ng-core/src/main/java/org/apache/flume/sink/ flume-ng-core/src/main/java/org/apac...
Date Tue, 06 Sep 2011 15:21:43 GMT
Author: esammer
Date: Tue Sep  6 15:21:42 2011
New Revision: 1165705

URL: http://svn.apache.org/viewvc?rev=1165705&view=rev
Log:
- Removed context param from LifecycleAware methods. Those that want config data should
  implement Configurable from now on.

Modified:
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/EventDrivenSourceRunner.java
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/LogicalNode.java
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleAware.java
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleSupervisor.java
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractEventSink.java
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/PollableSinkRunner.java
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/AbstractSource.java
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSource.java
    incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/PollableSourceRunner.java
    incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/TestLogicalNode.java
    incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleController.java
    incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleSupervisor.java
    incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoggerSink.java
    incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java
    incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestPollableSourceRunner.java
    incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestSequenceGeneratorSource.java
    incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/Application.java
    incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/FlumeNode.java
    incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/nodemanager/DefaultLogicalNodeManager.java
    incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractLogicalNodeManager.java
    incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestDefaultLogicalNodeManager.java
    incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestFlumeNode.java
    incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/source/TestNetcatSource.java

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/EventDrivenSourceRunner.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/EventDrivenSourceRunner.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/EventDrivenSourceRunner.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/EventDrivenSourceRunner.java
Tue Sep  6 15:21:42 2011
@@ -13,14 +13,14 @@ public class EventDrivenSourceRunner imp
   }
 
   @Override
-  public void start(Context context) {
-    source.start(context);
+  public void start() {
+    source.start();
     lifecycleState = LifecycleState.START;
   }
 
   @Override
-  public void stop(Context context) {
-    source.stop(context);
+  public void stop() {
+    source.stop();
     lifecycleState = LifecycleState.STOP;
   }
 

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/LogicalNode.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/LogicalNode.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/LogicalNode.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/LogicalNode.java
Tue Sep  6 15:21:42 2011
@@ -16,19 +16,19 @@ public class LogicalNode implements Life
   }
 
   @Override
-  public void start(Context context) {
+  public void start() {
 
-    sourceRunner.start(context);
-    sinkRunner.start(context);
+    sourceRunner.start();
+    sinkRunner.start();
 
     lifecycleState = LifecycleState.START;
   }
 
   @Override
-  public void stop(Context context) {
+  public void stop() {
 
-    sourceRunner.stop(context);
-    sinkRunner.stop(context);
+    sourceRunner.stop();
+    sinkRunner.stop();
 
     lifecycleState = LifecycleState.STOP;
   }

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleAware.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleAware.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleAware.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleAware.java
Tue Sep  6 15:21:42 2011
@@ -1,6 +1,5 @@
 package org.apache.flume.lifecycle;
 
-import org.apache.flume.Context;
 
 /**
  * <p>
@@ -12,8 +11,8 @@ import org.apache.flume.Context;
  * starting, stopping, and reporting their current state. Additionally, this
  * interface creates a standard method of communicating failure to perform a
  * lifecycle operation to the caller (i.e. via {@link LifecycleException}). It
- * is never considered valid to call {@link #start(Context)} or
- * {@link #stop(Context)} more than once or to call them in the wrong order.
+ * is never considered valid to call {@link #start()} or
+ * {@link #stop()} more than once or to call them in the wrong order.
  * While this is not strictly enforced, it may be in the future.
  * </p>
  * <p>
@@ -73,11 +72,10 @@ public interface LifecycleAware {
    * the return value of {@link #getLifecycleState()} accordingly.
    * </p>
    * 
-   * @param context
    * @throws LifecycleException
    * @throws InterruptedException
    */
-  public void start(Context context);
+  public void start();
 
   /**
    * <p>
@@ -88,11 +86,10 @@ public interface LifecycleAware {
    * the return value of {@link #getLifecycleState()} accordingly.
    * </p>
    * 
-   * @param context
    * @throws LifecycleException
    * @throws InterruptedException
    */
-  public void stop(Context context);
+  public void stop();
 
   /**
    * <p>

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleSupervisor.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleSupervisor.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleSupervisor.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/lifecycle/LifecycleSupervisor.java
Tue Sep  6 15:21:42 2011
@@ -7,7 +7,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.flume.Context;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,7 +34,7 @@ public class LifecycleSupervisor impleme
   }
 
   @Override
-  public synchronized void start(Context context) {
+  public synchronized void start() {
 
     logger.info("Starting lifecycle supervisor {}", Thread.currentThread()
         .getId());
@@ -58,7 +57,7 @@ public class LifecycleSupervisor impleme
   }
 
   @Override
-  public synchronized void stop(Context context) {
+  public synchronized void stop() {
 
     logger.info("Stopping lifecycle supervisor {}", Thread.currentThread()
         .getId());
@@ -80,7 +79,7 @@ public class LifecycleSupervisor impleme
         .entrySet()) {
 
       if (entry.getKey().getLifecycleState().equals(LifecycleState.START)) {
-        entry.getKey().stop(context);
+        entry.getKey().stop();
       }
     }
 
@@ -184,12 +183,10 @@ public class LifecycleSupervisor impleme
                     supervisoree.status.desiredState,
                     supervisoree.status.failures });
 
-        Context context = new Context();
-
         switch (supervisoree.status.desiredState) {
         case START:
           try {
-            lifecycleAware.start(context);
+            lifecycleAware.start();
           } catch (Exception e) {
             logger.error("Unable to start " + lifecycleAware
                 + " - Exception follows.", e);
@@ -198,7 +195,7 @@ public class LifecycleSupervisor impleme
           break;
         case STOP:
           try {
-            lifecycleAware.stop(context);
+            lifecycleAware.stop();
           } catch (Exception e) {
             logger.error("Unable to stop " + lifecycleAware
                 + " - Exception follows.", e);

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractEventSink.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractEventSink.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractEventSink.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/AbstractEventSink.java
Tue Sep  6 15:21:42 2011
@@ -1,7 +1,6 @@
 package org.apache.flume.sink;
 
 import org.apache.flume.Channel;
-import org.apache.flume.Context;
 import org.apache.flume.EventSink;
 import org.apache.flume.lifecycle.LifecycleAware;
 import org.apache.flume.lifecycle.LifecycleState;
@@ -17,12 +16,12 @@ abstract public class AbstractEventSink 
   }
 
   @Override
-  public synchronized void start(Context context) {
+  public synchronized void start() {
     lifecycleState = LifecycleState.START;
   }
 
   @Override
-  public synchronized void stop(Context context) {
+  public synchronized void stop() {
     lifecycleState = LifecycleState.STOP;
   }
 

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/PollableSinkRunner.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/PollableSinkRunner.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/PollableSinkRunner.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/PollableSinkRunner.java
Tue Sep  6 15:21:42 2011
@@ -2,7 +2,6 @@ package org.apache.flume.sink;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.flume.Context;
 import org.apache.flume.CounterGroup;
 import org.apache.flume.EventDeliveryException;
 import org.apache.flume.PollableSink;
@@ -30,7 +29,7 @@ public class PollableSinkRunner implemen
   }
 
   @Override
-  public void start(Context context) {
+  public void start() {
     runner = new PollingRunner();
 
     runner.sink = sink;
@@ -44,7 +43,7 @@ public class PollableSinkRunner implemen
   }
 
   @Override
-  public void stop(Context context) {
+  public void stop() {
 
     if (runnerThread != null) {
       runner.shouldStop.set(true);

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/sink/RollingFileSink.java
Tue Sep  6 15:21:42 2011
@@ -68,9 +68,9 @@ public class RollingFileSink extends Abs
   }
 
   @Override
-  public void start(Context context) {
+  public void start() {
 
-    super.start(context);
+    super.start();
 
     pathController.setBaseDirectory(directory);
 
@@ -165,9 +165,9 @@ public class RollingFileSink extends Abs
   }
 
   @Override
-  public void stop(Context context) {
+  public void stop() {
 
-    super.stop(context);
+    super.stop();
 
     if (outputStream != null) {
       logger.debug("Closing file {}", pathController.getCurrentFile());

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/AbstractSource.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/AbstractSource.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/AbstractSource.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/AbstractSource.java
Tue Sep  6 15:21:42 2011
@@ -1,7 +1,6 @@
 package org.apache.flume.source;
 
 import org.apache.flume.Channel;
-import org.apache.flume.Context;
 import org.apache.flume.EventSource;
 import org.apache.flume.lifecycle.LifecycleState;
 
@@ -16,12 +15,12 @@ abstract public class AbstractSource imp
   }
 
   @Override
-  public void start(Context context) {
+  public void start() {
     lifecycleState = LifecycleState.START;
   }
 
   @Override
-  public void stop(Context context) {
+  public void stop() {
     lifecycleState = LifecycleState.STOP;
   }
 

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSource.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSource.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSource.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/NetcatSource.java
Tue Sep  6 15:21:42 2011
@@ -60,11 +60,11 @@ public class NetcatSource extends Abstra
   }
 
   @Override
-  public void start(Context context) {
+  public void start() {
 
     logger.info("Source starting");
 
-    super.start(context);
+    super.start();
 
     counterGroup.incrementAndGet("open.attempts");
 
@@ -101,10 +101,10 @@ public class NetcatSource extends Abstra
   }
 
   @Override
-  public void stop(Context context) {
+  public void stop() {
     logger.info("Source stopping");
 
-    super.stop(context);
+    super.stop();
 
     acceptThreadShouldStop.set(true);
 

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/PollableSourceRunner.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/PollableSourceRunner.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/PollableSourceRunner.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/main/java/org/apache/flume/source/PollableSourceRunner.java
Tue Sep  6 15:21:42 2011
@@ -2,7 +2,6 @@ package org.apache.flume.source;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.flume.Context;
 import org.apache.flume.CounterGroup;
 import org.apache.flume.PollableSource;
 import org.apache.flume.SourceRunner;
@@ -30,9 +29,9 @@ public class PollableSourceRunner implem
   }
 
   @Override
-  public void start(Context context) {
+  public void start() {
 
-    source.start(context);
+    source.start();
 
     runner = new PollingRunner();
 
@@ -47,7 +46,7 @@ public class PollableSourceRunner implem
   }
 
   @Override
-  public void stop(Context context) {
+  public void stop() {
 
     runner.shouldStop.set(true);
 
@@ -62,7 +61,7 @@ public class PollableSourceRunner implem
       Thread.currentThread().interrupt();
     }
 
-    source.stop(context);
+    source.stop();
 
     lifecycleState = LifecycleState.STOP;
   }

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/TestLogicalNode.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/TestLogicalNode.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/TestLogicalNode.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/TestLogicalNode.java
Tue Sep  6 15:21:42 2011
@@ -30,9 +30,7 @@ public class TestLogicalNode {
 
   @Test
   public void testLifecycle() throws LifecycleException, InterruptedException {
-    Context context = new Context();
-
-    node.start(context);
+    node.start();
     boolean reached = LifecycleController.waitForOneOf(node,
         new LifecycleState[] { LifecycleState.START, LifecycleState.ERROR },
         5000);
@@ -40,7 +38,7 @@ public class TestLogicalNode {
     Assert.assertTrue("Matched a lifecycle state", reached);
     Assert.assertEquals(LifecycleState.START, node.getLifecycleState());
 
-    node.stop(context);
+    node.stop();
     reached = LifecycleController.waitForOneOf(node, new LifecycleState[] {
         LifecycleState.STOP, LifecycleState.ERROR }, 5000);
 
@@ -62,7 +60,6 @@ public class TestLogicalNode {
 
         @Override
         public void run() {
-          Context context = new Context();
           LogicalNode node = new LogicalNode();
 
           node.setName("test-node-" + j);
@@ -70,7 +67,7 @@ public class TestLogicalNode {
           node.setSinkRunner(new EmptySinkRunner());
 
           try {
-            node.start(context);
+            node.start();
 
             boolean reached = LifecycleController.waitForOneOf(node,
                 new LifecycleState[] { LifecycleState.START,
@@ -81,7 +78,7 @@ public class TestLogicalNode {
 
             Thread.sleep(500);
 
-            node.stop(context);
+            node.stop();
             reached = LifecycleController.waitForOneOf(node,
                 new LifecycleState[] { LifecycleState.STOP,
                     LifecycleState.ERROR }, 5000);
@@ -110,12 +107,12 @@ public class TestLogicalNode {
     private LifecycleState lifecycleState = LifecycleState.IDLE;
 
     @Override
-    public void start(Context context) {
+    public void start() {
       lifecycleState = LifecycleState.START;
     }
 
     @Override
-    public void stop(Context context) {
+    public void stop() {
       lifecycleState = LifecycleState.STOP;
     }
 
@@ -131,12 +128,12 @@ public class TestLogicalNode {
     private LifecycleState lifecycleState = LifecycleState.IDLE;
 
     @Override
-    public void start(Context context) {
+    public void start() {
       lifecycleState = LifecycleState.START;
     }
 
     @Override
-    public void stop(Context context) {
+    public void stop() {
       lifecycleState = LifecycleState.STOP;
     }
 

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleController.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleController.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleController.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleController.java
Tue Sep  6 15:21:42 2011
@@ -2,7 +2,6 @@ package org.apache.flume.lifecycle;
 
 import junit.framework.Assert;
 
-import org.apache.flume.Context;
 import org.junit.Test;
 
 public class TestLifecycleController {
@@ -11,12 +10,11 @@ public class TestLifecycleController {
   public void testWaitForState() throws LifecycleException,
       InterruptedException {
 
-    Context context = new Context();
     LifecycleAware delegate = new SleeperLifecycleDelegate();
 
     Assert.assertTrue(delegate.getLifecycleState().equals(LifecycleState.IDLE));
 
-    delegate.start(context);
+    delegate.start();
 
     boolean reached = LifecycleController.waitForState(delegate,
         LifecycleState.START, 2000);
@@ -24,7 +22,7 @@ public class TestLifecycleController {
     Assert.assertEquals(true, reached);
     Assert.assertEquals(LifecycleState.START, delegate.getLifecycleState());
 
-    delegate.stop(context);
+    delegate.stop();
 
     reached = LifecycleController.waitForState(delegate, LifecycleState.STOP,
         2000);
@@ -32,7 +30,7 @@ public class TestLifecycleController {
     Assert.assertEquals(true, reached);
     Assert.assertEquals(LifecycleState.STOP, delegate.getLifecycleState());
 
-    delegate.start(context);
+    delegate.start();
 
     reached = LifecycleController.waitForState(delegate, LifecycleState.IDLE,
         500);
@@ -46,12 +44,11 @@ public class TestLifecycleController {
   public void testWaitForOneOf() throws LifecycleException,
       InterruptedException {
 
-    Context context = new Context();
     LifecycleAware delegate = new SleeperLifecycleDelegate();
 
     Assert.assertEquals(LifecycleState.IDLE, delegate.getLifecycleState());
 
-    delegate.start(context);
+    delegate.start();
 
     boolean reached = LifecycleController.waitForOneOf(delegate,
         new LifecycleState[] { LifecycleState.STOP, LifecycleState.START },
@@ -72,7 +69,7 @@ public class TestLifecycleController {
     }
 
     @Override
-    public void start(Context context) {
+    public void start() {
       try {
         Thread.sleep(sleepTime);
       } catch (InterruptedException e) {
@@ -83,7 +80,7 @@ public class TestLifecycleController {
     }
 
     @Override
-    public void stop(Context context) {
+    public void stop() {
       try {
         Thread.sleep(sleepTime);
       } catch (InterruptedException e) {

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleSupervisor.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleSupervisor.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleSupervisor.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/lifecycle/TestLifecycleSupervisor.java
Tue Sep  6 15:21:42 2011
@@ -1,6 +1,5 @@
 package org.apache.flume.lifecycle;
 
-import org.apache.flume.Context;
 import org.apache.flume.CounterGroup;
 import org.apache.flume.lifecycle.LifecycleSupervisor.SupervisorPolicy;
 import org.junit.Assert;
@@ -18,17 +17,13 @@ public class TestLifecycleSupervisor {
 
   @Test
   public void testLifecycle() throws LifecycleException, InterruptedException {
-    Context context = new Context();
-
-    supervisor.start(context);
-    supervisor.stop(context);
+    supervisor.start();
+    supervisor.stop();
   }
 
   @Test
   public void testSupervise() throws LifecycleException, InterruptedException {
-    Context context = new Context();
-
-    supervisor.start(context);
+    supervisor.start();
 
     /* Attempt to supervise a known-to-fail config. */
     /*
@@ -51,25 +46,24 @@ public class TestLifecycleSupervisor {
 
     Thread.sleep(5000);
 
-    supervisor.stop(context);
+    supervisor.stop();
   }
 
   @Test
   public void testSuperviseBroken() throws LifecycleException,
       InterruptedException {
-    Context context = new Context();
 
-    supervisor.start(context);
+    supervisor.start();
 
     /* Attempt to supervise a known-to-fail config. */
     LifecycleAware node = new LifecycleAware() {
 
       @Override
-      public void stop(Context context) {
+      public void stop() {
       }
 
       @Override
-      public void start(Context context) {
+      public void start() {
         throw new NullPointerException("Boom!");
       }
 
@@ -84,16 +78,14 @@ public class TestLifecycleSupervisor {
 
     Thread.sleep(5000);
 
-    supervisor.stop(context);
+    supervisor.stop();
   }
 
   @Test
   public void testSuperviseSupervisor() throws LifecycleException,
       InterruptedException {
 
-    Context context = new Context();
-
-    supervisor.start(context);
+    supervisor.start();
 
     LifecycleSupervisor supervisor2 = new LifecycleSupervisor();
 
@@ -107,15 +99,14 @@ public class TestLifecycleSupervisor {
 
     Thread.sleep(10000);
 
-    supervisor.stop(context);
+    supervisor.stop();
   }
 
   @Test
   public void testUnsuperviseServce() throws LifecycleException,
       InterruptedException {
-    Context context = new Context();
 
-    supervisor.start(context);
+    supervisor.start();
 
     LifecycleAware service = new CountingLifecycleAware();
     SupervisorPolicy policy = new SupervisorPolicy.OnceOnlyPolicy();
@@ -123,16 +114,14 @@ public class TestLifecycleSupervisor {
     supervisor.supervise(service, policy, LifecycleState.START);
     supervisor.unsupervise(service);
 
-    service.stop(context);
+    service.stop();
 
-    supervisor.stop(context);
+    supervisor.stop();
   }
 
   @Test
   public void testStopServce() throws LifecycleException, InterruptedException {
-    Context context = new Context();
-
-    supervisor.start(context);
+    supervisor.start();
 
     CountingLifecycleAware service = new CountingLifecycleAware();
     SupervisorPolicy policy = new SupervisorPolicy.OnceOnlyPolicy();
@@ -154,7 +143,7 @@ public class TestLifecycleSupervisor {
     Assert.assertEquals(Long.valueOf(1), service.counterGroup.get("start"));
     Assert.assertEquals(Long.valueOf(1), service.counterGroup.get("stop"));
 
-    supervisor.stop(context);
+    supervisor.stop();
   }
 
   public static class CountingLifecycleAware implements LifecycleAware {
@@ -168,7 +157,7 @@ public class TestLifecycleSupervisor {
     }
 
     @Override
-    public void start(Context context) {
+    public void start() {
 
       counterGroup.incrementAndGet("start");
 
@@ -176,7 +165,7 @@ public class TestLifecycleSupervisor {
     }
 
     @Override
-    public void stop(Context context) {
+    public void stop() {
 
       counterGroup.incrementAndGet("stop");
 

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoggerSink.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoggerSink.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoggerSink.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestLoggerSink.java
Tue Sep  6 15:21:42 2011
@@ -32,7 +32,7 @@ public class TestLoggerSink {
     Configurables.configure(sink, context);
 
     sink.setChannel(channel);
-    sink.start(context);
+    sink.start();
 
     for (int i = 0; i < 10; i++) {
       Event event = EventBuilder.withBody(("Test " + i).getBytes());
@@ -41,7 +41,7 @@ public class TestLoggerSink {
       sink.process();
     }
 
-    sink.stop(context);
+    sink.stop();
   }
 
 }

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/sink/TestRollingFileSink.java
Tue Sep  6 15:21:42 2011
@@ -50,8 +50,8 @@ public class TestRollingFileSink {
 
     Configurables.configure(sink, context);
 
-    sink.start(context);
-    sink.stop(context);
+    sink.start();
+    sink.stop();
   }
 
   @Test
@@ -68,7 +68,7 @@ public class TestRollingFileSink {
     Channel channel = new MemoryChannel();
 
     sink.setChannel(channel);
-    sink.start(context);
+    sink.start();
 
     for (int i = 0; i < 10; i++) {
       Event event = new SimpleEvent();
@@ -81,7 +81,7 @@ public class TestRollingFileSink {
       Thread.sleep(500);
     }
 
-    sink.stop(context);
+    sink.stop();
 
     for (String file : sink.getDirectory().list()) {
       BufferedReader reader = new BufferedReader(new FileReader(new File(

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestPollableSourceRunner.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestPollableSourceRunner.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestPollableSourceRunner.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestPollableSourceRunner.java
Tue Sep  6 15:21:42 2011
@@ -3,7 +3,6 @@ package org.apache.flume.source;
 import java.util.concurrent.CountDownLatch;
 
 import org.apache.flume.Channel;
-import org.apache.flume.Context;
 import org.apache.flume.Event;
 import org.apache.flume.EventDeliveryException;
 import org.apache.flume.PollableSource;
@@ -55,12 +54,12 @@ public class TestPollableSourceRunner {
       }
 
       @Override
-      public void start(Context context) {
+      public void start() {
         // Unused.
       }
 
       @Override
-      public void stop(Context context) {
+      public void stop() {
         // Unused.
       }
 
@@ -72,14 +71,12 @@ public class TestPollableSourceRunner {
 
     };
 
-    Context context = new Context();
-
     sourceRunner.setSource(source);
-    sourceRunner.start(context);
+    sourceRunner.start();
 
     latch.await();
 
-    sourceRunner.stop(context);
+    sourceRunner.stop();
   }
 
 }

Modified: incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestSequenceGeneratorSource.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestSequenceGeneratorSource.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestSequenceGeneratorSource.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-core/src/test/java/org/apache/flume/source/TestSequenceGeneratorSource.java
Tue Sep  6 15:21:42 2011
@@ -55,7 +55,7 @@ public class TestSequenceGeneratorSource
     Configurables.configure(source, context);
 
     source.setChannel(channel);
-    source.start(context);
+    source.start();
 
     for (long i = 0; i < 100; i++) {
       source.process();
@@ -65,7 +65,7 @@ public class TestSequenceGeneratorSource
           new String(event.getBody()).getBytes());
     }
 
-    source.stop(context);
+    source.stop();
   }
 
 }

Modified: incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/Application.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/Application.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/Application.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/Application.java
Tue Sep  6 15:21:42 2011
@@ -143,7 +143,6 @@ public class Application {
   public void run() throws LifecycleException, InterruptedException,
       InstantiationException {
 
-    final Context context = new Context();
     final FlumeNode node = new FlumeNode();
     NodeManager nodeManager = new DefaultLogicalNodeManager();
 
@@ -154,12 +153,12 @@ public class Application {
 
       @Override
       public void run() {
-        node.stop(context);
+        node.stop();
       }
 
     });
 
-    node.start(context);
+    node.start();
     LifecycleController.waitForOneOf(node, LifecycleState.START_OR_ERROR);
 
     if (node.getLifecycleState().equals(LifecycleState.START)) {

Modified: incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/FlumeNode.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/FlumeNode.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/FlumeNode.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/FlumeNode.java
Tue Sep  6 15:21:42 2011
@@ -1,6 +1,5 @@
 package org.apache.flume.node;
 
-import org.apache.flume.Context;
 import org.apache.flume.lifecycle.LifecycleAware;
 import org.apache.flume.lifecycle.LifecycleState;
 import org.apache.flume.lifecycle.LifecycleSupervisor;
@@ -25,13 +24,13 @@ public class FlumeNode implements Lifecy
   }
 
   @Override
-  public void start(Context context) {
+  public void start() {
 
     Preconditions.checkState(name != null, "Node name can not be null");
     Preconditions.checkState(nodeManager != null,
         "Node manager can not be null");
 
-    supervisor.start(context);
+    supervisor.start();
 
     logger.info("Flume node starting - {}", name);
 
@@ -42,11 +41,11 @@ public class FlumeNode implements Lifecy
   }
 
   @Override
-  public void stop(Context context) {
+  public void stop() {
 
     logger.info("Flume node stopping - {}", name);
 
-    supervisor.stop(context);
+    supervisor.stop();
 
     lifecycleState = LifecycleState.STOP;
   }

Modified: incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/nodemanager/DefaultLogicalNodeManager.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/nodemanager/DefaultLogicalNodeManager.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/nodemanager/DefaultLogicalNodeManager.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-node/src/main/java/org/apache/flume/node/nodemanager/DefaultLogicalNodeManager.java
Tue Sep  6 15:21:42 2011
@@ -1,13 +1,12 @@
 package org.apache.flume.node.nodemanager;
 
-import org.apache.flume.Context;
 import org.apache.flume.EventSink;
 import org.apache.flume.EventSource;
 import org.apache.flume.LogicalNode;
 import org.apache.flume.PollableSource;
 import org.apache.flume.SinkFactory;
-import org.apache.flume.SourceRunner;
 import org.apache.flume.SourceFactory;
+import org.apache.flume.SourceRunner;
 import org.apache.flume.lifecycle.LifecycleState;
 import org.apache.flume.lifecycle.LifecycleSupervisor;
 import org.apache.flume.lifecycle.LifecycleSupervisor.SupervisorPolicy;
@@ -116,11 +115,11 @@ public class DefaultLogicalNodeManager e
   }
 
   @Override
-  public void start(Context context) {
+  public void start() {
 
     logger.info("Node manager starting");
 
-    nodeSupervisor.start(context);
+    nodeSupervisor.start();
 
     logger.debug("Node manager started");
 
@@ -128,11 +127,11 @@ public class DefaultLogicalNodeManager e
   }
 
   @Override
-  public void stop(Context context) {
+  public void stop() {
 
     logger.info("Node manager stopping");
 
-    nodeSupervisor.stop(context);
+    nodeSupervisor.stop();
 
     logger.debug("Node manager stopped");
 

Modified: incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractLogicalNodeManager.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractLogicalNodeManager.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractLogicalNodeManager.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestAbstractLogicalNodeManager.java
Tue Sep  6 15:21:42 2011
@@ -2,7 +2,6 @@ package org.apache.flume.node;
 
 import junit.framework.Assert;
 
-import org.apache.flume.Context;
 import org.apache.flume.LogicalNode;
 import org.apache.flume.channel.MemoryChannel;
 import org.apache.flume.lifecycle.LifecycleController;
@@ -32,10 +31,10 @@ public class TestAbstractLogicalNodeMana
       private LifecycleState lifecycleState = LifecycleState.IDLE;
 
       @Override
-      public void stop(Context context) {
+      public void stop() {
 
         for (LogicalNode node : getNodes()) {
-          node.stop(context);
+          node.stop();
 
           boolean reached = false;
 
@@ -61,10 +60,10 @@ public class TestAbstractLogicalNodeMana
       }
 
       @Override
-      public void start(Context context) {
+      public void start() {
 
         for (LogicalNode node : getNodes()) {
-          node.start(context);
+          node.start();
 
           boolean reached = false;
 
@@ -99,16 +98,15 @@ public class TestAbstractLogicalNodeMana
   @Test
   public void testEmptyLifecycle() throws LifecycleException,
       InterruptedException {
-    Context context = new Context();
 
-    nodeManager.start(context);
+    nodeManager.start();
     boolean reached = LifecycleController.waitForOneOf(nodeManager,
         LifecycleState.START_OR_ERROR);
 
     Assert.assertTrue(reached);
     Assert.assertEquals(LifecycleState.START, nodeManager.getLifecycleState());
 
-    nodeManager.stop(context);
+    nodeManager.stop();
     reached = LifecycleController.waitForOneOf(nodeManager,
         LifecycleState.STOP_OR_ERROR);
 
@@ -119,8 +117,6 @@ public class TestAbstractLogicalNodeMana
   @Test
   public void testLifecycle() throws LifecycleException, InterruptedException {
 
-    Context context = new Context();
-
     LogicalNode node = new LogicalNode();
     node.setName("test");
 
@@ -135,14 +131,14 @@ public class TestAbstractLogicalNodeMana
 
     nodeManager.add(node);
 
-    nodeManager.start(context);
+    nodeManager.start();
     boolean reached = LifecycleController.waitForOneOf(nodeManager,
         LifecycleState.START_OR_ERROR);
 
     Assert.assertTrue(reached);
     Assert.assertEquals(LifecycleState.START, nodeManager.getLifecycleState());
 
-    nodeManager.stop(context);
+    nodeManager.stop();
     reached = LifecycleController.waitForOneOf(nodeManager,
         LifecycleState.STOP_OR_ERROR);
 
@@ -172,9 +168,7 @@ public class TestAbstractLogicalNodeMana
     nodeManager.add(node);
 
     for (int i = 0; i < 10; i++) {
-      Context context = new Context();
-
-      nodeManager.start(context);
+      nodeManager.start();
       boolean reached = LifecycleController.waitForOneOf(nodeManager,
           LifecycleState.START_OR_ERROR);
 
@@ -182,7 +176,7 @@ public class TestAbstractLogicalNodeMana
       Assert
           .assertEquals(LifecycleState.START, nodeManager.getLifecycleState());
 
-      nodeManager.stop(context);
+      nodeManager.stop();
       reached = LifecycleController.waitForOneOf(nodeManager,
           LifecycleState.STOP_OR_ERROR);
 

Modified: incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestDefaultLogicalNodeManager.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestDefaultLogicalNodeManager.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestDefaultLogicalNodeManager.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestDefaultLogicalNodeManager.java
Tue Sep  6 15:21:42 2011
@@ -3,7 +3,6 @@ package org.apache.flume.node;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.flume.Context;
 import org.apache.flume.LogicalNode;
 import org.apache.flume.channel.MemoryChannel;
 import org.apache.flume.lifecycle.LifecycleController;
@@ -27,14 +26,12 @@ public class TestDefaultLogicalNodeManag
 
   @Test
   public void testLifecycle() throws LifecycleException, InterruptedException {
-    Context context = new Context();
-
-    nodeManager.start(context);
+    nodeManager.start();
     Assert.assertTrue("Node manager didn't reach START or ERROR",
         LifecycleController.waitForOneOf(nodeManager,
             LifecycleState.START_OR_ERROR, 5000));
 
-    nodeManager.stop(context);
+    nodeManager.stop();
     Assert.assertTrue("Node manager didn't reach STOP or ERROR",
         LifecycleController.waitForOneOf(nodeManager,
             LifecycleState.STOP_OR_ERROR, 5000));
@@ -44,9 +41,7 @@ public class TestDefaultLogicalNodeManag
   public void testLifecycleWithNodes() throws LifecycleException,
       InterruptedException {
 
-    Context context = new Context();
-
-    nodeManager.start(context);
+    nodeManager.start();
     Assert.assertTrue("Node manager didn't reach START or ERROR",
         LifecycleController.waitForOneOf(nodeManager,
             LifecycleState.START_OR_ERROR, 5000));
@@ -68,7 +63,7 @@ public class TestDefaultLogicalNodeManag
 
     Thread.sleep(5000);
 
-    nodeManager.stop(context);
+    nodeManager.stop();
     Assert.assertTrue("Node manager didn't reach STOP or ERROR",
         LifecycleController.waitForOneOf(nodeManager,
             LifecycleState.STOP_OR_ERROR, 5000));
@@ -95,9 +90,7 @@ public class TestDefaultLogicalNodeManag
       testNodes.add(node);
     }
 
-    Context context = new Context();
-
-    nodeManager.start(context);
+    nodeManager.start();
     Assert.assertTrue("Node manager didn't reach START or ERROR",
         LifecycleController.waitForOneOf(nodeManager,
             LifecycleState.START_OR_ERROR, 5000));
@@ -108,7 +101,7 @@ public class TestDefaultLogicalNodeManag
 
     Thread.sleep(5000);
 
-    nodeManager.stop(context);
+    nodeManager.stop();
     Assert.assertTrue("Node manager didn't reach STOP or ERROR",
         LifecycleController.waitForOneOf(nodeManager,
             LifecycleState.STOP_OR_ERROR, 5000));
@@ -134,9 +127,7 @@ public class TestDefaultLogicalNodeManag
       testNodes.add(node);
     }
 
-    Context context = new Context();
-
-    nodeManager.start(context);
+    nodeManager.start();
     Assert.assertTrue("Node manager didn't reach START or ERROR",
         LifecycleController.waitForOneOf(nodeManager,
             LifecycleState.START_OR_ERROR, 5000));
@@ -147,7 +138,7 @@ public class TestDefaultLogicalNodeManag
 
     Thread.sleep(5000);
 
-    nodeManager.stop(context);
+    nodeManager.stop();
     Assert.assertTrue("Node manager didn't reach STOP or ERROR",
         LifecycleController.waitForOneOf(nodeManager,
             LifecycleState.STOP_OR_ERROR, 5000));

Modified: incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestFlumeNode.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestFlumeNode.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestFlumeNode.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/node/TestFlumeNode.java
Tue Sep  6 15:21:42 2011
@@ -1,6 +1,5 @@
 package org.apache.flume.node;
 
-import org.apache.flume.Context;
 import org.apache.flume.LogicalNode;
 import org.apache.flume.lifecycle.LifecycleController;
 import org.apache.flume.lifecycle.LifecycleException;
@@ -24,16 +23,14 @@ public class TestFlumeNode {
 
   @Test
   public void testLifecycle() throws InterruptedException, LifecycleException {
-    Context context = new Context();
-
-    node.start(context);
+    node.start();
     boolean reached = LifecycleController.waitForOneOf(node,
         LifecycleState.START_OR_ERROR, 5000);
 
     Assert.assertTrue("Matched a known state", reached);
     Assert.assertEquals(LifecycleState.START, node.getLifecycleState());
 
-    node.stop(context);
+    node.stop();
     reached = LifecycleController.waitForOneOf(node,
         LifecycleState.STOP_OR_ERROR, 5000);
 
@@ -43,9 +40,7 @@ public class TestFlumeNode {
 
   @Test
   public void testAddNodes() throws InterruptedException, LifecycleException {
-    Context context = new Context();
-
-    node.start(context);
+    node.start();
     boolean reached = LifecycleController.waitForOneOf(node,
         LifecycleState.START_OR_ERROR, 5000);
 
@@ -56,7 +51,7 @@ public class TestFlumeNode {
 
     node.getNodeManager().add(n1);
 
-    node.stop(context);
+    node.stop();
     reached = LifecycleController.waitForOneOf(node,
         LifecycleState.STOP_OR_ERROR, 5000);
 
@@ -74,12 +69,12 @@ public class TestFlumeNode {
     }
 
     @Override
-    public void start(Context context) {
+    public void start() {
       lifecycleState = LifecycleState.START;
     }
 
     @Override
-    public void stop(Context context) {
+    public void stop() {
       lifecycleState = LifecycleState.STOP;
     }
 

Modified: incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/source/TestNetcatSource.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/source/TestNetcatSource.java?rev=1165705&r1=1165704&r2=1165705&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/source/TestNetcatSource.java
(original)
+++ incubator/flume/branches/flume-728/flume-ng-node/src/test/java/org/apache/flume/source/TestNetcatSource.java
Tue Sep  6 15:21:42 2011
@@ -47,7 +47,7 @@ public class TestNetcatSource {
 
     Configurables.configure(source, context);
 
-    source.start(context);
+    source.start();
 
     /* FIXME: Ensure proper send / received semantics. */
 
@@ -88,7 +88,7 @@ public class TestNetcatSource {
       executor.awaitTermination(500, TimeUnit.MILLISECONDS);
     }
 
-    source.stop(context);
+    source.stop();
   }
 
 }



Mime
View raw message