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 2017/06/06 15:07:15 UTC

[GitHub] brooklyn-server pull request #718: WIP - DO NOT MERGE - improve OSGi seriali...

GitHub user ahgittin opened a pull request:

    https://github.com/apache/brooklyn-server/pull/718

    WIP - DO NOT MERGE - improve OSGi serialization strategies

    Discovered that the current deserialization strategy for OSGi does not work in the following case: 
     `foo-wrapped` extending yaml type `foo` which depends on bundle `foo-bundle` and extends java type `foo.Foo` in `foo-bundle`.  The class-loading-context for `foo-wrapped` does not look in `foo-bundle`.  Creating `foo-wrapped` works because it creates `foo` but trying to load class `foo.Foo` using the CLC for `foo-wrapped` fails.  This impacts _specs_:  we fail if we rebind a cluster whose member spec is a `foo-wrapped` for this reason.  The serialized form of `EntitySpec` records `type` as `foo.Foo` and -- for lookup purposes -- `catalogItemId` `foo-wrapped`.
    
    So far this just adds a failing test, but provides a better place for discussing than IRC.  I will summarise IRC discussion here and @geomacy @neykov @aledsage we can continue the discussion.

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

    $ git pull https://github.com/ahgittin/brooklyn-server osgi-serialization-better

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

    https://github.com/apache/brooklyn-server/pull/718.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 #718
    
----
commit a3d256d7ddf28f76bd6e0b00b0b17b446025752b
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2017-06-06T13:20:21Z

    failing test to illustrate osgi entity spec descendant rebind bug

----


---
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] brooklyn-server issue #718: improvement to OSGi serialization strategies (wr...

Posted by neykov <gi...@git.apache.org>.
Github user neykov commented on the issue:

    https://github.com/apache/brooklyn-server/pull/718
  
    Thanks @ahgittin. Local tests are passing so merging.


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120765509
  
    --- Diff: core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlSerializer.java ---
    @@ -89,33 +96,49 @@ protected MapperWrapper wrapMapper(MapperWrapper next) {
         }
     
         /**
    -     * JCC is used when class names are serialized/deserialized and no alias is defined;
    -     * it is configured in XStream *without* access to the XStream mapper.
    +     * JCC is used when Class instances are serialized/deserialized as a value 
    +     * (not as tags) and there are no aliases configured for that type.
    +     * It is configured in XStream default *without* access to the XStream mapper,
    +     * which is meant to apply when serializing the type name for instances of that type.
    +     * <p>
          * However we need a few selected mappers (see {@link #wrapMapperForAllLowLevelMentions(Mapper)} )
    -     * in order to effect renames at the low level, but many of the mappers must NOT be used,
    +     * to apply to all class renames, but many of the mappers must NOT be used,
          * e.g. because some might intercept all Class<? extends Entity> references
          * (and that interception is only wanted when serializing <i>instances</i>,
          * as in {@link #wrapMapperForNormalUsage(Mapper)}).
          * <p>
    -     * This can typically be done simply by registering our own instance (due to order guarantee of PrioritizedList),
    +     * This can typically be done simply by registering our own instance of this (due to order guarantee of PrioritizedList),
          * after the instance added by XStream.setupConverters()
          */
         private JavaClassConverter newCustomJavaClassConverter() {
             return new JavaClassConverter(wrapMapperForAllLowLevelMentions(new DefaultMapper(xstream.getClassLoaderReference()))) {};
         }
         
    -    /** Adds mappers needed for *any* reference to a class, e.g. when names are used for inner classes, or classes are renamed;
    -     * this *excludes* basic mentions, however, because most rewrites should *not* be applied at this deep level;
    -     * mappers which effect aliases or intercept references to entities are usually NOT be invoked in this low-level pathway.
    -     * See {@link #newCustomJavaClassConverter()}. */
    +    /** Adds mappers needed for *any* reference to a class, both "normal" usage (when xstream wants a mapper)
    +     * and Class conversion (when xstream needs to serialize an instance of Class and doesn't have an alias).
    +     * <p>
    +     * This should apply when nice names are used for inner classes, or classes are renamed;
    +     * however mappers which affect aliases or intercept references to entities are usually 
    +     * NOT be invoked in this low-level pathway. See {@link #newCustomJavaClassConverter()}. */
    +    // Developer note - this is called by the xstream subclass constructor in the constructor of this class,
    +    // so very few fields are populated
         protected MapperWrapper wrapMapperForAllLowLevelMentions(Mapper next) {
             MapperWrapper result = new CompilerIndependentOuterClassFieldMapper(next);
    +        
             Supplier<ClassLoader> classLoaderSupplier = new Supplier<ClassLoader>() {
                 @Override public ClassLoader get() {
                     return xstream.getClassLoaderReference().getReference();
                 }
             };
    -        return new ClassRenamingMapper(result, deserializingClassRenames, classLoaderSupplier);
    +        result = new ClassRenamingMapper(result, deserializingClassRenames, classLoaderSupplier);
    +        result = new OsgiClassnameMapper(new Supplier<XStream>() {
    +            @Override public XStream get() { return xstream; } }, result);
    --- End diff --
    
    +1 constructor call yuck - half curious what a more elegant solution is.  though also happy to leave this alone as it's approximately once a year this needs looking at and if we do yoml we can leave all this.


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120764791
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/mgmt/classloading/ClassLoaderFromStackOfBrooklynClassLoadingContext.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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 org.apache.brooklyn.core.mgmt.classloading;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Stack;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContextSequential;
    +import org.apache.brooklyn.core.mgmt.classloading.ClassLoaderFromBrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.JavaBrooklynClassLoadingContext;
    +import org.apache.brooklyn.util.core.ClassLoaderUtils;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.ClassLoadingContext;
    +
    +/** Provides a stack where {@link ClassLoadingContext} instances can be pushed and popped,
    + * with the most recently pushed one in effect at any given time.
    + * <p>
    + * This is useful when traversing a tree some of whose elements may bring custom search paths,
    + * and the worker wants to have a simple view of the loader to use at any point in time.
    + * For example XStream keeps a class-loader in effect but when deserializing things 
    + * some of those things may define bundles to use. */
    +public class ClassLoaderFromStackOfBrooklynClassLoadingContext extends ClassLoader {
    +    
    +    private final Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    private final Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    private final AtomicReference<Thread> lockOwner = new AtomicReference<Thread>();
    +    private ManagementContext mgmt;
    +    private ClassLoader currentClassLoader;
    +    private AtomicReference<ClassLoaderUtils> currentLoader = new AtomicReference<>();
    +    private int lockCount;
    +    
    +    public ClassLoaderFromStackOfBrooklynClassLoadingContext(ClassLoader classLoader) {
    +        setCurrentClassLoader(classLoader);
    +    }
    +    
    +    public void setManagementContext(ManagementContext mgmt) {
    +        this.mgmt = checkNotNull(mgmt, "mgmt");
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +
    +    @Override
    +    protected Class<?> findClass(String name) throws ClassNotFoundException {
    +        return currentLoader.get().loadClass(name);
    +    }
    +    
    +    @Override
    +    protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
    +        return findClass(name);
    +    }
    +
    +    /** Must be accompanied by a corresponding {@link #popClassLoadingContext()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    public void pushClassLoadingContext(BrooklynClassLoadingContext clcNew) {
    +        acquireLock();
    +        BrooklynClassLoadingContext oldClc;
    +        if (!contexts.isEmpty()) {
    +            oldClc = contexts.peek();
    +        } else {
    +            // TODO xml serializers using this should take a BCLC instead of a CL
    +            oldClc = JavaBrooklynClassLoadingContext.create(mgmt, getCurrentClassLoader());
    +        }
    +        BrooklynClassLoadingContextSequential clcMerged = new BrooklynClassLoadingContextSequential(mgmt, oldClc, clcNew);
    +        ClassLoader newCL = ClassLoaderFromBrooklynClassLoadingContext.of(clcMerged);
    +        contexts.push(clcMerged);
    +        cls.push(getCurrentClassLoader());
    +        setCurrentClassLoader(newCL);
    +    }
    +
    +    public void popClassLoadingContext() {
    +        synchronized (lockOwner) {
    +            releaseXstreamLock();
    +            setCurrentClassLoader(cls.pop());
    +            contexts.pop();
    +        }
    +    }
    +    
    +    private ClassLoader getCurrentClassLoader() {
    +        return currentClassLoader;
    +    }
    +    
    +    private void setCurrentClassLoader(ClassLoader classLoader) {
    +        currentClassLoader = checkNotNull(classLoader);
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +    
    +    protected void acquireLock() {
    +        synchronized (lockOwner) {
    --- End diff --
    
    I thought I took `xstream` out of the names but I missed at least one!
    
    I'm really not sure what the threading guarantees/requirements are.  It was made just for xstream to support osgi and follows whatever was needed there.  i was amused to discover despite its name it had nothing to do with osgi or xstream!  i thought i needed to refactor it for tests though i ended up doing the tests differents; still the rename felt worth keeping.
    
    my 2p it's good to be reminded of this, there might well be other places such a thing might be useful, and if there is then we'll better think through threading and locking.


---
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] brooklyn-server issue #718: WIP - DO NOT MERGE - improve OSGi serialization ...

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

    https://github.com/apache/brooklyn-server/pull/718
  
    we also need a way to fix existing deployments that suffer this problem.  any of the above fixes should suffice, but i wonder whether there is a workaround that doesn't involve upgrading to a new version once we've fixed it.
    
    if you're using a recent brooklyn, we can take advantage of the fact that `ClassRenamingMapper` applies to "low-level mentions", and understands OSGi prefixes.  we can define a rename from `foo.Foo` to `foo-bundle:foo.Foo` and that works.
    
    however i've a deployment that predates the above behaviour, and i'd like to be able to load its state.  i can't find a way, not even with edits to the persisted state.  :(


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120659368
  
    --- Diff: core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlSerializer.java ---
    @@ -89,33 +96,49 @@ protected MapperWrapper wrapMapper(MapperWrapper next) {
         }
     
         /**
    -     * JCC is used when class names are serialized/deserialized and no alias is defined;
    -     * it is configured in XStream *without* access to the XStream mapper.
    +     * JCC is used when Class instances are serialized/deserialized as a value 
    +     * (not as tags) and there are no aliases configured for that type.
    +     * It is configured in XStream default *without* access to the XStream mapper,
    +     * which is meant to apply when serializing the type name for instances of that type.
    +     * <p>
          * However we need a few selected mappers (see {@link #wrapMapperForAllLowLevelMentions(Mapper)} )
    -     * in order to effect renames at the low level, but many of the mappers must NOT be used,
    +     * to apply to all class renames, but many of the mappers must NOT be used,
          * e.g. because some might intercept all Class<? extends Entity> references
          * (and that interception is only wanted when serializing <i>instances</i>,
          * as in {@link #wrapMapperForNormalUsage(Mapper)}).
          * <p>
    -     * This can typically be done simply by registering our own instance (due to order guarantee of PrioritizedList),
    +     * This can typically be done simply by registering our own instance of this (due to order guarantee of PrioritizedList),
          * after the instance added by XStream.setupConverters()
          */
         private JavaClassConverter newCustomJavaClassConverter() {
             return new JavaClassConverter(wrapMapperForAllLowLevelMentions(new DefaultMapper(xstream.getClassLoaderReference()))) {};
         }
         
    -    /** Adds mappers needed for *any* reference to a class, e.g. when names are used for inner classes, or classes are renamed;
    -     * this *excludes* basic mentions, however, because most rewrites should *not* be applied at this deep level;
    -     * mappers which effect aliases or intercept references to entities are usually NOT be invoked in this low-level pathway.
    -     * See {@link #newCustomJavaClassConverter()}. */
    +    /** Adds mappers needed for *any* reference to a class, both "normal" usage (when xstream wants a mapper)
    +     * and Class conversion (when xstream needs to serialize an instance of Class and doesn't have an alias).
    +     * <p>
    +     * This should apply when nice names are used for inner classes, or classes are renamed;
    +     * however mappers which affect aliases or intercept references to entities are usually 
    +     * NOT be invoked in this low-level pathway. See {@link #newCustomJavaClassConverter()}. */
    +    // Developer note - this is called by the xstream subclass constructor in the constructor of this class,
    +    // so very few fields are populated
         protected MapperWrapper wrapMapperForAllLowLevelMentions(Mapper next) {
    --- End diff --
    
    Should this really be `protected`?  Do you want derived classes to be able to modify this? Is the desired extension point not the `wrapMapperForNormalUsage`?


---
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] brooklyn-server issue #718: WIP - DO NOT MERGE - improve OSGi serialization ...

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

    https://github.com/apache/brooklyn-server/pull/718
  
    in `SpecConverter`, if the catalog item ID is set i think that implies the java type so we _could_ skip writing and _should_ ignore it (just use the catalog item ID) when reading.  that would solve the problem identified here.
    
    a different solution would be to make sure the CLC for `foo-wrapped` includes bundles declared in ancestor types.  that's probably (?) wanted in any case, so feels like we should do both of these.
    
    those are quicker, necessary, and independent fixes instead of the suggestion in the previous post about the wider osgi serialization approach.  (that approach should also look at comments in `ClassRenamingMapper` about unifying it with `OsgiClassnameMapper`; and a related improvement is for the search path to refer to bundles.)


---
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] brooklyn-server issue #718: WIP - DO NOT MERGE - improve OSGi serialization ...

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

    https://github.com/apache/brooklyn-server/pull/718
  
    scratch the `ClassLoadingContext` suggestion:  it's hard, and I've come to the conclusion it's misguided.  it serves a different, narrower purpose than a resulting instance's search path (where ancestors probably are wanted), but in short it makes sense to defer that "ancestor-bundles-are-transitively-visible" decision to the instantiation engine (which might want to hide it) rather than mandate it as part of the CLC for an item.  aside from `EntitySpec` we don't have a strong use case for supertype bundles being visible at the type:  the instantiation engines don't need them (apart from XML deserialization which is ugly).
    
    also to do it properly i now think (after a reverted attempt included here in case we change our minds!) means making actual supertypes part of the type API, which is a sensible thing to do (and the existing `RegisteredType.getSupertypes()` is hokey) but requires a more sophisticated initialization (because currently we don't attempt to parse and discover supertypes until instances are used).  it's overkill to do that just for the problem we're solving here!
    
    comments to this effect in the last commit, https://github.com/apache/brooklyn-server/pull/718/commits/3b39ca65d1b871506a2dd61f6790fc1bd2473472


---
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] brooklyn-server issue #718: WIP - DO NOT MERGE - improve OSGi serialization ...

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

    https://github.com/apache/brooklyn-server/pull/718
  
    currently when xstream writes a reference to a java type it checks whether that class comes from a bundle and if so it prefixes the reference with the bundle name.  in the past some things have been white-listed but we are white-listing fewer and fewer of these.  the code does this for most "normal" references (though I'm still a bit unclear in `XmlSerializer` what "normal usage" is, as contrasted to "low-level mentions").
    
    normal usage seems to exclude the `SpecConverter.{,un}marshal` so specs always write the unadorned java name, and can't understand a prefix.
    
    normal usage _does_ include things like miscellaneous classes being written, like a `ConfigKey<Object>` whose value is a type `com.custom.bundle.MyEnum`.
    
    i think most people are agreed that in an ideal world we'd serialize only registered types, and json primitives, not arbitrary java.  but given what's out there already, and the work needed on supporting arbitrary types in the registry (mainly in API/UI, the backend is pretty good) we have to be able to read/write many java items, and increasingly these are from osgi.
    
    so the suggested solution is:
    
    1) if there is a registered type which is a simple wrapper for a java type, refer to that type name when reading and writing; that contains all the bundle info to create the type, and our update strategies for registered types can apply to them.  attend to search paths when reading so we prefer things in a local bundle.
    
    2) if not, WARN, then-
    
    * on write:  prefix with the bundle name
    
    * on read: try loading it ignoring the bundle but using the search path (bundles, and for legacy readings also catalog item(s)); if that doesn't work, then look in the declared bundle



---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120658487
  
    --- Diff: core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlSerializer.java ---
    @@ -89,33 +96,49 @@ protected MapperWrapper wrapMapper(MapperWrapper next) {
         }
     
         /**
    -     * JCC is used when class names are serialized/deserialized and no alias is defined;
    -     * it is configured in XStream *without* access to the XStream mapper.
    +     * JCC is used when Class instances are serialized/deserialized as a value 
    +     * (not as tags) and there are no aliases configured for that type.
    +     * It is configured in XStream default *without* access to the XStream mapper,
    +     * which is meant to apply when serializing the type name for instances of that type.
    +     * <p>
          * However we need a few selected mappers (see {@link #wrapMapperForAllLowLevelMentions(Mapper)} )
    -     * in order to effect renames at the low level, but many of the mappers must NOT be used,
    +     * to apply to all class renames, but many of the mappers must NOT be used,
          * e.g. because some might intercept all Class<? extends Entity> references
          * (and that interception is only wanted when serializing <i>instances</i>,
          * as in {@link #wrapMapperForNormalUsage(Mapper)}).
          * <p>
    -     * This can typically be done simply by registering our own instance (due to order guarantee of PrioritizedList),
    +     * This can typically be done simply by registering our own instance of this (due to order guarantee of PrioritizedList),
          * after the instance added by XStream.setupConverters()
          */
         private JavaClassConverter newCustomJavaClassConverter() {
             return new JavaClassConverter(wrapMapperForAllLowLevelMentions(new DefaultMapper(xstream.getClassLoaderReference()))) {};
         }
         
    -    /** Adds mappers needed for *any* reference to a class, e.g. when names are used for inner classes, or classes are renamed;
    -     * this *excludes* basic mentions, however, because most rewrites should *not* be applied at this deep level;
    -     * mappers which effect aliases or intercept references to entities are usually NOT be invoked in this low-level pathway.
    -     * See {@link #newCustomJavaClassConverter()}. */
    +    /** Adds mappers needed for *any* reference to a class, both "normal" usage (when xstream wants a mapper)
    +     * and Class conversion (when xstream needs to serialize an instance of Class and doesn't have an alias).
    +     * <p>
    +     * This should apply when nice names are used for inner classes, or classes are renamed;
    +     * however mappers which affect aliases or intercept references to entities are usually 
    +     * NOT be invoked in this low-level pathway. See {@link #newCustomJavaClassConverter()}. */
    +    // Developer note - this is called by the xstream subclass constructor in the constructor of this class,
    +    // so very few fields are populated
         protected MapperWrapper wrapMapperForAllLowLevelMentions(Mapper next) {
             MapperWrapper result = new CompilerIndependentOuterClassFieldMapper(next);
    +        
             Supplier<ClassLoader> classLoaderSupplier = new Supplier<ClassLoader>() {
                 @Override public ClassLoader get() {
                     return xstream.getClassLoaderReference().getReference();
                 }
             };
    -        return new ClassRenamingMapper(result, deserializingClassRenames, classLoaderSupplier);
    +        result = new ClassRenamingMapper(result, deserializingClassRenames, classLoaderSupplier);
    +        result = new OsgiClassnameMapper(new Supplier<XStream>() {
    +            @Override public XStream get() { return xstream; } }, result);
    --- End diff --
    
    Why a supplier if you are caching the value in the anonymous class? The xstream reference doesn't change, as the classloader one does, so could pass it directly.


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120654803
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/mgmt/classloading/ClassLoaderFromStackOfBrooklynClassLoadingContext.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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 org.apache.brooklyn.core.mgmt.classloading;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Stack;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContextSequential;
    +import org.apache.brooklyn.core.mgmt.classloading.ClassLoaderFromBrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.JavaBrooklynClassLoadingContext;
    +import org.apache.brooklyn.util.core.ClassLoaderUtils;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.ClassLoadingContext;
    +
    +/** Provides a stack where {@link ClassLoadingContext} instances can be pushed and popped,
    + * with the most recently pushed one in effect at any given time.
    + * <p>
    + * This is useful when traversing a tree some of whose elements may bring custom search paths,
    + * and the worker wants to have a simple view of the loader to use at any point in time.
    + * For example XStream keeps a class-loader in effect but when deserializing things 
    + * some of those things may define bundles to use. */
    +public class ClassLoaderFromStackOfBrooklynClassLoadingContext extends ClassLoader {
    +    
    +    private final Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    private final Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    private final AtomicReference<Thread> lockOwner = new AtomicReference<Thread>();
    +    private ManagementContext mgmt;
    +    private ClassLoader currentClassLoader;
    +    private AtomicReference<ClassLoaderUtils> currentLoader = new AtomicReference<>();
    +    private int lockCount;
    +    
    +    public ClassLoaderFromStackOfBrooklynClassLoadingContext(ClassLoader classLoader) {
    +        setCurrentClassLoader(classLoader);
    +    }
    +    
    +    public void setManagementContext(ManagementContext mgmt) {
    +        this.mgmt = checkNotNull(mgmt, "mgmt");
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +
    +    @Override
    +    protected Class<?> findClass(String name) throws ClassNotFoundException {
    +        return currentLoader.get().loadClass(name);
    +    }
    +    
    +    @Override
    +    protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
    +        return findClass(name);
    +    }
    +
    +    /** Must be accompanied by a corresponding {@link #popClassLoadingContext()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    public void pushClassLoadingContext(BrooklynClassLoadingContext clcNew) {
    +        acquireLock();
    +        BrooklynClassLoadingContext oldClc;
    +        if (!contexts.isEmpty()) {
    +            oldClc = contexts.peek();
    +        } else {
    +            // TODO xml serializers using this should take a BCLC instead of a CL
    +            oldClc = JavaBrooklynClassLoadingContext.create(mgmt, getCurrentClassLoader());
    +        }
    +        BrooklynClassLoadingContextSequential clcMerged = new BrooklynClassLoadingContextSequential(mgmt, oldClc, clcNew);
    +        ClassLoader newCL = ClassLoaderFromBrooklynClassLoadingContext.of(clcMerged);
    +        contexts.push(clcMerged);
    +        cls.push(getCurrentClassLoader());
    +        setCurrentClassLoader(newCL);
    +    }
    +
    +    public void popClassLoadingContext() {
    +        synchronized (lockOwner) {
    +            releaseXstreamLock();
    +            setCurrentClassLoader(cls.pop());
    +            contexts.pop();
    +        }
    +    }
    +    
    +    private ClassLoader getCurrentClassLoader() {
    +        return currentClassLoader;
    +    }
    +    
    +    private void setCurrentClassLoader(ClassLoader classLoader) {
    +        currentClassLoader = checkNotNull(classLoader);
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +    
    +    protected void acquireLock() {
    +        synchronized (lockOwner) {
    +            while (true) {
    +                if (lockOwner.compareAndSet(null, Thread.currentThread()) || 
    +                    Thread.currentThread().equals( lockOwner.get() )) {
    +                    break;
    +                }
    +                try {
    +                    lockOwner.wait(1000);
    +                } catch (InterruptedException e) {
    +                    throw Exceptions.propagate(e);
    +                }
    +            }
    +            lockCount++;
    +        }
    +    }
    +
    +    protected void releaseXstreamLock() {
    --- End diff --
    
    No longer xstream specific.


---
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] brooklyn-server issue #718: WIP - DO NOT MERGE - improve OSGi serialization ...

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

    https://github.com/apache/brooklyn-server/pull/718
  
    Also the `SpecConverter` is obnoxiously hard to overwrite to make it ignore the `type` field, so that "quick" suggestion is also out.  Thinking the original fix is actually the best.


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120765248
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/mgmt/classloading/ClassLoaderFromStackOfBrooklynClassLoadingContext.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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 org.apache.brooklyn.core.mgmt.classloading;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Stack;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContextSequential;
    +import org.apache.brooklyn.core.mgmt.classloading.ClassLoaderFromBrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.JavaBrooklynClassLoadingContext;
    +import org.apache.brooklyn.util.core.ClassLoaderUtils;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.ClassLoadingContext;
    +
    +/** Provides a stack where {@link ClassLoadingContext} instances can be pushed and popped,
    + * with the most recently pushed one in effect at any given time.
    + * <p>
    + * This is useful when traversing a tree some of whose elements may bring custom search paths,
    + * and the worker wants to have a simple view of the loader to use at any point in time.
    + * For example XStream keeps a class-loader in effect but when deserializing things 
    + * some of those things may define bundles to use. */
    +public class ClassLoaderFromStackOfBrooklynClassLoadingContext extends ClassLoader {
    +    
    +    private final Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    private final Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    private final AtomicReference<Thread> lockOwner = new AtomicReference<Thread>();
    +    private ManagementContext mgmt;
    +    private ClassLoader currentClassLoader;
    +    private AtomicReference<ClassLoaderUtils> currentLoader = new AtomicReference<>();
    +    private int lockCount;
    +    
    +    public ClassLoaderFromStackOfBrooklynClassLoadingContext(ClassLoader classLoader) {
    +        setCurrentClassLoader(classLoader);
    +    }
    +    
    +    public void setManagementContext(ManagementContext mgmt) {
    +        this.mgmt = checkNotNull(mgmt, "mgmt");
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +
    +    @Override
    +    protected Class<?> findClass(String name) throws ClassNotFoundException {
    +        return currentLoader.get().loadClass(name);
    +    }
    +    
    +    @Override
    +    protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
    +        return findClass(name);
    +    }
    +
    +    /** Must be accompanied by a corresponding {@link #popClassLoadingContext()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    public void pushClassLoadingContext(BrooklynClassLoadingContext clcNew) {
    +        acquireLock();
    +        BrooklynClassLoadingContext oldClc;
    +        if (!contexts.isEmpty()) {
    +            oldClc = contexts.peek();
    +        } else {
    +            // TODO xml serializers using this should take a BCLC instead of a CL
    +            oldClc = JavaBrooklynClassLoadingContext.create(mgmt, getCurrentClassLoader());
    +        }
    +        BrooklynClassLoadingContextSequential clcMerged = new BrooklynClassLoadingContextSequential(mgmt, oldClc, clcNew);
    +        ClassLoader newCL = ClassLoaderFromBrooklynClassLoadingContext.of(clcMerged);
    +        contexts.push(clcMerged);
    +        cls.push(getCurrentClassLoader());
    +        setCurrentClassLoader(newCL);
    +    }
    +
    +    public void popClassLoadingContext() {
    +        synchronized (lockOwner) {
    +            releaseXstreamLock();
    +            setCurrentClassLoader(cls.pop());
    +            contexts.pop();
    +        }
    +    }
    +    
    +    private ClassLoader getCurrentClassLoader() {
    +        return currentClassLoader;
    +    }
    +    
    +    private void setCurrentClassLoader(ClassLoader classLoader) {
    +        currentClassLoader = checkNotNull(classLoader);
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +    
    +    protected void acquireLock() {
    +        synchronized (lockOwner) {
    +            while (true) {
    +                if (lockOwner.compareAndSet(null, Thread.currentThread()) || 
    +                    Thread.currentThread().equals( lockOwner.get() )) {
    +                    break;
    +                }
    +                try {
    +                    lockOwner.wait(1000);
    +                } catch (InterruptedException e) {
    +                    throw Exceptions.propagate(e);
    +                }
    +            }
    +            lockCount++;
    +        }
    +    }
    +
    +    protected void releaseXstreamLock() {
    --- End diff --
    
    good to see all three reviewers picked up on this!


---
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] brooklyn-server issue #718: WIP - DO NOT MERGE - improve OSGi serialization ...

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

    https://github.com/apache/brooklyn-server/pull/718
  
    Should be good for now.  I went with the quite-simple of always writing the osgi bundle prefix (part of step 2 in the long term solution).  Lots of comments about how it works and moving to step 1.
    
    Re workaround for existing deployments, I don't have an easy answer apart from upgrading.  Currently you have to upgrade and then define the classes in question in `org.apache.brooklyn.classrename.cfg` (karaf only).  Meh - but shouldn't be that hard in practise.


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120675289
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/mgmt/classloading/ClassLoaderFromStackOfBrooklynClassLoadingContext.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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 org.apache.brooklyn.core.mgmt.classloading;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Stack;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContextSequential;
    +import org.apache.brooklyn.core.mgmt.classloading.ClassLoaderFromBrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.JavaBrooklynClassLoadingContext;
    +import org.apache.brooklyn.util.core.ClassLoaderUtils;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.ClassLoadingContext;
    +
    +/** Provides a stack where {@link ClassLoadingContext} instances can be pushed and popped,
    + * with the most recently pushed one in effect at any given time.
    + * <p>
    + * This is useful when traversing a tree some of whose elements may bring custom search paths,
    + * and the worker wants to have a simple view of the loader to use at any point in time.
    + * For example XStream keeps a class-loader in effect but when deserializing things 
    + * some of those things may define bundles to use. */
    +public class ClassLoaderFromStackOfBrooklynClassLoadingContext extends ClassLoader {
    +    
    +    private final Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    private final Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    private final AtomicReference<Thread> lockOwner = new AtomicReference<Thread>();
    +    private ManagementContext mgmt;
    +    private ClassLoader currentClassLoader;
    +    private AtomicReference<ClassLoaderUtils> currentLoader = new AtomicReference<>();
    +    private int lockCount;
    +    
    +    public ClassLoaderFromStackOfBrooklynClassLoadingContext(ClassLoader classLoader) {
    +        setCurrentClassLoader(classLoader);
    +    }
    +    
    +    public void setManagementContext(ManagementContext mgmt) {
    +        this.mgmt = checkNotNull(mgmt, "mgmt");
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +
    +    @Override
    +    protected Class<?> findClass(String name) throws ClassNotFoundException {
    +        return currentLoader.get().loadClass(name);
    +    }
    +    
    +    @Override
    +    protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
    +        return findClass(name);
    +    }
    +
    +    /** Must be accompanied by a corresponding {@link #popClassLoadingContext()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    public void pushClassLoadingContext(BrooklynClassLoadingContext clcNew) {
    +        acquireLock();
    +        BrooklynClassLoadingContext oldClc;
    +        if (!contexts.isEmpty()) {
    +            oldClc = contexts.peek();
    +        } else {
    +            // TODO xml serializers using this should take a BCLC instead of a CL
    +            oldClc = JavaBrooklynClassLoadingContext.create(mgmt, getCurrentClassLoader());
    +        }
    +        BrooklynClassLoadingContextSequential clcMerged = new BrooklynClassLoadingContextSequential(mgmt, oldClc, clcNew);
    +        ClassLoader newCL = ClassLoaderFromBrooklynClassLoadingContext.of(clcMerged);
    +        contexts.push(clcMerged);
    +        cls.push(getCurrentClassLoader());
    +        setCurrentClassLoader(newCL);
    +    }
    +
    +    public void popClassLoadingContext() {
    +        synchronized (lockOwner) {
    +            releaseXstreamLock();
    +            setCurrentClassLoader(cls.pop());
    +            contexts.pop();
    +        }
    +    }
    +    
    +    private ClassLoader getCurrentClassLoader() {
    +        return currentClassLoader;
    +    }
    +    
    +    private void setCurrentClassLoader(ClassLoader classLoader) {
    +        currentClassLoader = checkNotNull(classLoader);
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +    
    +    protected void acquireLock() {
    +        synchronized (lockOwner) {
    --- End diff --
    
    TL;DR: ignore this comment in this PR!
    
    The threading and acquire/release feels confusing here. It would be good to document the assumptions/behaviour, I think. For example, calling acquire from a different thread has pushed onto the stack, until that other thread has popped everything. Do we really need that? Is the behaviour we want more like thread-local storage, with a different stack for each thread - or do we deliberately want other threads to see a shared `getCurrentClassLoader()` but not be allowed to push onto the stack?
    
    Actually, I see you mostly just moved this code from `XmlMementoSerializer.OsgiClassLoader` so it's fine to leave as-is rather than trying to document or change it more here!
    
    (but it is strange to have "xstream" in the method names, now that it's moved to package `org.apache.brooklyn.core.mgmt.classloading`).


---
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] brooklyn-server issue #718: improvement to OSGi serialization strategies (wr...

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

    https://github.com/apache/brooklyn-server/pull/718
  
    Fixed all items discussed.  I think I've improved the horrible "low-level" name, and better explained the suppliers @neykov, and explained (and figured out) the reasons for the threading model of the classloading stack @aledsage.
    
    Thanks all for the quick review.  If someone wants to eyeball my last changes then we can merge.


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120704212
  
    --- Diff: core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlSerializer.java ---
    @@ -89,33 +96,49 @@ protected MapperWrapper wrapMapper(MapperWrapper next) {
         }
     
         /**
    -     * JCC is used when class names are serialized/deserialized and no alias is defined;
    -     * it is configured in XStream *without* access to the XStream mapper.
    +     * JCC is used when Class instances are serialized/deserialized as a value 
    +     * (not as tags) and there are no aliases configured for that type.
    +     * It is configured in XStream default *without* access to the XStream mapper,
    +     * which is meant to apply when serializing the type name for instances of that type.
    +     * <p>
          * However we need a few selected mappers (see {@link #wrapMapperForAllLowLevelMentions(Mapper)} )
    -     * in order to effect renames at the low level, but many of the mappers must NOT be used,
    +     * to apply to all class renames, but many of the mappers must NOT be used,
          * e.g. because some might intercept all Class<? extends Entity> references
          * (and that interception is only wanted when serializing <i>instances</i>,
          * as in {@link #wrapMapperForNormalUsage(Mapper)}).
          * <p>
    -     * This can typically be done simply by registering our own instance (due to order guarantee of PrioritizedList),
    +     * This can typically be done simply by registering our own instance of this (due to order guarantee of PrioritizedList),
          * after the instance added by XStream.setupConverters()
          */
         private JavaClassConverter newCustomJavaClassConverter() {
             return new JavaClassConverter(wrapMapperForAllLowLevelMentions(new DefaultMapper(xstream.getClassLoaderReference()))) {};
         }
         
    -    /** Adds mappers needed for *any* reference to a class, e.g. when names are used for inner classes, or classes are renamed;
    -     * this *excludes* basic mentions, however, because most rewrites should *not* be applied at this deep level;
    -     * mappers which effect aliases or intercept references to entities are usually NOT be invoked in this low-level pathway.
    -     * See {@link #newCustomJavaClassConverter()}. */
    +    /** Adds mappers needed for *any* reference to a class, both "normal" usage (when xstream wants a mapper)
    +     * and Class conversion (when xstream needs to serialize an instance of Class and doesn't have an alias).
    +     * <p>
    +     * This should apply when nice names are used for inner classes, or classes are renamed;
    +     * however mappers which affect aliases or intercept references to entities are usually 
    +     * NOT be invoked in this low-level pathway. See {@link #newCustomJavaClassConverter()}. */
    +    // Developer note - this is called by the xstream subclass constructor in the constructor of this class,
    +    // so very few fields are populated
         protected MapperWrapper wrapMapperForAllLowLevelMentions(Mapper next) {
             MapperWrapper result = new CompilerIndependentOuterClassFieldMapper(next);
    +        
             Supplier<ClassLoader> classLoaderSupplier = new Supplier<ClassLoader>() {
                 @Override public ClassLoader get() {
                     return xstream.getClassLoaderReference().getReference();
                 }
             };
    -        return new ClassRenamingMapper(result, deserializingClassRenames, classLoaderSupplier);
    +        result = new ClassRenamingMapper(result, deserializingClassRenames, classLoaderSupplier);
    +        result = new OsgiClassnameMapper(new Supplier<XStream>() {
    +            @Override public XStream get() { return xstream; } }, result);
    --- End diff --
    
    Makes sense now, thanks.


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120765196
  
    --- Diff: core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlSerializer.java ---
    @@ -89,33 +96,49 @@ protected MapperWrapper wrapMapper(MapperWrapper next) {
         }
     
         /**
    -     * JCC is used when class names are serialized/deserialized and no alias is defined;
    -     * it is configured in XStream *without* access to the XStream mapper.
    +     * JCC is used when Class instances are serialized/deserialized as a value 
    +     * (not as tags) and there are no aliases configured for that type.
    +     * It is configured in XStream default *without* access to the XStream mapper,
    +     * which is meant to apply when serializing the type name for instances of that type.
    +     * <p>
          * However we need a few selected mappers (see {@link #wrapMapperForAllLowLevelMentions(Mapper)} )
    -     * in order to effect renames at the low level, but many of the mappers must NOT be used,
    +     * to apply to all class renames, but many of the mappers must NOT be used,
          * e.g. because some might intercept all Class<? extends Entity> references
          * (and that interception is only wanted when serializing <i>instances</i>,
          * as in {@link #wrapMapperForNormalUsage(Mapper)}).
          * <p>
    -     * This can typically be done simply by registering our own instance (due to order guarantee of PrioritizedList),
    +     * This can typically be done simply by registering our own instance of this (due to order guarantee of PrioritizedList),
          * after the instance added by XStream.setupConverters()
          */
         private JavaClassConverter newCustomJavaClassConverter() {
             return new JavaClassConverter(wrapMapperForAllLowLevelMentions(new DefaultMapper(xstream.getClassLoaderReference()))) {};
         }
         
    -    /** Adds mappers needed for *any* reference to a class, e.g. when names are used for inner classes, or classes are renamed;
    -     * this *excludes* basic mentions, however, because most rewrites should *not* be applied at this deep level;
    -     * mappers which effect aliases or intercept references to entities are usually NOT be invoked in this low-level pathway.
    -     * See {@link #newCustomJavaClassConverter()}. */
    +    /** Adds mappers needed for *any* reference to a class, both "normal" usage (when xstream wants a mapper)
    +     * and Class conversion (when xstream needs to serialize an instance of Class and doesn't have an alias).
    +     * <p>
    +     * This should apply when nice names are used for inner classes, or classes are renamed;
    +     * however mappers which affect aliases or intercept references to entities are usually 
    +     * NOT be invoked in this low-level pathway. See {@link #newCustomJavaClassConverter()}. */
    +    // Developer note - this is called by the xstream subclass constructor in the constructor of this class,
    +    // so very few fields are populated
         protected MapperWrapper wrapMapperForAllLowLevelMentions(Mapper next) {
    --- End diff --
    
    either should be extensible.  it's not a `lowLevelWrapping` but a `wrapperExtensionPointForLowLevelActivities`.
    
    in fact the name low-level is pissing me off.  i'm tempted to rename it `wrapMapperForLotsOfShitImNotSureWhat` and "normal usage" of course becomes `wrapMapperForSlightlyMoreSpecificUsageButAgainImNotTellingYouWhat`.
    
    (no prizes for guessing what numpty was responsible for the original names :) )


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120669009
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/mgmt/classloading/ClassLoaderFromStackOfBrooklynClassLoadingContext.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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 org.apache.brooklyn.core.mgmt.classloading;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.Stack;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContextSequential;
    +import org.apache.brooklyn.core.mgmt.classloading.ClassLoaderFromBrooklynClassLoadingContext;
    +import org.apache.brooklyn.core.mgmt.classloading.JavaBrooklynClassLoadingContext;
    +import org.apache.brooklyn.util.core.ClassLoaderUtils;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.ClassLoadingContext;
    +
    +/** Provides a stack where {@link ClassLoadingContext} instances can be pushed and popped,
    + * with the most recently pushed one in effect at any given time.
    + * <p>
    + * This is useful when traversing a tree some of whose elements may bring custom search paths,
    + * and the worker wants to have a simple view of the loader to use at any point in time.
    + * For example XStream keeps a class-loader in effect but when deserializing things 
    + * some of those things may define bundles to use. */
    +public class ClassLoaderFromStackOfBrooklynClassLoadingContext extends ClassLoader {
    +    
    +    private final Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
    +    private final Stack<ClassLoader> cls = new Stack<ClassLoader>();
    +    private final AtomicReference<Thread> lockOwner = new AtomicReference<Thread>();
    +    private ManagementContext mgmt;
    +    private ClassLoader currentClassLoader;
    +    private AtomicReference<ClassLoaderUtils> currentLoader = new AtomicReference<>();
    +    private int lockCount;
    +    
    +    public ClassLoaderFromStackOfBrooklynClassLoadingContext(ClassLoader classLoader) {
    +        setCurrentClassLoader(classLoader);
    +    }
    +    
    +    public void setManagementContext(ManagementContext mgmt) {
    +        this.mgmt = checkNotNull(mgmt, "mgmt");
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +
    +    @Override
    +    protected Class<?> findClass(String name) throws ClassNotFoundException {
    +        return currentLoader.get().loadClass(name);
    +    }
    +    
    +    @Override
    +    protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
    +        return findClass(name);
    +    }
    +
    +    /** Must be accompanied by a corresponding {@link #popClassLoadingContext()} when finished. */
    +    @SuppressWarnings("deprecation")
    +    public void pushClassLoadingContext(BrooklynClassLoadingContext clcNew) {
    +        acquireLock();
    +        BrooklynClassLoadingContext oldClc;
    +        if (!contexts.isEmpty()) {
    +            oldClc = contexts.peek();
    +        } else {
    +            // TODO xml serializers using this should take a BCLC instead of a CL
    +            oldClc = JavaBrooklynClassLoadingContext.create(mgmt, getCurrentClassLoader());
    +        }
    +        BrooklynClassLoadingContextSequential clcMerged = new BrooklynClassLoadingContextSequential(mgmt, oldClc, clcNew);
    +        ClassLoader newCL = ClassLoaderFromBrooklynClassLoadingContext.of(clcMerged);
    +        contexts.push(clcMerged);
    +        cls.push(getCurrentClassLoader());
    +        setCurrentClassLoader(newCL);
    +    }
    +
    +    public void popClassLoadingContext() {
    +        synchronized (lockOwner) {
    +            releaseXstreamLock();
    +            setCurrentClassLoader(cls.pop());
    +            contexts.pop();
    +        }
    +    }
    +    
    +    private ClassLoader getCurrentClassLoader() {
    +        return currentClassLoader;
    +    }
    +    
    +    private void setCurrentClassLoader(ClassLoader classLoader) {
    +        currentClassLoader = checkNotNull(classLoader);
    +        currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
    +    }
    +    
    +    protected void acquireLock() {
    +        synchronized (lockOwner) {
    +            while (true) {
    +                if (lockOwner.compareAndSet(null, Thread.currentThread()) || 
    +                    Thread.currentThread().equals( lockOwner.get() )) {
    +                    break;
    +                }
    +                try {
    +                    lockOwner.wait(1000);
    +                } catch (InterruptedException e) {
    +                    throw Exceptions.propagate(e);
    +                }
    +            }
    +            lockCount++;
    +        }
    +    }
    +
    +    protected void releaseXstreamLock() {
    --- End diff --
    
    You renamed the `acquireXStreamLock` as `acquireLock`, maybe do the same here


---
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] brooklyn-server pull request #718: improvement to OSGi serialization strateg...

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

    https://github.com/apache/brooklyn-server/pull/718#discussion_r120678060
  
    --- Diff: core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlSerializer.java ---
    @@ -89,33 +96,49 @@ protected MapperWrapper wrapMapper(MapperWrapper next) {
         }
     
         /**
    -     * JCC is used when class names are serialized/deserialized and no alias is defined;
    -     * it is configured in XStream *without* access to the XStream mapper.
    +     * JCC is used when Class instances are serialized/deserialized as a value 
    +     * (not as tags) and there are no aliases configured for that type.
    +     * It is configured in XStream default *without* access to the XStream mapper,
    +     * which is meant to apply when serializing the type name for instances of that type.
    +     * <p>
          * However we need a few selected mappers (see {@link #wrapMapperForAllLowLevelMentions(Mapper)} )
    -     * in order to effect renames at the low level, but many of the mappers must NOT be used,
    +     * to apply to all class renames, but many of the mappers must NOT be used,
          * e.g. because some might intercept all Class<? extends Entity> references
          * (and that interception is only wanted when serializing <i>instances</i>,
          * as in {@link #wrapMapperForNormalUsage(Mapper)}).
          * <p>
    -     * This can typically be done simply by registering our own instance (due to order guarantee of PrioritizedList),
    +     * This can typically be done simply by registering our own instance of this (due to order guarantee of PrioritizedList),
          * after the instance added by XStream.setupConverters()
          */
         private JavaClassConverter newCustomJavaClassConverter() {
             return new JavaClassConverter(wrapMapperForAllLowLevelMentions(new DefaultMapper(xstream.getClassLoaderReference()))) {};
         }
         
    -    /** Adds mappers needed for *any* reference to a class, e.g. when names are used for inner classes, or classes are renamed;
    -     * this *excludes* basic mentions, however, because most rewrites should *not* be applied at this deep level;
    -     * mappers which effect aliases or intercept references to entities are usually NOT be invoked in this low-level pathway.
    -     * See {@link #newCustomJavaClassConverter()}. */
    +    /** Adds mappers needed for *any* reference to a class, both "normal" usage (when xstream wants a mapper)
    +     * and Class conversion (when xstream needs to serialize an instance of Class and doesn't have an alias).
    +     * <p>
    +     * This should apply when nice names are used for inner classes, or classes are renamed;
    +     * however mappers which affect aliases or intercept references to entities are usually 
    +     * NOT be invoked in this low-level pathway. See {@link #newCustomJavaClassConverter()}. */
    +    // Developer note - this is called by the xstream subclass constructor in the constructor of this class,
    +    // so very few fields are populated
         protected MapperWrapper wrapMapperForAllLowLevelMentions(Mapper next) {
             MapperWrapper result = new CompilerIndependentOuterClassFieldMapper(next);
    +        
             Supplier<ClassLoader> classLoaderSupplier = new Supplier<ClassLoader>() {
                 @Override public ClassLoader get() {
                     return xstream.getClassLoaderReference().getReference();
                 }
             };
    -        return new ClassRenamingMapper(result, deserializingClassRenames, classLoaderSupplier);
    +        result = new ClassRenamingMapper(result, deserializingClassRenames, classLoaderSupplier);
    +        result = new OsgiClassnameMapper(new Supplier<XStream>() {
    +            @Override public XStream get() { return xstream; } }, result);
    --- End diff --
    
    @neykov I think @ahgittin said it will be null at this point, because we're calling this method from inside the `XmlSerializer` constructor (yuck!), and worse it gets called from inside the constructor of the `XStream` object.
    
    The `new XStream()` constructor calls `buildMapper()`, which calls `wrapMapper()`, which we override in the anonymous XStream class to call `wrapMapperForNormalUsage`, which calls `wrapMapperForAllLowLevelMentions`. So when we get to this line of code, we haven't managed to set the `xstream` field yet. Crazy!


---
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] brooklyn-server issue #718: improvement to OSGi serialization strategies (wr...

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

    https://github.com/apache/brooklyn-server/pull/718
  
    | Svet and I changed the serialization so all classes have their bundle prefix included in persisted state, even if they are white-labelled. That felt much more consistent. If a classname doesn't have a prefix, we'll still look in the list of white-labelled bundle (mainly for backwards compatibility).
    
    @aledsage 100% in favour of this.  did you see the suggested steps (1) and (2) near the start of this issue for a long-term vision, and a way to avoid needing to do this at all?


---
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] brooklyn-server issue #718: WIP - DO NOT MERGE - improve OSGi serialization ...

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

    https://github.com/apache/brooklyn-server/pull/718
  
    BTW there was a lot of refactoring / clean-up I did, split into commits for some ease of reviewing, and  a lot of comments and tests.
    
    The only functional change should be the shift of `OsgiClassnameMapper` from normal usage to low-level mention, which AFAIK only impacts reading and writing class names when xstream needs a specific instance of `Class`, and now supports osgi-prefixes in that context.  Old serialized state should read in _if_ the old version could read it in, because the prefix isn't required, but this will now add that prefix when writing (so if you have buggy instances and define some renames those only need to be applied once).


---
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.
---