Repository: brooklyn-server
Updated Branches:
  refs/heads/master 3679b0127 -> 16ebd0c61


Remove lock on DSL objects - they are stateless

The same DSL object could be used in mutliple entities created from the same 
spec, so one entity resolving would block the rest even if they already have 
the values. The objects are stateless so need for a lock.


Project: http://git-wip-us.apache.org/repos/asf/brooklyn-server/repo
Commit: http://git-wip-us.apache.org/repos/asf/brooklyn-server/commit/9c0b1f83
Tree: http://git-wip-us.apache.org/repos/asf/brooklyn-server/tree/9c0b1f83
Diff: http://git-wip-us.apache.org/repos/asf/brooklyn-server/diff/9c0b1f83

Branch: refs/heads/master
Commit: 9c0b1f837499c8cae60e0e0cc63a12c6491ae9c9
Parents: 1de3fb8
Author: Svetoslav Neykov <svetoslav.ney...@cloudsoftcorp.com>
Authored: Thu Oct 13 19:42:26 2016 +0300
Committer: Svetoslav Neykov <svetoslav.ney...@cloudsoftcorp.com>
Committed: Thu Oct 13 19:45:14 2016 +0300

----------------------------------------------------------------------
 .../spi/dsl/BrooklynDslDeferredSupplier.java    |  83 +++-----
 .../spi/dsl/methods/BrooklynDslCommon.java      |   6 +-
 .../camp/brooklyn/EntitiesYamlTest.java         |  16 +-
 .../brooklyn/camp/brooklyn/dsl/DslTest.java     | 206 +++++++++++++++++++
 .../brooklyn/util/core/task/TasksTest.java      |   2 +-
 .../util/core/task/ValueResolverTest.java       |  20 +-
 6 files changed, 253 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c0b1f83/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/BrooklynDslDeferredSupplier.java
----------------------------------------------------------------------
diff --git 
a/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/BrooklynDslDeferredSupplier.java
 
b/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/BrooklynDslDeferredSupplier.java
index 48a0283..4e302b3 100644
--- 
a/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/BrooklynDslDeferredSupplier.java
+++ 
b/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/BrooklynDslDeferredSupplier.java
@@ -18,10 +18,8 @@
  */
 package org.apache.brooklyn.camp.brooklyn.spi.dsl;
 
-import java.io.IOException;
-import java.io.ObjectInputStream;
 import java.io.Serializable;
-import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.ExecutionException;
 
 import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.mgmt.ExecutionContext;
@@ -44,26 +42,27 @@ import org.slf4j.LoggerFactory;
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
 
-/** provide an object suitable to resolve chained invocations in a parsed YAML 
/ Deployment Plan DSL,
+/**
+ * Provide an object suitable to resolve chained invocations in a parsed YAML 
/ Deployment Plan DSL,
  * which also implements {@link DeferredSupplier} so that they can be resolved 
when needed
  * (e.g. when entity-lookup and execution contexts are available).
  * <p>
- * implementations of this abstract class are expected to be immutable,
- * as instances must support usage in multiple {@link Assembly} instances 
- * created from a single {@link AssemblyTemplate}  
+ * Implementations of this abstract class are expected to be immutable and 
thread safe,
+ * as instances must support usage in multiple {@link Assembly} instances
+ * created from a single {@link AssemblyTemplate}. The object can be used in 
parallel
+ * from multiple threads and no locking is done as all extending objects are 
assumed to be stateless.
  * <p>
- * subclasses which return a deferred value are typically only
+ * Subclasses which return a deferred value are typically only
  * resolvable in the context of a {@link Task} on an {@link Entity}; 
  * these should be only used as the value of a {@link ConfigKey} set in the 
YAML,
- * and should not accessed until after the components / entities are created 
+ * and should not accessed until after the components / entities are created
  * and are being started.
  * (TODO the precise semantics of this are under development.)
- * 
+ * <p>
  * The threading model is that only one thread can call {@link #get()} at a 
time. An interruptible
  * lock is obtained using {@link #lock} for the duration of that method. It is 
important to not
  * use {@code synchronized} because that is not interruptible - if someone 
tries to get the value
  * and interrupts after a short wait, then we must release the lock 
immediately and return.
- * <p>
  **/
 public abstract class BrooklynDslDeferredSupplier<T> implements 
DeferredSupplier<T>, TaskFactory<Task<T>>, Serializable {
 
@@ -71,15 +70,6 @@ public abstract class BrooklynDslDeferredSupplier<T> 
implements DeferredSupplier
 
     private static final Logger log = 
LoggerFactory.getLogger(BrooklynDslDeferredSupplier.class);
 
-    /**
-     * Lock to be used, rather than {@code synchronized} blocks, for anything 
long-running.
-     * Use {@link #getLock()} rather than this field directly, to ensure it is 
reinitialised 
-     * after rebinding.
-     * 
-     * @see https://issues.apache.org/jira/browse/BROOKLYN-214
-     */
-    private transient ReentrantLock lock;
-    
     // TODO json of this object should *be* this, not wrapped this 
($brooklyn:literal is a bit of a hack, though it might work!)
     @JsonInclude
     @JsonProperty(value="$brooklyn:literal")
@@ -89,7 +79,6 @@ public abstract class BrooklynDslDeferredSupplier<T> 
implements DeferredSupplier
     public BrooklynDslDeferredSupplier() {
         PlanInterpretationNode sourceNode = 
BrooklynDslInterpreter.currentNode();
         dsl = sourceNode!=null ? sourceNode.getOriginalValue() : null;
-        lock = new ReentrantLock();
     }
     
     /** returns the current entity; for use in implementations of {@link 
#get()} */
@@ -107,46 +96,28 @@ public abstract class BrooklynDslDeferredSupplier<T> 
implements DeferredSupplier
 
     @Override
     public final T get() {
-        try {
-            getLock().lockInterruptibly();
-        } catch (InterruptedException e) {
-            throw Exceptions.propagate(e);
+        if (log.isDebugEnabled())
+            log.debug("Queuing task to resolve "+dsl+", called by 
"+Tasks.current());
+
+        EntityInternal entity = (EntityInternal) 
BrooklynTaskTags.getTargetOrContextEntity(Tasks.current());
+        ExecutionContext exec =
+                (entity != null) ? entity.getExecutionContext()
+                                 : 
BasicExecutionContext.getCurrentExecutionContext();
+        if (exec == null) {
+            throw new IllegalStateException("No execution context available to 
resolve " + dsl);
         }
-        
-        try {
-            if (log.isDebugEnabled())
-                log.debug("Queuing task to resolve "+dsl+", called by 
"+Tasks.current());
-
-            EntityInternal entity = (EntityInternal) 
BrooklynTaskTags.getTargetOrContextEntity(Tasks.current());
-            ExecutionContext exec =
-                    (entity != null) ? entity.getExecutionContext()
-                                     : 
BasicExecutionContext.getCurrentExecutionContext();
-            if (exec == null) {
-                throw new IllegalStateException("No execution context 
available to resolve " + dsl);
-            }
-
-            Task<T> task = newTask();
-            T result = exec.submit(task).get();
-
-            if (log.isDebugEnabled())
-                log.debug("Resolved "+result+" from "+dsl);
-            return result;
 
-        } catch (Exception e) {
+        Task<T> task = newTask();
+        T result;
+        try {
+            result = exec.submit(task).get();
+        } catch (InterruptedException | ExecutionException e) {
             throw Exceptions.propagate(e);
-        } finally {
-            getLock().unlock();
         }
-    }
 
-    // Use this method, rather than the direct field, to ensure it is 
initialised after rebinding.
-    protected ReentrantLock getLock() {
-        synchronized (this) {
-            if (lock == null) {
-                lock = new ReentrantLock();
-            }
-        }
-        return lock;
+        if (log.isDebugEnabled())
+            log.debug("Resolved "+result+" from "+dsl);
+        return result;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c0b1f83/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/BrooklynDslCommon.java
----------------------------------------------------------------------
diff --git 
a/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/BrooklynDslCommon.java
 
b/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/BrooklynDslCommon.java
index f579507..a4bedf7 100644
--- 
a/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/BrooklynDslCommon.java
+++ 
b/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/BrooklynDslCommon.java
@@ -52,11 +52,9 @@ import org.apache.brooklyn.util.collections.MutableMap;
 import org.apache.brooklyn.util.core.ClassLoaderUtils;
 import org.apache.brooklyn.util.core.config.ConfigBag;
 import org.apache.brooklyn.util.core.flags.FlagUtils;
-import org.apache.brooklyn.util.core.flags.TypeCoercions;
 import org.apache.brooklyn.util.core.task.Tasks;
 import org.apache.brooklyn.util.exceptions.Exceptions;
 import org.apache.brooklyn.util.javalang.Reflections;
-import org.apache.brooklyn.util.javalang.coerce.ClassCoercionException;
 import org.apache.brooklyn.util.text.StringEscapes.JavaStringEscapes;
 import org.apache.brooklyn.util.text.Strings;
 import org.apache.commons.beanutils.BeanUtils;
@@ -284,6 +282,8 @@ public class BrooklynDslCommon {
 
     protected static class DslRegexReplacement extends 
BrooklynDslDeferredSupplier<String> {
 
+        private static final long serialVersionUID = 737189899361183341L;
+
         private Object source;
         private Object pattern;
         private Object replacement;
@@ -586,6 +586,8 @@ public class BrooklynDslCommon {
 
         protected static class DslRegexReplacer extends 
BrooklynDslDeferredSupplier<Function<String, String>> {
 
+            private static final long serialVersionUID = -2900037495440842269L;
+
             private Object pattern;
             private Object replacement;
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c0b1f83/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/EntitiesYamlTest.java
----------------------------------------------------------------------
diff --git 
a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/EntitiesYamlTest.java
 
b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/EntitiesYamlTest.java
index f7b13b3..bdb331e 100644
--- 
a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/EntitiesYamlTest.java
+++ 
b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/EntitiesYamlTest.java
@@ -420,24 +420,24 @@ public class EntitiesYamlTest extends AbstractYamlTest {
         final Entity app = 
createAndStartApplication(loadYaml("test-referencing-entities.yaml"));
         waitForApplicationTasks(app);
         
-        Entity root1 = Tasks.resolving(new DslComponent(Scope.ROOT, 
"xxx").newTask(), Entity.class).context( 
((EntityInternal)app).getExecutionContext() ).embedResolutionInTask(true).get();
+        Entity root1 = Tasks.resolving(new DslComponent(Scope.ROOT, 
"xxx").newTask(), Entity.class).context(app).embedResolutionInTask(true).get();
         Assert.assertEquals(root1, app);
         
-        Entity c1 = Tasks.resolving(new DslComponent("c1").newTask(), 
Entity.class).context( ((EntityInternal)app).getExecutionContext() 
).embedResolutionInTask(true).get();
+        Entity c1 = Tasks.resolving(new DslComponent("c1").newTask(), 
Entity.class).context(app).embedResolutionInTask(true).get();
         Assert.assertEquals(c1, 
Iterables.getOnlyElement(Entities.descendantsAndSelf(app, 
EntityPredicates.displayNameEqualTo("child 1"))));
         
-        Entity e1 = Tasks.resolving(new DslComponent(Scope.PARENT, 
"xxx").newTask(), Entity.class).context( 
((EntityInternal)c1).getExecutionContext() ).embedResolutionInTask(true).get();
+        Entity e1 = Tasks.resolving(new DslComponent(Scope.PARENT, 
"xxx").newTask(), Entity.class).context(c1).embedResolutionInTask(true).get();
         Assert.assertEquals(e1, 
Iterables.getOnlyElement(Entities.descendantsAndSelf(app, 
EntityPredicates.displayNameEqualTo("entity 1"))));
         
-        Entity root2 = Tasks.resolving(new DslComponent(Scope.ROOT, 
"xxx").newTask(), Entity.class).context( 
((EntityInternal)c1).getExecutionContext() ).embedResolutionInTask(true).get();
+        Entity root2 = Tasks.resolving(new DslComponent(Scope.ROOT, 
"xxx").newTask(), Entity.class).context(c1).embedResolutionInTask(true).get();
         Assert.assertEquals(root2, app);
         
-        Entity c1a = 
Tasks.resolving(BrooklynDslCommon.descendant("c1").newTask(), 
Entity.class).context( ((EntityInternal)e1).getExecutionContext() 
).embedResolutionInTask(true).get();
+        Entity c1a = 
Tasks.resolving(BrooklynDslCommon.descendant("c1").newTask(), 
Entity.class).context(e1).embedResolutionInTask(true).get();
         Assert.assertEquals(c1a, c1);
-        Entity e1a = 
Tasks.resolving(BrooklynDslCommon.ancestor("e1").newTask(), 
Entity.class).context( ((EntityInternal)c1).getExecutionContext() 
).embedResolutionInTask(true).get();
+        Entity e1a = 
Tasks.resolving(BrooklynDslCommon.ancestor("e1").newTask(), 
Entity.class).context(c1).embedResolutionInTask(true).get();
         Assert.assertEquals(e1a, e1);
         try {
-            Tasks.resolving(BrooklynDslCommon.ancestor("c1").newTask(), 
Entity.class).context( ((EntityInternal)e1).getExecutionContext() 
).embedResolutionInTask(true).get();
+            Tasks.resolving(BrooklynDslCommon.ancestor("c1").newTask(), 
Entity.class).context(e1).embedResolutionInTask(true).get();
             Assert.fail("Should not have found c1 as ancestor of e1");
         } catch (Exception e) { /* expected */ }
     }
@@ -595,7 +595,7 @@ public class EntitiesYamlTest extends AbstractYamlTest {
         return Tasks.resolving(Tasks.<Entity>builder().body(
             
Functionals.callable(Suppliers.compose(EntityFunctions.config(key), 
Suppliers.ofInstance(entity))) ).build())
             .as(Entity.class)
-            .context( ((EntityInternal)entity).getExecutionContext() 
).embedResolutionInTask(true)
+            .context(entity).embedResolutionInTask(true)
             .getMaybe();
     }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c0b1f83/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/dsl/DslTest.java
----------------------------------------------------------------------
diff --git 
a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/dsl/DslTest.java
 
b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/dsl/DslTest.java
new file mode 100644
index 0000000..29cc7d2
--- /dev/null
+++ 
b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/dsl/DslTest.java
@@ -0,0 +1,206 @@
+/*
+ * Copyright 2016 The Apache Software Foundation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.brooklyn.camp.brooklyn.dsl;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.entity.EntitySpec;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.api.sensor.AttributeSensor;
+import org.apache.brooklyn.camp.brooklyn.spi.dsl.BrooklynDslDeferredSupplier;
+import org.apache.brooklyn.camp.brooklyn.spi.dsl.methods.BrooklynDslCommon;
+import org.apache.brooklyn.core.test.BrooklynAppUnitTestSupport;
+import org.apache.brooklyn.core.test.entity.TestApplication;
+import org.apache.brooklyn.core.test.entity.TestEntity;
+import org.apache.brooklyn.util.core.task.Tasks;
+import org.apache.brooklyn.util.core.task.ValueResolver;
+import org.apache.brooklyn.util.guava.Maybe;
+import org.apache.brooklyn.util.text.Identifiers;
+import org.apache.brooklyn.util.time.Duration;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Supplier;
+
+public class DslTest extends BrooklynAppUnitTestSupport {
+
+    private static final int MAX_PARALLEL_RESOLVERS = 50;
+    private static final int RESOLVER_ITERATIONS = 1000;
+
+    @Test
+    public void testAttributeWhenReadyEmptyDoesNotBlock() {
+        BrooklynDslDeferredSupplier<?> dsl = 
BrooklynDslCommon.attributeWhenReady(TestApplication.MY_ATTRIBUTE.getName());
+        Maybe<? super String> actualValue = 
Tasks.resolving(dsl).as(TestEntity.NAME.getType())
+                .context(app)
+                .description("Computing sensor "+TestEntity.NAME+" from "+dsl)
+                .timeout(ValueResolver.REAL_REAL_QUICK_WAIT)
+                .getMaybe();
+        assertTrue(actualValue.isAbsent());
+    }
+
+    @Test
+    public void testAttributeWhenReady() {
+        BrooklynDslDeferredSupplier<?> dsl = 
BrooklynDslCommon.attributeWhenReady(TestEntity.NAME.getName());
+        new AttributeWhenReadyTestWorker(app, TestEntity.NAME, dsl).run();
+    }
+
+    @Test(groups="Integration")
+    public void testAttributeWhenReadyConcurrent() {
+        final BrooklynDslDeferredSupplier<?> dsl = 
BrooklynDslCommon.attributeWhenReady(TestEntity.NAME.getName());
+        runConcurrentWorker(new Supplier<Runnable>() {
+            @Override
+            public Runnable get() {
+                return new AttributeWhenReadyTestWorker(app, TestEntity.NAME, 
dsl);
+            }
+        });
+    }
+
+    @Test
+    public void testSelf() {
+        BrooklynDslDeferredSupplier<?> dsl = BrooklynDslCommon.self();
+        new SelfTestWorker(app, dsl).run();
+    }
+
+    @Test(groups="Integration")
+    public void testSelfConcurrent() {
+        final BrooklynDslDeferredSupplier<?> dsl = BrooklynDslCommon.self();
+        runConcurrentWorker(new Supplier<Runnable>() {
+            @Override
+            public Runnable get() {
+                return new SelfTestWorker(app, dsl);
+            }
+        });
+    }
+
+    @Test
+    public void testParent() {
+        BrooklynDslDeferredSupplier<?> dsl = BrooklynDslCommon.parent();
+        new ParentTestWorker(app, dsl).run();
+    }
+
+    @Test(groups="Integration")
+    public void testParentConcurrent() {
+        final BrooklynDslDeferredSupplier<?> dsl = BrooklynDslCommon.parent();
+        runConcurrentWorker(new Supplier<Runnable>() {
+            @Override
+            public Runnable get() {
+                return new ParentTestWorker(app, dsl);
+            }
+        });
+    }
+
+    public void runConcurrentWorker(Supplier<Runnable> taskSupplier) {
+        Collection<Task<?>> results = new ArrayList<>();
+        for (int i = 0; i < MAX_PARALLEL_RESOLVERS; i++) {
+            Task<?> result = 
mgmt.getExecutionManager().submit(taskSupplier.get());
+            results.add(result);
+        }
+        for (Task<?> result : results) {
+            result.getUnchecked();
+        }
+    }
+    
+    private static class DslTestWorker implements Runnable {
+        protected TestApplication parent;
+        protected BrooklynDslDeferredSupplier<?> dsl;
+        protected EntitySpec<TestEntity> childSpec = 
EntitySpec.create(TestEntity.class);
+        protected Class<?> type;
+
+        public DslTestWorker(TestApplication parent, 
BrooklynDslDeferredSupplier<?> dsl, Class<?> type) {
+            this.parent = parent;
+            this.dsl = dsl;
+            this.type = type;
+        }
+
+        @Override
+        public void run() {
+            TestEntity entity = parent.createAndManageChild(childSpec);
+            for (int i = 0; i < RESOLVER_ITERATIONS; i++) {
+                preResolve(entity);
+                Maybe<?> actualValue = Tasks.resolving(dsl).as(type)
+                        .context(entity)
+                        .description("Computing sensor "+type+" from "+dsl)
+                        .timeout(Duration.ONE_MINUTE)
+                        .getMaybe();
+                postResolve(actualValue);
+            }
+        }
+
+        protected void preResolve(TestEntity entity) {
+        }
+
+        protected void postResolve(Maybe<?> actualValue) {
+        }
+    }
+
+    private static class AttributeWhenReadyTestWorker extends DslTestWorker {
+        private AttributeSensor<String> sensor;
+        private String expectedValue;
+
+        public AttributeWhenReadyTestWorker(TestApplication parent, 
AttributeSensor<String> sensor, BrooklynDslDeferredSupplier<?> dsl) {
+            super(parent, dsl, sensor.getType());
+            this.sensor = sensor;
+        }
+
+        @Override
+        protected void preResolve(TestEntity entity) {
+            expectedValue = Identifiers.makeRandomId(10);
+            entity.sensors().set(sensor, expectedValue);
+        }
+
+
+        @Override
+        protected void postResolve(Maybe<?> actualValue) {
+            assertEquals(actualValue.get(), expectedValue);
+        }
+
+    }
+
+    private static class SelfTestWorker extends DslTestWorker {
+        private TestEntity entity;
+
+        public SelfTestWorker(TestApplication parent, 
BrooklynDslDeferredSupplier<?> dsl) {
+            super(parent, dsl, Entity.class);
+        }
+
+        @Override
+        protected void preResolve(TestEntity entity) {
+            this.entity = entity;
+        }
+
+        @Override
+        protected void postResolve(Maybe<?> actualValue) {
+            assertEquals(actualValue.get(), entity);
+        }
+
+    }
+
+    private static class ParentTestWorker extends DslTestWorker {
+        public ParentTestWorker(TestApplication parent, 
BrooklynDslDeferredSupplier<?> dsl) {
+            super(parent, dsl, Entity.class);
+        }
+
+        @Override
+        protected void postResolve(Maybe<?> actualValue) {
+            assertEquals(actualValue.get(), parent);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c0b1f83/core/src/test/java/org/apache/brooklyn/util/core/task/TasksTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/brooklyn/util/core/task/TasksTest.java 
b/core/src/test/java/org/apache/brooklyn/util/core/task/TasksTest.java
index af4bb93..074c14a 100644
--- a/core/src/test/java/org/apache/brooklyn/util/core/task/TasksTest.java
+++ b/core/src/test/java/org/apache/brooklyn/util/core/task/TasksTest.java
@@ -119,7 +119,7 @@ public class TasksTest extends BrooklynAppUnitTestSupport {
     public void testErrorsResolvingPropagatesOrSwallowedAllCorrectly() throws 
Exception {
         app.config().set(TestEntity.CONF_OBJECT, 
ValueResolverTest.newThrowTask(Duration.ZERO));
         Task<Object> t = 
Tasks.builder().body(Functionals.callable(EntityFunctions.config(TestEntity.CONF_OBJECT),
 app)).build();
-        ValueResolver<Object> v = 
Tasks.resolving(t).as(Object.class).context(app.getExecutionContext());
+        ValueResolver<Object> v = 
Tasks.resolving(t).as(Object.class).context(app);
         
         ValueResolverTest.assertThrowsOnMaybe(v);
         ValueResolverTest.assertThrowsOnGet(v);

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c0b1f83/core/src/test/java/org/apache/brooklyn/util/core/task/ValueResolverTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/brooklyn/util/core/task/ValueResolverTest.java 
b/core/src/test/java/org/apache/brooklyn/util/core/task/ValueResolverTest.java
index 7eb8926..6c5e990 100644
--- 
a/core/src/test/java/org/apache/brooklyn/util/core/task/ValueResolverTest.java
+++ 
b/core/src/test/java/org/apache/brooklyn/util/core/task/ValueResolverTest.java
@@ -20,11 +20,8 @@ package org.apache.brooklyn.util.core.task;
 
 import java.util.concurrent.Callable;
 
-import org.apache.brooklyn.api.mgmt.ExecutionContext;
 import org.apache.brooklyn.api.mgmt.Task;
 import org.apache.brooklyn.core.test.BrooklynAppUnitTestSupport;
-import org.apache.brooklyn.util.core.task.Tasks;
-import org.apache.brooklyn.util.core.task.ValueResolver;
 import org.apache.brooklyn.util.guava.Maybe;
 import org.apache.brooklyn.util.time.Duration;
 import org.apache.brooklyn.util.time.Time;
@@ -38,13 +35,10 @@ import org.testng.annotations.Test;
 @Test
 public class ValueResolverTest extends BrooklynAppUnitTestSupport {
 
-    private ExecutionContext executionContext;
-
     @BeforeMethod(alwaysRun=true)
     @Override
     public void setUp() throws Exception {
         super.setUp();
-        executionContext = app.getExecutionContext();
     }
     
     public static final Task<String> newSleepTask(final Duration timeout, 
final String result) {
@@ -66,18 +60,18 @@ public class ValueResolverTest extends 
BrooklynAppUnitTestSupport {
     }
     
     public void testTimeoutZero() {
-        Maybe<String> result = 
Tasks.resolving(newSleepTask(Duration.TEN_SECONDS, 
"foo")).as(String.class).context(executionContext).timeout(Duration.ZERO).getMaybe();
+        Maybe<String> result = 
Tasks.resolving(newSleepTask(Duration.TEN_SECONDS, 
"foo")).as(String.class).context(app).timeout(Duration.ZERO).getMaybe();
         Assert.assertFalse(result.isPresent());
     }
     
     public void testTimeoutBig() {
-        Maybe<String> result = Tasks.resolving(newSleepTask(Duration.ZERO, 
"foo")).as(String.class).context(executionContext).timeout(Duration.TEN_SECONDS).getMaybe();
+        Maybe<String> result = Tasks.resolving(newSleepTask(Duration.ZERO, 
"foo")).as(String.class).context(app).timeout(Duration.TEN_SECONDS).getMaybe();
         Assert.assertEquals(result.get(), "foo");
     }
 
     public void testNoExecutionContextOnCompleted() {
         Task<String> t = newSleepTask(Duration.ZERO, "foo");
-        executionContext.submit(t).getUnchecked();
+        app.getExecutionContext().submit(t).getUnchecked();
         Maybe<String> result = 
Tasks.resolving(t).as(String.class).timeout(Duration.ZERO).getMaybe();
         Assert.assertEquals(result.get(), "foo");
     }
@@ -106,26 +100,26 @@ public class ValueResolverTest extends 
BrooklynAppUnitTestSupport {
     }
     
     public void testSwallowError() {
-        ValueResolver<String> result = 
Tasks.resolving(newThrowTask(Duration.ZERO)).as(String.class).context(executionContext).swallowExceptions();
+        ValueResolver<String> result = 
Tasks.resolving(newThrowTask(Duration.ZERO)).as(String.class).context(app).swallowExceptions();
         assertMaybeIsAbsent(result);
         assertThrowsOnGet(result);
     }
 
 
     public void testDontSwallowError() {
-        ValueResolver<String> result = 
Tasks.resolving(newThrowTask(Duration.ZERO)).as(String.class).context(executionContext);
+        ValueResolver<String> result = 
Tasks.resolving(newThrowTask(Duration.ZERO)).as(String.class).context(app);
         assertThrowsOnMaybe(result);
         assertThrowsOnGet(result);
     }
 
     public void testDefaultWhenSwallowError() {
-        ValueResolver<String> result = 
Tasks.resolving(newThrowTask(Duration.ZERO)).as(String.class).context(executionContext).swallowExceptions().defaultValue("foo");
+        ValueResolver<String> result = 
Tasks.resolving(newThrowTask(Duration.ZERO)).as(String.class).context(app).swallowExceptions().defaultValue("foo");
         assertMaybeIsAbsent(result);
         Assert.assertEquals(result.get(), "foo");
     }
 
     public void testDefaultBeforeDelayAndError() {
-        ValueResolver<String> result = 
Tasks.resolving(newThrowTask(Duration.TEN_SECONDS)).as(String.class).context(executionContext).timeout(Duration.ZERO).defaultValue("foo");
+        ValueResolver<String> result = 
Tasks.resolving(newThrowTask(Duration.TEN_SECONDS)).as(String.class).context(app).timeout(Duration.ZERO).defaultValue("foo");
         assertMaybeIsAbsent(result);
         Assert.assertEquals(result.get(), "foo");
     }

Reply via email to