You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@brooklyn.apache.org by ahgittin <gi...@git.apache.org> on 2015/02/06 13:53:38 UTC

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

GitHub user ahgittin opened a pull request:

    https://github.com/apache/incubator-brooklyn/pull/506

    Active partial rebind allowing catalog item version change dynamically

    No GUI support (yet) for changing versions, but you can test this in groovy console with
    
        mgmt.rebindManager.rebindPartialActive(
            brooklyn.entity.rebind.transformer.CompoundTransformer.builder().changeCatalogItemId(
                "item", "1.0", "item", "2.0" /* old and new catalog item+version */).build(),
            "abcd1234" /* entity id */);


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ahgittin/incubator-brooklyn versioning-switch-on-rebind

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-brooklyn/pull/506.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #506
    
----
commit 1f29f47faf78dddac55f2516b34b57bf18234cbd
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-01-29T11:26:13Z

    Partial rebind - basically working, but for mgmt confusion
    
    makes RebindIteration abstract, with InitialFullRI having the old (full) behaviour,
    and new ActivePartialRebindIteration supporting a subset of nodes;
    minor fixes elsewhere to prevent problems

commit 905604f2641eb5e3fd08c04bf91f118bc3912853
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-03T12:48:22Z

    Partial rebind - using new explicit BrooklynObjectManagementMode

commit 323550cb4acc57ee15d8da982b1cde1748aca63e
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-04T12:33:22Z

    Partial rebind - add test and refactor / tidy of new code

commit fa6b2f6b80f7149f5bdd4e43d09655ce18cc26d5
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-05T18:28:29Z

    Partial rebind - test that version has successfully changed

commit c1ccccce8bbd7acb123ab2eb4991af033c491ac4
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-05T18:36:15Z

    Switch OSGi more entites sayHI effector to include version info
    
    as per @aledsage's suggestion

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24326247
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/persister/XmlMementoSerializer.java ---
    @@ -341,5 +353,151 @@ public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext co
                 return lookupContext.lookupManagementContext();
             }
         }
    +
    +    /** When reading/writing specs, it checks whether there is a catalog item id set and uses it to load */
    +    public class SpecConverter extends ReflectionConverter {
    +        SpecConverter() {
    +            super(xstream.getMapper(), xstream.getReflectionProvider());
    +        }
    +        @Override
    +        public boolean canConvert(@SuppressWarnings("rawtypes") Class type) {
    +            return AbstractBrooklynObjectSpec.class.isAssignableFrom(type);
    +        }
    +        @Override
    +        public void marshal(Object source, HierarchicalStreamWriter writer, MarshallingContext context) {
    +            if (source == null) return;
    +            AbstractBrooklynObjectSpec<?, ?> spec = (AbstractBrooklynObjectSpec<?, ?>) source;
    +            String catalogItemId = spec.getCatalogItemId();
    +            if (Strings.isNonBlank(catalogItemId)) {
    +                // write this field first, so we can peek at it when we read
    +                writer.startNode("catalogItemId");
    +                writer.setValue(catalogItemId);
    +                writer.endNode();
    +                
    +                // we're going to write the catalogItemId field twice :( but that's okay.
    +                // better solution would be to have mark/reset on reader so we can peek for such a field;
    +                // see comment below
    +                super.marshal(source, writer, context);
    +            } else {
    +                super.marshal(source, writer, context);
    +            }
    +        }
    +        @Override
    +        public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            String catalogItemId = null;
    +            instantiateNewInstanceSettingCache(reader, context);
    +            
    +            if (reader instanceof PathTrackingReader) {
    +                // have to assume this is first; there is no mark/reset support on these readers
    +                // (if there were then it would be easier, we could just look for that child anywhere,
    +                // and not need a custom writer!)
    +                if ("catalogItemId".equals( ((PathTrackingReader)reader).peekNextChild() )) {
    +                    // cache the instance
    +                    
    +                    reader.moveDown();
    +                    catalogItemId = reader.getValue();
    +                    reader.moveUp();
    +                }
    +            }
    +            boolean customLoaderSet = false;
    +            try {
    +                if (Strings.isNonBlank(catalogItemId)) {
    +                    if (lookupContext==null) throw new NullPointerException("lookupContext required to load catalog item "+catalogItemId);
    +                    CatalogItem<?, ?> cat = CatalogUtils.getCatalogItemOptionalVersion(lookupContext.lookupManagementContext(), catalogItemId);
    +                    if (cat==null) throw new NoSuchElementException("catalog item: "+catalogItemId);
    +                    BrooklynClassLoadingContext clcNew = CatalogUtils.newClassLoadingContext(lookupContext.lookupManagementContext(), cat);
    +                    pushXstreamCustomClassLoader(clcNew);
    +                    customLoaderSet = true;
    +                }
    +                
    +                AbstractBrooklynObjectSpec<?, ?> result = (AbstractBrooklynObjectSpec<?, ?>) super.unmarshal(reader, context);
    +                // we wrote it twice so this shouldn't be necessary; but if we fix it so we only write once, we'd need this
    +                result.catalogItemId(catalogItemId);
    +                return result;
    +            } finally {
    +                instance = null;
    +                if (customLoaderSet) {
    +                    popXstreamCustomClassLoader();
    +                }
    +            }
    +        }
    +
    +        Object instance;
    +        
    +        @Override
    +        protected Object instantiateNewInstance(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            // the super calls getAttribute which requires that we have not yet done moveDown,
    +            // so we do this earlier and cache it for when we call super.unmarshal
    +            if (instance==null)
    +                throw new IllegalStateException("Instance should be created and cached");
    +            return instance;
    +        }
    +        protected void instantiateNewInstanceSettingCache(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            instance = super.instantiateNewInstance(reader, context);
    +        }
    +    }
    +    
    +    Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    AtomicReference<Thread> xstreamLockOwner = new AtomicReference<Thread>();
    +    int lockCount;
         
    +    /** Must be accompanied by a corresponding {@link #popXstreamCustomClassLoader()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    protected void pushXstreamCustomClassLoader(BrooklynClassLoadingContext clcNew) {
    +        acquireXstreamLock();
    +        BrooklynClassLoadingContext oldClc;
    +        if (!contexts.isEmpty()) {
    +            oldClc = contexts.peek();
    --- End diff --
    
    Looks dangerous; no synchronization on `xstreamLockOwner` so is assuming no other thread will have popped in between the check to `isEmpty` and `peek`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24327857
  
    --- Diff: api/src/main/java/brooklyn/basic/AbstractBrooklynObjectSpec.java ---
    @@ -121,4 +121,21 @@ protected final void checkIsImplementation(Class<?> val, Class<? super T> requir
             if (Modifier.isAbstract(val.getModifiers())) throw new IllegalStateException("Implementation "+val+" is abstract, but must be a non-abstract class");
         }
     
    +    @Override
    +    public boolean equals(Object obj) {
    +        if (obj==null) return false;
    --- End diff --
    
    You're right - it's not braking the official hashCode contract. It would just be wrong to put a spec in a `HashSet` and then modify it.
    
    The spec is used like a builder, so tricky to make it immutable. We could have a `build` method (or `toImmutable` or some such), but I don't think it's a big deal.
    
    Agree moving it to a test utility would be much harder to find (i.e. bad).
    
    I'm happy to leave as-is in this PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
GitHub user ahgittin reopened a pull request:

    https://github.com/apache/incubator-brooklyn/pull/506

    Active partial rebind allowing catalog item version change dynamically

    No GUI support (yet) for changing versions, but you can test this in groovy console with
    
        mgmt.rebindManager.rebindPartialActive(
            brooklyn.entity.rebind.transformer.CompoundTransformer.builder().changeCatalogItemId(
                "item", "1.0", "item", "2.0" /* old and new catalog item+version */).build(),
            "abcd1234" /* entity id */);


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ahgittin/incubator-brooklyn versioning-switch-on-rebind

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-brooklyn/pull/506.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #506
    
----
commit d423522cacf97c3fefa0df3c641718a49f9103a5
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-01-27T17:47:41Z

    CLI better default, port/https control, and version reporting
    
    * adds config key for web-console port
    * adds CLI for https
    * port defaults to 8443+ if no port specified and using https
    * default if no CLI options is to show help and an error
    * some https test fixes
    * discovers and reports Brooklyn OSGi metadata in log, including git SHA1 (and `brooklyn info`)

commit 0d078c9c97a4989923cf1063b6bd61484efb532f
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-06T22:11:36Z

    This closes #478

commit 0fb0c633d269f48cf474c036b8c14bfaba03b5c5
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-01-29T11:26:13Z

    Partial rebind - basically working, but for mgmt confusion
    
    makes RebindIteration abstract, with InitialFullRI having the old (full) behaviour,
    and new ActivePartialRebindIteration supporting a subset of nodes;
    minor fixes elsewhere to prevent problems

commit 3f7d7508e125d751933dc40914d72441ce27381e
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-03T12:48:22Z

    Partial rebind - using new explicit BrooklynObjectManagementMode

commit be6fd97ec5b318af02a7b5d663a9c7fe1b365c05
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-04T12:33:22Z

    Partial rebind - add test and refactor / tidy of new code

commit 07aaa8994237c2ef260d1996660b94fd04064b71
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-05T18:28:29Z

    Partial rebind - test that version has successfully changed

commit 26811b9ae4f6e7102e30743998bf8be31ae9464a
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-05T18:36:15Z

    Switch OSGi more entites sayHI effector to include version info
    
    as per @aledsage's suggestion

commit 689f0512fda78cd02f7cdf3425a2b55c99c3ff74
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-06T17:56:06Z

    allow DynamicCluster to hold things which aren't Startable

commit 45820728671c18e183e0e296a197fc5d46305d0a
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-06T17:56:38Z

    better dev-environment detection and osgi error reporting

commit 6156a7743bdeacadfd533ead28b4c7460fdc7610
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-06T17:57:41Z

    add convenience method for creating spec from catalog item

commit 0dc533d116cc92a7d5811338cec060a7868e64a0
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-06T17:58:41Z

    special xml serialization for EntitySpec to support CatalogItem libraries

commit 71c82da572ae28d73c31f1c102f4ac4c5a7f2280
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-06T17:59:49Z

    test that changing version applies to specs

commit 35142c8049d3068775f448d6c48edeb365f53bfa
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-09T10:23:20Z

    fix proxy leak on extra promotion to master

commit 3c97a5774e30be0e9e6b7dfa0fb0613b2cc419e3
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-09T11:10:10Z

    some tests and comments exploring location unmanagement
    
    as there is a slow leak around locations, no one unmanages them after use by an entity

commit 491ca49efdfe4f8a7bb2bf5cbba5cd592bf4798a
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-09T12:32:29Z

    Partial rebind - address code review comments

commit b13cb63a4a23e847a3595a79a6fa2aab2609008a
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-02-09T14:23:18Z

    partial rebind - more code review, esp much better javadoc

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24320780
  
    --- Diff: api/src/main/java/brooklyn/basic/AbstractBrooklynObjectSpec.java ---
    @@ -121,4 +121,21 @@ protected final void checkIsImplementation(Class<?> val, Class<? super T> requir
             if (Modifier.isAbstract(val.getModifiers())) throw new IllegalStateException("Implementation "+val+" is abstract, but must be a non-abstract class");
         }
     
    +    @Override
    +    public boolean equals(Object obj) {
    +        if (obj==null) return false;
    --- End diff --
    
    I'm slightly hesitant about equals/hashCode on mutable objects. It potentially breaks the contract if two objects that were equal then become not-equal and the hashCode changes (http://stackoverflow.com/a/4718059/1393883).
    
    Guessing this is just used for tests? Perhaps having a utility class for comparisons would be safer?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24324258
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.Semaphore;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.basic.BrooklynObjectInternal;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore;
    +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics;
    +import brooklyn.entity.rebind.transformer.CompoundTransformer;
    +import brooklyn.management.ha.ManagementNodeState;
    +import brooklyn.mementos.BrooklynMementoPersister;
    +import brooklyn.mementos.BrooklynMementoRawData;
    +import brooklyn.mementos.BrooklynMementoRawData.Builder;
    +import brooklyn.mementos.Memento;
    +import brooklyn.util.collections.MutableList;
    +import brooklyn.util.collections.MutableSet;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back.
    + */
    +public class ActivePartialRebindIteration extends RebindIteration {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(ActivePartialRebindIteration.class);
    +    
    +    protected Iterator<BrooklynObject> objectsToRebindInitial;
    +    protected Collection<BrooklynObject> objectsToRebindFinal;
    +    protected List<CompoundTransformer> transformers = MutableList.of();
    +    
    +    public ActivePartialRebindIteration(RebindManagerImpl rebindManager, 
    +            ManagementNodeState mode,
    +            ClassLoader classLoader, RebindExceptionHandler exceptionHandler,
    +            Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess
    +            ) {
    +        super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +    }
    +
    +    @Override
    +    protected boolean isRebindingActiveAgain() {
    +        return true;
    +    }
    +    
    +    public void setObjectIterator(Iterator<BrooklynObject> objectsToRebind) {
    +        this.objectsToRebindInitial = objectsToRebind;
    +    }
    +    
    +    public void applyTransformer(CompoundTransformer transformer) {
    +        if (transformer!=null)
    --- End diff --
    
    i don't agree but i've changed it :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24321929
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.Semaphore;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.basic.BrooklynObjectInternal;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore;
    +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics;
    +import brooklyn.entity.rebind.transformer.CompoundTransformer;
    +import brooklyn.management.ha.ManagementNodeState;
    +import brooklyn.mementos.BrooklynMementoPersister;
    +import brooklyn.mementos.BrooklynMementoRawData;
    +import brooklyn.mementos.BrooklynMementoRawData.Builder;
    +import brooklyn.mementos.Memento;
    +import brooklyn.util.collections.MutableList;
    +import brooklyn.util.collections.MutableSet;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back.
    + */
    +public class ActivePartialRebindIteration extends RebindIteration {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(ActivePartialRebindIteration.class);
    +    
    +    protected Iterator<BrooklynObject> objectsToRebindInitial;
    +    protected Collection<BrooklynObject> objectsToRebindFinal;
    +    protected List<CompoundTransformer> transformers = MutableList.of();
    +    
    +    public ActivePartialRebindIteration(RebindManagerImpl rebindManager, 
    +            ManagementNodeState mode,
    +            ClassLoader classLoader, RebindExceptionHandler exceptionHandler,
    +            Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess
    +            ) {
    +        super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +    }
    +
    +    @Override
    +    protected boolean isRebindingActiveAgain() {
    +        return true;
    +    }
    +    
    +    public void setObjectIterator(Iterator<BrooklynObject> objectsToRebind) {
    +        this.objectsToRebindInitial = objectsToRebind;
    +    }
    +    
    +    public void applyTransformer(CompoundTransformer transformer) {
    +        if (transformer!=null)
    --- End diff --
    
    I still have a general preference for us not to ignore nulls like this. It's a serious risk of ignoring problems with surprising behaviour, versus the convenience of the caller not having to check for null. The time I lean towards accepting nulls is if it's in a fluent API.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24321455
  
    --- Diff: core/src/main/java/brooklyn/catalog/internal/CatalogUtils.java ---
    @@ -227,4 +228,13 @@ public static String getVersionedId(String id, String version) {
             }
         }
     
    +    @Beta
    +    public static EntitySpec<?> createEntitySpec(ManagementContext mgmt, CatalogItem<?, ?> catalogItem) {
    --- End diff --
    
    This is currently only used for tests - should we move it to `src/test/java` instead?
    
    My concern is that this doesn't necessarily do the full thing that the original `CatalogItem` did - for example, the `CatalogItem.getPlanYaml()` is ignored (i.e. whatever other config was supplied in that yaml will be ignored), and also it assumes the `CatalogItem` is for an `EntitySpec`. That's all fine to assume for tests, but doesn't feel right for core code (even if it is marked `@Beta`).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#issuecomment-73509740
  
    @ahgittin I've finished reviewing. Lots of great stuff here.
    
    Two general comments: 1) worth e-mailing brooklyn-dev list about this, as it is a significant new feature to be able to do in-life upgrades without restarting the server; 2) worth some more javadoc at the class-level to help future maintainers.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24330252
  
    --- Diff: core/src/main/java/brooklyn/management/internal/LocalEntityManager.java ---
    @@ -354,23 +362,23 @@ private void unmanage(final Entity e, ManagementTransitionMode mode, boolean has
             
             if (hasBeenReplaced) {
                 // we are unmanaging an old instance after having replaced it
    -            // (called from manage(...)
    +            // don't unmanage or even clear its fields, because there might be references to it
                 
    -            if (mode==ManagementTransitionMode.REBINDING_NO_LONGER_PRIMARY) {
    -                // when migrating away, these all need to be called
    +            if (mode.wasReadOnly()) {
    +                // if coming *from* read only; nothing needed
    +            } else {
    +                if (!mode.wasPrimary()) {
    +                    log.warn("Unexpected mode "+mode+" for unmanage-replace "+e+" (applying anyway)");
    --- End diff --
    
    My ideal would be a model where tests fail if they write any WARN messages at all.  (Maybe we support `@AllowLogWarnings` for some cases.)
    
    I'm in the habit of watching out for WARNs when running things.  Everyone should be.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24325998
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/RebindManagerImpl.java ---
    @@ -361,6 +364,38 @@ public void stop() {
             if (persistenceStoreAccess != null) persistenceStoreAccess.stop(true);
         }
         
    +        
    +    public void rebindPartialActive(CompoundTransformer transformer, Iterator<BrooklynObject> objectsToRebind) {
    +        final ClassLoader classLoader = 
    +            managementContext.getCatalog().getRootClassLoader();
    +        final RebindExceptionHandler exceptionHandler = 
    +            RebindExceptionHandlerImpl.builder()
    +                .danglingRefFailureMode(danglingRefFailureMode)
    +                .danglingRefQuorumRequiredHealthy(danglingRefsQuorumRequiredHealthy)
    +                .rebindFailureMode(rebindFailureMode)
    +                .addPolicyFailureMode(addPolicyFailureMode)
    +                .loadPolicyFailureMode(loadPolicyFailureMode)
    +                .build();
    +        final ManagementNodeState mode = getRebindMode();
    +
    +        ActivePartialRebindIteration iteration = new ActivePartialRebindIteration(this, mode, classLoader, exceptionHandler,
    +            rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +
    +        iteration.setObjectIterator(objectsToRebind);
    +        iteration.applyTransformer(transformer);
    +        iteration.run();
    +    }
    +    
    +    public void rebindPartialActive(CompoundTransformer transformer, String ...objectsToRebindIds) {
    +        List<BrooklynObject> objectsToRebind = MutableList.of();
    +        for (String objectId: objectsToRebindIds) {
    +            BrooklynObject obj = managementContext.lookup(objectId);
    +            if (obj instanceof Entity) obj = Entities.deproxy((Entity)obj);
    --- End diff --
    
    agree


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24320820
  
    --- Diff: core/src/main/java/brooklyn/BrooklynVersion.java ---
    @@ -19,80 +19,215 @@
     package brooklyn;
     
     import static com.google.common.base.Preconditions.checkNotNull;
    -import static java.lang.String.format;
     
     import java.io.IOException;
     import java.io.InputStream;
    +import java.net.URL;
    +import java.util.Enumeration;
     import java.util.Properties;
    +import java.util.concurrent.atomic.AtomicReference;
    +import java.util.jar.Attributes;
    +
    +import javax.annotation.Nullable;
     
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    +import brooklyn.util.exceptions.Exceptions;
    +import brooklyn.util.osgi.Osgis;
    +import brooklyn.util.osgi.Osgis.ManifestHelper;
    +import brooklyn.util.text.Strings;
    +
    +/**
    + * Wraps the version of Brooklyn.
    + * <p>
    + * Also retrieves the SHA-1 from any OSGi bundle, and checks that the maven and osgi versions match.
    + */
     public class BrooklynVersion {
     
       private static final Logger log = LoggerFactory.getLogger(BrooklynVersion.class);
       
    -  private static final String VERSION_RESOURCE_FILE = "META-INF/maven/io.brooklyn/brooklyn-core/pom.properties";
    -  private static final String VERSION_PROPERTY_NAME = "version";
    +  private static final String MVN_VERSION_RESOURCE_FILE = "META-INF/maven/org.apache.brooklyn/brooklyn-core/pom.properties";
    +  private static final String MANIFEST_PATH = "META-INF/MANIFEST.MF";
    +  private static final String BROOKLYN_CORE_SYMBOLIC_NAME = "org.apache.brooklyn.core";
    +  
    +  private static final String MVN_VERSION_PROPERTY_NAME = "version";
    +  private static final String OSGI_VERSION_PROPERTY_NAME = Attributes.Name.IMPLEMENTATION_VERSION.toString();
    +  private static final String OSGI_SHA1_PROPERTY_NAME = "Implementation-SHA-1";
    +
     
    +  private final static String VERSION_FROM_STATIC = "0.7.0-SNAPSHOT"; // BROOKLYN_VERSION
    +  private static final AtomicReference<Boolean> IS_DEV_ENV = new AtomicReference<Boolean>();
    +  
       public static final BrooklynVersion INSTANCE = new BrooklynVersion();
    +  
    +  private final Properties versionProperties = new Properties();
    +  
    +  public BrooklynVersion() {
    +      // we read the maven pom metadata and osgi metadata and make sure it's sensible
    +      // everything is put into a single map for now (good enough, but should be cleaned up)
    +      readPropertiesFromMavenResource(BrooklynVersion.class.getClassLoader());
    +      readPropertiesFromOsgiResource(BrooklynVersion.class.getClassLoader(), "org.apache.brooklyn.core");
    --- End diff --
    
    Use constant `BROOKLYN_CORE_SYMBOLIC_NAME`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24323665
  
    --- Diff: core/src/main/java/brooklyn/BrooklynVersion.java ---
    @@ -19,80 +19,215 @@
     package brooklyn;
     
     import static com.google.common.base.Preconditions.checkNotNull;
    -import static java.lang.String.format;
     
     import java.io.IOException;
     import java.io.InputStream;
    +import java.net.URL;
    +import java.util.Enumeration;
     import java.util.Properties;
    +import java.util.concurrent.atomic.AtomicReference;
    +import java.util.jar.Attributes;
    +
    +import javax.annotation.Nullable;
     
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    +import brooklyn.util.exceptions.Exceptions;
    +import brooklyn.util.osgi.Osgis;
    +import brooklyn.util.osgi.Osgis.ManifestHelper;
    +import brooklyn.util.text.Strings;
    +
    +/**
    + * Wraps the version of Brooklyn.
    + * <p>
    + * Also retrieves the SHA-1 from any OSGi bundle, and checks that the maven and osgi versions match.
    + */
     public class BrooklynVersion {
     
       private static final Logger log = LoggerFactory.getLogger(BrooklynVersion.class);
       
    -  private static final String VERSION_RESOURCE_FILE = "META-INF/maven/io.brooklyn/brooklyn-core/pom.properties";
    -  private static final String VERSION_PROPERTY_NAME = "version";
    +  private static final String MVN_VERSION_RESOURCE_FILE = "META-INF/maven/org.apache.brooklyn/brooklyn-core/pom.properties";
    +  private static final String MANIFEST_PATH = "META-INF/MANIFEST.MF";
    +  private static final String BROOKLYN_CORE_SYMBOLIC_NAME = "org.apache.brooklyn.core";
    +  
    +  private static final String MVN_VERSION_PROPERTY_NAME = "version";
    +  private static final String OSGI_VERSION_PROPERTY_NAME = Attributes.Name.IMPLEMENTATION_VERSION.toString();
    +  private static final String OSGI_SHA1_PROPERTY_NAME = "Implementation-SHA-1";
    +
     
    +  private final static String VERSION_FROM_STATIC = "0.7.0-SNAPSHOT"; // BROOKLYN_VERSION
    +  private static final AtomicReference<Boolean> IS_DEV_ENV = new AtomicReference<Boolean>();
    +  
       public static final BrooklynVersion INSTANCE = new BrooklynVersion();
    +  
    +  private final Properties versionProperties = new Properties();
    +  
    +  public BrooklynVersion() {
    +      // we read the maven pom metadata and osgi metadata and make sure it's sensible
    +      // everything is put into a single map for now (good enough, but should be cleaned up)
    +      readPropertiesFromMavenResource(BrooklynVersion.class.getClassLoader());
    +      readPropertiesFromOsgiResource(BrooklynVersion.class.getClassLoader(), "org.apache.brooklyn.core");
    +      // TODO there is also build-metadata.properties used in ServerResource /v1/server/version endpoint
    +      // see comments on that about folding it into this class instead
     
    -  private final String versionFromClasspath;
    -  // static useful when running from the IDE
    -  // TODO is the classpath version ever useful? should we always use the static?
    -  private final String versionFromStatic = "0.7.0-SNAPSHOT"; // BROOKLYN_VERSION
    -  private final String version;
    +      checkVersions();
    +  }
     
    -  public BrooklynVersion() {
    -    this.versionFromClasspath = readVersionPropertyFromClasspath(BrooklynVersion.class.getClassLoader());
    -    if (isValid(versionFromClasspath)) {
    -        this.version = versionFromClasspath;
    -        if (!this.version.equals(versionFromStatic)) {
    -            // should always be the same, and we can drop classpath detection ...
    -            log.warn("Version detected from classpath as "+versionFromClasspath+" (preferring that), but in code it is recorded as "+versionFromStatic);
    -        }
    -    } else {
    -        this.version = versionFromStatic;
    -    }
    +  public void checkVersions() {
    +      String mvnVersion = getVersionFromMavenProperties();
    +      if (mvnVersion!=null && !VERSION_FROM_STATIC.equals(mvnVersion)) {
    +          throw new IllegalStateException("Version error: maven "+mvnVersion+" / code "+VERSION_FROM_STATIC);
    +      }
    +      
    +      String osgiVersion = versionProperties.getProperty(OSGI_VERSION_PROPERTY_NAME);
    +      // TODO does the OSGi version include other slightly differ gubbins/style ?
    +      if (osgiVersion!=null && !VERSION_FROM_STATIC.equals(osgiVersion)) {
    +          throw new IllegalStateException("Version error: osgi "+osgiVersion+" / code "+VERSION_FROM_STATIC);
    +      }
       }
    -  
    +
    +  /** Returns version as inferred from classpath/osgi, if possible, or 0.0.0-SNAPSHOT.
    +   * See also {@link #getVersionFromMavenProperties()} and {@link #getVersionFromOsgiManifest()}.
    +   * @deprecated since 0.7.0, in favour of the more specific methods (and does anyone need that default value?)
    +   */
    +  @Deprecated
       public String getVersionFromClasspath() {
    -    return versionFromClasspath;
    +      String v = getVersionFromMavenProperties();
    +      if (Strings.isNonBlank(v)) return v;
    +      v = getVersionFromOsgiManifest();
    +      if (Strings.isNonBlank(v)) return v;
    +      return "0.0.0-SNAPSHOT";
       }
       
    -  public String getVersion() {
    -    return version;
    +  @Nullable
    +  public String getVersionFromMavenProperties() {
    +      return versionProperties.getProperty(MVN_VERSION_PROPERTY_NAME);
    +  }
    +
    +  @Nullable
    +  public String getVersionFromOsgiManifest() {
    +      return versionProperties.getProperty(OSGI_VERSION_PROPERTY_NAME);
       }
       
    -  public String getVersionFromStatic() {
    -    return versionFromStatic;
    +  @Nullable
    +  /** SHA1 of the last commit to brooklyn at the time this build was made.
    +   * For SNAPSHOT builds of course there may have been further non-committed changes. */
    +  public String getSha1FromOsgiManifest() {
    +      return versionProperties.getProperty(OSGI_SHA1_PROPERTY_NAME);
       }
    -
    -  public boolean isSnapshot() {
    -      return (getVersion().indexOf("-SNAPSHOT")>=0);
    +  
    +  public String getVersion() {
    +    return VERSION_FROM_STATIC;
       }
       
    -  private static boolean isValid(String v) {
    -    if (v==null) return false;
    -    if (v.equals("0.0.0") || v.equals("0.0")) return false;
    -    if (v.startsWith("0.0.0-") || v.startsWith("0.0-")) return false;
    -    return true;
    +  public boolean isSnapshot() {
    +      return (getVersion().indexOf("-SNAPSHOT")>=0);
       }
    -
    -  private String readVersionPropertyFromClasspath(ClassLoader resourceLoader) {
    -    Properties versionProperties = new Properties();
    +    
    +  private void readPropertiesFromMavenResource(ClassLoader resourceLoader) {
         try {
    -      InputStream versionStream = resourceLoader.getResourceAsStream(VERSION_RESOURCE_FILE);
    -      if (versionStream==null) return null;
    +      InputStream versionStream = resourceLoader.getResourceAsStream(MVN_VERSION_RESOURCE_FILE);
    +      if (versionStream==null) {
    +          if (isDevelopmentEnvironment()) {
    +              // allowed for dev env
    +              log.trace("No maven resource file "+MVN_VERSION_RESOURCE_FILE+" available");
    +          } else {
    +              log.warn("No maven resource file "+MVN_VERSION_RESOURCE_FILE+" available");
    +          }
    +          return;
    +      }
           versionProperties.load(checkNotNull(versionStream));
    -    } catch (IOException exception) {
    -      throw new IllegalStateException(format("Unable to load version resource file '%s'", VERSION_RESOURCE_FILE), exception);
    +    } catch (IOException e) {
    +      log.warn("Error reading maven resource file "+MVN_VERSION_RESOURCE_FILE+": "+e, e);
         }
    -    return checkNotNull(versionProperties.getProperty(VERSION_PROPERTY_NAME), VERSION_PROPERTY_NAME);
    +  }
    +
    +  /** reads properties from brooklyn-core's manifest */
    +  private void readPropertiesFromOsgiResource(ClassLoader resourceLoader, String symbolicName) {
    +      Enumeration<URL> paths;
    +      try {
    +          paths = BrooklynVersion.class.getClassLoader().getResources(MANIFEST_PATH);
    +      } catch (IOException e) {
    +          // shouldn't happen
    +          throw Exceptions.propagate(e);
    +      }
    +      while (paths.hasMoreElements()) {
    +          URL u = paths.nextElement();
    +          try {
    +              ManifestHelper mh = Osgis.ManifestHelper.forManifest(u.openStream());
    +              if (BROOKLYN_CORE_SYMBOLIC_NAME.equals(mh.getSymbolicName())) {
    +                  Attributes attrs = mh.getManifest().getMainAttributes();
    +                  for (Object key: attrs.keySet()) {
    +                      // key is an Attribute.Name; toString converts to string
    +                      versionProperties.put(key.toString(), attrs.getValue(key.toString()));
    +                  }
    +                  return;
    +              }
    +          } catch (Exception e) {
    +              Exceptions.propagateIfFatal(e);
    +              log.warn("Error reading OSGi manifest from "+u+" when determining version properties: "+e, e);
    +          }
    +      }
    +      if (isDevelopmentEnvironment()) {
    +          // allowed for dev env
    +          log.trace("No OSGi manifest available to determine version properties");
    +      } else {
    +          log.warn("No OSGi manifest available to determine version properties");
    +      }
    +  }
    +
    +  /** 
    +   * Returns whether this is a Brooklyn dev environment,
    +   * specifically core/target/classes/ is on the classpath for the org.apache.brooklyn.core project.
    +   * <p>
    +   * In a packaged or library build of Brooklyn (normal usage) this should return false,
    +   * and all OSGi components should be available.
    +   */
    +  public static boolean isDevelopmentEnvironment() {
    +      Boolean isDevEnv = IS_DEV_ENV.get();
    +      if (isDevEnv!=null) return isDevEnv;
    +      synchronized (IS_DEV_ENV) {
    +          isDevEnv = computeIsDevelopmentEnvironment();
    +          IS_DEV_ENV.set(isDevEnv);
    +          return isDevEnv;
    +      }
    +  }
    +  
    +  private static boolean computeIsDevelopmentEnvironment() {
    +      Enumeration<URL> paths;
    +      try {
    +          paths = BrooklynVersion.class.getClassLoader().getResources("brooklyn/BrooklynVersion.class");
    +      } catch (IOException e) {
    +          // shouldn't happen
    +          throw Exceptions.propagate(e);
    +      }
    +      while (paths.hasMoreElements()) {
    +          URL u = paths.nextElement();
    +          if (u.getPath().endsWith("core/target/classes/brooklyn/BrooklynVersion.class")) {
    +              try {
    +                  log.debug("Brooklyn debug environment detected; BrooklynVersion class is at: "+u);
    +                  return true;
    +              } catch (Exception e) {
    +                  Exceptions.propagateIfFatal(e);
    +                  log.warn("Error reading manifest to determine whether this is a development environment: "+e, e);
    +              }
    +          }
    +      }
    +      return false;
    +  }
    +
    +  public void logSummary() {
    +      log.debug("Brooklyn version "+getVersion()+" (git SHA1 "+getSha1FromOsgiManifest()+")");
    +  }
    +
    +  /** @deprecated since 0.7.0, redundant with {@link #get()} */ @Deprecated
    +  public static String getVersionFromStatic() {
    +      return VERSION_FROM_STATIC;
       }
     
       public static String get() {
    -    return INSTANCE.version;
    +    return getVersionFromStatic();
    --- End diff --
    
    now does `return INSTANCE.getVersion();`.  the other option was to return the static constant, but feels safer to have the method fail if `INSTANCE` couldn't be initialized.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24322062
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.Semaphore;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.basic.BrooklynObjectInternal;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore;
    +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics;
    +import brooklyn.entity.rebind.transformer.CompoundTransformer;
    +import brooklyn.management.ha.ManagementNodeState;
    +import brooklyn.mementos.BrooklynMementoPersister;
    +import brooklyn.mementos.BrooklynMementoRawData;
    +import brooklyn.mementos.BrooklynMementoRawData.Builder;
    +import brooklyn.mementos.Memento;
    +import brooklyn.util.collections.MutableList;
    +import brooklyn.util.collections.MutableSet;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back.
    + */
    +public class ActivePartialRebindIteration extends RebindIteration {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(ActivePartialRebindIteration.class);
    +    
    +    protected Iterator<BrooklynObject> objectsToRebindInitial;
    +    protected Collection<BrooklynObject> objectsToRebindFinal;
    +    protected List<CompoundTransformer> transformers = MutableList.of();
    +    
    +    public ActivePartialRebindIteration(RebindManagerImpl rebindManager, 
    +            ManagementNodeState mode,
    +            ClassLoader classLoader, RebindExceptionHandler exceptionHandler,
    +            Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess
    +            ) {
    +        super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +    }
    +
    +    @Override
    +    protected boolean isRebindingActiveAgain() {
    +        return true;
    +    }
    +    
    +    public void setObjectIterator(Iterator<BrooklynObject> objectsToRebind) {
    +        this.objectsToRebindInitial = objectsToRebind;
    +    }
    +    
    +    public void applyTransformer(CompoundTransformer transformer) {
    +        if (transformer!=null)
    +            transformers.add(transformer);
    +    }
    +    
    +    @Override
    +    protected void doRun() throws Exception {
    +        checkEnteringPhase(1);
    +        Preconditions.checkState(readOnlyRebindCount.get()==Integer.MIN_VALUE, "Partial rebind in read-only mode not supported");
    --- End diff --
    
    This seems a strange ay to check if the state is read-only mode. I wouldn't have guess it from looking at the variable name.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24326330
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/persister/XmlMementoSerializer.java ---
    @@ -341,5 +353,151 @@ public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext co
                 return lookupContext.lookupManagementContext();
             }
         }
    +
    +    /** When reading/writing specs, it checks whether there is a catalog item id set and uses it to load */
    +    public class SpecConverter extends ReflectionConverter {
    +        SpecConverter() {
    +            super(xstream.getMapper(), xstream.getReflectionProvider());
    +        }
    +        @Override
    +        public boolean canConvert(@SuppressWarnings("rawtypes") Class type) {
    +            return AbstractBrooklynObjectSpec.class.isAssignableFrom(type);
    +        }
    +        @Override
    +        public void marshal(Object source, HierarchicalStreamWriter writer, MarshallingContext context) {
    +            if (source == null) return;
    +            AbstractBrooklynObjectSpec<?, ?> spec = (AbstractBrooklynObjectSpec<?, ?>) source;
    +            String catalogItemId = spec.getCatalogItemId();
    +            if (Strings.isNonBlank(catalogItemId)) {
    +                // write this field first, so we can peek at it when we read
    +                writer.startNode("catalogItemId");
    +                writer.setValue(catalogItemId);
    +                writer.endNode();
    +                
    +                // we're going to write the catalogItemId field twice :( but that's okay.
    +                // better solution would be to have mark/reset on reader so we can peek for such a field;
    +                // see comment below
    +                super.marshal(source, writer, context);
    +            } else {
    +                super.marshal(source, writer, context);
    +            }
    +        }
    +        @Override
    +        public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            String catalogItemId = null;
    +            instantiateNewInstanceSettingCache(reader, context);
    +            
    +            if (reader instanceof PathTrackingReader) {
    +                // have to assume this is first; there is no mark/reset support on these readers
    +                // (if there were then it would be easier, we could just look for that child anywhere,
    +                // and not need a custom writer!)
    +                if ("catalogItemId".equals( ((PathTrackingReader)reader).peekNextChild() )) {
    +                    // cache the instance
    +                    
    +                    reader.moveDown();
    +                    catalogItemId = reader.getValue();
    +                    reader.moveUp();
    +                }
    +            }
    +            boolean customLoaderSet = false;
    +            try {
    +                if (Strings.isNonBlank(catalogItemId)) {
    +                    if (lookupContext==null) throw new NullPointerException("lookupContext required to load catalog item "+catalogItemId);
    +                    CatalogItem<?, ?> cat = CatalogUtils.getCatalogItemOptionalVersion(lookupContext.lookupManagementContext(), catalogItemId);
    +                    if (cat==null) throw new NoSuchElementException("catalog item: "+catalogItemId);
    +                    BrooklynClassLoadingContext clcNew = CatalogUtils.newClassLoadingContext(lookupContext.lookupManagementContext(), cat);
    +                    pushXstreamCustomClassLoader(clcNew);
    +                    customLoaderSet = true;
    +                }
    +                
    +                AbstractBrooklynObjectSpec<?, ?> result = (AbstractBrooklynObjectSpec<?, ?>) super.unmarshal(reader, context);
    +                // we wrote it twice so this shouldn't be necessary; but if we fix it so we only write once, we'd need this
    +                result.catalogItemId(catalogItemId);
    +                return result;
    +            } finally {
    +                instance = null;
    +                if (customLoaderSet) {
    +                    popXstreamCustomClassLoader();
    +                }
    +            }
    +        }
    +
    +        Object instance;
    +        
    +        @Override
    +        protected Object instantiateNewInstance(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            // the super calls getAttribute which requires that we have not yet done moveDown,
    +            // so we do this earlier and cache it for when we call super.unmarshal
    +            if (instance==null)
    +                throw new IllegalStateException("Instance should be created and cached");
    +            return instance;
    +        }
    +        protected void instantiateNewInstanceSettingCache(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            instance = super.instantiateNewInstance(reader, context);
    +        }
    +    }
    +    
    +    Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    AtomicReference<Thread> xstreamLockOwner = new AtomicReference<Thread>();
    +    int lockCount;
         
    +    /** Must be accompanied by a corresponding {@link #popXstreamCustomClassLoader()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    protected void pushXstreamCustomClassLoader(BrooklynClassLoadingContext clcNew) {
    +        acquireXstreamLock();
    +        BrooklynClassLoadingContext oldClc;
    +        if (!contexts.isEmpty()) {
    +            oldClc = contexts.peek();
    --- End diff --
    
    he has acquired the xstream lock at this point


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin closed the pull request at:

    https://github.com/apache/incubator-brooklyn/pull/506


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24325003
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/RebindContextLookupContext.java ---
    @@ -60,6 +66,9 @@ public RebindContextLookupContext(ManagementContext managementContext, RebindCon
         @Override public Entity lookupEntity(String id) {
             Entity result = rebindContext.getEntity(id);
             if (result == null) {
    +            result = managementContext.lookup(id, Entity.class);
    --- End diff --
    
    Should this be guarded with `lookInManagementContext`? That field is never used.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24322344
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.Semaphore;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.basic.BrooklynObjectInternal;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore;
    +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics;
    +import brooklyn.entity.rebind.transformer.CompoundTransformer;
    +import brooklyn.management.ha.ManagementNodeState;
    +import brooklyn.mementos.BrooklynMementoPersister;
    +import brooklyn.mementos.BrooklynMementoRawData;
    +import brooklyn.mementos.BrooklynMementoRawData.Builder;
    +import brooklyn.mementos.Memento;
    +import brooklyn.util.collections.MutableList;
    +import brooklyn.util.collections.MutableSet;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back.
    + */
    +public class ActivePartialRebindIteration extends RebindIteration {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(ActivePartialRebindIteration.class);
    +    
    +    protected Iterator<BrooklynObject> objectsToRebindInitial;
    +    protected Collection<BrooklynObject> objectsToRebindFinal;
    +    protected List<CompoundTransformer> transformers = MutableList.of();
    +    
    +    public ActivePartialRebindIteration(RebindManagerImpl rebindManager, 
    +            ManagementNodeState mode,
    +            ClassLoader classLoader, RebindExceptionHandler exceptionHandler,
    +            Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess
    +            ) {
    +        super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +    }
    +
    +    @Override
    +    protected boolean isRebindingActiveAgain() {
    +        return true;
    +    }
    +    
    +    public void setObjectIterator(Iterator<BrooklynObject> objectsToRebind) {
    +        this.objectsToRebindInitial = objectsToRebind;
    +    }
    +    
    +    public void applyTransformer(CompoundTransformer transformer) {
    +        if (transformer!=null)
    +            transformers.add(transformer);
    +    }
    +    
    +    @Override
    +    protected void doRun() throws Exception {
    +        checkEnteringPhase(1);
    +        Preconditions.checkState(readOnlyRebindCount.get()==Integer.MIN_VALUE, "Partial rebind in read-only mode not supported");
    +        Preconditions.checkNotNull(objectsToRebindInitial, "Objects to rebind must be set");
    +
    +        LOG.debug("Partial rebind Rebinding ("+mode+") from "+rebindManager.getPersister().getBackingStoreDescription()+"...");
    +
    +        Builder mementoRawBuilder = BrooklynMementoRawData.builder();
    +
    +        /*
    +         * Unmanagement is done as part of the "manage" call, entity by entity.
    +         */
    +
    +        objectsToRebindFinal = MutableSet.of();
    +        while (objectsToRebindInitial.hasNext()) {
    +            BrooklynObject bo = objectsToRebindInitial.next();
    +            objectsToRebindFinal.add(bo);
    +            
    +            if (bo instanceof Entity) {
    +                // if it's an entity, add all adjuncts. (if doing some sort of pause, that's maybe not necessary...)
    +                objectsToRebindFinal.addAll( ((EntityInternal)bo).getPolicies() );
    +                objectsToRebindFinal.addAll( ((EntityInternal)bo).getEnrichers() );
    +                objectsToRebindFinal.addAll( ((EntityInternal)bo).feeds().getFeeds() );
    +            }
    +        }
    +        
    +        // get serialization
    +        for (BrooklynObject bo: objectsToRebindFinal) {
    +            Memento m = ((BrooklynObjectInternal)bo).getRebindSupport().getMemento();
    +            BrooklynMementoPersister p = rebindManager.getPersister();
    +            String mr = ((BrooklynMementoPersisterToObjectStore)p).getMementoSerializer().toString(m);
    --- End diff --
    
    Shame we have to cast this. It makes it feel pointless to leave the old implementations of `BrooklynMementoPersister` deprecated if one will potentially get runtime exceptions using it (better to have deleted the classes to give compile-time errors, depending under what circumstances this code-path will be hit).
    
    The `BrooklynMementoPersisterInMemory` feels like a useful thing to keep in the test packages, but maybe not even worth that if we just accept the performance overhead in our test cases of always using `BrooklynMementoPersisterToObjectStore`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#issuecomment-73520715
  
    spurious failure?  port 65536?  i think there's another PR or comment about fixing that... trying again as it seems unrelated


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24325060
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/RebindContextLookupContext.java ---
    @@ -60,6 +66,9 @@ public RebindContextLookupContext(ManagementContext managementContext, RebindCon
         @Override public Entity lookupEntity(String id) {
             Entity result = rebindContext.getEntity(id);
             if (result == null) {
    +            result = managementContext.lookup(id, Entity.class);
    --- End diff --
    
    But neither is the constructor to set `lookInManagementContext` to true. Can we just delete that?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24325561
  
    --- Diff: core/src/main/java/brooklyn/management/internal/ManagementTransitionMode.java ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.
    + */
    +package brooklyn.management.internal;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +public class ManagementTransitionMode {
    --- End diff --
    
    Could do with some javadoc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24326515
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/persister/XmlMementoSerializer.java ---
    @@ -341,5 +353,151 @@ public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext co
                 return lookupContext.lookupManagementContext();
             }
         }
    +
    +    /** When reading/writing specs, it checks whether there is a catalog item id set and uses it to load */
    +    public class SpecConverter extends ReflectionConverter {
    +        SpecConverter() {
    +            super(xstream.getMapper(), xstream.getReflectionProvider());
    +        }
    +        @Override
    +        public boolean canConvert(@SuppressWarnings("rawtypes") Class type) {
    +            return AbstractBrooklynObjectSpec.class.isAssignableFrom(type);
    +        }
    +        @Override
    +        public void marshal(Object source, HierarchicalStreamWriter writer, MarshallingContext context) {
    +            if (source == null) return;
    +            AbstractBrooklynObjectSpec<?, ?> spec = (AbstractBrooklynObjectSpec<?, ?>) source;
    +            String catalogItemId = spec.getCatalogItemId();
    +            if (Strings.isNonBlank(catalogItemId)) {
    +                // write this field first, so we can peek at it when we read
    +                writer.startNode("catalogItemId");
    +                writer.setValue(catalogItemId);
    +                writer.endNode();
    +                
    +                // we're going to write the catalogItemId field twice :( but that's okay.
    +                // better solution would be to have mark/reset on reader so we can peek for such a field;
    +                // see comment below
    +                super.marshal(source, writer, context);
    +            } else {
    +                super.marshal(source, writer, context);
    +            }
    +        }
    +        @Override
    +        public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            String catalogItemId = null;
    +            instantiateNewInstanceSettingCache(reader, context);
    +            
    +            if (reader instanceof PathTrackingReader) {
    +                // have to assume this is first; there is no mark/reset support on these readers
    +                // (if there were then it would be easier, we could just look for that child anywhere,
    +                // and not need a custom writer!)
    +                if ("catalogItemId".equals( ((PathTrackingReader)reader).peekNextChild() )) {
    +                    // cache the instance
    +                    
    +                    reader.moveDown();
    +                    catalogItemId = reader.getValue();
    +                    reader.moveUp();
    +                }
    +            }
    +            boolean customLoaderSet = false;
    +            try {
    +                if (Strings.isNonBlank(catalogItemId)) {
    +                    if (lookupContext==null) throw new NullPointerException("lookupContext required to load catalog item "+catalogItemId);
    +                    CatalogItem<?, ?> cat = CatalogUtils.getCatalogItemOptionalVersion(lookupContext.lookupManagementContext(), catalogItemId);
    +                    if (cat==null) throw new NoSuchElementException("catalog item: "+catalogItemId);
    +                    BrooklynClassLoadingContext clcNew = CatalogUtils.newClassLoadingContext(lookupContext.lookupManagementContext(), cat);
    +                    pushXstreamCustomClassLoader(clcNew);
    +                    customLoaderSet = true;
    +                }
    +                
    +                AbstractBrooklynObjectSpec<?, ?> result = (AbstractBrooklynObjectSpec<?, ?>) super.unmarshal(reader, context);
    +                // we wrote it twice so this shouldn't be necessary; but if we fix it so we only write once, we'd need this
    +                result.catalogItemId(catalogItemId);
    +                return result;
    +            } finally {
    +                instance = null;
    +                if (customLoaderSet) {
    +                    popXstreamCustomClassLoader();
    +                }
    +            }
    +        }
    +
    +        Object instance;
    +        
    +        @Override
    +        protected Object instantiateNewInstance(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            // the super calls getAttribute which requires that we have not yet done moveDown,
    +            // so we do this earlier and cache it for when we call super.unmarshal
    +            if (instance==null)
    +                throw new IllegalStateException("Instance should be created and cached");
    +            return instance;
    +        }
    +        protected void instantiateNewInstanceSettingCache(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            instance = super.instantiateNewInstance(reader, context);
    +        }
    +    }
    +    
    +    Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    AtomicReference<Thread> xstreamLockOwner = new AtomicReference<Thread>();
    +    int lockCount;
         
    +    /** Must be accompanied by a corresponding {@link #popXstreamCustomClassLoader()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    protected void pushXstreamCustomClassLoader(BrooklynClassLoadingContext clcNew) {
    --- End diff --
    
    Gut feel is that this feels complicated, but not sure how to simplify it so happy as-is (but see other related comments).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24326303
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/persister/XmlMementoSerializer.java ---
    @@ -341,5 +353,151 @@ public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext co
                 return lookupContext.lookupManagementContext();
             }
         }
    +
    +    /** When reading/writing specs, it checks whether there is a catalog item id set and uses it to load */
    +    public class SpecConverter extends ReflectionConverter {
    +        SpecConverter() {
    +            super(xstream.getMapper(), xstream.getReflectionProvider());
    +        }
    +        @Override
    +        public boolean canConvert(@SuppressWarnings("rawtypes") Class type) {
    +            return AbstractBrooklynObjectSpec.class.isAssignableFrom(type);
    +        }
    +        @Override
    +        public void marshal(Object source, HierarchicalStreamWriter writer, MarshallingContext context) {
    +            if (source == null) return;
    +            AbstractBrooklynObjectSpec<?, ?> spec = (AbstractBrooklynObjectSpec<?, ?>) source;
    +            String catalogItemId = spec.getCatalogItemId();
    +            if (Strings.isNonBlank(catalogItemId)) {
    +                // write this field first, so we can peek at it when we read
    +                writer.startNode("catalogItemId");
    +                writer.setValue(catalogItemId);
    +                writer.endNode();
    +                
    +                // we're going to write the catalogItemId field twice :( but that's okay.
    +                // better solution would be to have mark/reset on reader so we can peek for such a field;
    +                // see comment below
    +                super.marshal(source, writer, context);
    +            } else {
    +                super.marshal(source, writer, context);
    +            }
    +        }
    +        @Override
    +        public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            String catalogItemId = null;
    +            instantiateNewInstanceSettingCache(reader, context);
    +            
    +            if (reader instanceof PathTrackingReader) {
    +                // have to assume this is first; there is no mark/reset support on these readers
    +                // (if there were then it would be easier, we could just look for that child anywhere,
    +                // and not need a custom writer!)
    +                if ("catalogItemId".equals( ((PathTrackingReader)reader).peekNextChild() )) {
    +                    // cache the instance
    +                    
    +                    reader.moveDown();
    +                    catalogItemId = reader.getValue();
    +                    reader.moveUp();
    +                }
    +            }
    +            boolean customLoaderSet = false;
    +            try {
    +                if (Strings.isNonBlank(catalogItemId)) {
    +                    if (lookupContext==null) throw new NullPointerException("lookupContext required to load catalog item "+catalogItemId);
    +                    CatalogItem<?, ?> cat = CatalogUtils.getCatalogItemOptionalVersion(lookupContext.lookupManagementContext(), catalogItemId);
    +                    if (cat==null) throw new NoSuchElementException("catalog item: "+catalogItemId);
    +                    BrooklynClassLoadingContext clcNew = CatalogUtils.newClassLoadingContext(lookupContext.lookupManagementContext(), cat);
    +                    pushXstreamCustomClassLoader(clcNew);
    +                    customLoaderSet = true;
    +                }
    +                
    +                AbstractBrooklynObjectSpec<?, ?> result = (AbstractBrooklynObjectSpec<?, ?>) super.unmarshal(reader, context);
    +                // we wrote it twice so this shouldn't be necessary; but if we fix it so we only write once, we'd need this
    +                result.catalogItemId(catalogItemId);
    +                return result;
    +            } finally {
    +                instance = null;
    +                if (customLoaderSet) {
    +                    popXstreamCustomClassLoader();
    +                }
    +            }
    +        }
    +
    +        Object instance;
    +        
    +        @Override
    +        protected Object instantiateNewInstance(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            // the super calls getAttribute which requires that we have not yet done moveDown,
    +            // so we do this earlier and cache it for when we call super.unmarshal
    +            if (instance==null)
    +                throw new IllegalStateException("Instance should be created and cached");
    +            return instance;
    +        }
    +        protected void instantiateNewInstanceSettingCache(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            instance = super.instantiateNewInstance(reader, context);
    +        }
    +    }
    +    
    +    Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    AtomicReference<Thread> xstreamLockOwner = new AtomicReference<Thread>();
    +    int lockCount;
         
    +    /** Must be accompanied by a corresponding {@link #popXstreamCustomClassLoader()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    protected void pushXstreamCustomClassLoader(BrooklynClassLoadingContext clcNew) {
    +        acquireXstreamLock();
    +        BrooklynClassLoadingContext oldClc;
    +        if (!contexts.isEmpty()) {
    +            oldClc = contexts.peek();
    +        } else {
    +            // TODO XmlMementoSerializer should take a BCLC instead of a CL
    +            oldClc = JavaBrooklynClassLoadingContext.create(lookupContext.lookupManagementContext(), xstream.getClassLoader());
    +        }
    +        BrooklynClassLoadingContextSequential clcMerged = new BrooklynClassLoadingContextSequential(lookupContext.lookupManagementContext(),
    +            oldClc, clcNew);
    +        contexts.push(clcMerged);
    +        cls.push(xstream.getClassLoader());
    +        ClassLoader newCL = ClassLoaderFromBrooklynClassLoadingContext.of(clcMerged);
    +        xstream.setClassLoader(newCL);
    +    }
    +
    +    protected void popXstreamCustomClassLoader() {
    +        synchronized (xstreamLockOwner) {
    +            releaseXstreamLock();
    +            xstream.setClassLoader(cls.pop());
    +            contexts.pop();
    +        }
    +    }
    +    
    +    protected void acquireXstreamLock() {
    +        synchronized (xstreamLockOwner) {
    +            while (true) {
    +                if (xstreamLockOwner.compareAndSet(null, Thread.currentThread()) || 
    +                    Thread.currentThread().equals( xstreamLockOwner.get() )) {
    +                    break;
    +                }
    +                try {
    +                    xstreamLockOwner.wait(1000);
    --- End diff --
    
    added `notifyAll` on release


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24324430
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.Semaphore;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.basic.BrooklynObjectInternal;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore;
    +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics;
    +import brooklyn.entity.rebind.transformer.CompoundTransformer;
    +import brooklyn.management.ha.ManagementNodeState;
    +import brooklyn.mementos.BrooklynMementoPersister;
    +import brooklyn.mementos.BrooklynMementoRawData;
    +import brooklyn.mementos.BrooklynMementoRawData.Builder;
    +import brooklyn.mementos.Memento;
    +import brooklyn.util.collections.MutableList;
    +import brooklyn.util.collections.MutableSet;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back.
    + */
    +public class ActivePartialRebindIteration extends RebindIteration {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(ActivePartialRebindIteration.class);
    +    
    +    protected Iterator<BrooklynObject> objectsToRebindInitial;
    +    protected Collection<BrooklynObject> objectsToRebindFinal;
    +    protected List<CompoundTransformer> transformers = MutableList.of();
    +    
    +    public ActivePartialRebindIteration(RebindManagerImpl rebindManager, 
    +            ManagementNodeState mode,
    +            ClassLoader classLoader, RebindExceptionHandler exceptionHandler,
    +            Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess
    +            ) {
    +        super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +    }
    +
    +    @Override
    +    protected boolean isRebindingActiveAgain() {
    +        return true;
    +    }
    +    
    +    public void setObjectIterator(Iterator<BrooklynObject> objectsToRebind) {
    +        this.objectsToRebindInitial = objectsToRebind;
    +    }
    +    
    +    public void applyTransformer(CompoundTransformer transformer) {
    +        if (transformer!=null)
    +            transformers.add(transformer);
    +    }
    +    
    +    @Override
    +    protected void doRun() throws Exception {
    +        checkEnteringPhase(1);
    +        Preconditions.checkState(readOnlyRebindCount.get()==Integer.MIN_VALUE, "Partial rebind in read-only mode not supported");
    +        Preconditions.checkNotNull(objectsToRebindInitial, "Objects to rebind must be set");
    +
    +        LOG.debug("Partial rebind Rebinding ("+mode+") from "+rebindManager.getPersister().getBackingStoreDescription()+"...");
    +
    +        Builder mementoRawBuilder = BrooklynMementoRawData.builder();
    +
    +        /*
    +         * Unmanagement is done as part of the "manage" call, entity by entity.
    +         */
    +
    +        objectsToRebindFinal = MutableSet.of();
    +        while (objectsToRebindInitial.hasNext()) {
    +            BrooklynObject bo = objectsToRebindInitial.next();
    +            objectsToRebindFinal.add(bo);
    +            
    +            if (bo instanceof Entity) {
    +                // if it's an entity, add all adjuncts. (if doing some sort of pause, that's maybe not necessary...)
    +                objectsToRebindFinal.addAll( ((EntityInternal)bo).getPolicies() );
    +                objectsToRebindFinal.addAll( ((EntityInternal)bo).getEnrichers() );
    +                objectsToRebindFinal.addAll( ((EntityInternal)bo).feeds().getFeeds() );
    +            }
    +        }
    +        
    +        // get serialization
    +        for (BrooklynObject bo: objectsToRebindFinal) {
    +            Memento m = ((BrooklynObjectInternal)bo).getRebindSupport().getMemento();
    +            BrooklynMementoPersister p = rebindManager.getPersister();
    +            String mr = ((BrooklynMementoPersisterToObjectStore)p).getMementoSerializer().toString(m);
    --- End diff --
    
    there is a `SingleThreadedInMemoryStoreObjectAccessor` for use with the `BrooklynMementoPersisterToObjectStore`, giving minimal overhead, and a more faithful test pathway.
    
    the deprecated classes aren't used and can be deleted; we just need to cut 070-GA. i don't think there's point in adding support or compile-time checks for the deprecated classes in this pathway.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24324699
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.Semaphore;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.basic.BrooklynObjectInternal;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore;
    +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics;
    +import brooklyn.entity.rebind.transformer.CompoundTransformer;
    +import brooklyn.management.ha.ManagementNodeState;
    +import brooklyn.mementos.BrooklynMementoPersister;
    +import brooklyn.mementos.BrooklynMementoRawData;
    +import brooklyn.mementos.BrooklynMementoRawData.Builder;
    +import brooklyn.mementos.Memento;
    +import brooklyn.util.collections.MutableList;
    +import brooklyn.util.collections.MutableSet;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back.
    + */
    +public class ActivePartialRebindIteration extends RebindIteration {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(ActivePartialRebindIteration.class);
    +    
    +    protected Iterator<BrooklynObject> objectsToRebindInitial;
    +    protected Collection<BrooklynObject> objectsToRebindFinal;
    +    protected List<CompoundTransformer> transformers = MutableList.of();
    +    
    +    public ActivePartialRebindIteration(RebindManagerImpl rebindManager, 
    +            ManagementNodeState mode,
    +            ClassLoader classLoader, RebindExceptionHandler exceptionHandler,
    +            Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess
    +            ) {
    +        super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +    }
    +
    +    @Override
    +    protected boolean isRebindingActiveAgain() {
    +        return true;
    +    }
    +    
    +    public void setObjectIterator(Iterator<BrooklynObject> objectsToRebind) {
    +        this.objectsToRebindInitial = objectsToRebind;
    +    }
    +    
    +    public void applyTransformer(CompoundTransformer transformer) {
    +        if (transformer!=null)
    +            transformers.add(transformer);
    +    }
    +    
    +    @Override
    +    protected void doRun() throws Exception {
    --- End diff --
    
    yes, that's cleaner as it turns out


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24323604
  
    --- Diff: core/src/test/java/brooklyn/BrooklynVersionTest.java ---
    @@ -19,18 +19,63 @@
     package brooklyn;
     
     import static org.testng.Assert.assertEquals;
    +
    +import java.net.URL;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.Assert;
     import org.testng.annotations.Test;
     
    +import brooklyn.util.text.Strings;
    +
     public class BrooklynVersionTest {
     
    +  private static final Logger log = LoggerFactory.getLogger(BrooklynVersionTest.class);
    +    
       @Test
       public void testGetVersion() {
    -      assertEquals(BrooklynVersion.get(), BrooklynVersion.INSTANCE.getVersionFromStatic());
    +      assertEquals(BrooklynVersion.get(), BrooklynVersion.INSTANCE.getVersion());
       }
         
       @Test
       public void testGetHardcodedClasspathVersion() {
    -    assertEquals(BrooklynVersion.INSTANCE.getVersionFromClasspath(), "0.0.0-SNAPSHOT");
    +      @SuppressWarnings("deprecation")
    +      String v = BrooklynVersion.INSTANCE.getVersionFromClasspath();
    +      Assert.assertTrue(BrooklynVersion.get().equals(v) || "0.0.0-SNAPSHOT".equals(v), v);
    +  }
    +  
    +  @Test
    +  public void testGetFromMaven() {
    +      String v = BrooklynVersion.INSTANCE.getVersionFromMavenProperties();
    +      Assert.assertTrue(v==null || BrooklynVersion.get().equals(v), v);
    +  }
    +  
    +  @Test
    +  public void testGetFromOsgi() {
    +      String v = BrooklynVersion.INSTANCE.getVersionFromOsgiManifest();
    +      Assert.assertTrue(v==null || BrooklynVersion.get().equals(v), v);
    +  }
    +  
    +  @Test
    +  public void testGetOsgiSha1() {
    +      String sha1 = BrooklynVersion.INSTANCE.getSha1FromOsgiManifest();
    +      log.info("sha1: "+sha1);
    +      if (Strings.isNonBlank(sha1) || BrooklynVersion.isDevelopmentEnvironment())
    +          return;
    +      // we might not have a SHA1 if it's a standalone source build; just log warn in that case
    --- End diff --
    
    the warning is useful if you're running the test interactive
    
    yes it would be cleaner to have a group skipped on non-git source builds, or a flag indicating that status, but not really worth the effort now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24327310
  
    --- Diff: core/src/test/java/brooklyn/entity/rebind/ActivePartialRebindTest.java ---
    @@ -0,0 +1,84 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.Assert;
    +import org.testng.annotations.Test;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.AbstractEntity;
    +import brooklyn.entity.basic.Entities;
    +import brooklyn.entity.proxying.EntitySpec;
    +import brooklyn.test.entity.TestEntity;
    +import brooklyn.util.text.Strings;
    +
    +public class ActivePartialRebindTest extends RebindTestFixtureWithApp {
    +
    +    private static final Logger log = LoggerFactory.getLogger(ActivePartialRebindTest.class);
    +    
    +    protected void doPartialRebindOfIds(String ...objectsToRebindIds) {
    +        RebindManagerImpl rm = (RebindManagerImpl) origManagementContext.getRebindManager();
    +        rm.rebindPartialActive(null, objectsToRebindIds);        
    +    }
    +    
    +    @Test
    +    public void testRebindOneSimple() throws Exception {
    +        TestEntity c1 = origApp.addChild(EntitySpec.create(TestEntity.class));
    +        Entities.manage(c1);
    +        AbstractEntity c1r = Entities.deproxy(c1);
    +        
    +        doPartialRebindOfIds(c1.getId());
    +        
    +        BrooklynObject c2 = origManagementContext.lookup(c1.getId());
    +        AbstractEntity c2r = Entities.deproxy((Entity)c2);
    +        
    +        Assert.assertTrue(c2 == c1, "Proxy instance should be the same: "+c1+" / "+c2);
    +        Assert.assertFalse(c2r == c1r, "Real instance should NOT be the same: "+c1r+" / "+c2r);
    +    }
    +
    +    @Test(groups="Integration")
    +    public void testRebindCheckingMemoryLeak() throws Exception {
    +        TestEntity c1 = origApp.addChild(EntitySpec.create(TestEntity.class));
    +        Entities.manage(c1);
    +        c1.setConfig(TestEntity.CONF_NAME, Strings.makeRandomId(1000000));
    +        
    +        gcAndLog("before");
    +        long used0 = Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory();
    --- End diff --
    
    Minor preference: I'd set assign this to -1 or some such, given it will always be overwritten when `i==5`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24324325
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.Semaphore;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.basic.BrooklynObjectInternal;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore;
    +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics;
    +import brooklyn.entity.rebind.transformer.CompoundTransformer;
    +import brooklyn.management.ha.ManagementNodeState;
    +import brooklyn.mementos.BrooklynMementoPersister;
    +import brooklyn.mementos.BrooklynMementoRawData;
    +import brooklyn.mementos.BrooklynMementoRawData.Builder;
    +import brooklyn.mementos.Memento;
    +import brooklyn.util.collections.MutableList;
    +import brooklyn.util.collections.MutableSet;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back.
    + */
    +public class ActivePartialRebindIteration extends RebindIteration {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(ActivePartialRebindIteration.class);
    +    
    +    protected Iterator<BrooklynObject> objectsToRebindInitial;
    +    protected Collection<BrooklynObject> objectsToRebindFinal;
    +    protected List<CompoundTransformer> transformers = MutableList.of();
    +    
    +    public ActivePartialRebindIteration(RebindManagerImpl rebindManager, 
    +            ManagementNodeState mode,
    +            ClassLoader classLoader, RebindExceptionHandler exceptionHandler,
    +            Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess
    +            ) {
    +        super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +    }
    +
    +    @Override
    +    protected boolean isRebindingActiveAgain() {
    +        return true;
    +    }
    +    
    +    public void setObjectIterator(Iterator<BrooklynObject> objectsToRebind) {
    +        this.objectsToRebindInitial = objectsToRebind;
    +    }
    +    
    +    public void applyTransformer(CompoundTransformer transformer) {
    +        if (transformer!=null)
    +            transformers.add(transformer);
    +    }
    +    
    +    @Override
    +    protected void doRun() throws Exception {
    +        checkEnteringPhase(1);
    +        Preconditions.checkState(readOnlyRebindCount.get()==Integer.MIN_VALUE, "Partial rebind in read-only mode not supported");
    --- End diff --
    
    have added another check and improved messages


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-brooklyn/pull/506


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24321297
  
    --- Diff: core/src/main/java/brooklyn/BrooklynVersion.java ---
    @@ -19,80 +19,215 @@
     package brooklyn;
     
     import static com.google.common.base.Preconditions.checkNotNull;
    -import static java.lang.String.format;
     
     import java.io.IOException;
     import java.io.InputStream;
    +import java.net.URL;
    +import java.util.Enumeration;
     import java.util.Properties;
    +import java.util.concurrent.atomic.AtomicReference;
    +import java.util.jar.Attributes;
    +
    +import javax.annotation.Nullable;
     
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    +import brooklyn.util.exceptions.Exceptions;
    +import brooklyn.util.osgi.Osgis;
    +import brooklyn.util.osgi.Osgis.ManifestHelper;
    +import brooklyn.util.text.Strings;
    +
    +/**
    + * Wraps the version of Brooklyn.
    + * <p>
    + * Also retrieves the SHA-1 from any OSGi bundle, and checks that the maven and osgi versions match.
    + */
     public class BrooklynVersion {
     
       private static final Logger log = LoggerFactory.getLogger(BrooklynVersion.class);
       
    -  private static final String VERSION_RESOURCE_FILE = "META-INF/maven/io.brooklyn/brooklyn-core/pom.properties";
    -  private static final String VERSION_PROPERTY_NAME = "version";
    +  private static final String MVN_VERSION_RESOURCE_FILE = "META-INF/maven/org.apache.brooklyn/brooklyn-core/pom.properties";
    +  private static final String MANIFEST_PATH = "META-INF/MANIFEST.MF";
    +  private static final String BROOKLYN_CORE_SYMBOLIC_NAME = "org.apache.brooklyn.core";
    +  
    +  private static final String MVN_VERSION_PROPERTY_NAME = "version";
    +  private static final String OSGI_VERSION_PROPERTY_NAME = Attributes.Name.IMPLEMENTATION_VERSION.toString();
    +  private static final String OSGI_SHA1_PROPERTY_NAME = "Implementation-SHA-1";
    +
     
    +  private final static String VERSION_FROM_STATIC = "0.7.0-SNAPSHOT"; // BROOKLYN_VERSION
    +  private static final AtomicReference<Boolean> IS_DEV_ENV = new AtomicReference<Boolean>();
    +  
       public static final BrooklynVersion INSTANCE = new BrooklynVersion();
    +  
    +  private final Properties versionProperties = new Properties();
    +  
    +  public BrooklynVersion() {
    +      // we read the maven pom metadata and osgi metadata and make sure it's sensible
    +      // everything is put into a single map for now (good enough, but should be cleaned up)
    +      readPropertiesFromMavenResource(BrooklynVersion.class.getClassLoader());
    +      readPropertiesFromOsgiResource(BrooklynVersion.class.getClassLoader(), "org.apache.brooklyn.core");
    +      // TODO there is also build-metadata.properties used in ServerResource /v1/server/version endpoint
    +      // see comments on that about folding it into this class instead
     
    -  private final String versionFromClasspath;
    -  // static useful when running from the IDE
    -  // TODO is the classpath version ever useful? should we always use the static?
    -  private final String versionFromStatic = "0.7.0-SNAPSHOT"; // BROOKLYN_VERSION
    -  private final String version;
    +      checkVersions();
    +  }
     
    -  public BrooklynVersion() {
    -    this.versionFromClasspath = readVersionPropertyFromClasspath(BrooklynVersion.class.getClassLoader());
    -    if (isValid(versionFromClasspath)) {
    -        this.version = versionFromClasspath;
    -        if (!this.version.equals(versionFromStatic)) {
    -            // should always be the same, and we can drop classpath detection ...
    -            log.warn("Version detected from classpath as "+versionFromClasspath+" (preferring that), but in code it is recorded as "+versionFromStatic);
    -        }
    -    } else {
    -        this.version = versionFromStatic;
    -    }
    +  public void checkVersions() {
    +      String mvnVersion = getVersionFromMavenProperties();
    +      if (mvnVersion!=null && !VERSION_FROM_STATIC.equals(mvnVersion)) {
    +          throw new IllegalStateException("Version error: maven "+mvnVersion+" / code "+VERSION_FROM_STATIC);
    +      }
    +      
    +      String osgiVersion = versionProperties.getProperty(OSGI_VERSION_PROPERTY_NAME);
    +      // TODO does the OSGi version include other slightly differ gubbins/style ?
    +      if (osgiVersion!=null && !VERSION_FROM_STATIC.equals(osgiVersion)) {
    +          throw new IllegalStateException("Version error: osgi "+osgiVersion+" / code "+VERSION_FROM_STATIC);
    +      }
       }
    -  
    +
    +  /** Returns version as inferred from classpath/osgi, if possible, or 0.0.0-SNAPSHOT.
    +   * See also {@link #getVersionFromMavenProperties()} and {@link #getVersionFromOsgiManifest()}.
    +   * @deprecated since 0.7.0, in favour of the more specific methods (and does anyone need that default value?)
    +   */
    +  @Deprecated
       public String getVersionFromClasspath() {
    -    return versionFromClasspath;
    +      String v = getVersionFromMavenProperties();
    +      if (Strings.isNonBlank(v)) return v;
    +      v = getVersionFromOsgiManifest();
    +      if (Strings.isNonBlank(v)) return v;
    +      return "0.0.0-SNAPSHOT";
       }
       
    -  public String getVersion() {
    -    return version;
    +  @Nullable
    +  public String getVersionFromMavenProperties() {
    +      return versionProperties.getProperty(MVN_VERSION_PROPERTY_NAME);
    +  }
    +
    +  @Nullable
    +  public String getVersionFromOsgiManifest() {
    +      return versionProperties.getProperty(OSGI_VERSION_PROPERTY_NAME);
       }
       
    -  public String getVersionFromStatic() {
    -    return versionFromStatic;
    +  @Nullable
    +  /** SHA1 of the last commit to brooklyn at the time this build was made.
    +   * For SNAPSHOT builds of course there may have been further non-committed changes. */
    +  public String getSha1FromOsgiManifest() {
    +      return versionProperties.getProperty(OSGI_SHA1_PROPERTY_NAME);
       }
    -
    -  public boolean isSnapshot() {
    -      return (getVersion().indexOf("-SNAPSHOT")>=0);
    +  
    +  public String getVersion() {
    +    return VERSION_FROM_STATIC;
       }
       
    -  private static boolean isValid(String v) {
    -    if (v==null) return false;
    -    if (v.equals("0.0.0") || v.equals("0.0")) return false;
    -    if (v.startsWith("0.0.0-") || v.startsWith("0.0-")) return false;
    -    return true;
    +  public boolean isSnapshot() {
    +      return (getVersion().indexOf("-SNAPSHOT")>=0);
       }
    -
    -  private String readVersionPropertyFromClasspath(ClassLoader resourceLoader) {
    -    Properties versionProperties = new Properties();
    +    
    +  private void readPropertiesFromMavenResource(ClassLoader resourceLoader) {
         try {
    -      InputStream versionStream = resourceLoader.getResourceAsStream(VERSION_RESOURCE_FILE);
    -      if (versionStream==null) return null;
    +      InputStream versionStream = resourceLoader.getResourceAsStream(MVN_VERSION_RESOURCE_FILE);
    +      if (versionStream==null) {
    +          if (isDevelopmentEnvironment()) {
    +              // allowed for dev env
    +              log.trace("No maven resource file "+MVN_VERSION_RESOURCE_FILE+" available");
    +          } else {
    +              log.warn("No maven resource file "+MVN_VERSION_RESOURCE_FILE+" available");
    +          }
    +          return;
    +      }
           versionProperties.load(checkNotNull(versionStream));
    -    } catch (IOException exception) {
    -      throw new IllegalStateException(format("Unable to load version resource file '%s'", VERSION_RESOURCE_FILE), exception);
    +    } catch (IOException e) {
    +      log.warn("Error reading maven resource file "+MVN_VERSION_RESOURCE_FILE+": "+e, e);
         }
    -    return checkNotNull(versionProperties.getProperty(VERSION_PROPERTY_NAME), VERSION_PROPERTY_NAME);
    +  }
    +
    +  /** reads properties from brooklyn-core's manifest */
    +  private void readPropertiesFromOsgiResource(ClassLoader resourceLoader, String symbolicName) {
    +      Enumeration<URL> paths;
    +      try {
    +          paths = BrooklynVersion.class.getClassLoader().getResources(MANIFEST_PATH);
    +      } catch (IOException e) {
    +          // shouldn't happen
    +          throw Exceptions.propagate(e);
    +      }
    +      while (paths.hasMoreElements()) {
    +          URL u = paths.nextElement();
    +          try {
    +              ManifestHelper mh = Osgis.ManifestHelper.forManifest(u.openStream());
    +              if (BROOKLYN_CORE_SYMBOLIC_NAME.equals(mh.getSymbolicName())) {
    +                  Attributes attrs = mh.getManifest().getMainAttributes();
    +                  for (Object key: attrs.keySet()) {
    +                      // key is an Attribute.Name; toString converts to string
    +                      versionProperties.put(key.toString(), attrs.getValue(key.toString()));
    +                  }
    +                  return;
    +              }
    +          } catch (Exception e) {
    +              Exceptions.propagateIfFatal(e);
    +              log.warn("Error reading OSGi manifest from "+u+" when determining version properties: "+e, e);
    +          }
    +      }
    +      if (isDevelopmentEnvironment()) {
    +          // allowed for dev env
    +          log.trace("No OSGi manifest available to determine version properties");
    +      } else {
    +          log.warn("No OSGi manifest available to determine version properties");
    +      }
    +  }
    +
    +  /** 
    +   * Returns whether this is a Brooklyn dev environment,
    +   * specifically core/target/classes/ is on the classpath for the org.apache.brooklyn.core project.
    +   * <p>
    +   * In a packaged or library build of Brooklyn (normal usage) this should return false,
    +   * and all OSGi components should be available.
    +   */
    +  public static boolean isDevelopmentEnvironment() {
    +      Boolean isDevEnv = IS_DEV_ENV.get();
    +      if (isDevEnv!=null) return isDevEnv;
    +      synchronized (IS_DEV_ENV) {
    +          isDevEnv = computeIsDevelopmentEnvironment();
    +          IS_DEV_ENV.set(isDevEnv);
    +          return isDevEnv;
    +      }
    +  }
    +  
    +  private static boolean computeIsDevelopmentEnvironment() {
    +      Enumeration<URL> paths;
    +      try {
    +          paths = BrooklynVersion.class.getClassLoader().getResources("brooklyn/BrooklynVersion.class");
    +      } catch (IOException e) {
    +          // shouldn't happen
    +          throw Exceptions.propagate(e);
    +      }
    +      while (paths.hasMoreElements()) {
    +          URL u = paths.nextElement();
    +          if (u.getPath().endsWith("core/target/classes/brooklyn/BrooklynVersion.class")) {
    +              try {
    +                  log.debug("Brooklyn debug environment detected; BrooklynVersion class is at: "+u);
    +                  return true;
    +              } catch (Exception e) {
    +                  Exceptions.propagateIfFatal(e);
    +                  log.warn("Error reading manifest to determine whether this is a development environment: "+e, e);
    +              }
    +          }
    +      }
    +      return false;
    +  }
    +
    +  public void logSummary() {
    +      log.debug("Brooklyn version "+getVersion()+" (git SHA1 "+getSha1FromOsgiManifest()+")");
    +  }
    +
    +  /** @deprecated since 0.7.0, redundant with {@link #get()} */ @Deprecated
    +  public static String getVersionFromStatic() {
    +      return VERSION_FROM_STATIC;
       }
     
       public static String get() {
    -    return INSTANCE.version;
    +    return getVersionFromStatic();
    --- End diff --
    
    Would prefer this to call `getVersion()`, rather than the deprecated `getVersionFromStatic()`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24322893
  
    --- Diff: api/src/main/java/brooklyn/basic/AbstractBrooklynObjectSpec.java ---
    @@ -121,4 +121,21 @@ protected final void checkIsImplementation(Class<?> val, Class<? super T> requir
             if (Modifier.isAbstract(val.getModifiers())) throw new IllegalStateException("Implementation "+val+" is abstract, but must be a non-abstract class");
         }
     
    +    @Override
    +    public boolean equals(Object obj) {
    +        if (obj==null) return false;
    --- End diff --
    
    i don't think it breaks the contract of hashcode -- if a spec is modified in such a way that the hashcode changes, then the equality semantics will have changed in the expected way.
    
    as we already have `Object.equals/hashcode` i think this implementation is an improvement, and putting things into a test convenience would just make things harder to find.
    
    making the spec immutable would be the elegant way to solve this; i wouldn't be strongly opposed although i don't see any major benefit at this time for the extra code. (is spec mutability confusion really a problem?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#issuecomment-73322740
  
    requires #478 
    
    also fixes deserialization of entity specs with catalog items.  see the two `ActivePartial...Test` classes. /cc @neykov @bhairavi25


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24322448
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.Semaphore;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.basic.BrooklynObjectInternal;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore;
    +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics;
    +import brooklyn.entity.rebind.transformer.CompoundTransformer;
    +import brooklyn.management.ha.ManagementNodeState;
    +import brooklyn.mementos.BrooklynMementoPersister;
    +import brooklyn.mementos.BrooklynMementoRawData;
    +import brooklyn.mementos.BrooklynMementoRawData.Builder;
    +import brooklyn.mementos.Memento;
    +import brooklyn.util.collections.MutableList;
    +import brooklyn.util.collections.MutableSet;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back.
    + */
    +public class ActivePartialRebindIteration extends RebindIteration {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(ActivePartialRebindIteration.class);
    +    
    +    protected Iterator<BrooklynObject> objectsToRebindInitial;
    +    protected Collection<BrooklynObject> objectsToRebindFinal;
    +    protected List<CompoundTransformer> transformers = MutableList.of();
    +    
    +    public ActivePartialRebindIteration(RebindManagerImpl rebindManager, 
    +            ManagementNodeState mode,
    +            ClassLoader classLoader, RebindExceptionHandler exceptionHandler,
    +            Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess
    +            ) {
    +        super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +    }
    +
    +    @Override
    +    protected boolean isRebindingActiveAgain() {
    +        return true;
    +    }
    +    
    +    public void setObjectIterator(Iterator<BrooklynObject> objectsToRebind) {
    +        this.objectsToRebindInitial = objectsToRebind;
    +    }
    +    
    +    public void applyTransformer(CompoundTransformer transformer) {
    +        if (transformer!=null)
    +            transformers.add(transformer);
    +    }
    +    
    +    @Override
    +    protected void doRun() throws Exception {
    --- End diff --
    
    Could this be implemented by just overriding `loadManifestFiles()` and `rebuildCatalog()`, rather than `doRun()`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24323876
  
    --- Diff: core/src/main/java/brooklyn/catalog/internal/CatalogUtils.java ---
    @@ -227,4 +228,13 @@ public static String getVersionedId(String id, String version) {
             }
         }
     
    +    @Beta
    +    public static EntitySpec<?> createEntitySpec(ManagementContext mgmt, CatalogItem<?, ?> catalogItem) {
    --- End diff --
    
    good spot.  i completely agree -- i had missed that config etc would be ignored (d'oh).
    
    have moved to `CatalogTestUtils.createEssentialEntitySpec`.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24325782
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/RebindManagerImpl.java ---
    @@ -361,6 +364,38 @@ public void stop() {
             if (persistenceStoreAccess != null) persistenceStoreAccess.stop(true);
         }
         
    +        
    +    public void rebindPartialActive(CompoundTransformer transformer, Iterator<BrooklynObject> objectsToRebind) {
    +        final ClassLoader classLoader = 
    +            managementContext.getCatalog().getRootClassLoader();
    +        final RebindExceptionHandler exceptionHandler = 
    +            RebindExceptionHandlerImpl.builder()
    +                .danglingRefFailureMode(danglingRefFailureMode)
    +                .danglingRefQuorumRequiredHealthy(danglingRefsQuorumRequiredHealthy)
    +                .rebindFailureMode(rebindFailureMode)
    +                .addPolicyFailureMode(addPolicyFailureMode)
    +                .loadPolicyFailureMode(loadPolicyFailureMode)
    +                .build();
    +        final ManagementNodeState mode = getRebindMode();
    +
    +        ActivePartialRebindIteration iteration = new ActivePartialRebindIteration(this, mode, classLoader, exceptionHandler,
    +            rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess);
    +
    +        iteration.setObjectIterator(objectsToRebind);
    +        iteration.applyTransformer(transformer);
    +        iteration.run();
    +    }
    +    
    +    public void rebindPartialActive(CompoundTransformer transformer, String ...objectsToRebindIds) {
    +        List<BrooklynObject> objectsToRebind = MutableList.of();
    +        for (String objectId: objectsToRebindIds) {
    +            BrooklynObject obj = managementContext.lookup(objectId);
    +            if (obj instanceof Entity) obj = Entities.deproxy((Entity)obj);
    --- End diff --
    
    Do these have to be deproxy'ed here? Callers of `rebindPartialActive(CompoundTransformer, Iterator<BrooklynObject>) ` probably wouldn't realise that they can't just pass proxy objects. Therefore I'd be inclined to have the deproxy'ing happen somehwere in that other `rebindPartialActive` method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#issuecomment-73516077
  
    Thanks @aledsage.  Great comments.  I've added a bunch more javadoc, and I was already in the process of drafting a mail to dev.
    
    I think they're all addressed, and will merge if the latest push passes buildhive, unless you have any further comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24325712
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/RebindManagerImpl.java ---
    @@ -361,6 +364,38 @@ public void stop() {
             if (persistenceStoreAccess != null) persistenceStoreAccess.stop(true);
         }
         
    +        
    +    public void rebindPartialActive(CompoundTransformer transformer, Iterator<BrooklynObject> objectsToRebind) {
    +        final ClassLoader classLoader = 
    +            managementContext.getCatalog().getRootClassLoader();
    +        final RebindExceptionHandler exceptionHandler = 
    +            RebindExceptionHandlerImpl.builder()
    +                .danglingRefFailureMode(danglingRefFailureMode)
    --- End diff --
    
    Do we definitely want to use the same exception handler as was used at start-up? What happens if there is an unhandled exception? What state will the entities be left in? I presume that if the exception is early (e.g. when loading/transforming state etc) then it's like a no-op, but if during the entity's manage() then we'll have changed some entities but not others?
    
    Doing the partial-rebind at run-time feels more dangerous than at startup so possibly want different exception handling (e.g. on startup, might be happy to fail-fast, so JVM exits).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24324151
  
    --- Diff: core/src/main/java/brooklyn/entity/group/DynamicClusterImpl.java ---
    @@ -130,12 +130,14 @@ public ZoneFailureDetector apply(final String input) {
     
         private static final Function<Collection<Entity>, Entity> defaultRemovalStrategy = new Function<Collection<Entity>, Entity>() {
             @Override public Entity apply(Collection<Entity> contenders) {
    -            // choose newest entity that is stoppable
    +            // choose newest entity that is stoppable, or if none are stoppable take the newest non-stoppable
                 long newestTime = 0;
                 Entity newest = null;
     
                 for (Entity contender : contenders) {
    -                if (contender instanceof Startable && contender.getCreationTime() > newestTime) {
    +                boolean newer = contender.getCreationTime() > newestTime;
    --- End diff --
    
    have added


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24326582
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.
    + */
    +package brooklyn.entity.rebind;
    +
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.Semaphore;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import brooklyn.basic.BrooklynObject;
    +import brooklyn.basic.BrooklynObjectInternal;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore;
    +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics;
    +import brooklyn.entity.rebind.transformer.CompoundTransformer;
    +import brooklyn.management.ha.ManagementNodeState;
    +import brooklyn.mementos.BrooklynMementoPersister;
    +import brooklyn.mementos.BrooklynMementoRawData;
    +import brooklyn.mementos.BrooklynMementoRawData.Builder;
    +import brooklyn.mementos.Memento;
    +import brooklyn.util.collections.MutableList;
    +import brooklyn.util.collections.MutableSet;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back.
    --- End diff --
    
    Could do with more javadoc, to say that this is for use on a running primary management node etc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#issuecomment-73488919
  
    Test failure is `brooklyn.management.ha.HighAvailabilityManagerInMemoryTest.tearDown`:
    
        Caused by: java.lang.IllegalStateException: Has different management context: null; expected LocalManagementContext[A6LXdOF0-oBWEBSQn]
        	at brooklyn.management.internal.EntityManagementSupport.onManagementStopping(EntityManagementSupport.java:271)
        	at brooklyn.management.internal.LocalEntityManager$3.apply(LocalEntityManager.java:400)
        	at brooklyn.management.internal.LocalEntityManager$3.apply(LocalEntityManager.java:397)
        	at brooklyn.management.internal.LocalEntityManager.recursively(LocalEntityManager.java:526)
        	at brooklyn.management.internal.LocalEntityManager.recursively(LocalEntityManager.java:531)
        	at brooklyn.management.internal.LocalEntityManager.unmanage(LocalEntityManager.java:397)
        	at brooklyn.management.internal.LocalEntityManager.unmanage(LocalEntityManager.java:356)
        	at brooklyn.management.internal.LocalEntityManager.unmanage(LocalEntityManager.java:352)
        	at brooklyn.entity.basic.AbstractApplication.stop(AbstractApplication.java:222)



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24326173
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/persister/XmlMementoSerializer.java ---
    @@ -341,5 +353,151 @@ public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext co
                 return lookupContext.lookupManagementContext();
             }
         }
    +
    +    /** When reading/writing specs, it checks whether there is a catalog item id set and uses it to load */
    +    public class SpecConverter extends ReflectionConverter {
    +        SpecConverter() {
    +            super(xstream.getMapper(), xstream.getReflectionProvider());
    +        }
    +        @Override
    +        public boolean canConvert(@SuppressWarnings("rawtypes") Class type) {
    +            return AbstractBrooklynObjectSpec.class.isAssignableFrom(type);
    +        }
    +        @Override
    +        public void marshal(Object source, HierarchicalStreamWriter writer, MarshallingContext context) {
    +            if (source == null) return;
    +            AbstractBrooklynObjectSpec<?, ?> spec = (AbstractBrooklynObjectSpec<?, ?>) source;
    +            String catalogItemId = spec.getCatalogItemId();
    +            if (Strings.isNonBlank(catalogItemId)) {
    +                // write this field first, so we can peek at it when we read
    +                writer.startNode("catalogItemId");
    +                writer.setValue(catalogItemId);
    +                writer.endNode();
    +                
    +                // we're going to write the catalogItemId field twice :( but that's okay.
    +                // better solution would be to have mark/reset on reader so we can peek for such a field;
    +                // see comment below
    +                super.marshal(source, writer, context);
    +            } else {
    +                super.marshal(source, writer, context);
    +            }
    +        }
    +        @Override
    +        public Object unmarshal(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            String catalogItemId = null;
    +            instantiateNewInstanceSettingCache(reader, context);
    +            
    +            if (reader instanceof PathTrackingReader) {
    +                // have to assume this is first; there is no mark/reset support on these readers
    +                // (if there were then it would be easier, we could just look for that child anywhere,
    +                // and not need a custom writer!)
    +                if ("catalogItemId".equals( ((PathTrackingReader)reader).peekNextChild() )) {
    +                    // cache the instance
    +                    
    +                    reader.moveDown();
    +                    catalogItemId = reader.getValue();
    +                    reader.moveUp();
    +                }
    +            }
    +            boolean customLoaderSet = false;
    +            try {
    +                if (Strings.isNonBlank(catalogItemId)) {
    +                    if (lookupContext==null) throw new NullPointerException("lookupContext required to load catalog item "+catalogItemId);
    +                    CatalogItem<?, ?> cat = CatalogUtils.getCatalogItemOptionalVersion(lookupContext.lookupManagementContext(), catalogItemId);
    +                    if (cat==null) throw new NoSuchElementException("catalog item: "+catalogItemId);
    +                    BrooklynClassLoadingContext clcNew = CatalogUtils.newClassLoadingContext(lookupContext.lookupManagementContext(), cat);
    +                    pushXstreamCustomClassLoader(clcNew);
    +                    customLoaderSet = true;
    +                }
    +                
    +                AbstractBrooklynObjectSpec<?, ?> result = (AbstractBrooklynObjectSpec<?, ?>) super.unmarshal(reader, context);
    +                // we wrote it twice so this shouldn't be necessary; but if we fix it so we only write once, we'd need this
    +                result.catalogItemId(catalogItemId);
    +                return result;
    +            } finally {
    +                instance = null;
    +                if (customLoaderSet) {
    +                    popXstreamCustomClassLoader();
    +                }
    +            }
    +        }
    +
    +        Object instance;
    +        
    +        @Override
    +        protected Object instantiateNewInstance(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            // the super calls getAttribute which requires that we have not yet done moveDown,
    +            // so we do this earlier and cache it for when we call super.unmarshal
    +            if (instance==null)
    +                throw new IllegalStateException("Instance should be created and cached");
    +            return instance;
    +        }
    +        protected void instantiateNewInstanceSettingCache(HierarchicalStreamReader reader, UnmarshallingContext context) {
    +            instance = super.instantiateNewInstance(reader, context);
    +        }
    +    }
    +    
    +    Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    AtomicReference<Thread> xstreamLockOwner = new AtomicReference<Thread>();
    +    int lockCount;
         
    +    /** Must be accompanied by a corresponding {@link #popXstreamCustomClassLoader()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    protected void pushXstreamCustomClassLoader(BrooklynClassLoadingContext clcNew) {
    +        acquireXstreamLock();
    +        BrooklynClassLoadingContext oldClc;
    +        if (!contexts.isEmpty()) {
    +            oldClc = contexts.peek();
    +        } else {
    +            // TODO XmlMementoSerializer should take a BCLC instead of a CL
    +            oldClc = JavaBrooklynClassLoadingContext.create(lookupContext.lookupManagementContext(), xstream.getClassLoader());
    +        }
    +        BrooklynClassLoadingContextSequential clcMerged = new BrooklynClassLoadingContextSequential(lookupContext.lookupManagementContext(),
    +            oldClc, clcNew);
    +        contexts.push(clcMerged);
    +        cls.push(xstream.getClassLoader());
    +        ClassLoader newCL = ClassLoaderFromBrooklynClassLoadingContext.of(clcMerged);
    +        xstream.setClassLoader(newCL);
    +    }
    +
    +    protected void popXstreamCustomClassLoader() {
    +        synchronized (xstreamLockOwner) {
    +            releaseXstreamLock();
    +            xstream.setClassLoader(cls.pop());
    +            contexts.pop();
    +        }
    +    }
    +    
    +    protected void acquireXstreamLock() {
    +        synchronized (xstreamLockOwner) {
    +            while (true) {
    +                if (xstreamLockOwner.compareAndSet(null, Thread.currentThread()) || 
    +                    Thread.currentThread().equals( xstreamLockOwner.get() )) {
    +                    break;
    +                }
    +                try {
    +                    xstreamLockOwner.wait(1000);
    --- End diff --
    
    Why is this calling `wait(1000)` when nothing calls `notifyAll()`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24320959
  
    --- Diff: core/src/test/java/brooklyn/BrooklynVersionTest.java ---
    @@ -19,18 +19,63 @@
     package brooklyn;
     
     import static org.testng.Assert.assertEquals;
    +
    +import java.net.URL;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.Assert;
     import org.testng.annotations.Test;
     
    +import brooklyn.util.text.Strings;
    +
     public class BrooklynVersionTest {
     
    +  private static final Logger log = LoggerFactory.getLogger(BrooklynVersionTest.class);
    +    
       @Test
       public void testGetVersion() {
    -      assertEquals(BrooklynVersion.get(), BrooklynVersion.INSTANCE.getVersionFromStatic());
    +      assertEquals(BrooklynVersion.get(), BrooklynVersion.INSTANCE.getVersion());
       }
         
       @Test
       public void testGetHardcodedClasspathVersion() {
    -    assertEquals(BrooklynVersion.INSTANCE.getVersionFromClasspath(), "0.0.0-SNAPSHOT");
    +      @SuppressWarnings("deprecation")
    +      String v = BrooklynVersion.INSTANCE.getVersionFromClasspath();
    +      Assert.assertTrue(BrooklynVersion.get().equals(v) || "0.0.0-SNAPSHOT".equals(v), v);
    +  }
    +  
    +  @Test
    +  public void testGetFromMaven() {
    +      String v = BrooklynVersion.INSTANCE.getVersionFromMavenProperties();
    +      Assert.assertTrue(v==null || BrooklynVersion.get().equals(v), v);
    +  }
    +  
    +  @Test
    +  public void testGetFromOsgi() {
    +      String v = BrooklynVersion.INSTANCE.getVersionFromOsgiManifest();
    +      Assert.assertTrue(v==null || BrooklynVersion.get().equals(v), v);
    +  }
    +  
    +  @Test
    +  public void testGetOsgiSha1() {
    +      String sha1 = BrooklynVersion.INSTANCE.getSha1FromOsgiManifest();
    +      log.info("sha1: "+sha1);
    +      if (Strings.isNonBlank(sha1) || BrooklynVersion.isDevelopmentEnvironment())
    +          return;
    +      // we might not have a SHA1 if it's a standalone source build; just log warn in that case
    --- End diff --
    
    Longer term, would prefer to create a special test group for things to be skipped with a source build. Folk aren't going to notice this warning when running the test suite.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24326988
  
    --- Diff: core/src/main/java/brooklyn/management/internal/LocalEntityManager.java ---
    @@ -354,23 +362,23 @@ private void unmanage(final Entity e, ManagementTransitionMode mode, boolean has
             
             if (hasBeenReplaced) {
                 // we are unmanaging an old instance after having replaced it
    -            // (called from manage(...)
    +            // don't unmanage or even clear its fields, because there might be references to it
                 
    -            if (mode==ManagementTransitionMode.REBINDING_NO_LONGER_PRIMARY) {
    -                // when migrating away, these all need to be called
    +            if (mode.wasReadOnly()) {
    +                // if coming *from* read only; nothing needed
    +            } else {
    +                if (!mode.wasPrimary()) {
    +                    log.warn("Unexpected mode "+mode+" for unmanage-replace "+e+" (applying anyway)");
    --- End diff --
    
    I'm a bit worried about these kind of warnings around the place, in that we might not notice them happening until it's too late (e.g. until a customer reports a problem). Longer term, is there something we can do to more strongly assert what modes are expected/allowed in different circumstances?
    
    (I'm not specifically talking about this particular piece of code; more in general so those maintaining the code know exactly what each code branch is for. On balance, agree it's often better to warn and continue than fail completely in production - but in tests and QA then failing fast is arguably better! Perhaps it should be an `assert`?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24325437
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/RebindContextLookupContext.java ---
    @@ -60,6 +66,9 @@ public RebindContextLookupContext(ManagementContext managementContext, RebindCon
         @Override public Entity lookupEntity(String id) {
             Entity result = rebindContext.getEntity(id);
             if (result == null) {
    +            result = managementContext.lookup(id, Entity.class);
    --- End diff --
    
    yes - i didn't want to break compatibility, but now i see that i did, and that the only usages are during rebind.  have removed that field and added javadoc that this class looks in both (which helps justify its existence besides the rebind context).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24321734
  
    --- Diff: core/src/main/java/brooklyn/entity/group/DynamicClusterImpl.java ---
    @@ -130,12 +130,14 @@ public ZoneFailureDetector apply(final String input) {
     
         private static final Function<Collection<Entity>, Entity> defaultRemovalStrategy = new Function<Collection<Entity>, Entity>() {
             @Override public Entity apply(Collection<Entity> contenders) {
    -            // choose newest entity that is stoppable
    +            // choose newest entity that is stoppable, or if none are stoppable take the newest non-stoppable
                 long newestTime = 0;
                 Entity newest = null;
     
                 for (Entity contender : contenders) {
    -                if (contender instanceof Startable && contender.getCreationTime() > newestTime) {
    +                boolean newer = contender.getCreationTime() > newestTime;
    --- End diff --
    
    Is there a unit test for this, using the `DynamicCluster` with non-startable entities?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#discussion_r24326262
  
    --- Diff: core/src/main/java/brooklyn/entity/rebind/RebindManagerImpl.java ---
    @@ -361,6 +364,38 @@ public void stop() {
             if (persistenceStoreAccess != null) persistenceStoreAccess.stop(true);
         }
         
    +        
    +    public void rebindPartialActive(CompoundTransformer transformer, Iterator<BrooklynObject> objectsToRebind) {
    +        final ClassLoader classLoader = 
    +            managementContext.getCatalog().getRootClassLoader();
    +        final RebindExceptionHandler exceptionHandler = 
    +            RebindExceptionHandlerImpl.builder()
    +                .danglingRefFailureMode(danglingRefFailureMode)
    --- End diff --
    
    have added a TODO
    
    the update-proxy-then-unmanage-old is mostly atomic so most failures should be pretty safe, but i'm sure there will be edge conditions
    
    when we better understand the failure dimensions (and have visualization) then we can tidy this up


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: Active partial rebind allowing ca...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/506#issuecomment-73492555
  
    i've fixed the failed test, hopefully buildhive agrees.  will look through your other comments when you're finished.
    
    also some new comments on a slow location persistence leak (no one unmanages locations created for an entity when that entity is cleaned up, /cc @neykov)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---