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/11/10 15:47:43 UTC

[GitHub] incubator-brooklyn pull request: Flesh out type creation using the...

GitHub user ahgittin opened a pull request:

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

    Flesh out type creation using the new TypeRegistry

    Switch many uses of Catalog to RegisteredType and improve the API around the TypePlanTransformer.  See individual commits for detail.
    
    Next is:
    * enable CAMP to use bean creation and register some beans
    * swap CAMP to use a new simpler implementation for parsing, enabling reflection-based doc generation and code completion proposals
    * persist RegisteredType instances instead of CatalogItems
    
    Note that this builds on #1016 (which should be quick to review) so ignore commit  036ba79.

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

    $ git pull https://github.com/ahgittin/incubator-brooklyn type-registry

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

    https://github.com/apache/incubator-brooklyn/pull/1017.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 #1017
    
----
commit 036ba79ae3c63c5fa078f73cc1b77cbb854e7631
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-11-10T14:09:38Z

    expand assertions and http assertions for string containment, updating rest tests

commit 9fd33e4b4f5a4231addd39c666065d6c7d5eaaa6
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-11-02T16:06:44Z

    Start to flesh out how types get created.
    
    also move inner classes to top level

commit 8ca99fefdc7e325c14d4b5d827df59963882c7b4
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-11-04T15:47:22Z

    Working CampTypePlanTransformer

commit 90a9c23ece9e26fca8a86fe2a64ab1b0deb6af2a
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-11-09T16:09:56Z

    Big refactoring of new type registry code
    
    * call the class RegisteredTypeLoadingContext instead of constraint, improve method names based on better understood semantics
    * clean up how plans can be passed directly, without needing an anonymous RegisteredType
    * move BrooklynClassLoadingContext to API project (keeping temporary sub-interface in core)

commit 2ec76352e469b7765f0a3b558907ea680ee324ef
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-11-09T16:53:17Z

    XML-based illustration of new-style XML plan creation
    
    and tidy API and update deprecation

commit 3868344796e2f799baef8429671f473fe672f80f
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Date:   2015-11-09T21:42:02Z

    More conversion to new-style TypePlanTransformer, and fixes

----


---
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: Flesh out type creation using the...

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/1017#discussion_r45041185
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/plan/PlanToSpecFactory.java ---
    @@ -36,12 +40,30 @@
     import com.google.common.base.Function;
     import com.google.common.collect.ImmutableList;
     
    +/** @deprecated since 0.9.0 use {@link TypePlanTransformers} as part of switch to {@link BrooklynTypePlanTransformer};
    + * mark transformers as deprecated if there is a preferred corresponding {@link BrooklynTypePlanTransformer} */
    +@Deprecated 
     public class PlanToSpecFactory {
         
         private static final Logger log = LoggerFactory.getLogger(PlanToSpecFactory.class);
     
    -    private static Collection<PlanToSpecTransformer> getAll() {
    -        return ImmutableList.copyOf(ServiceLoader.load(PlanToSpecTransformer.class));
    +    private static Collection<PlanToSpecTransformer> getAll(boolean includeDeprecated) {
    --- End diff --
    
    +1


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44647715
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/SpecParameterInMetaTest.java ---
    @@ -43,18 +49,40 @@
     public class SpecParameterInMetaTest {
         private ManagementContext mgmt;
         private BrooklynCatalog catalog;
    -    private String spec;
    +    private String specId;
     
         @BeforeMethod(alwaysRun=true)
         public void setUp() {
             mgmt = LocalManagementContextForTests.newInstanceWithOsgi();
             catalog = mgmt.getCatalog();
    -        spec = TestToSpecTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    +        StaticTypePlanTransformer.forceInstall();
    +        PlanToSpecFactory.forceAvailable(TestToSpecTransformer.class, JavaCatalogToSpecTransformer.class);
    +        specId = StaticTypePlanTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
         }
     
    +    @AfterMethod(alwaysRun=true)
    +    public void tearDown() {
    +        StaticTypePlanTransformer.clearForced();
    +        PlanToSpecFactory.clearForced();
    +    }
    +
    +    @Test
    +    public void testCanRetrieveWithNew() {
    +        AbstractBrooklynObjectSpec<?, ?> spec = mgmt.getTypeRegistry().createSpecFromPlan(null, specId, null, null);
    +        Assert.assertNotNull(spec);
    +    }
    +
    +    // it's not actually added to the catalog; probably it would be cleaner if it is;
    +    // but for now when we resolve in PlanToSpecFactory we make explicit reference to StaticTypePlanTransformer
    --- End diff --
    
    Don't get this one.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44640867
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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.typereg;
    +
    +import groovy.xml.Entity;
    +
    +import java.util.Set;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.objs.BrooklynObject;
    +import org.apache.brooklyn.api.objs.BrooklynObjectType;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +public class RegisteredTypeLoadingContexts {
    +
    +    private static final Logger log = LoggerFactory.getLogger(RegisteredTypeLoadingContexts.BasicRegisteredTypeLoadingContext.class);
    +    
    +    /** Immutable (from caller's perspective) record of a constraint */
    +    public final static class BasicRegisteredTypeLoadingContext implements RegisteredTypeLoadingContext {
    +        @Nullable private RegisteredTypeKind kind;
    +        @Nullable private Class<?> expectedSuperType;
    +        @Nonnull private Set<String> encounteredTypes = ImmutableSet.of();
    +        @Nullable BrooklynClassLoadingContext loader;
    +        
    +        private BasicRegisteredTypeLoadingContext() {}
    +        
    +        public BasicRegisteredTypeLoadingContext(@Nullable RegisteredTypeLoadingContext source) {
    +            if (source==null) return;
    +            
    +            this.kind = source.getExpectedKind();
    +            this.expectedSuperType = source.getExpectedJavaSuperType();
    +            this.encounteredTypes = source.getAlreadyEncounteredTypes();
    +            this.loader = (BrooklynClassLoadingContext) source.getLoader();
    +        }
    +
    +        @Override
    +        public RegisteredTypeKind getExpectedKind() {
    +            return kind;
    +        }
    +        
    +        @Override
    +        public Class<?> getExpectedJavaSuperType() {
    +            if (expectedSuperType==null) return Object.class;
    +            return expectedSuperType;
    +        }
    +
    +        @Override
    +        public Set<String> getAlreadyEncounteredTypes() {
    +            if (encounteredTypes==null) return ImmutableSet.of();
    +            return ImmutableSet.<String>copyOf(encounteredTypes);
    +        }
    +        
    +        @Override
    +        public BrooklynClassLoadingContext getLoader() {
    +            return loader;
    +        }
    +        
    +        @Override
    +        public String toString() {
    +            return JavaClassNames.cleanSimpleClassName(this)+"["+kind+","+expectedSuperType+","+encounteredTypes+"]";
    +        }
    +    }
    +
    +    /** returns a constraint which allows anything */
    +    public static RegisteredTypeLoadingContext any() {
    +        return new BasicRegisteredTypeLoadingContext();
    +    }
    +
    +    public static RegisteredTypeLoadingContext alreadyEncountered(Set<String> encounteredTypeSymbolicNames) {
    +        BasicRegisteredTypeLoadingContext result = new BasicRegisteredTypeLoadingContext();
    +        result.encounteredTypes = encounteredTypeSymbolicNames == null ? ImmutableSet.<String>of() : ImmutableSet.copyOf(encounteredTypeSymbolicNames);
    +        return result;
    +    }
    +    public static RegisteredTypeLoadingContext alreadyEncountered(Set<String> encounteredTypeSymbolicNames, String anotherEncounteredType) {
    +        BasicRegisteredTypeLoadingContext result = new BasicRegisteredTypeLoadingContext();
    +        MutableSet<String> encounteredTypes = MutableSet.copyOf(encounteredTypeSymbolicNames);
    +        encounteredTypes.addIfNotNull(anotherEncounteredType);
    +        result.encounteredTypes = encounteredTypes.asUnmodifiable();
    +        return result;
    +    }
    +
    +    public static RegisteredTypeLoadingContext loaderAlreadyEncountered(BrooklynClassLoadingContext loader, Set<String> encounteredTypeSymbolicNames) {
    +        return loaderAlreadyEncountered(loader, encounteredTypeSymbolicNames, null);
    +    }
    +    public static RegisteredTypeLoadingContext loaderAlreadyEncountered(BrooklynClassLoadingContext loader, Set<String> encounteredTypeSymbolicNames, String anotherEncounteredType) {
    +        return withLoader(alreadyEncountered(encounteredTypeSymbolicNames, anotherEncounteredType), loader);
    +    }
    +
    +    private static RegisteredTypeLoadingContext of(RegisteredTypeKind kind, Class<?> javaSuperType) {
    +        BasicRegisteredTypeLoadingContext result = new BasicRegisteredTypeLoadingContext();
    +        result.kind = kind;
    +        result.expectedSuperType = javaSuperType;
    +        return result;
    +    }
    +
    +    public static RegisteredTypeLoadingContext bean(Class<?> javaSuperType) {
    +        return of(RegisteredTypeKind.BEAN, javaSuperType);
    +    }
    +
    +    public static RegisteredTypeLoadingContext spec(Class<? extends BrooklynObject> javaSuperType) {
    +        return of(RegisteredTypeKind.SPEC, javaSuperType);
    +    }
    +    
    +    public static <T> RegisteredTypeLoadingContext withBeanSuperType(@Nullable RegisteredTypeLoadingContext source, @Nullable Class<T> beanSuperType) {
    +        Class<T> superType = beanSuperType;
    +        BasicRegisteredTypeLoadingContext constraint = new BasicRegisteredTypeLoadingContext(source);
    +        if (source==null) source = constraint;
    +        if (superType==null) return source;
    +        constraint.expectedSuperType = superType;
    +        if (source.getExpectedJavaSuperType()==null || source.getExpectedJavaSuperType().isAssignableFrom( superType )) {
    +            // the old constraint was weaker than present; return the new constraint
    +            return constraint;
    +        }
    +        if (superType.isAssignableFrom( source.getExpectedJavaSuperType() )) {
    +            // the constraint was already for something more specific; ignore what we've inferred here
    +            return source;
    +        }
    +        log.warn("Ambiguous bean supertypes ("+beanSuperType+" for target "+source.getExpectedJavaSuperType()+"); "
    +            + "it is recommended that any registered type constraint for a spec be compatible with the spec type");
    +        return source;
    +    }
    +
    +    /** Takes a Spec java type and adds an expected java type to the {@link RegisteredTypeLoadingContext} */
    +    public static <T extends AbstractBrooklynObjectSpec<?,?>> RegisteredTypeLoadingContext withSpecSuperType(@Nullable RegisteredTypeLoadingContext source, @Nullable Class<T> specSuperType) {
    +        Class<?> superType = lookupTargetTypeForSpec(specSuperType);
    +        BasicRegisteredTypeLoadingContext constraint = new BasicRegisteredTypeLoadingContext(source);
    +        if (source==null) source = constraint;
    +        if (superType==null) return source;
    +        constraint.expectedSuperType = superType;
    +        if (source.getExpectedJavaSuperType()==null || source.getExpectedJavaSuperType().isAssignableFrom( superType )) {
    +            // the old constraint was weaker than present; return the new constraint
    +            return constraint;
    +        }
    +        if (superType.isAssignableFrom( source.getExpectedJavaSuperType() )) {
    +            // the constraint was already for something more specific; ignore what we've inferred here
    +            return source;
    +        }
    +        // trickier situation; the constraint had a type not compatible with the spec type; log a warning and leave alone
    +        // (e.g. caller specified some java super type which is not a super or sub of the spec target type;
    +        // this may be because the caller specified a Spec as the type supertype, which is wrong;
    +        // or they may have specified an interface along a different hierarchy, which we discouraged
    +        // as it will make filtering/indexing more complex)
    +        log.warn("Ambiguous spec supertypes ("+specSuperType+" for target "+source.getExpectedJavaSuperType()+"); "
    +            + "it is recommended that any registered type constraint for a spec be compatible with the spec type");
    +        return source;
    +    }
    +        
    +    /** given a spec, returns the class of the item it targets, for instance returns {@link Entity} given {@link EntitySpec};
    +     * see also {@link #lookupSpecTypeForTarget(Class)} */
    +    static <T extends AbstractBrooklynObjectSpec<?,?>> Class<? extends BrooklynObject> lookupTargetTypeForSpec(Class<T> specSuperType) {
    +        if (specSuperType==null) return BrooklynObject.class;
    +        BrooklynObjectType best = null;
    +
    +        for (BrooklynObjectType t: BrooklynObjectType.values()) {
    +            if (t.getSpecType()==null) continue;
    +            if (!t.getSpecType().isAssignableFrom(specSuperType)) continue;
    +            // on equality, exit immediately
    +            if (t.getSpecType().equals(specSuperType)) return t.getInterfaceType();
    +            // else pick which is best
    +            if (best==null) { best = t; continue; }
    +            // if t is more specific, it is better (handles case when e.g. a Policy is a subclass of Entity)
    +            if (best.getSpecType().isAssignableFrom(t.getSpecType())) { best = t; continue; }
    +        }
    +        if (best==null) {
    +            log.warn("Unexpected spec supertype ("+specSuperType+"); treating as any "+BrooklynObject.class, new Throwable("Trace for unexpected spec supertype"));
    +            return BrooklynObject.class;
    +        }
    +        // the spec is more specific, but we're not familiar with it here; return the best
    +        return best.getInterfaceType();
    +    }
    +
    +    /** given a {@link BrooklynObject}, returns the spec class which would generate it, for instance returns {@link EntitySpec} given {@link Entity},
    +     * or null if not known */
    +    static <BO extends BrooklynObject> Class<? extends AbstractBrooklynObjectSpec<?,?>> lookupSpecTypeForTarget(Class<BO> targetSuperType) {
    +        if (targetSuperType==null) return null;
    +        BrooklynObjectType best = null;
    +
    +        for (BrooklynObjectType t: BrooklynObjectType.values()) {
    +            if (t.getSpecType()==null) continue;
    --- End diff --
    
    Should be `getInterfaceType()`


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#issuecomment-157447650
  
    All addressed, and merged/resolved with #1036. Ready for merging if tests pass on the server.
    
    @neykov the simpler signatures `TypeRegistry.get` are nice, and i've added validation and a simple test for the `StaticTypePlanTransformer`.  The old `Test...Transformer` is gone as I see you've moved the tests to `usage/camp` where it's not needed.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r45057599
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/BrooklynTypeRegistry.java ---
    @@ -45,24 +45,25 @@
         /** @return The item matching the given given 
          * {@link RegisteredType#getSymbolicName() symbolicName} 
          * and optionally {@link RegisteredType#getVersion()},
    -     * filtered for the optionally supplied {@link RegisteredTypeConstraint}, 
    +     * filtered for the optionally supplied {@link RegisteredTypeLoadingContext}, 
          * taking the best version if the version is null or a default marker,
          * returning null if no matches are found. */
    -    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} with no constraints */
    +    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} with no constraints */
         RegisteredType get(String symbolicName, String version);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} but allows <code>"name:version"</code> 
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} but allows <code>"name:version"</code> 
          * (the {@link RegisteredType#getId()}) in addition to the unversioned name,
          * using a default marker if no version can be inferred */
    -    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, RegisteredTypeConstraint)} but with no constraints */
    +    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, RegisteredTypeLoadingContext)} but with no constraints */
         RegisteredType get(String symbolicNameWithOptionalVersion);
     
         // NB the seemingly more correct generics <T,SpecT extends AbstractBrooklynObjectSpec<T,SpecT>> 
         // cause compile errors, not in Eclipse, but in maven (?) 
    -    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint optionalConstraint, Class<SpecT> optionalSpecSuperType);
    +    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<SpecT> optionalSpecSuperType);
    +    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpecFromPlan(String planFormat, Object planData, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<SpecT> optionalSpecSuperType);
         
    -    // TODO when we support beans
    -//    <T> T createBean(RegisteredType type, @Nullable RegisteredTypeConstraint optionalConstraint, Class<T> optionalResultSuperType);
    +    <T> T createBean(RegisteredType type, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<T> optionalResultSuperType);
    +    <T> T createBeanFromPlan(String planFormat, Object planData, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<T> optionalBeanSuperType);
    --- End diff --
    
    No need for a new entry point on the context, a `new TypeInstantiator(mgmt).createBean(...)` will do.


---
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: Flesh out type creation using the...

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/1017#discussion_r45051796
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    --- End diff --
    
    i'd like to be able to make them non-null, when the creation API is good enough


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44536000
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/BrooklynTypeRegistry.java ---
    @@ -45,24 +45,25 @@
         /** @return The item matching the given given 
          * {@link RegisteredType#getSymbolicName() symbolicName} 
          * and optionally {@link RegisteredType#getVersion()},
    -     * filtered for the optionally supplied {@link RegisteredTypeConstraint}, 
    +     * filtered for the optionally supplied {@link RegisteredTypeLoadingContext}, 
          * taking the best version if the version is null or a default marker,
          * returning null if no matches are found. */
    -    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} with no constraints */
    +    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} with no constraints */
         RegisteredType get(String symbolicName, String version);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} but allows <code>"name:version"</code> 
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} but allows <code>"name:version"</code> 
          * (the {@link RegisteredType#getId()}) in addition to the unversioned name,
          * using a default marker if no version can be inferred */
    -    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, RegisteredTypeConstraint)} but with no constraints */
    +    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, RegisteredTypeLoadingContext)} but with no constraints */
         RegisteredType get(String symbolicNameWithOptionalVersion);
     
         // NB the seemingly more correct generics <T,SpecT extends AbstractBrooklynObjectSpec<T,SpecT>> 
         // cause compile errors, not in Eclipse, but in maven (?) 
    -    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint optionalConstraint, Class<SpecT> optionalSpecSuperType);
    +    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<SpecT> optionalSpecSuperType);
    +    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpecFromPlan(String planFormat, Object planData, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<SpecT> optionalSpecSuperType);
         
    -    // TODO when we support beans
    -//    <T> T createBean(RegisteredType type, @Nullable RegisteredTypeConstraint optionalConstraint, Class<T> optionalResultSuperType);
    +    <T> T createBean(RegisteredType type, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<T> optionalResultSuperType);
    +    <T> T createBeanFromPlan(String planFormat, Object planData, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<T> optionalBeanSuperType);
    --- End diff --
    
    I'd say that all the `createXXX` methods don't belong to the registry. Makes more sense if another piece of code does this, having a reference to the registry. Even `createSpec` on the CI was out of place for me, but as a single method was a 'don't bother' case.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44661576
  
    --- Diff: usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogParametersTest.java ---
    @@ -273,7 +273,54 @@ public void testAppSpecInheritsCatalogParameters() {
         }
     
         @Test
    -    public void testParametersCoercedOnSetAndReferences() throws Exception {
    +    public void testParametersOnItemCoercedOnSetAndReferences() throws Exception {
    +        Integer testValue = Integer.valueOf(55);
    +        addCatalogItems(
    +                "brooklyn.catalog:",
    +                "  id: " + SYMBOLIC_NAME,
    +                "  version: " + TEST_VERSION,
    +                "  item:",
    +                "    type: " + BasicApplication.class.getName(),
    +                "    brooklyn.parameters:",
    +                "    - name: num",
    +                "      type: integer",
    +                "    brooklyn.children:",
    +                "    - type: " + ConfigEntityForTest.class.getName(),
    +                "      brooklyn.config:",
    +                "        refConfig: $brooklyn:scopeRoot().config(\"num\")",
    +                "    - type: " + ConfigEntityForTest.class.getName(),
    +                "      brooklyn.config:",
    +                "        refConfig: $brooklyn:config(\"num\")"); //inherited config
    +
    +        Entity app = createAndStartApplication(
    +                "services:",
    +                "- type: " + BasicApplication.class.getName(),
    +                "  brooklyn.children:",
    +                "  - type: " + ver(SYMBOLIC_NAME),
    +                "    brooklyn.config:",
    +                "      num: \"" + testValue + "\"");
    +
    +        Entity scopeRoot = Iterables.getOnlyElement(app.getChildren());
    +
    +        ConfigKey<Object> numKey = ConfigKeys.newConfigKey(Object.class, "num");
    +        assertEquals(scopeRoot.config().get(numKey), testValue);
    +
    +        ConfigKey<Object> refConfigKey = ConfigKeys.newConfigKey(Object.class, "refConfig");
    +
    +        Iterator<Entity> childIter = scopeRoot.getChildren().iterator();
    +        Entity c1 = childIter.next();
    +        assertEquals(c1.config().get(refConfigKey), testValue);
    +        Entity c2 = childIter.next();
    +        assertEquals(c2.config().get(refConfigKey), testValue);
    +        assertFalse(childIter.hasNext());
    +    }
    +    
    +    // XXX TODO parameters on the root don't work with new type registry; 
    +    // they require the CI being able to keep them,
    +    // or else modifying the plan. TODO should they be permitted as metadata in this way?
    +    // or treaded like a declaration of config keys on the entity?  i (alex) prefer the latter.
    --- End diff --
    
    Do you plan on supporting this short term? If not remove/update mention in the catalog docs.
    
    I have a slight preference for parameters in meta, but don't mind removing support.


---
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: Flesh out type creation using the...

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/1017#discussion_r45040452
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/BrooklynTypeRegistry.java ---
    @@ -45,24 +45,25 @@
         /** @return The item matching the given given 
          * {@link RegisteredType#getSymbolicName() symbolicName} 
          * and optionally {@link RegisteredType#getVersion()},
    -     * filtered for the optionally supplied {@link RegisteredTypeConstraint}, 
    +     * filtered for the optionally supplied {@link RegisteredTypeLoadingContext}, 
          * taking the best version if the version is null or a default marker,
          * returning null if no matches are found. */
    -    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} with no constraints */
    +    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} with no constraints */
         RegisteredType get(String symbolicName, String version);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} but allows <code>"name:version"</code> 
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} but allows <code>"name:version"</code> 
          * (the {@link RegisteredType#getId()}) in addition to the unversioned name,
          * using a default marker if no version can be inferred */
    -    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, RegisteredTypeConstraint)} but with no constraints */
    +    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, RegisteredTypeLoadingContext)} but with no constraints */
         RegisteredType get(String symbolicNameWithOptionalVersion);
     
         // NB the seemingly more correct generics <T,SpecT extends AbstractBrooklynObjectSpec<T,SpecT>> 
         // cause compile errors, not in Eclipse, but in maven (?) 
    -    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint optionalConstraint, Class<SpecT> optionalSpecSuperType);
    +    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<SpecT> optionalSpecSuperType);
    +    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpecFromPlan(String planFormat, Object planData, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<SpecT> optionalSpecSuperType);
         
    -    // TODO when we support beans
    -//    <T> T createBean(RegisteredType type, @Nullable RegisteredTypeConstraint optionalConstraint, Class<T> optionalResultSuperType);
    +    <T> T createBean(RegisteredType type, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<T> optionalResultSuperType);
    +    <T> T createBeanFromPlan(String planFormat, Object planData, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<T> optionalBeanSuperType);
    --- End diff --
    
    Interesting point.  Logically they could be moved.  But where would we move them to?  Would we have another `managementContext.getTypeInstantiator()` which uses the `TypeRegistry` ?  That doesn't seem so bad.  Let's think 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] incubator-brooklyn pull request: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r45058958
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypePredicates.java ---
    @@ -109,36 +109,46 @@ public boolean apply(@Nullable RegisteredType item) {
             }
         }
     
    -    public static <T> Predicate<RegisteredType> javaType(final Predicate<Class<T>> filter) {
    -        return new JavaTypeMatches(filter);
    +    public static <T> Predicate<RegisteredType> anySuperType(final Predicate<Class<T>> filter) {
    --- End diff --
    
    Good 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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44552658
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampTypePlanTransformer.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.camp.brooklyn.spi.creation;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredType.TypeImplementationPlan;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.typereg.AbstractCustomImplementationPlan;
    +import org.apache.brooklyn.core.typereg.AbstractTypePlanTransformer;
    +import org.apache.brooklyn.core.typereg.BasicTypeImplementationPlan;
    +import org.apache.brooklyn.core.typereg.RegisteredTypes;
    +import org.apache.brooklyn.util.guava.Maybe;
    +
    +import com.google.common.collect.ImmutableList;
    +
    +public class CampTypePlanTransformer extends AbstractTypePlanTransformer {
    +
    +    private static final List<String> FORMATS = ImmutableList.of("brooklyn-camp", "camp", "brooklyn");
    --- End diff --
    
    Why introduce alternative identifiers if they were never used before? It would make sense only to keep backwards compatibility, but not for new functionality.
    Better stick to a single one.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44634838
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BrooklynTypePlanTransformer.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.List;
    +import java.util.ServiceLoader;
    +
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
    +
    +/**
    + * Interface for use by schemes which with to be able to transform plans.
    + * <p>
    + * To add a new plan transformation scheme, simply create an implementation and declare it
    + * as a java service (cf {@link ServiceLoader}).
    + * <p>
    + * Implementations may wish to extend {@link AbstractTypePlanTransformer} which simplifies the process.
    + */
    +public interface BrooklynTypePlanTransformer extends ManagementContextInjectable {
    --- End diff --
    
    The name is confusing - it's working on `RegisteredType`s, not `BrooklynType`s?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44665943
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    --- End diff --
    
    `symbolicName`, `version` are `@Nullable` as well


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44646760
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/StaticTypePlanTransformer.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.catalog.internal;
    +
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.typereg.AbstractTypePlanTransformer;
    +import org.apache.brooklyn.core.typereg.JavaClassNameTypePlanTransformer;
    +import org.apache.brooklyn.core.typereg.TypePlanTransformers;
    +import org.apache.brooklyn.util.text.Identifiers;
    +
    +/**
    + * Resolves previously registered specs by id.
    + * First create a spec and register it, keeping the returned ID:
    + * <pre> {@code
    + * String specId = StaticTypePlanTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    + * }</pre>
    + *
    + * Then build a plan to be resolved such as:
    + * <pre> {@code
    + *  brooklyn.catalog:
    + *    id: test.inputs
    + *    version: 0.0.1
    + *    item: <specId>
    + * } </pre>
    + */
    +public class StaticTypePlanTransformer extends AbstractTypePlanTransformer {
    +    
    +    public StaticTypePlanTransformer() {
    +        super("static-types", "Static Type", "Static transformer for use in tests");
    +    }
    +
    +    private static final Map<String, AbstractBrooklynObjectSpec<?, ?>> REGISTERED_SPECS = new ConcurrentHashMap<>();
    +
    +    public static void forceInstall() {
    +        TypePlanTransformers.forceAvailable(StaticTypePlanTransformer.class, JavaClassNameTypePlanTransformer.class);
    --- End diff --
    
    How is this class related to `JavaClassNameTypePlanTransformer`?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44637757
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.SPEC, symbolicName, version, plan), superType);
         }
    -    
    -    public abstract static class AbstractRegisteredType implements RegisteredType {
     
    -        final String symbolicName;
    -        final String version;
    +    /** returns the {@link Class} object corresponding to the given java type name,
    +     * using the cache on the type and the loader defined on the type
    +     * @param mgmt */
    +    @Beta
    +    // TODO should this be on the AbstractTypePlanTransformer ?
    +    public static Class<?> loadActualJavaType(String javaTypeName, ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) throws Exception {
    +        Class<?> result = ((BasicRegisteredType)type).getCache().get(ACTUAL_JAVA_TYPE);
    +        if (result!=null) return result;
             
    -        List<OsgiBundleWithUrl> bundles;
    -        String displayName;
    -        String description;
    -        String iconUrl;
    -        boolean deprecated;
    -        boolean disabled;
    -
    -        // TODO ensure this is re-populated on rebind
    -        transient Class<?> javaType;
    +        result = CatalogUtils.newClassLoadingContext(mgmt, type, context==null ? null : context.getLoader()).loadClass( javaTypeName );
    +        Preconditions.checkNotNull(result, "Could not load class "+javaTypeName+"; returned null (should have thrown a different exception!)");
    --- End diff --
    
    `loadClass` returns non null or throws - better mark it in its contract than do defensive programming at call sites.


---
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: Flesh out type creation using the...

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/1017#discussion_r45049272
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/test/BrooklynMgmtUnitTestSupport.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.test;
    +
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
    +import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.AfterMethod;
    +import org.testng.annotations.BeforeMethod;
    +
    +/**
    + * To be extended by unit/integration tests.
    + * <p>
    + * Uses a light-weight management context that will not read {@code ~/.brooklyn/brooklyn.properties}.
    + */
    +public class BrooklynMgmtUnitTestSupport {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(BrooklynMgmtUnitTestSupport.class);
    +
    +    protected ManagementContextInternal mgmt;
    +
    +    @BeforeMethod(alwaysRun=true)
    +    public void setUp() throws Exception {
    +        if (mgmt == null) {
    +            mgmt = LocalManagementContextForTests.newInstance();
    +        }
    +    }
    +
    +    @AfterMethod(alwaysRun=true)
    +    public void tearDown() throws Exception {
    +        try {
    +            if (mgmt != null) Entities.destroyAll(mgmt);
    +        } catch (Throwable t) {
    +            LOG.error("Caught exception in tearDown method", t);
    --- End diff --
    
                // we should fail here, except almost always that masks a primary failure in the test itself,
                // so it would be extremely unhelpful to do so. if we could check if test has not already failed,
                // that would be ideal, but i'm not sure if that's possible with TestNG. ?



---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44650500
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/SpecParameterInMetaTest.java ---
    @@ -43,18 +49,40 @@
     public class SpecParameterInMetaTest {
         private ManagementContext mgmt;
         private BrooklynCatalog catalog;
    -    private String spec;
    +    private String specId;
     
         @BeforeMethod(alwaysRun=true)
         public void setUp() {
             mgmt = LocalManagementContextForTests.newInstanceWithOsgi();
             catalog = mgmt.getCatalog();
    -        spec = TestToSpecTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    +        StaticTypePlanTransformer.forceInstall();
    +        PlanToSpecFactory.forceAvailable(TestToSpecTransformer.class, JavaCatalogToSpecTransformer.class);
    --- End diff --
    
    Part of the tests depend on StaticTypePlanTransformer, other part on PlanToSpecFactory, but not at the same time? If so it would be nice to split them in separate files so the dependencies are clear.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44534762
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/mgmt/classloading/BrooklynClassLoadingContext.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.api.mgmt.classloading;
    +
    +import java.net.URL;
    +
    +import javax.annotation.Nullable;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.util.guava.Maybe;
    +
    +/** 
    + * Provides functionality for loading classes based on the current context
    + * (e.g. the bundles of a registered type from which an entity is created)
    + */
    +public interface BrooklynClassLoadingContext {
    +
    +    public ManagementContext getManagementContext();
    +    public Class<?> loadClass(String className);
    +    public <T> Class<? extends T> loadClass(String className, @Nullable Class<T> supertype);
    +
    +    public Maybe<Class<?>> tryLoadClass(String className);
    +    public <T> Maybe<Class<? extends T>> tryLoadClass(String className, @Nullable Class<T> supertype);
    +
    +    /** As {@link ClassLoader#getResource(String)} */
    +    public URL getResource(String name);
    +
    +    /**
    +     * As {@link ClassLoader#getResources(String)} but returning an {@link Iterable} rather than
    +     * an {@link java.util.Enumeration}.
    +     */
    +    public Iterable<URL> getResources(String name);
    --- End diff --
    
    Not friendly to return `Iterable`, may be it's the right time to change to `Collection`?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44644467
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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.typereg;
    +
    +import groovy.xml.Entity;
    +
    +import java.util.Set;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.objs.BrooklynObject;
    +import org.apache.brooklyn.api.objs.BrooklynObjectType;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +public class RegisteredTypeLoadingContexts {
    +
    +    private static final Logger log = LoggerFactory.getLogger(RegisteredTypeLoadingContexts.BasicRegisteredTypeLoadingContext.class);
    --- End diff --
    
    Weird logger, is it intentional?


---
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: Flesh out type creation using the...

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/1017#discussion_r45040042
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/internal/AbstractBrooklynObjectSpec.java ---
    @@ -160,5 +172,9 @@ public boolean equals(Object obj) {
         public int hashCode() {
             return Objects.hashCode(getCatalogItemId(), getDisplayName(), getType(), getTags());
         }
    +
    +    /** strings inserted as flags, config keys inserted as config keys; 
    +     * if you want to force one or the other, create a ConfigBag and convert to the appropriate map type */
    +    public abstract SpecT configure(Map<?,?> val);
    --- End diff --
    
    you're right, that simplifies things -- have promoted all flag/config methods, and done related tidy


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44647701
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/SpecParameterInMetaTest.java ---
    @@ -43,18 +49,40 @@
     public class SpecParameterInMetaTest {
         private ManagementContext mgmt;
         private BrooklynCatalog catalog;
    -    private String spec;
    +    private String specId;
     
         @BeforeMethod(alwaysRun=true)
         public void setUp() {
             mgmt = LocalManagementContextForTests.newInstanceWithOsgi();
             catalog = mgmt.getCatalog();
    -        spec = TestToSpecTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    +        StaticTypePlanTransformer.forceInstall();
    +        PlanToSpecFactory.forceAvailable(TestToSpecTransformer.class, JavaCatalogToSpecTransformer.class);
    +        specId = StaticTypePlanTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
         }
     
    +    @AfterMethod(alwaysRun=true)
    +    public void tearDown() {
    +        StaticTypePlanTransformer.clearForced();
    +        PlanToSpecFactory.clearForced();
    +    }
    +
    +    @Test
    +    public void testCanRetrieveWithNew() {
    +        AbstractBrooklynObjectSpec<?, ?> spec = mgmt.getTypeRegistry().createSpecFromPlan(null, specId, null, null);
    +        Assert.assertNotNull(spec);
    +    }
    +
    +    // it's not actually added to the catalog; probably it would be cleaner if it is;
    --- End diff --
    
    Being able to create a spec from a plan is different from adding a catalog item so don't agree, it's a separate thing. The mechanism could be used for application specs as well, it's not specific to the catalog.
    
    Could add a utility method somewhere to add a catalog item for the registered spec, but not useful for the following tests.



---
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: Flesh out type creation using the...

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/1017#discussion_r45048101
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/StaticTypePlanTransformer.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.catalog.internal;
    +
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.typereg.AbstractTypePlanTransformer;
    +import org.apache.brooklyn.core.typereg.JavaClassNameTypePlanTransformer;
    +import org.apache.brooklyn.core.typereg.TypePlanTransformers;
    +import org.apache.brooklyn.util.text.Identifiers;
    +
    +/**
    + * Resolves previously registered specs by id.
    + * First create a spec and register it, keeping the returned ID:
    + * <pre> {@code
    + * String specId = StaticTypePlanTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    + * }</pre>
    + *
    + * Then build a plan to be resolved such as:
    + * <pre> {@code
    + *  brooklyn.catalog:
    + *    id: test.inputs
    + *    version: 0.0.1
    + *    item: <specId>
    + * } </pre>
    + */
    +public class StaticTypePlanTransformer extends AbstractTypePlanTransformer {
    +    
    +    public StaticTypePlanTransformer() {
    +        super("static-types", "Static Type", "Static transformer for use in tests");
    +    }
    +
    +    private static final Map<String, AbstractBrooklynObjectSpec<?, ?>> REGISTERED_SPECS = new ConcurrentHashMap<>();
    +
    +    public static void forceInstall() {
    +        TypePlanTransformers.forceAvailable(StaticTypePlanTransformer.class, JavaClassNameTypePlanTransformer.class);
    --- End diff --
    
    different.  this one requires the caller to provide the specs.  `JCNTPT` does a `Class.forName(typeName)` to create the spec.  added javadoc to this effect, plus a few other cleanups (clears the `REGISTERED_SPECS` map!)


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r45057997
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/BrooklynTypeRegistry.java ---
    @@ -45,24 +45,25 @@
         /** @return The item matching the given given 
          * {@link RegisteredType#getSymbolicName() symbolicName} 
          * and optionally {@link RegisteredType#getVersion()},
    -     * filtered for the optionally supplied {@link RegisteredTypeConstraint}, 
    +     * filtered for the optionally supplied {@link RegisteredTypeLoadingContext}, 
          * taking the best version if the version is null or a default marker,
          * returning null if no matches are found. */
    -    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} with no constraints */
    +    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} with no constraints */
         RegisteredType get(String symbolicName, String version);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} but allows <code>"name:version"</code> 
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} but allows <code>"name:version"</code> 
          * (the {@link RegisteredType#getId()}) in addition to the unversioned name,
          * using a default marker if no version can be inferred */
    -    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, RegisteredTypeConstraint)} but with no constraints */
    +    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeLoadingContext constraint);
    --- End diff --
    
    All three points above apply to the instantiator code path only, not the lookup.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44550433
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BrooklynTypePlanTransformer.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.List;
    +import java.util.ServiceLoader;
    +
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
    +
    +/**
    + * Interface for use by schemes which with to be able to transform plans.
    --- End diff --
    
    "which with" - is something missing?


---
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: Flesh out type creation using the...

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/1017#discussion_r45049790
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/api/AssemblyTemplateSpecInstantiator.java ---
    @@ -23,19 +23,23 @@
     
     import org.apache.brooklyn.api.entity.Application;
     import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
     import org.apache.brooklyn.camp.CampPlatform;
     import org.apache.brooklyn.camp.spi.AssemblyTemplate;
     import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
    -import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
     
     public interface AssemblyTemplateSpecInstantiator extends AssemblyTemplateInstantiator {
     
    +    @Deprecated /** @deprecaed since 0.9.0 include encountered types */
    +    EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader);
    +    
         /**
          * Gets the single item returned by {@link #createServiceSpecs}
          * and wraps it in an Application if needed, applying top-level
          * attributes and locations to the root entity.
          */
    -    EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader);
    +    EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader, Set<String> encounteredCatalogTypes);
    --- End diff --
    
    agree, but the `CampResolver` methods use it, currently calling to this when they should be calilng services.  rather than repair that i hope we can deprecate all of this in favour of a new much simpler `CampTypePlanTransformer`.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#issuecomment-157390530
  
    Awesome thorough review.  I've addressed all comments except where I'm commented.  Much better now.  Going to merge #1036 however and then clean this up once more before 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] incubator-brooklyn pull request: Flesh out type creation using the...

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/1017#discussion_r45044594
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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.typereg;
    +
    +import groovy.xml.Entity;
    +
    +import java.util.Set;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.objs.BrooklynObject;
    +import org.apache.brooklyn.api.objs.BrooklynObjectType;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +public class RegisteredTypeLoadingContexts {
    +
    +    private static final Logger log = LoggerFactory.getLogger(RegisteredTypeLoadingContexts.BasicRegisteredTypeLoadingContext.class);
    +    
    +    /** Immutable (from caller's perspective) record of a constraint */
    +    public final static class BasicRegisteredTypeLoadingContext implements RegisteredTypeLoadingContext {
    +        @Nullable private RegisteredTypeKind kind;
    +        @Nullable private Class<?> expectedSuperType;
    +        @Nonnull private Set<String> encounteredTypes = ImmutableSet.of();
    +        @Nullable BrooklynClassLoadingContext loader;
    +        
    +        private BasicRegisteredTypeLoadingContext() {}
    +        
    +        public BasicRegisteredTypeLoadingContext(@Nullable RegisteredTypeLoadingContext source) {
    +            if (source==null) return;
    +            
    +            this.kind = source.getExpectedKind();
    +            this.expectedSuperType = source.getExpectedJavaSuperType();
    +            this.encounteredTypes = source.getAlreadyEncounteredTypes();
    +            this.loader = (BrooklynClassLoadingContext) source.getLoader();
    +        }
    +
    +        @Override
    +        public RegisteredTypeKind getExpectedKind() {
    +            return kind;
    +        }
    +        
    +        @Override
    +        public Class<?> getExpectedJavaSuperType() {
    +            if (expectedSuperType==null) return Object.class;
    +            return expectedSuperType;
    +        }
    +
    +        @Override
    +        public Set<String> getAlreadyEncounteredTypes() {
    +            if (encounteredTypes==null) return ImmutableSet.of();
    +            return ImmutableSet.<String>copyOf(encounteredTypes);
    +        }
    +        
    +        @Override
    +        public BrooklynClassLoadingContext getLoader() {
    +            return loader;
    +        }
    +        
    +        @Override
    +        public String toString() {
    +            return JavaClassNames.cleanSimpleClassName(this)+"["+kind+","+expectedSuperType+","+encounteredTypes+"]";
    +        }
    +    }
    +
    +    /** returns a constraint which allows anything */
    +    public static RegisteredTypeLoadingContext any() {
    +        return new BasicRegisteredTypeLoadingContext();
    +    }
    +
    +    public static RegisteredTypeLoadingContext alreadyEncountered(Set<String> encounteredTypeSymbolicNames) {
    +        BasicRegisteredTypeLoadingContext result = new BasicRegisteredTypeLoadingContext();
    +        result.encounteredTypes = encounteredTypeSymbolicNames == null ? ImmutableSet.<String>of() : ImmutableSet.copyOf(encounteredTypeSymbolicNames);
    +        return result;
    +    }
    +    public static RegisteredTypeLoadingContext alreadyEncountered(Set<String> encounteredTypeSymbolicNames, String anotherEncounteredType) {
    +        BasicRegisteredTypeLoadingContext result = new BasicRegisteredTypeLoadingContext();
    +        MutableSet<String> encounteredTypes = MutableSet.copyOf(encounteredTypeSymbolicNames);
    +        encounteredTypes.addIfNotNull(anotherEncounteredType);
    +        result.encounteredTypes = encounteredTypes.asUnmodifiable();
    +        return result;
    +    }
    +
    +    public static RegisteredTypeLoadingContext loaderAlreadyEncountered(BrooklynClassLoadingContext loader, Set<String> encounteredTypeSymbolicNames) {
    +        return loaderAlreadyEncountered(loader, encounteredTypeSymbolicNames, null);
    +    }
    +    public static RegisteredTypeLoadingContext loaderAlreadyEncountered(BrooklynClassLoadingContext loader, Set<String> encounteredTypeSymbolicNames, String anotherEncounteredType) {
    +        return withLoader(alreadyEncountered(encounteredTypeSymbolicNames, anotherEncounteredType), loader);
    +    }
    +
    +    private static RegisteredTypeLoadingContext of(RegisteredTypeKind kind, Class<?> javaSuperType) {
    +        BasicRegisteredTypeLoadingContext result = new BasicRegisteredTypeLoadingContext();
    +        result.kind = kind;
    +        result.expectedSuperType = javaSuperType;
    +        return result;
    +    }
    +
    +    public static RegisteredTypeLoadingContext bean(Class<?> javaSuperType) {
    +        return of(RegisteredTypeKind.BEAN, javaSuperType);
    +    }
    +
    +    public static RegisteredTypeLoadingContext spec(Class<? extends BrooklynObject> javaSuperType) {
    +        return of(RegisteredTypeKind.SPEC, javaSuperType);
    +    }
    +    
    +    public static <T> RegisteredTypeLoadingContext withBeanSuperType(@Nullable RegisteredTypeLoadingContext source, @Nullable Class<T> beanSuperType) {
    +        Class<T> superType = beanSuperType;
    +        BasicRegisteredTypeLoadingContext constraint = new BasicRegisteredTypeLoadingContext(source);
    +        if (source==null) source = constraint;
    +        if (superType==null) return source;
    +        constraint.expectedSuperType = superType;
    +        if (source.getExpectedJavaSuperType()==null || source.getExpectedJavaSuperType().isAssignableFrom( superType )) {
    +            // the old constraint was weaker than present; return the new constraint
    +            return constraint;
    +        }
    +        if (superType.isAssignableFrom( source.getExpectedJavaSuperType() )) {
    +            // the constraint was already for something more specific; ignore what we've inferred here
    +            return source;
    +        }
    +        log.warn("Ambiguous bean supertypes ("+beanSuperType+" for target "+source.getExpectedJavaSuperType()+"); "
    +            + "it is recommended that any registered type constraint for a spec be compatible with the spec type");
    +        return source;
    +    }
    +
    +    /** Takes a Spec java type and adds an expected java type to the {@link RegisteredTypeLoadingContext} */
    +    public static <T extends AbstractBrooklynObjectSpec<?,?>> RegisteredTypeLoadingContext withSpecSuperType(@Nullable RegisteredTypeLoadingContext source, @Nullable Class<T> specSuperType) {
    +        Class<?> superType = lookupTargetTypeForSpec(specSuperType);
    +        BasicRegisteredTypeLoadingContext constraint = new BasicRegisteredTypeLoadingContext(source);
    +        if (source==null) source = constraint;
    +        if (superType==null) return source;
    +        constraint.expectedSuperType = superType;
    +        if (source.getExpectedJavaSuperType()==null || source.getExpectedJavaSuperType().isAssignableFrom( superType )) {
    +            // the old constraint was weaker than present; return the new constraint
    +            return constraint;
    +        }
    +        if (superType.isAssignableFrom( source.getExpectedJavaSuperType() )) {
    +            // the constraint was already for something more specific; ignore what we've inferred here
    +            return source;
    +        }
    +        // trickier situation; the constraint had a type not compatible with the spec type; log a warning and leave alone
    +        // (e.g. caller specified some java super type which is not a super or sub of the spec target type;
    +        // this may be because the caller specified a Spec as the type supertype, which is wrong;
    +        // or they may have specified an interface along a different hierarchy, which we discouraged
    +        // as it will make filtering/indexing more complex)
    +        log.warn("Ambiguous spec supertypes ("+specSuperType+" for target "+source.getExpectedJavaSuperType()+"); "
    +            + "it is recommended that any registered type constraint for a spec be compatible with the spec type");
    +        return source;
    +    }
    +        
    +    /** given a spec, returns the class of the item it targets, for instance returns {@link Entity} given {@link EntitySpec};
    +     * see also {@link #lookupSpecTypeForTarget(Class)} */
    +    static <T extends AbstractBrooklynObjectSpec<?,?>> Class<? extends BrooklynObject> lookupTargetTypeForSpec(Class<T> specSuperType) {
    +        if (specSuperType==null) return BrooklynObject.class;
    +        BrooklynObjectType best = null;
    +
    +        for (BrooklynObjectType t: BrooklynObjectType.values()) {
    +            if (t.getSpecType()==null) continue;
    +            if (!t.getSpecType().isAssignableFrom(specSuperType)) continue;
    +            // on equality, exit immediately
    +            if (t.getSpecType().equals(specSuperType)) return t.getInterfaceType();
    +            // else pick which is best
    +            if (best==null) { best = t; continue; }
    +            // if t is more specific, it is better (handles case when e.g. a Policy is a subclass of Entity)
    +            if (best.getSpecType().isAssignableFrom(t.getSpecType())) { best = t; continue; }
    +        }
    +        if (best==null) {
    +            log.warn("Unexpected spec supertype ("+specSuperType+"); treating as any "+BrooklynObject.class, new Throwable("Trace for unexpected spec supertype"));
    +            return BrooklynObject.class;
    +        }
    +        // the spec is more specific, but we're not familiar with it here; return the best
    +        return best.getInterfaceType();
    +    }
    +
    +    /** given a {@link BrooklynObject}, returns the spec class which would generate it, for instance returns {@link EntitySpec} given {@link Entity},
    +     * or null if not known */
    +    static <BO extends BrooklynObject> Class<? extends AbstractBrooklynObjectSpec<?,?>> lookupSpecTypeForTarget(Class<BO> targetSuperType) {
    +        if (targetSuperType==null) return null;
    +        BrooklynObjectType best = null;
    +
    +        for (BrooklynObjectType t: BrooklynObjectType.values()) {
    +            if (t.getSpecType()==null) continue;
    +            if (!t.getInterfaceType().isAssignableFrom(targetSuperType)) continue;
    +            // on equality, exit immediately
    +            if (t.getInterfaceType().equals(targetSuperType)) return t.getSpecType();
    +            // else pick which is best
    +            if (best==null) { best = t; continue; }
    +            // if t is more specific, it is better (handles case when e.g. a Policy is a subclass of Entity)
    +            if (best.getSpecType().isAssignableFrom(t.getSpecType())) { best = t; continue; }
    --- End diff --
    
    good spots


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44551536
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BrooklynTypePlanTransformer.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.List;
    +import java.util.ServiceLoader;
    +
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
    +
    +/**
    + * Interface for use by schemes which with to be able to transform plans.
    + * <p>
    + * To add a new plan transformation scheme, simply create an implementation and declare it
    + * as a java service (cf {@link ServiceLoader}).
    + * <p>
    + * Implementations may wish to extend {@link AbstractTypePlanTransformer} which simplifies the process.
    + */
    +public interface BrooklynTypePlanTransformer extends ManagementContextInjectable {
    +
    +    /** @return a code to identify type implementations created specifying the use of this plan transformer. */
    +    String getFormatCode();
    +    /** @return a display name for this transformer. */
    +    String getFormatName();
    +    /** @return a description for this transformer */
    +    String getFormatDescription();
    +
    +    /** @return how appropriate is this transformer for the {@link RegisteredType#getPlan()} of the type;
    +     * 0 (or less) if not, 1 for absolutely, and in some autodetect cases a value between 0 and 1 indicate a ranking.
    +     * <p>
    +     * The framework guarantees arguments are nonnull, and that the {@link RegisteredType#getPlan()} is also not-null.
    --- End diff --
    
    Worth annotating the arguments with `@Nonnull`, more visible than the javadoc. The return values for the methods you mention as well.


---
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: Flesh out type creation using the...

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/1017#discussion_r45049289
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/test/BrooklynMgmtUnitTestSupport.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.test;
    +
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
    +import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.AfterMethod;
    +import org.testng.annotations.BeforeMethod;
    +
    +/**
    + * To be extended by unit/integration tests.
    + * <p>
    + * Uses a light-weight management context that will not read {@code ~/.brooklyn/brooklyn.properties}.
    + */
    +public class BrooklynMgmtUnitTestSupport {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(BrooklynMgmtUnitTestSupport.class);
    +
    +    protected ManagementContextInternal mgmt;
    +
    +    @BeforeMethod(alwaysRun=true)
    +    public void setUp() throws Exception {
    +        if (mgmt == null) {
    +            mgmt = LocalManagementContextForTests.newInstance();
    +        }
    +    }
    +
    +    @AfterMethod(alwaysRun=true)
    +    public void tearDown() throws Exception {
    +        try {
    +            if (mgmt != null) Entities.destroyAll(mgmt);
    +        } catch (Throwable t) {
    +            LOG.error("Caught exception in tearDown method", t);
    --- End diff --
    
    (agree, have added comment)


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r45060455
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/SpecParameterInMetaTest.java ---
    @@ -43,18 +49,40 @@
     public class SpecParameterInMetaTest {
         private ManagementContext mgmt;
         private BrooklynCatalog catalog;
    -    private String spec;
    +    private String specId;
     
         @BeforeMethod(alwaysRun=true)
         public void setUp() {
             mgmt = LocalManagementContextForTests.newInstanceWithOsgi();
             catalog = mgmt.getCatalog();
    -        spec = TestToSpecTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    +        StaticTypePlanTransformer.forceInstall();
    +        PlanToSpecFactory.forceAvailable(TestToSpecTransformer.class, JavaCatalogToSpecTransformer.class);
    +        specId = StaticTypePlanTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
         }
     
    +    @AfterMethod(alwaysRun=true)
    +    public void tearDown() {
    +        StaticTypePlanTransformer.clearForced();
    +        PlanToSpecFactory.clearForced();
    +    }
    +
    +    @Test
    +    public void testCanRetrieveWithNew() {
    +        AbstractBrooklynObjectSpec<?, ?> spec = mgmt.getTypeRegistry().createSpecFromPlan(null, specId, null, null);
    +        Assert.assertNotNull(spec);
    +    }
    +
    +    // it's not actually added to the catalog; probably it would be cleaner if it is;
    --- End diff --
    
    The idea of `TestToSpecTransformer` is to convert a plan to a pre-registered spec. It's not working on types but on plans. This means that it's able to convert a plan it understands (be it coming from a catalog item or used as an app plan) to a spec. The plan itself may be added as a catalog item, but it's a separate matter.
    
    A plan refers to a type, but is not itself a type, it's a composition of types. Calling `registerSpec` tells the transformer how to interpret the plan, not what the spec for a specific type is.


---
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: Flesh out type creation using the...

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

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


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44548055
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/plan/PlanNotRecognizedException.java ---
    @@ -18,8 +18,14 @@
      */
     package org.apache.brooklyn.core.plan;
     
    +import org.apache.brooklyn.core.typereg.BrooklynTypePlanTransformer;
    +import org.apache.brooklyn.core.typereg.UnsupportedTypePlanException;
    +
    +/** @deprecated since 0.9.0 use {@link UnsupportedTypePlanException} as part of switch to {@link BrooklynTypePlanTransformer} */
    +@Deprecated 
     public class PlanNotRecognizedException extends RuntimeException {
     
    +    /** {@link UnsupportedTypePlanException} */
    --- End diff --
    
    Don't see the point of the comment?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44637673
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.SPEC, symbolicName, version, plan), superType);
         }
    -    
    -    public abstract static class AbstractRegisteredType implements RegisteredType {
     
    -        final String symbolicName;
    -        final String version;
    +    /** returns the {@link Class} object corresponding to the given java type name,
    +     * using the cache on the type and the loader defined on the type
    +     * @param mgmt */
    +    @Beta
    +    // TODO should this be on the AbstractTypePlanTransformer ?
    +    public static Class<?> loadActualJavaType(String javaTypeName, ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) throws Exception {
    +        Class<?> result = ((BasicRegisteredType)type).getCache().get(ACTUAL_JAVA_TYPE);
    --- End diff --
    
    Isn't it better to have a local `WeakHashMap` cache, than assume that the parameter will be of specific type?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44659164
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampResolver.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.camp.brooklyn.spi.creation;
    +
    +import java.util.Set;
    +
    +import org.apache.brooklyn.api.entity.Application;
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.policy.Policy;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.camp.CampPlatform;
    +import org.apache.brooklyn.camp.brooklyn.api.AssemblyTemplateSpecInstantiator;
    +import org.apache.brooklyn.camp.spi.AssemblyTemplate;
    +import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
    +import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
    +import org.apache.brooklyn.core.mgmt.EntityManagementUtils;
    +import org.apache.brooklyn.core.typereg.RegisteredTypes;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.text.Strings;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +class CampResolver {
    +
    +    private ManagementContext mgmt;
    +    private RegisteredType type;
    +    private RegisteredTypeLoadingContext context;
    +
    +    /** whether to allow parsing of the 'full' syntax for applications,
    +     * where items are wrapped in a "services:" block, and if the wrapper is an application,
    +     * to promote it */
    +    boolean allowApplicationFullSyntax = true;
    +
    +    /** whether to allow parsing of the legacy 'full' syntax, 
    +     * where a non-application items are wrapped:
    +     * <li> in a "services:" block for entities,
    +     * <li> in a "brooklyn.locations" or "brooklyn.policies" block for locations and policies */
    +    boolean allowLegacyFullSyntax = true;
    +
    +    /** whether to allow parsing of the type syntax, where an item is a map with a "type:" field,
    +     * i.e. not wrapped in any "services:" or "brooklyn.{locations,policies}" block */
    +    boolean allowTypeSyntax = true;
    +
    +    public CampResolver(ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) {
    +        this.mgmt = mgmt;
    +        this.type = type;
    +        this.context = context;
    +    }
    +
    +    public AbstractBrooklynObjectSpec<?, ?> createSpec() {
    +        // TODO new-style approach:
    +        //            AbstractBrooklynObjectSpec<?, ?> spec = RegisteredTypes.newSpecInstance(mgmt, /* 'type' key */);
    +        //            spec.configure(keysAndValues);
    +        return createSpecFromFull(mgmt, type, context.getExpectedJavaSuperType(), context.getAlreadyEncounteredTypes(), context.getLoader());
    +    }
    +
    +    static AbstractBrooklynObjectSpec<?, ?> createSpecFromFull(ManagementContext mgmt, RegisteredType item, Class<?> expectedType, Set<String> parentEncounteredTypes, BrooklynClassLoadingContext loaderO) {
    +        // for this method, a prefix "services" or "brooklyn.{location,policies}" is required at the root;
    +        // we now prefer items to come in "{ type: .. }" format, except for application roots which
    +        // should have a "services: [ ... ]" block (and which may subsequently be unwrapped)
    +        BrooklynClassLoadingContext loader = CatalogUtils.newClassLoadingContext(mgmt, item, loaderO);
    +
    +        Set<String> encounteredTypes;
    +        // symbolicName could be null if coming from the catalog parser where it tries to load before knowing the id
    +        if (item.getSymbolicName() != null) {
    +            encounteredTypes = ImmutableSet.<String>builder()
    +                .addAll(parentEncounteredTypes)
    +                .add(item.getSymbolicName())
    +                .build();
    +        } else {
    +            encounteredTypes = parentEncounteredTypes;
    +        }
    +
    +        AbstractBrooklynObjectSpec<?, ?> spec;
    +        String planYaml = RegisteredTypes.getImplementationDataStringForSpec(item);
    +        MutableSet<Object> supers = MutableSet.copyOf(item.getSuperTypes());
    +        supers.addIfNotNull(expectedType);
    +        if (RegisteredTypes.isSubTypeOf(supers, Policy.class)) {
    +            spec = CampInternalUtils.createPolicySpec(planYaml, loader, encounteredTypes);
    +        } else if (RegisteredTypes.isSubTypeOf(supers, Location.class)) {
    +            spec = CampInternalUtils.createLocationSpec(planYaml, loader, encounteredTypes);
    +        } else if (RegisteredTypes.isSubTypeOf(supers, Application.class)) {
    +            spec = createEntitySpecFromServicesBlock(planYaml, loader, encounteredTypes, true);
    +        } else if (RegisteredTypes.isSubTypeOf(supers, Entity.class)) {
    +            spec = createEntitySpecFromServicesBlock(planYaml, loader, encounteredTypes, false);
    +        } else {
    +            // try any of them???
    +            
    +            throw new IllegalStateException("Cannot detect spec type from "+item.getSuperTypes()+" for "+item+"\n"+planYaml);
    +        }
    +
    +        ((AbstractBrooklynObjectSpec<?, ?>)spec).catalogItemId(item.getId());
    +
    +        if (Strings.isBlank( ((AbstractBrooklynObjectSpec<?, ?>)spec).getDisplayName() ))
    +            ((AbstractBrooklynObjectSpec<?, ?>)spec).displayName(item.getDisplayName());
    +
    +        return spec;
    +    }
    + 
    +    private static EntitySpec<?> createEntitySpecFromServicesBlock(String plan, BrooklynClassLoadingContext loader, Set<String> encounteredTypes, boolean isApplication) {
    +        CampPlatform camp = CampInternalUtils.getCampPlatform(loader.getManagementContext());
    +
    +        AssemblyTemplate at = CampInternalUtils.registerDeploymentPlan(plan, loader, camp);
    +        AssemblyTemplateInstantiator instantiator = CampInternalUtils.getInstantiator(at);
    +        if (instantiator instanceof AssemblyTemplateSpecInstantiator) {
    +            EntitySpec<? extends Application> appSpec = ((AssemblyTemplateSpecInstantiator)instantiator).createApplicationSpec(at, camp, loader, encounteredTypes);
    +
    +            if (!isApplication && EntityManagementUtils.canPromoteChildrenInWrappedApplication(appSpec) && appSpec.getChildren().size()==1) {
    --- End diff --
    
    `appSpec.getChildren().size()==1` already included in the `canPromoteChildrenInWrappedApplication` check.
    Also (minor) can move this block in a wrapper function instead of using a flag argument.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44542209
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/BrooklynTypeRegistry.java ---
    @@ -45,24 +45,25 @@
         /** @return The item matching the given given 
          * {@link RegisteredType#getSymbolicName() symbolicName} 
          * and optionally {@link RegisteredType#getVersion()},
    -     * filtered for the optionally supplied {@link RegisteredTypeConstraint}, 
    +     * filtered for the optionally supplied {@link RegisteredTypeLoadingContext}, 
          * taking the best version if the version is null or a default marker,
          * returning null if no matches are found. */
    -    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} with no constraints */
    +    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} with no constraints */
         RegisteredType get(String symbolicName, String version);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} but allows <code>"name:version"</code> 
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} but allows <code>"name:version"</code> 
          * (the {@link RegisteredType#getId()}) in addition to the unversioned name,
          * using a default marker if no version can be inferred */
    -    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, RegisteredTypeConstraint)} but with no constraints */
    +    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeLoadingContext constraint);
    --- End diff --
    
    It's not clear from the javadoc what does the `constraint` filter on `get` methods. Do we allow conflicting types - for example a catalog item and a bean with the same `symbolicName` where `version` is not enough of a discriminator?
       * if yes, then shouldn't the return type be a collection for the case where the `constraint


---
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: Flesh out type creation using the...

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/1017#discussion_r45050468
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampResolver.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.camp.brooklyn.spi.creation;
    +
    +import java.util.Set;
    +
    +import org.apache.brooklyn.api.entity.Application;
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.policy.Policy;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.camp.CampPlatform;
    +import org.apache.brooklyn.camp.brooklyn.api.AssemblyTemplateSpecInstantiator;
    +import org.apache.brooklyn.camp.spi.AssemblyTemplate;
    +import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
    +import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
    +import org.apache.brooklyn.core.mgmt.EntityManagementUtils;
    +import org.apache.brooklyn.core.typereg.RegisteredTypes;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.text.Strings;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +class CampResolver {
    +
    +    private ManagementContext mgmt;
    +    private RegisteredType type;
    +    private RegisteredTypeLoadingContext context;
    +
    +    /** whether to allow parsing of the 'full' syntax for applications,
    +     * where items are wrapped in a "services:" block, and if the wrapper is an application,
    +     * to promote it */
    +    boolean allowApplicationFullSyntax = true;
    +
    +    /** whether to allow parsing of the legacy 'full' syntax, 
    +     * where a non-application items are wrapped:
    +     * <li> in a "services:" block for entities,
    +     * <li> in a "brooklyn.locations" or "brooklyn.policies" block for locations and policies */
    +    boolean allowLegacyFullSyntax = true;
    +
    +    /** whether to allow parsing of the type syntax, where an item is a map with a "type:" field,
    +     * i.e. not wrapped in any "services:" or "brooklyn.{locations,policies}" block */
    +    boolean allowTypeSyntax = true;
    +
    +    public CampResolver(ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) {
    +        this.mgmt = mgmt;
    +        this.type = type;
    +        this.context = context;
    +    }
    +
    +    public AbstractBrooklynObjectSpec<?, ?> createSpec() {
    +        // TODO new-style approach:
    +        //            AbstractBrooklynObjectSpec<?, ?> spec = RegisteredTypes.newSpecInstance(mgmt, /* 'type' key */);
    +        //            spec.configure(keysAndValues);
    +        return createSpecFromFull(mgmt, type, context.getExpectedJavaSuperType(), context.getAlreadyEncounteredTypes(), context.getLoader());
    +    }
    +
    +    static AbstractBrooklynObjectSpec<?, ?> createSpecFromFull(ManagementContext mgmt, RegisteredType item, Class<?> expectedType, Set<String> parentEncounteredTypes, BrooklynClassLoadingContext loaderO) {
    +        // for this method, a prefix "services" or "brooklyn.{location,policies}" is required at the root;
    +        // we now prefer items to come in "{ type: .. }" format, except for application roots which
    +        // should have a "services: [ ... ]" block (and which may subsequently be unwrapped)
    +        BrooklynClassLoadingContext loader = CatalogUtils.newClassLoadingContext(mgmt, item, loaderO);
    +
    +        Set<String> encounteredTypes;
    +        // symbolicName could be null if coming from the catalog parser where it tries to load before knowing the id
    +        if (item.getSymbolicName() != null) {
    +            encounteredTypes = ImmutableSet.<String>builder()
    +                .addAll(parentEncounteredTypes)
    +                .add(item.getSymbolicName())
    +                .build();
    +        } else {
    +            encounteredTypes = parentEncounteredTypes;
    +        }
    +
    +        AbstractBrooklynObjectSpec<?, ?> spec;
    +        String planYaml = RegisteredTypes.getImplementationDataStringForSpec(item);
    +        MutableSet<Object> supers = MutableSet.copyOf(item.getSuperTypes());
    +        supers.addIfNotNull(expectedType);
    +        if (RegisteredTypes.isSubTypeOf(supers, Policy.class)) {
    +            spec = CampInternalUtils.createPolicySpec(planYaml, loader, encounteredTypes);
    +        } else if (RegisteredTypes.isSubTypeOf(supers, Location.class)) {
    +            spec = CampInternalUtils.createLocationSpec(planYaml, loader, encounteredTypes);
    +        } else if (RegisteredTypes.isSubTypeOf(supers, Application.class)) {
    +            spec = createEntitySpecFromServicesBlock(planYaml, loader, encounteredTypes, true);
    +        } else if (RegisteredTypes.isSubTypeOf(supers, Entity.class)) {
    +            spec = createEntitySpecFromServicesBlock(planYaml, loader, encounteredTypes, false);
    +        } else {
    +            // try any of them???
    +            
    +            throw new IllegalStateException("Cannot detect spec type from "+item.getSuperTypes()+" for "+item+"\n"+planYaml);
    +        }
    +
    +        ((AbstractBrooklynObjectSpec<?, ?>)spec).catalogItemId(item.getId());
    +
    +        if (Strings.isBlank( ((AbstractBrooklynObjectSpec<?, ?>)spec).getDisplayName() ))
    +            ((AbstractBrooklynObjectSpec<?, ?>)spec).displayName(item.getDisplayName());
    +
    +        return spec;
    +    }
    + 
    +    private static EntitySpec<?> createEntitySpecFromServicesBlock(String plan, BrooklynClassLoadingContext loader, Set<String> encounteredTypes, boolean isApplication) {
    +        CampPlatform camp = CampInternalUtils.getCampPlatform(loader.getManagementContext());
    +
    +        AssemblyTemplate at = CampInternalUtils.registerDeploymentPlan(plan, loader, camp);
    +        AssemblyTemplateInstantiator instantiator = CampInternalUtils.getInstantiator(at);
    +        if (instantiator instanceof AssemblyTemplateSpecInstantiator) {
    +            EntitySpec<? extends Application> appSpec = ((AssemblyTemplateSpecInstantiator)instantiator).createApplicationSpec(at, camp, loader, encounteredTypes);
    +
    +            if (!isApplication && EntityManagementUtils.canPromoteChildrenInWrappedApplication(appSpec) && appSpec.getChildren().size()==1) {
    +                CampInternalUtils.resetSpecIfTemplateHasNoExplicitParameters(at, appSpec);
    --- End diff --
    
    good spot


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44630807
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/AbstractTypePlanTransformer.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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.typereg;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Convenience supertype for {@link BrooklynTypePlanTransformer} instances.
    + */
    +public abstract class AbstractTypePlanTransformer implements BrooklynTypePlanTransformer {
    +
    +    private static final Logger log = LoggerFactory.getLogger(AbstractTypePlanTransformer.class);
    +    
    +    protected ManagementContext mgmt;
    +
    +    @Override
    +    public void injectManagementContext(ManagementContext mgmt) {
    +        this.mgmt = mgmt;
    +    }
    +
    +    private final String format;
    +    private final String formatName;
    +    private final String formatDescription;
    +    
    +    protected AbstractTypePlanTransformer(String format, String formatName, String formatDescription) {
    +        this.format = format;
    +        this.formatName = formatName;
    +        this.formatDescription = formatDescription;
    +    }
    +    
    +    @Override
    +    public String getFormatCode() {
    +        return format;
    +    }
    +
    +    @Override
    +    public String getFormatName() {
    +        return formatName;
    +    }
    +
    +    @Override
    +    public String getFormatDescription() {
    +        return formatDescription;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return getFormatCode()+":"+JavaClassNames.simpleClassName(this);
    +    }
    +    
    +    @Override
    +    public double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context) {
    +        if (getFormatCode().equals(type.getPlan().getPlanFormat())) return 1;
    +        if (type.getPlan().getPlanFormat()==null)
    +            return scoreForNullFormat(type.getPlan().getPlanData(), type, context);
    +        else
    +            return scoreForNonmatchingNonnullFormat(type.getPlan().getPlanFormat(), type.getPlan().getPlanData(), type, context);
    +    }
    +
    +    protected abstract double scoreForNullFormat(Object planData, RegisteredType type, RegisteredTypeLoadingContext context);
    +    protected abstract double scoreForNonmatchingNonnullFormat(String planFormat, Object planData, RegisteredType type, RegisteredTypeLoadingContext context);
    +
    +    /** delegates to more specific abstract create methods,
    +     * and performs common validation and customisation of the items created.
    +     * <p>
    +     * this includes:
    +     * <li> setting the {@link AbstractBrooklynObjectSpec#catalogItemId(String)}
    +     */
    +    @Override
    +    public Object create(final RegisteredType type, final RegisteredTypeLoadingContext context) {
    +        try {
    +            return validate(new RegisteredTypeKindVisitor<Object>() {
    +                @Override protected Object visitSpec(RegisteredType type) {
    +                    try { 
    +                        AbstractBrooklynObjectSpec<?, ?> result = createSpec(type, context);
    +                        result.catalogItemId(type.getId());
    +                        return result;
    +                    } catch (Exception e) { throw Exceptions.propagate(e); }
    +                }
    +                @Override protected Object visitBean(RegisteredType type) {
    +                    try { 
    +                        return createBean(type, context);
    +                    } catch (Exception e) { throw Exceptions.propagate(e); }
    +                }
    +                
    +            }.visit(type), type, context);
    +        } catch (UnsupportedTypePlanException e) {
    +            // no logging
    +            throw Exceptions.propagate(e);
    +        } catch (Exception e) {
    +            Exceptions.propagateIfFatal(e);
    +            log.debug("Could not instantiate "+type+" (rethrowing): "+Exceptions.collapseText(e));
    +            throw Exceptions.propagate(e);
    +        }
    +    }
    +    
    +    protected <T> T validate(T createdObject, RegisteredType type, RegisteredTypeLoadingContext context) {
    +        if (createdObject==null) return null;
    +        // TODO validation based on the constraint, throw UnsupportedTypePlanException with details if not matched
    +        return createdObject;
    +    }
    +
    +    protected abstract AbstractBrooklynObjectSpec<?,?> createSpec(RegisteredType type, RegisteredTypeLoadingContext context) throws Exception;
    --- End diff --
    
    `throws Exception` works for tests, but doesn't it mask failure points that should be handled in implementations?


---
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: Flesh out type creation using the...

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/1017#discussion_r45042573
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/AbstractTypePlanTransformer.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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.typereg;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Convenience supertype for {@link BrooklynTypePlanTransformer} instances.
    + */
    +public abstract class AbstractTypePlanTransformer implements BrooklynTypePlanTransformer {
    +
    +    private static final Logger log = LoggerFactory.getLogger(AbstractTypePlanTransformer.class);
    +    
    +    protected ManagementContext mgmt;
    +
    +    @Override
    +    public void injectManagementContext(ManagementContext mgmt) {
    +        this.mgmt = mgmt;
    +    }
    +
    +    private final String format;
    +    private final String formatName;
    +    private final String formatDescription;
    +    
    +    protected AbstractTypePlanTransformer(String format, String formatName, String formatDescription) {
    +        this.format = format;
    +        this.formatName = formatName;
    +        this.formatDescription = formatDescription;
    +    }
    +    
    +    @Override
    +    public String getFormatCode() {
    +        return format;
    +    }
    +
    +    @Override
    +    public String getFormatName() {
    +        return formatName;
    +    }
    +
    +    @Override
    +    public String getFormatDescription() {
    +        return formatDescription;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return getFormatCode()+":"+JavaClassNames.simpleClassName(this);
    +    }
    +    
    +    @Override
    +    public double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context) {
    +        if (getFormatCode().equals(type.getPlan().getPlanFormat())) return 1;
    +        if (type.getPlan().getPlanFormat()==null)
    +            return scoreForNullFormat(type.getPlan().getPlanData(), type, context);
    +        else
    +            return scoreForNonmatchingNonnullFormat(type.getPlan().getPlanFormat(), type.getPlan().getPlanData(), type, context);
    --- End diff --
    
    That wasn't the intention.  The idea is to simplify subclass implementations which typically do different things if the format is specified.  Have added javadoc to the class.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44641189
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypePredicates.java ---
    @@ -109,36 +109,46 @@ public boolean apply(@Nullable RegisteredType item) {
             }
         }
     
    -    public static <T> Predicate<RegisteredType> javaType(final Predicate<Class<T>> filter) {
    -        return new JavaTypeMatches(filter);
    +    public static <T> Predicate<RegisteredType> anySuperType(final Predicate<Class<T>> filter) {
    --- End diff --
    
    Generic type is not used, replace with wildcard.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44633926
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BasicBrooklynTypeRegistry.java ---
    @@ -90,31 +96,95 @@ public RegisteredType get(String symbolicNameWithOptionalVersion, RegisteredType
     
         @Override
         public RegisteredType get(String symbolicNameWithOptionalVersion) {
    -        return get(symbolicNameWithOptionalVersion, (RegisteredTypeConstraint)null);
    +        return get(symbolicNameWithOptionalVersion, (RegisteredTypeLoadingContext)null);
         }
     
         @SuppressWarnings({ "deprecation", "unchecked", "rawtypes" })
         @Override
    -    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint constraint, Class<SpecT> specSuperType) {
    -        if (!(type instanceof RegisteredSpecType)) { 
    -            throw new IllegalStateException("Cannot create spec from type "+type);
    +    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext constraint, Class<SpecT> specSuperType) {
    +        Preconditions.checkNotNull(type, "type");
    +        if (type.getKind()!=RegisteredTypeKind.SPEC) { 
    +            throw new IllegalStateException("Cannot create spec from type "+type+" (kind "+type.getKind()+")");
             }
             if (constraint!=null) {
    -            if (constraint.getKind()!=null && constraint.getKind()!=RegisteredTypeKind.SPEC) {
    +            if (constraint.getExpectedKind()!=null && constraint.getExpectedKind()!=RegisteredTypeKind.SPEC) {
                     throw new IllegalStateException("Cannot create spec with constraint "+constraint);
                 }
    -            if (constraint.getEncounteredTypes().contains(type.getSymbolicName())) {
    +            if (constraint.getAlreadyEncounteredTypes().contains(type.getSymbolicName())) {
                     // avoid recursive cycle
                     // TODO implement using java if permitted
                 }
             }
    -        constraint = RegisteredTypeConstraints.extendedWithSpecSuperType(constraint, specSuperType);
    +        constraint = RegisteredTypeLoadingContexts.withSpecSuperType(constraint, specSuperType);
     
    -        // TODO look up in the actual registry
    +        Maybe<Object> result = TypePlanTransformers.transform(mgmt, type, constraint);
    +        if (result.isPresent()) return (SpecT) result.get();
             
             // fallback: look up in (legacy) catalog
    +        // TODO remove once all transformers are available in the new style
             CatalogItem item = (CatalogItem) mgmt.getCatalog().getCatalogItem(type.getSymbolicName(), type.getVersion());
    -        return (SpecT) BasicBrooklynCatalog.internalCreateSpecWithTransformers(mgmt, item, constraint.getEncounteredTypes());
    +        if (item==null) {
    +            // if not in catalog (because loading a new item?) then look up item based on type
    +            // (only really used in tests; possibly also for any recursive legacy transformers we might have to create a CI; cross that bridge when we come to it)
    +            CatalogItemType ciType = CatalogItemType.ofTargetClass( (Class)constraint.getExpectedJavaSuperType() );
    --- End diff --
    
    No point in adding code to support tests, better adapt the tests?
    Also don't get the recursive transformers comment, can you give an example.


---
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: Flesh out type creation using the...

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/1017#discussion_r45042177
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BrooklynTypePlanTransformer.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.List;
    +import java.util.ServiceLoader;
    +
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
    +
    +/**
    + * Interface for use by schemes which with to be able to transform plans.
    + * <p>
    + * To add a new plan transformation scheme, simply create an implementation and declare it
    + * as a java service (cf {@link ServiceLoader}).
    + * <p>
    + * Implementations may wish to extend {@link AbstractTypePlanTransformer} which simplifies the process.
    + */
    +public interface BrooklynTypePlanTransformer extends ManagementContextInjectable {
    +
    +    /** @return a code to identify type implementations created specifying the use of this plan transformer. */
    +    String getFormatCode();
    +    /** @return a display name for this transformer. */
    +    String getFormatName();
    +    /** @return a description for this transformer */
    +    String getFormatDescription();
    +
    +    /** @return how appropriate is this transformer for the {@link RegisteredType#getPlan()} of the type;
    +     * 0 (or less) if not, 1 for absolutely, and in some autodetect cases a value between 0 and 1 indicate a ranking.
    +     * <p>
    +     * The framework guarantees arguments are nonnull, and that the {@link RegisteredType#getPlan()} is also not-null.
    +     * However the format in that plan may be null. */
    +    double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context);
    --- End diff --
    
    have expanded comments to try to help determine right score.  if we store formats right this won't be used very often (or possibly even ever in which case we can get rid of 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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44643874
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.SPEC, symbolicName, version, plan), superType);
         }
    -    
    -    public abstract static class AbstractRegisteredType implements RegisteredType {
     
    -        final String symbolicName;
    -        final String version;
    +    /** returns the {@link Class} object corresponding to the given java type name,
    +     * using the cache on the type and the loader defined on the type
    +     * @param mgmt */
    +    @Beta
    +    // TODO should this be on the AbstractTypePlanTransformer ?
    +    public static Class<?> loadActualJavaType(String javaTypeName, ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) throws Exception {
    +        Class<?> result = ((BasicRegisteredType)type).getCache().get(ACTUAL_JAVA_TYPE);
    +        if (result!=null) return result;
             
    -        List<OsgiBundleWithUrl> bundles;
    -        String displayName;
    -        String description;
    -        String iconUrl;
    -        boolean deprecated;
    -        boolean disabled;
    -
    -        // TODO ensure this is re-populated on rebind
    -        transient Class<?> javaType;
    +        result = CatalogUtils.newClassLoadingContext(mgmt, type, context==null ? null : context.getLoader()).loadClass( javaTypeName );
    +        Preconditions.checkNotNull(result, "Could not load class "+javaTypeName+"; returned null (should have thrown a different exception!)");
             
    -        public AbstractRegisteredType(String symbolicName, String version, Class<?> javaType) {
    -            this.symbolicName = symbolicName;
    -            this.version = version;
    -            this.javaType = javaType;
    -        }
    -
    -        @Override
    -        public String getId() {
    -            return symbolicName + (version!=null ? ":"+version : "");
    -        }
    +        ((BasicRegisteredType)type).getCache().put(ACTUAL_JAVA_TYPE, result);
    +        return result;
    +    }
     
    -        @Override
    -        public String getSymbolicName() {
    -            return symbolicName;
    +    @Beta
    +    public static RegisteredType addSuperType(RegisteredType type, @Nullable Class<?> superType) {
    +        if (superType!=null) {
    +            ((BasicRegisteredType)type).superTypes.add(superType);
             }
    +        return type;
    +    }
     
    -        @Override
    -        public String getVersion() {
    -            return version;
    -        }
    -        
    -        @Override
    -        public Collection<OsgiBundleWithUrl> getLibraries() {
    -            return bundles;
    +    @Beta
    +    public static RegisteredType addSuperType(RegisteredType type, @Nullable RegisteredType superType) {
    +        if (superType!=null) {
    +            if (isSubTypeOf(superType, type)) {
    +                throw new IllegalStateException(superType+" declares "+type+" as a supertype; cannot set "+superType+" as a supertype of "+type);
    +            }
    +            ((BasicRegisteredType)type).superTypes.add(superType);
             }
    +        return type;
    +    }
     
    -        @Override
    -        public String getDisplayName() {
    -            return displayName;
    -        }
    +    /** returns the implementation data for a spec if it is a string (e.g. plan yaml or java class name); else false */
    +    @Beta
    +    public static String getImplementationDataStringForSpec(RegisteredType item) {
    +        if (item==null || item.getPlan()==null) return null;
    +        Object data = item.getPlan().getPlanData();
    +        if (!(data instanceof String)) return null;
    +        return (String)data;
    +    }
     
    -        @Override
    -        public String getDescription() {
    -            return description;
    -        }
    +    /** returns an implementation of the spec class corresponding to the given target type;
    +     * for use in {@link BrooklynTypePlanTransformer#create(RegisteredType, RegisteredTypeLoadingContext)} 
    +     * implementations when dealing with a spec; returns null if none found
    +     * @param mgmt */
    +    @Beta
    +    public static AbstractBrooklynObjectSpec<?,?> newSpecInstance(ManagementContext mgmt, Class<? extends BrooklynObject> targetType) throws Exception {
    +        Class<? extends AbstractBrooklynObjectSpec<?, ?>> specType = RegisteredTypeLoadingContexts.lookupSpecTypeForTarget(targetType);
    +        if (specType==null) return null;
    +        Method createMethod = specType.getMethod("create", Class.class);
    +        return (AbstractBrooklynObjectSpec<?, ?>) createMethod.invoke(null, targetType);
    +    }
     
    -        @Override
    -        public String getIconUrl() {
    -            return iconUrl;
    -        }
    -        
    -        @Override
    -        public boolean isDisabled() {
    -            return disabled;
    -        }
    -        
    -        @Override
    -        public boolean isDeprecated() {
    -            return deprecated;
    -        }
    -        
    -        @Override
    -        public Class<?> getJavaType() {
    -            return javaType;
    -        }
    -        
    -        @Override
    -        public String toString() {
    -            return JavaClassNames.simpleClassName(this)+"["+getId()+
    -                (isDisabled() ? ";DISABLED" : "")+
    -                (isDeprecated() ? ";deprecated" : "")+
    -                "]";
    -        }
    +    /** Returns a wrapped map, if the object is YAML which parses as a map; 
    +     * otherwise returns absent capable of throwing an error with more details */
    +    @SuppressWarnings("unchecked")
    +    public static Maybe<Map<Object,Object>> getAsYamlMap(Object planData) {
    +        if (!(planData instanceof String)) return Maybe.absent("not a string");
    +        Iterable<Object> result;
    +        try {
    +            result = Yamls.parseAll((String)planData);
    +        } catch (Exception e) {
    +            Exceptions.propagateIfFatal(e);
    +            return Maybe.absent(e);
    +        }
    +        Iterator<Object> ri = result.iterator();
    +        if (!ri.hasNext()) return Maybe.absent("YAML has no elements in it");
    +        Object r1 = ri.next();
    +        if (ri.hasNext()) return Maybe.absent("YAML has multiple elements in it");
    +        if (r1 instanceof Map) return Maybe.of((Map<Object,Object>)r1);
    +        return Maybe.absent("YAML does not contain a map");
         }
     
    -    // TODO
    -//    public static class RegisteredBeanType extends AbstractRegisteredType {
    -//        
    -//    }
    +    /** 
    +     * Queries recursively the supertypes of {@link RegisteredType} to see whether it 
    +     * declares a supertype compatible with the given {@link Class} */
    +    public static boolean isSubTypeOf(RegisteredType type, Class<?> superType) {
    +        return isSubTypeOf(type.getSuperTypes(), superType);
    +    }
         
    -    public static class RegisteredSpecType extends AbstractRegisteredType {
    -
    -        private TypeImplementation impl;
    -        
    -        public RegisteredSpecType(String symbolicName, String version, Class<?> javaType, TypeImplementation impl) {
    -            super(symbolicName, version, javaType);
    -            this.impl = impl;
    +    /** 
    +     * Queries recursively the given types (either {@link Class} or {@link RegisteredType}) 
    +     * to see whether any are compatible with the given {@link Class} */
    +    public static boolean isSubTypeOf(Set<Object> allKnownTypes, Class<?> superType) {
    +        for (Object st: allKnownTypes) {
    +            if (st instanceof Class) {
    +                if (superType.isAssignableFrom((Class<?>)st)) return true;
    +            }
             }
    -
    -        public TypeImplementation getImplementation() {
    -            return impl;
    +        for (Object st: allKnownTypes) {
    +            if (st instanceof RegisteredType) {
    +                if (isSubTypeOf((RegisteredType)st, superType)) return true;
    +            }
             }
    +        return false;
    --- End diff --
    
    Looks the same as `RegisteredTypePredicates.assignableFrom(Class<?>)`? Re-use implementation and unify 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] incubator-brooklyn pull request: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44651468
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/TestToSpecTransformer.java ---
    @@ -20,43 +20,36 @@
     
     import java.util.Map;
     import java.util.Set;
    -import java.util.concurrent.ConcurrentHashMap;
     
    +import org.apache.brooklyn.api.catalog.BrooklynCatalog;
     import org.apache.brooklyn.api.catalog.CatalogItem;
     import org.apache.brooklyn.api.entity.Application;
     import org.apache.brooklyn.api.entity.EntitySpec;
     import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
     import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
     import org.apache.brooklyn.core.plan.PlanNotRecognizedException;
     import org.apache.brooklyn.core.plan.PlanToSpecTransformer;
    -import org.apache.brooklyn.util.text.Identifiers;
    +import org.apache.brooklyn.core.typereg.JavaClassNameTypePlanTransformer;
    +import org.apache.brooklyn.core.typereg.RegisteredTypeLoadingContexts;
    +import org.apache.brooklyn.util.collections.MutableSet;
     import org.apache.brooklyn.util.yaml.Yamls;
     
    +import com.google.common.collect.Iterables;
    +
     /**
    - * Resolves previously registered specs by id.
    - * First create a spec and register it, keeping the returned ID:
    - * <pre> {@code
    - * String specId = TestToSpecTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    - * }</pre>
    - *
    - * Then build a plan to be resolved such as:
    - * <pre> {@code
    - *  brooklyn.catalog:
    - *    id: test.inputs
    - *    version: 0.0.1
    - *    item: <specId>
    - * } </pre>
    + * For use in conjunction with {@link StaticTypePlanTransformer};
    + * this will lookup an item by ID or in a map "type: id".
    + * <p>
    + * Should be removed when catalog is able to function using new-style lookups.
      */
     public class TestToSpecTransformer implements PlanToSpecTransformer {
    --- End diff --
    
    Still can't wrap my head around the changes in this file. `StaticTypePlanTransformer` doesn't seem to be involved at all where this is used.
    Looks like it's just a minimalistic CAMP parser, delegating to the registry to resolve the type. It's used in a single test - not worth it's weight (especially implementing deprecated functionality).


---
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: Flesh out type creation using the...

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/1017#discussion_r45091965
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/RegisteredType.java ---
    @@ -38,19 +46,24 @@
         String getDescription();
         String getIconUrl();
     
    -    /** @return the java type or a supertype thereof that this registered type represents.
    +    /** @return all declared supertypes or super-interfaces of this registered type,
    +     * consisting of a collection of {@link Class} or {@link RegisteredType}
          * <p>
    -     * For beans, this is the type that the {@link BrooklynTypeRegistry} will create. 
    -     * For specs, this is what the spec that will be created points at 
    -     * (e.g. the concrete {@link Entity}, not the {@link EntitySpec});
    +     * This should normally include at least one {@link Class} object:
    +     * For beans, this should include the java type that the {@link BrooklynTypeRegistry} will create. 
    +     * For specs, this should refer to the {@link BrooklynObject} type that the created spec will point at 
    +     * (e.g. the concrete {@link Entity}, not the {@link EntitySpec}).
          * <p>
    -     * In some cases this may return an interface or a super-type of what will actually be created, 
    +     * This may not necessarily return the most specific java class or classes;
          * such as if the concrete type is private and callers should know only about a particular public interface,
          * or if precise type details are unavailable and all that is known at creation is some higher level interface/supertype
          * (e.g. this may return {@link Entity} even though the spec points at a specific subclass,
    -     * for instance because the YAML has not yet been parsed or OSGi bundles downloaded). 
    +     * for instance because the YAML has not yet been parsed or OSGi bundles downloaded).
    +     * <p>
    +     * This may include other registered types such as marker interfaces.
          */
    -    Class<?> getJavaType();
    +    @Beta
    +    @Nullable Set<Object> getSuperTypes();
    --- End diff --
    
    shouldn't be, will remove


---
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: Flesh out type creation using the...

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/1017#discussion_r45041122
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/BrooklynTypeRegistry.java ---
    @@ -45,24 +45,25 @@
         /** @return The item matching the given given 
          * {@link RegisteredType#getSymbolicName() symbolicName} 
          * and optionally {@link RegisteredType#getVersion()},
    -     * filtered for the optionally supplied {@link RegisteredTypeConstraint}, 
    +     * filtered for the optionally supplied {@link RegisteredTypeLoadingContext}, 
          * taking the best version if the version is null or a default marker,
          * returning null if no matches are found. */
    -    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} with no constraints */
    +    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} with no constraints */
         RegisteredType get(String symbolicName, String version);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} but allows <code>"name:version"</code> 
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} but allows <code>"name:version"</code> 
          * (the {@link RegisteredType#getId()}) in addition to the unversioned name,
          * using a default marker if no version can be inferred */
    -    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, RegisteredTypeConstraint)} but with no constraints */
    +    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeLoadingContext constraint);
    --- End diff --
    
    You're right.
    
    In general it serves a few purposes:
    * it supplies class-loading context
    * it helps with recursive resolution (`alreadyEncounteredType`)
    * it prevents returning incompatible classes
    
    But none of them are needed for a lookup.  Maybe we should remove; I'll look at that.
    
    BTW correct that we don't allow different instances of the same symbolic name+version.  However i have in my mind a concept of `aliases` which types could declare, and those could conflict, so if looking up an alias the context could be useful.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44662144
  
    --- Diff: utils/common/src/main/java/org/apache/brooklyn/util/text/Identifiers.java ---
    @@ -26,6 +26,9 @@
         
         public static final String JAVA_GOOD_START_CHARS = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz_";
         public static final String JAVA_GOOD_NONSTART_CHARS = JAVA_GOOD_START_CHARS+"1234567890";
    +    public static final String JAVA_SEGMENT_REGEX = "["+JAVA_GOOD_START_CHARS+"]"+"["+JAVA_GOOD_NONSTART_CHARS+"]*";
    +    public static final String JAVA_PACKAGE_OR_CLASS_REGEX = "("+JAVA_SEGMENT_REGEX+"\\."+")*"+JAVA_SEGMENT_REGEX;
    +    public static final String JAVA_BINARY_REGEX = JAVA_PACKAGE_OR_CLASS_REGEX+"(\\$["+JAVA_GOOD_NONSTART_CHARS+"]+)*";
    --- End diff --
    
    This includes a tiny subset of valid java identifiers, see http://docs.oracle.com/javase/specs/jls/se7/html/jls-3.html#jls-3.8. Could lead to subtle bugs.


---
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: Flesh out type creation using the...

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/1017#discussion_r45050927
  
    --- Diff: usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogParametersTest.java ---
    @@ -273,7 +273,54 @@ public void testAppSpecInheritsCatalogParameters() {
         }
     
         @Test
    -    public void testParametersCoercedOnSetAndReferences() throws Exception {
    +    public void testParametersOnItemCoercedOnSetAndReferences() throws Exception {
    +        Integer testValue = Integer.valueOf(55);
    +        addCatalogItems(
    +                "brooklyn.catalog:",
    +                "  id: " + SYMBOLIC_NAME,
    +                "  version: " + TEST_VERSION,
    +                "  item:",
    +                "    type: " + BasicApplication.class.getName(),
    +                "    brooklyn.parameters:",
    +                "    - name: num",
    +                "      type: integer",
    +                "    brooklyn.children:",
    +                "    - type: " + ConfigEntityForTest.class.getName(),
    +                "      brooklyn.config:",
    +                "        refConfig: $brooklyn:scopeRoot().config(\"num\")",
    +                "    - type: " + ConfigEntityForTest.class.getName(),
    +                "      brooklyn.config:",
    +                "        refConfig: $brooklyn:config(\"num\")"); //inherited config
    +
    +        Entity app = createAndStartApplication(
    +                "services:",
    +                "- type: " + BasicApplication.class.getName(),
    +                "  brooklyn.children:",
    +                "  - type: " + ver(SYMBOLIC_NAME),
    +                "    brooklyn.config:",
    +                "      num: \"" + testValue + "\"");
    +
    +        Entity scopeRoot = Iterables.getOnlyElement(app.getChildren());
    +
    +        ConfigKey<Object> numKey = ConfigKeys.newConfigKey(Object.class, "num");
    +        assertEquals(scopeRoot.config().get(numKey), testValue);
    +
    +        ConfigKey<Object> refConfigKey = ConfigKeys.newConfigKey(Object.class, "refConfig");
    +
    +        Iterator<Entity> childIter = scopeRoot.getChildren().iterator();
    +        Entity c1 = childIter.next();
    +        assertEquals(c1.config().get(refConfigKey), testValue);
    +        Entity c2 = childIter.next();
    +        assertEquals(c2.config().get(refConfigKey), testValue);
    +        assertFalse(childIter.hasNext());
    +    }
    +    
    +    // XXX TODO parameters on the root don't work with new type registry; 
    +    // they require the CI being able to keep them,
    +    // or else modifying the plan. TODO should they be permitted as metadata in this way?
    +    // or treaded like a declaration of config keys on the entity?  i (alex) prefer the latter.
    --- End diff --
    
    i think we've agreed we'll remove this?  we can remove the test (and this comment) when we do


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44638756
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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.typereg;
    +
    +import groovy.xml.Entity;
    +
    +import java.util.Set;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.objs.BrooklynObject;
    +import org.apache.brooklyn.api.objs.BrooklynObjectType;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +public class RegisteredTypeLoadingContexts {
    +
    +    private static final Logger log = LoggerFactory.getLogger(RegisteredTypeLoadingContexts.BasicRegisteredTypeLoadingContext.class);
    +    
    +    /** Immutable (from caller's perspective) record of a constraint */
    +    public final static class BasicRegisteredTypeLoadingContext implements RegisteredTypeLoadingContext {
    +        @Nullable private RegisteredTypeKind kind;
    +        @Nullable private Class<?> expectedSuperType;
    +        @Nonnull private Set<String> encounteredTypes = ImmutableSet.of();
    +        @Nullable BrooklynClassLoadingContext loader;
    +        
    +        private BasicRegisteredTypeLoadingContext() {}
    +        
    +        public BasicRegisteredTypeLoadingContext(@Nullable RegisteredTypeLoadingContext source) {
    +            if (source==null) return;
    +            
    +            this.kind = source.getExpectedKind();
    +            this.expectedSuperType = source.getExpectedJavaSuperType();
    +            this.encounteredTypes = source.getAlreadyEncounteredTypes();
    +            this.loader = (BrooklynClassLoadingContext) source.getLoader();
    --- End diff --
    
    Cast not needed.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44640885
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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.typereg;
    +
    +import groovy.xml.Entity;
    +
    +import java.util.Set;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.objs.BrooklynObject;
    +import org.apache.brooklyn.api.objs.BrooklynObjectType;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +public class RegisteredTypeLoadingContexts {
    +
    +    private static final Logger log = LoggerFactory.getLogger(RegisteredTypeLoadingContexts.BasicRegisteredTypeLoadingContext.class);
    +    
    +    /** Immutable (from caller's perspective) record of a constraint */
    +    public final static class BasicRegisteredTypeLoadingContext implements RegisteredTypeLoadingContext {
    +        @Nullable private RegisteredTypeKind kind;
    +        @Nullable private Class<?> expectedSuperType;
    +        @Nonnull private Set<String> encounteredTypes = ImmutableSet.of();
    +        @Nullable BrooklynClassLoadingContext loader;
    +        
    +        private BasicRegisteredTypeLoadingContext() {}
    +        
    +        public BasicRegisteredTypeLoadingContext(@Nullable RegisteredTypeLoadingContext source) {
    +            if (source==null) return;
    +            
    +            this.kind = source.getExpectedKind();
    +            this.expectedSuperType = source.getExpectedJavaSuperType();
    +            this.encounteredTypes = source.getAlreadyEncounteredTypes();
    +            this.loader = (BrooklynClassLoadingContext) source.getLoader();
    +        }
    +
    +        @Override
    +        public RegisteredTypeKind getExpectedKind() {
    +            return kind;
    +        }
    +        
    +        @Override
    +        public Class<?> getExpectedJavaSuperType() {
    +            if (expectedSuperType==null) return Object.class;
    +            return expectedSuperType;
    +        }
    +
    +        @Override
    +        public Set<String> getAlreadyEncounteredTypes() {
    +            if (encounteredTypes==null) return ImmutableSet.of();
    +            return ImmutableSet.<String>copyOf(encounteredTypes);
    +        }
    +        
    +        @Override
    +        public BrooklynClassLoadingContext getLoader() {
    +            return loader;
    +        }
    +        
    +        @Override
    +        public String toString() {
    +            return JavaClassNames.cleanSimpleClassName(this)+"["+kind+","+expectedSuperType+","+encounteredTypes+"]";
    +        }
    +    }
    +
    +    /** returns a constraint which allows anything */
    +    public static RegisteredTypeLoadingContext any() {
    +        return new BasicRegisteredTypeLoadingContext();
    +    }
    +
    +    public static RegisteredTypeLoadingContext alreadyEncountered(Set<String> encounteredTypeSymbolicNames) {
    +        BasicRegisteredTypeLoadingContext result = new BasicRegisteredTypeLoadingContext();
    +        result.encounteredTypes = encounteredTypeSymbolicNames == null ? ImmutableSet.<String>of() : ImmutableSet.copyOf(encounteredTypeSymbolicNames);
    +        return result;
    +    }
    +    public static RegisteredTypeLoadingContext alreadyEncountered(Set<String> encounteredTypeSymbolicNames, String anotherEncounteredType) {
    +        BasicRegisteredTypeLoadingContext result = new BasicRegisteredTypeLoadingContext();
    +        MutableSet<String> encounteredTypes = MutableSet.copyOf(encounteredTypeSymbolicNames);
    +        encounteredTypes.addIfNotNull(anotherEncounteredType);
    +        result.encounteredTypes = encounteredTypes.asUnmodifiable();
    +        return result;
    +    }
    +
    +    public static RegisteredTypeLoadingContext loaderAlreadyEncountered(BrooklynClassLoadingContext loader, Set<String> encounteredTypeSymbolicNames) {
    +        return loaderAlreadyEncountered(loader, encounteredTypeSymbolicNames, null);
    +    }
    +    public static RegisteredTypeLoadingContext loaderAlreadyEncountered(BrooklynClassLoadingContext loader, Set<String> encounteredTypeSymbolicNames, String anotherEncounteredType) {
    +        return withLoader(alreadyEncountered(encounteredTypeSymbolicNames, anotherEncounteredType), loader);
    +    }
    +
    +    private static RegisteredTypeLoadingContext of(RegisteredTypeKind kind, Class<?> javaSuperType) {
    +        BasicRegisteredTypeLoadingContext result = new BasicRegisteredTypeLoadingContext();
    +        result.kind = kind;
    +        result.expectedSuperType = javaSuperType;
    +        return result;
    +    }
    +
    +    public static RegisteredTypeLoadingContext bean(Class<?> javaSuperType) {
    +        return of(RegisteredTypeKind.BEAN, javaSuperType);
    +    }
    +
    +    public static RegisteredTypeLoadingContext spec(Class<? extends BrooklynObject> javaSuperType) {
    +        return of(RegisteredTypeKind.SPEC, javaSuperType);
    +    }
    +    
    +    public static <T> RegisteredTypeLoadingContext withBeanSuperType(@Nullable RegisteredTypeLoadingContext source, @Nullable Class<T> beanSuperType) {
    +        Class<T> superType = beanSuperType;
    +        BasicRegisteredTypeLoadingContext constraint = new BasicRegisteredTypeLoadingContext(source);
    +        if (source==null) source = constraint;
    +        if (superType==null) return source;
    +        constraint.expectedSuperType = superType;
    +        if (source.getExpectedJavaSuperType()==null || source.getExpectedJavaSuperType().isAssignableFrom( superType )) {
    +            // the old constraint was weaker than present; return the new constraint
    +            return constraint;
    +        }
    +        if (superType.isAssignableFrom( source.getExpectedJavaSuperType() )) {
    +            // the constraint was already for something more specific; ignore what we've inferred here
    +            return source;
    +        }
    +        log.warn("Ambiguous bean supertypes ("+beanSuperType+" for target "+source.getExpectedJavaSuperType()+"); "
    +            + "it is recommended that any registered type constraint for a spec be compatible with the spec type");
    +        return source;
    +    }
    +
    +    /** Takes a Spec java type and adds an expected java type to the {@link RegisteredTypeLoadingContext} */
    +    public static <T extends AbstractBrooklynObjectSpec<?,?>> RegisteredTypeLoadingContext withSpecSuperType(@Nullable RegisteredTypeLoadingContext source, @Nullable Class<T> specSuperType) {
    +        Class<?> superType = lookupTargetTypeForSpec(specSuperType);
    +        BasicRegisteredTypeLoadingContext constraint = new BasicRegisteredTypeLoadingContext(source);
    +        if (source==null) source = constraint;
    +        if (superType==null) return source;
    +        constraint.expectedSuperType = superType;
    +        if (source.getExpectedJavaSuperType()==null || source.getExpectedJavaSuperType().isAssignableFrom( superType )) {
    +            // the old constraint was weaker than present; return the new constraint
    +            return constraint;
    +        }
    +        if (superType.isAssignableFrom( source.getExpectedJavaSuperType() )) {
    +            // the constraint was already for something more specific; ignore what we've inferred here
    +            return source;
    +        }
    +        // trickier situation; the constraint had a type not compatible with the spec type; log a warning and leave alone
    +        // (e.g. caller specified some java super type which is not a super or sub of the spec target type;
    +        // this may be because the caller specified a Spec as the type supertype, which is wrong;
    +        // or they may have specified an interface along a different hierarchy, which we discouraged
    +        // as it will make filtering/indexing more complex)
    +        log.warn("Ambiguous spec supertypes ("+specSuperType+" for target "+source.getExpectedJavaSuperType()+"); "
    +            + "it is recommended that any registered type constraint for a spec be compatible with the spec type");
    +        return source;
    +    }
    +        
    +    /** given a spec, returns the class of the item it targets, for instance returns {@link Entity} given {@link EntitySpec};
    +     * see also {@link #lookupSpecTypeForTarget(Class)} */
    +    static <T extends AbstractBrooklynObjectSpec<?,?>> Class<? extends BrooklynObject> lookupTargetTypeForSpec(Class<T> specSuperType) {
    +        if (specSuperType==null) return BrooklynObject.class;
    +        BrooklynObjectType best = null;
    +
    +        for (BrooklynObjectType t: BrooklynObjectType.values()) {
    +            if (t.getSpecType()==null) continue;
    +            if (!t.getSpecType().isAssignableFrom(specSuperType)) continue;
    +            // on equality, exit immediately
    +            if (t.getSpecType().equals(specSuperType)) return t.getInterfaceType();
    +            // else pick which is best
    +            if (best==null) { best = t; continue; }
    +            // if t is more specific, it is better (handles case when e.g. a Policy is a subclass of Entity)
    +            if (best.getSpecType().isAssignableFrom(t.getSpecType())) { best = t; continue; }
    +        }
    +        if (best==null) {
    +            log.warn("Unexpected spec supertype ("+specSuperType+"); treating as any "+BrooklynObject.class, new Throwable("Trace for unexpected spec supertype"));
    +            return BrooklynObject.class;
    +        }
    +        // the spec is more specific, but we're not familiar with it here; return the best
    +        return best.getInterfaceType();
    +    }
    +
    +    /** given a {@link BrooklynObject}, returns the spec class which would generate it, for instance returns {@link EntitySpec} given {@link Entity},
    +     * or null if not known */
    +    static <BO extends BrooklynObject> Class<? extends AbstractBrooklynObjectSpec<?,?>> lookupSpecTypeForTarget(Class<BO> targetSuperType) {
    +        if (targetSuperType==null) return null;
    +        BrooklynObjectType best = null;
    +
    +        for (BrooklynObjectType t: BrooklynObjectType.values()) {
    +            if (t.getSpecType()==null) continue;
    +            if (!t.getInterfaceType().isAssignableFrom(targetSuperType)) continue;
    +            // on equality, exit immediately
    +            if (t.getInterfaceType().equals(targetSuperType)) return t.getSpecType();
    +            // else pick which is best
    +            if (best==null) { best = t; continue; }
    +            // if t is more specific, it is better (handles case when e.g. a Policy is a subclass of Entity)
    +            if (best.getSpecType().isAssignableFrom(t.getSpecType())) { best = t; continue; }
    --- End diff --
    
    Again should be `getInterfaceType()`


---
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: Flesh out type creation using the...

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/1017#discussion_r45043213
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BasicBrooklynTypeRegistry.java ---
    @@ -90,31 +96,95 @@ public RegisteredType get(String symbolicNameWithOptionalVersion, RegisteredType
     
         @Override
         public RegisteredType get(String symbolicNameWithOptionalVersion) {
    -        return get(symbolicNameWithOptionalVersion, (RegisteredTypeConstraint)null);
    +        return get(symbolicNameWithOptionalVersion, (RegisteredTypeLoadingContext)null);
         }
     
         @SuppressWarnings({ "deprecation", "unchecked", "rawtypes" })
         @Override
    -    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint constraint, Class<SpecT> specSuperType) {
    -        if (!(type instanceof RegisteredSpecType)) { 
    -            throw new IllegalStateException("Cannot create spec from type "+type);
    +    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext constraint, Class<SpecT> specSuperType) {
    +        Preconditions.checkNotNull(type, "type");
    +        if (type.getKind()!=RegisteredTypeKind.SPEC) { 
    +            throw new IllegalStateException("Cannot create spec from type "+type+" (kind "+type.getKind()+")");
             }
             if (constraint!=null) {
    -            if (constraint.getKind()!=null && constraint.getKind()!=RegisteredTypeKind.SPEC) {
    +            if (constraint.getExpectedKind()!=null && constraint.getExpectedKind()!=RegisteredTypeKind.SPEC) {
                     throw new IllegalStateException("Cannot create spec with constraint "+constraint);
                 }
    -            if (constraint.getEncounteredTypes().contains(type.getSymbolicName())) {
    +            if (constraint.getAlreadyEncounteredTypes().contains(type.getSymbolicName())) {
                     // avoid recursive cycle
                     // TODO implement using java if permitted
                 }
             }
    -        constraint = RegisteredTypeConstraints.extendedWithSpecSuperType(constraint, specSuperType);
    +        constraint = RegisteredTypeLoadingContexts.withSpecSuperType(constraint, specSuperType);
     
    -        // TODO look up in the actual registry
    +        Maybe<Object> result = TypePlanTransformers.transform(mgmt, type, constraint);
    +        if (result.isPresent()) return (SpecT) result.get();
             
             // fallback: look up in (legacy) catalog
    +        // TODO remove once all transformers are available in the new style
             CatalogItem item = (CatalogItem) mgmt.getCatalog().getCatalogItem(type.getSymbolicName(), type.getVersion());
    -        return (SpecT) BasicBrooklynCatalog.internalCreateSpecWithTransformers(mgmt, item, constraint.getEncounteredTypes());
    +        if (item==null) {
    +            // if not in catalog (because loading a new item?) then look up item based on type
    +            // (only really used in tests; possibly also for any recursive legacy transformers we might have to create a CI; cross that bridge when we come to it)
    +            CatalogItemType ciType = CatalogItemType.ofTargetClass( (Class)constraint.getExpectedJavaSuperType() );
    --- End diff --
    
    this is a port of old logic; the TODO above notes we want to remove the block


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44652999
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/test/BrooklynMgmtUnitTestSupport.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.test;
    +
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
    +import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.AfterMethod;
    +import org.testng.annotations.BeforeMethod;
    +
    +/**
    + * To be extended by unit/integration tests.
    + * <p>
    + * Uses a light-weight management context that will not read {@code ~/.brooklyn/brooklyn.properties}.
    + */
    +public class BrooklynMgmtUnitTestSupport {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(BrooklynMgmtUnitTestSupport.class);
    +
    +    protected ManagementContextInternal mgmt;
    +
    +    @BeforeMethod(alwaysRun=true)
    +    public void setUp() throws Exception {
    +        if (mgmt == null) {
    +            mgmt = LocalManagementContextForTests.newInstance();
    +        }
    +    }
    +
    +    @AfterMethod(alwaysRun=true)
    +    public void tearDown() throws Exception {
    +        try {
    +            if (mgmt != null) Entities.destroyAll(mgmt);
    +        } catch (Throwable t) {
    +            LOG.error("Caught exception in tearDown method", t);
    --- End diff --
    
    I see you just copied this, but still rings an alarm in my head - aren't exceptions in tearDown part of the test failing as well? No strong opinion though.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44658611
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampResolver.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.camp.brooklyn.spi.creation;
    +
    +import java.util.Set;
    +
    +import org.apache.brooklyn.api.entity.Application;
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.policy.Policy;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.camp.CampPlatform;
    +import org.apache.brooklyn.camp.brooklyn.api.AssemblyTemplateSpecInstantiator;
    +import org.apache.brooklyn.camp.spi.AssemblyTemplate;
    +import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
    +import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
    +import org.apache.brooklyn.core.mgmt.EntityManagementUtils;
    +import org.apache.brooklyn.core.typereg.RegisteredTypes;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.text.Strings;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +class CampResolver {
    +
    +    private ManagementContext mgmt;
    +    private RegisteredType type;
    +    private RegisteredTypeLoadingContext context;
    +
    +    /** whether to allow parsing of the 'full' syntax for applications,
    +     * where items are wrapped in a "services:" block, and if the wrapper is an application,
    +     * to promote it */
    +    boolean allowApplicationFullSyntax = true;
    +
    +    /** whether to allow parsing of the legacy 'full' syntax, 
    +     * where a non-application items are wrapped:
    +     * <li> in a "services:" block for entities,
    +     * <li> in a "brooklyn.locations" or "brooklyn.policies" block for locations and policies */
    +    boolean allowLegacyFullSyntax = true;
    +
    +    /** whether to allow parsing of the type syntax, where an item is a map with a "type:" field,
    +     * i.e. not wrapped in any "services:" or "brooklyn.{locations,policies}" block */
    +    boolean allowTypeSyntax = true;
    +
    +    public CampResolver(ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) {
    +        this.mgmt = mgmt;
    +        this.type = type;
    +        this.context = context;
    +    }
    +
    +    public AbstractBrooklynObjectSpec<?, ?> createSpec() {
    +        // TODO new-style approach:
    +        //            AbstractBrooklynObjectSpec<?, ?> spec = RegisteredTypes.newSpecInstance(mgmt, /* 'type' key */);
    +        //            spec.configure(keysAndValues);
    +        return createSpecFromFull(mgmt, type, context.getExpectedJavaSuperType(), context.getAlreadyEncounteredTypes(), context.getLoader());
    +    }
    +
    +    static AbstractBrooklynObjectSpec<?, ?> createSpecFromFull(ManagementContext mgmt, RegisteredType item, Class<?> expectedType, Set<String> parentEncounteredTypes, BrooklynClassLoadingContext loaderO) {
    +        // for this method, a prefix "services" or "brooklyn.{location,policies}" is required at the root;
    +        // we now prefer items to come in "{ type: .. }" format, except for application roots which
    +        // should have a "services: [ ... ]" block (and which may subsequently be unwrapped)
    +        BrooklynClassLoadingContext loader = CatalogUtils.newClassLoadingContext(mgmt, item, loaderO);
    +
    +        Set<String> encounteredTypes;
    +        // symbolicName could be null if coming from the catalog parser where it tries to load before knowing the id
    +        if (item.getSymbolicName() != null) {
    +            encounteredTypes = ImmutableSet.<String>builder()
    +                .addAll(parentEncounteredTypes)
    +                .add(item.getSymbolicName())
    +                .build();
    +        } else {
    +            encounteredTypes = parentEncounteredTypes;
    +        }
    +
    +        AbstractBrooklynObjectSpec<?, ?> spec;
    +        String planYaml = RegisteredTypes.getImplementationDataStringForSpec(item);
    +        MutableSet<Object> supers = MutableSet.copyOf(item.getSuperTypes());
    +        supers.addIfNotNull(expectedType);
    +        if (RegisteredTypes.isSubTypeOf(supers, Policy.class)) {
    +            spec = CampInternalUtils.createPolicySpec(planYaml, loader, encounteredTypes);
    +        } else if (RegisteredTypes.isSubTypeOf(supers, Location.class)) {
    +            spec = CampInternalUtils.createLocationSpec(planYaml, loader, encounteredTypes);
    +        } else if (RegisteredTypes.isSubTypeOf(supers, Application.class)) {
    +            spec = createEntitySpecFromServicesBlock(planYaml, loader, encounteredTypes, true);
    +        } else if (RegisteredTypes.isSubTypeOf(supers, Entity.class)) {
    +            spec = createEntitySpecFromServicesBlock(planYaml, loader, encounteredTypes, false);
    +        } else {
    +            // try any of them???
    +            
    +            throw new IllegalStateException("Cannot detect spec type from "+item.getSuperTypes()+" for "+item+"\n"+planYaml);
    +        }
    +
    +        ((AbstractBrooklynObjectSpec<?, ?>)spec).catalogItemId(item.getId());
    +
    +        if (Strings.isBlank( ((AbstractBrooklynObjectSpec<?, ?>)spec).getDisplayName() ))
    +            ((AbstractBrooklynObjectSpec<?, ?>)spec).displayName(item.getDisplayName());
    +
    +        return spec;
    +    }
    + 
    +    private static EntitySpec<?> createEntitySpecFromServicesBlock(String plan, BrooklynClassLoadingContext loader, Set<String> encounteredTypes, boolean isApplication) {
    +        CampPlatform camp = CampInternalUtils.getCampPlatform(loader.getManagementContext());
    +
    +        AssemblyTemplate at = CampInternalUtils.registerDeploymentPlan(plan, loader, camp);
    +        AssemblyTemplateInstantiator instantiator = CampInternalUtils.getInstantiator(at);
    +        if (instantiator instanceof AssemblyTemplateSpecInstantiator) {
    +            EntitySpec<? extends Application> appSpec = ((AssemblyTemplateSpecInstantiator)instantiator).createApplicationSpec(at, camp, loader, encounteredTypes);
    +
    +            if (!isApplication && EntityManagementUtils.canPromoteChildrenInWrappedApplication(appSpec) && appSpec.getChildren().size()==1) {
    +                CampInternalUtils.resetSpecIfTemplateHasNoExplicitParameters(at, appSpec);
    --- End diff --
    
    Already happened in `createWrapperApp`


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44645118
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/TypePlanTransformers.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.TreeMap;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.PropagatedRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.Beta;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.collect.ArrayListMultimap;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Iterables;
    +import com.google.common.collect.Multimap;
    +
    +public class TypePlanTransformers {
    +
    +    private static final Logger log = LoggerFactory.getLogger(TypePlanTransformers.class);
    +
    +    private static Collection<BrooklynTypePlanTransformer> getAll() {
    +        return ImmutableList.copyOf(ServiceLoader.load(BrooklynTypePlanTransformer.class));
    +    }
    +
    +    private static Collection<Class<? extends BrooklynTypePlanTransformer>> OVERRIDE;
    +    @SafeVarargs
    +    @VisibleForTesting
    +    public synchronized static void forceAvailable(Class<? extends BrooklynTypePlanTransformer> ...classes) {
    +        OVERRIDE = Arrays.asList(classes);
    +    }
    +    public synchronized static void clearForced() {
    +        OVERRIDE = null;
    +    }
    +
    +    public static Collection<BrooklynTypePlanTransformer> all(ManagementContext mgmt) {
    +        // TODO cache these in the TypeRegistry, looking for new ones periodically or supplying a way to register them
    --- End diff --
    
    Should switch to an OSGi solution instead of trying to improve this longer term.


---
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: Flesh out type creation using the...

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/1017#discussion_r45067501
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/BrooklynTypeRegistry.java ---
    @@ -45,24 +45,25 @@
         /** @return The item matching the given given 
          * {@link RegisteredType#getSymbolicName() symbolicName} 
          * and optionally {@link RegisteredType#getVersion()},
    -     * filtered for the optionally supplied {@link RegisteredTypeConstraint}, 
    +     * filtered for the optionally supplied {@link RegisteredTypeLoadingContext}, 
          * taking the best version if the version is null or a default marker,
          * returning null if no matches are found. */
    -    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} with no constraints */
    +    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} with no constraints */
         RegisteredType get(String symbolicName, String version);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} but allows <code>"name:version"</code> 
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} but allows <code>"name:version"</code> 
          * (the {@link RegisteredType#getId()}) in addition to the unversioned name,
          * using a default marker if no version can be inferred */
    -    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, RegisteredTypeConstraint)} but with no constraints */
    +    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeLoadingContext constraint);
    --- End diff --
    
    Yup I reached the same conclusion.  Testing your change 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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44655440
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampResolver.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.camp.brooklyn.spi.creation;
    +
    +import java.util.Set;
    +
    +import org.apache.brooklyn.api.entity.Application;
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.policy.Policy;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.camp.CampPlatform;
    +import org.apache.brooklyn.camp.brooklyn.api.AssemblyTemplateSpecInstantiator;
    +import org.apache.brooklyn.camp.spi.AssemblyTemplate;
    +import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
    +import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
    +import org.apache.brooklyn.core.mgmt.EntityManagementUtils;
    +import org.apache.brooklyn.core.typereg.RegisteredTypes;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.text.Strings;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +class CampResolver {
    +
    +    private ManagementContext mgmt;
    +    private RegisteredType type;
    +    private RegisteredTypeLoadingContext context;
    +
    +    /** whether to allow parsing of the 'full' syntax for applications,
    +     * where items are wrapped in a "services:" block, and if the wrapper is an application,
    +     * to promote it */
    +    boolean allowApplicationFullSyntax = true;
    +
    +    /** whether to allow parsing of the legacy 'full' syntax, 
    +     * where a non-application items are wrapped:
    +     * <li> in a "services:" block for entities,
    +     * <li> in a "brooklyn.locations" or "brooklyn.policies" block for locations and policies */
    +    boolean allowLegacyFullSyntax = true;
    +
    +    /** whether to allow parsing of the type syntax, where an item is a map with a "type:" field,
    +     * i.e. not wrapped in any "services:" or "brooklyn.{locations,policies}" block */
    +    boolean allowTypeSyntax = true;
    +
    +    public CampResolver(ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) {
    +        this.mgmt = mgmt;
    +        this.type = type;
    +        this.context = context;
    +    }
    +
    +    public AbstractBrooklynObjectSpec<?, ?> createSpec() {
    +        // TODO new-style approach:
    +        //            AbstractBrooklynObjectSpec<?, ?> spec = RegisteredTypes.newSpecInstance(mgmt, /* 'type' key */);
    +        //            spec.configure(keysAndValues);
    +        return createSpecFromFull(mgmt, type, context.getExpectedJavaSuperType(), context.getAlreadyEncounteredTypes(), context.getLoader());
    +    }
    +
    +    static AbstractBrooklynObjectSpec<?, ?> createSpecFromFull(ManagementContext mgmt, RegisteredType item, Class<?> expectedType, Set<String> parentEncounteredTypes, BrooklynClassLoadingContext loaderO) {
    +        // for this method, a prefix "services" or "brooklyn.{location,policies}" is required at the root;
    +        // we now prefer items to come in "{ type: .. }" format, except for application roots which
    +        // should have a "services: [ ... ]" block (and which may subsequently be unwrapped)
    +        BrooklynClassLoadingContext loader = CatalogUtils.newClassLoadingContext(mgmt, item, loaderO);
    +
    +        Set<String> encounteredTypes;
    +        // symbolicName could be null if coming from the catalog parser where it tries to load before knowing the id
    +        if (item.getSymbolicName() != null) {
    +            encounteredTypes = ImmutableSet.<String>builder()
    +                .addAll(parentEncounteredTypes)
    +                .add(item.getSymbolicName())
    +                .build();
    +        } else {
    +            encounteredTypes = parentEncounteredTypes;
    +        }
    +
    +        AbstractBrooklynObjectSpec<?, ?> spec;
    +        String planYaml = RegisteredTypes.getImplementationDataStringForSpec(item);
    +        MutableSet<Object> supers = MutableSet.copyOf(item.getSuperTypes());
    +        supers.addIfNotNull(expectedType);
    --- End diff --
    
    Check `item` and `expectedType` compatibility?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44633396
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BasicBrooklynTypeRegistry.java ---
    @@ -90,31 +96,95 @@ public RegisteredType get(String symbolicNameWithOptionalVersion, RegisteredType
     
         @Override
         public RegisteredType get(String symbolicNameWithOptionalVersion) {
    -        return get(symbolicNameWithOptionalVersion, (RegisteredTypeConstraint)null);
    +        return get(symbolicNameWithOptionalVersion, (RegisteredTypeLoadingContext)null);
         }
     
         @SuppressWarnings({ "deprecation", "unchecked", "rawtypes" })
         @Override
    -    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint constraint, Class<SpecT> specSuperType) {
    -        if (!(type instanceof RegisteredSpecType)) { 
    -            throw new IllegalStateException("Cannot create spec from type "+type);
    +    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext constraint, Class<SpecT> specSuperType) {
    +        Preconditions.checkNotNull(type, "type");
    +        if (type.getKind()!=RegisteredTypeKind.SPEC) { 
    +            throw new IllegalStateException("Cannot create spec from type "+type+" (kind "+type.getKind()+")");
             }
             if (constraint!=null) {
    -            if (constraint.getKind()!=null && constraint.getKind()!=RegisteredTypeKind.SPEC) {
    +            if (constraint.getExpectedKind()!=null && constraint.getExpectedKind()!=RegisteredTypeKind.SPEC) {
                     throw new IllegalStateException("Cannot create spec with constraint "+constraint);
                 }
    -            if (constraint.getEncounteredTypes().contains(type.getSymbolicName())) {
    +            if (constraint.getAlreadyEncounteredTypes().contains(type.getSymbolicName())) {
                     // avoid recursive cycle
                     // TODO implement using java if permitted
                 }
             }
    -        constraint = RegisteredTypeConstraints.extendedWithSpecSuperType(constraint, specSuperType);
    +        constraint = RegisteredTypeLoadingContexts.withSpecSuperType(constraint, specSuperType);
     
    -        // TODO look up in the actual registry
    +        Maybe<Object> result = TypePlanTransformers.transform(mgmt, type, constraint);
    +        if (result.isPresent()) return (SpecT) result.get();
             
             // fallback: look up in (legacy) catalog
    +        // TODO remove once all transformers are available in the new style
    --- End diff --
    
    Won't relying on the deprecated transformers make them stick around forever. Why not implement them the recommended way and be done with 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: Flesh out type creation using the...

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/1017#discussion_r45049102
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/plan/XmlPlanToSpecTransformer.java ---
    @@ -41,6 +41,8 @@
     /** Example implementation of {@link PlanToSpecTransformer} showing 
      * how implementations are meant to be written. */
     public class XmlPlanToSpecTransformer implements PlanToSpecTransformer {
    +    
    +    // this is REPLACED by ExampleXmlTypePlanTransformer
    --- End diff --
    
    when we remove the PlanToSpecTransformer, yes


---
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: Flesh out type creation using the...

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/1017#discussion_r45043399
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BrooklynTypePlanTransformer.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.List;
    +import java.util.ServiceLoader;
    +
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
    +
    +/**
    + * Interface for use by schemes which with to be able to transform plans.
    + * <p>
    + * To add a new plan transformation scheme, simply create an implementation and declare it
    + * as a java service (cf {@link ServiceLoader}).
    + * <p>
    + * Implementations may wish to extend {@link AbstractTypePlanTransformer} which simplifies the process.
    + */
    +public interface BrooklynTypePlanTransformer extends ManagementContextInjectable {
    +
    +    /** @return a code to identify type implementations created specifying the use of this plan transformer. */
    +    String getFormatCode();
    +    /** @return a display name for this transformer. */
    +    String getFormatName();
    +    /** @return a description for this transformer */
    +    String getFormatDescription();
    +
    +    /** @return how appropriate is this transformer for the {@link RegisteredType#getPlan()} of the type;
    +     * 0 (or less) if not, 1 for absolutely, and in some autodetect cases a value between 0 and 1 indicate a ranking.
    +     * <p>
    +     * The framework guarantees arguments are nonnull, and that the {@link RegisteredType#getPlan()} is also not-null.
    +     * However the format in that plan may be null. */
    +    double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context);
    +    /** Creates a new instance of the indicated type, or throws if not supported;
    +     * this method is used by the {@link BrooklynTypeRegistry} when it creates instances,
    +     * so implementations must respect the {@link RegisteredTypeKind} semantics and the {@link RegisteredTypeLoadingContext}
    +     * if they return an instance.
    +     * <p>
    +     * The framework guarantees this will only be invoked when {@link #scoreForType(RegisteredType, RegisteredTypeLoadingContext)} 
    +     * has returned a positive value.
    +     * <p>
    +     * Implementations should either return null or throw {@link UnsupportedTypePlanException} 
    +     * if the {@link RegisteredType#getPlan()} is not supported. */
    +    Object create(RegisteredType type, RegisteredTypeLoadingContext context);
    +    
    +    double scoreForTypeDefinition(String formatCode, Object catalogData);
    +    List<RegisteredType> createFromTypeDefinition(String formatCode, Object catalogData);
    --- End diff --
    
    TBD.  My thinking is that this is where `brooklyn.catalog` syntax could be handled.  But it is very much a beta placeholder until we have implementations.
    
    There is duplication with `BrooklynTypeRegistery.createXxxFromPlan` which will naturally be resolved as part of that work.


---
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: Flesh out type creation using the...

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/1017#discussion_r45040279
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/mgmt/classloading/BrooklynClassLoadingContext.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.api.mgmt.classloading;
    +
    +import java.net.URL;
    +
    +import javax.annotation.Nullable;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.util.guava.Maybe;
    +
    +/** 
    + * Provides functionality for loading classes based on the current context
    + * (e.g. the bundles of a registered type from which an entity is created)
    + */
    +public interface BrooklynClassLoadingContext {
    +
    +    public ManagementContext getManagementContext();
    +    public Class<?> loadClass(String className);
    +    public <T> Class<? extends T> loadClass(String className, @Nullable Class<T> supertype);
    +
    +    public Maybe<Class<?>> tryLoadClass(String className);
    +    public <T> Maybe<Class<? extends T>> tryLoadClass(String className, @Nullable Class<T> supertype);
    +
    +    /** As {@link ClassLoader#getResource(String)} */
    +    public URL getResource(String name);
    +
    +    /**
    +     * As {@link ClassLoader#getResources(String)} but returning an {@link Iterable} rather than
    +     * an {@link java.util.Enumeration}.
    +     */
    +    public Iterable<URL> getResources(String name);
    --- End diff --
    
    what's unfriendly about `Iterable`?  personally i ilke it, and it means implementations are more efficient.  (it's better than `Enumeration` which java's `ClassLoader` returns, and presumably implementations there support lazy eval.)



---
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: Flesh out type creation using the...

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/1017#discussion_r45047390
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/TypePlanTransformers.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.TreeMap;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.PropagatedRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.Beta;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.collect.ArrayListMultimap;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Iterables;
    +import com.google.common.collect.Multimap;
    +
    +public class TypePlanTransformers {
    +
    +    private static final Logger log = LoggerFactory.getLogger(TypePlanTransformers.class);
    +
    +    private static Collection<BrooklynTypePlanTransformer> getAll() {
    +        return ImmutableList.copyOf(ServiceLoader.load(BrooklynTypePlanTransformer.class));
    +    }
    +
    +    private static Collection<Class<? extends BrooklynTypePlanTransformer>> OVERRIDE;
    +    @SafeVarargs
    +    @VisibleForTesting
    +    public synchronized static void forceAvailable(Class<? extends BrooklynTypePlanTransformer> ...classes) {
    +        OVERRIDE = Arrays.asList(classes);
    +    }
    +    public synchronized static void clearForced() {
    +        OVERRIDE = null;
    +    }
    +
    +    public static Collection<BrooklynTypePlanTransformer> all(ManagementContext mgmt) {
    +        // TODO cache these in the TypeRegistry, looking for new ones periodically or supplying a way to register them
    +        Collection<Class<? extends BrooklynTypePlanTransformer>> override = OVERRIDE;
    +        Collection<BrooklynTypePlanTransformer> result = new ArrayList<BrooklynTypePlanTransformer>();
    +        if (override!=null) {
    +            for (Class<? extends BrooklynTypePlanTransformer> o1: override) {
    +                try {
    +                    result.add(o1.newInstance());
    +                } catch (Exception e) {
    +                    Exceptions.propagate(e);
    +                }
    +            }
    +        } else {
    +            result.addAll(getAll());
    +        }
    +        for(BrooklynTypePlanTransformer t : result) {
    +            t.injectManagementContext(mgmt);
    +        }
    +        return result;
    +    }
    +
    +    /** returns a list of {@link BrooklynTypePlanTransformer} instances for this {@link ManagementContext}
    +     * which may be able to handle the given plan; the list is sorted with highest-score transformer first */
    +    @Beta
    +    public static List<BrooklynTypePlanTransformer> forType(ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext constraint) {
    +        Multimap<Double,BrooklynTypePlanTransformer> byScoreMulti = ArrayListMultimap.create(); 
    +        Collection<BrooklynTypePlanTransformer> transformers = all(mgmt);
    +        for (BrooklynTypePlanTransformer transformer : transformers) {
    +            double score = transformer.scoreForType(type, constraint);
    +            if (score>0) byScoreMulti.put(score, transformer);
    +        }
    +        Map<Double, Collection<BrooklynTypePlanTransformer>> tree = new TreeMap<Double, Collection<BrooklynTypePlanTransformer>>(byScoreMulti.asMap());
    +        List<Collection<BrooklynTypePlanTransformer>> highestFirst = new ArrayList<Collection<BrooklynTypePlanTransformer>>(tree.values());
    +        Collections.reverse(highestFirst);
    +        return MutableList.copyOf(Iterables.concat(highestFirst)).asUnmodifiable();
    --- End diff --
    
    agree immutable.
    concat is to flatten the (collection) entries in the treemap.
    reverse as an explicit line makes the logic clearer imho.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44654463
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/api/AssemblyTemplateSpecInstantiator.java ---
    @@ -23,19 +23,23 @@
     
     import org.apache.brooklyn.api.entity.Application;
     import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
     import org.apache.brooklyn.camp.CampPlatform;
     import org.apache.brooklyn.camp.spi.AssemblyTemplate;
     import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
    -import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
     
     public interface AssemblyTemplateSpecInstantiator extends AssemblyTemplateInstantiator {
     
    +    @Deprecated /** @deprecaed since 0.9.0 include encountered types */
    +    EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader);
    --- End diff --
    
    No point in keeping this method. There will be compile errors anyway because of the new one and it doesn't even get called (false sense of security).


---
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: Flesh out type creation using the...

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/1017#discussion_r45047135
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/TypePlanTransformers.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.TreeMap;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.PropagatedRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.Beta;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.collect.ArrayListMultimap;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Iterables;
    +import com.google.common.collect.Multimap;
    +
    +public class TypePlanTransformers {
    +
    +    private static final Logger log = LoggerFactory.getLogger(TypePlanTransformers.class);
    +
    +    private static Collection<BrooklynTypePlanTransformer> getAll() {
    +        return ImmutableList.copyOf(ServiceLoader.load(BrooklynTypePlanTransformer.class));
    +    }
    +
    +    private static Collection<Class<? extends BrooklynTypePlanTransformer>> OVERRIDE;
    +    @SafeVarargs
    +    @VisibleForTesting
    +    public synchronized static void forceAvailable(Class<? extends BrooklynTypePlanTransformer> ...classes) {
    --- End diff --
    
    I liked the idea of instances but because instances need the management context injected that would prevent their use for rebind tests.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44756034
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogUtils.java ---
    @@ -85,16 +89,23 @@ public static BrooklynClassLoadingContext getClassLoadingContext(Entity entity)
         }
     
         public static BrooklynClassLoadingContext newClassLoadingContext(@Nullable ManagementContext mgmt, String catalogItemId, Collection<? extends OsgiBundleWithUrl> libraries) {
    +        return newClassLoadingContext(mgmt, catalogItemId, libraries, null);
    +    }
    +    
    +    public static BrooklynClassLoadingContext newClassLoadingContext(@Nullable ManagementContext mgmt, String catalogItemId, Collection<? extends OsgiBundleWithUrl> libraries, BrooklynClassLoadingContext loader) {
             BrooklynClassLoadingContextSequential result = new BrooklynClassLoadingContextSequential(mgmt);
     
             if (libraries!=null && !libraries.isEmpty()) {
                 result.add(new OsgiBrooklynClassLoadingContext(mgmt, catalogItemId, libraries));
             }
     
    -        BrooklynClassLoadingContext loader = BrooklynLoaderTracker.getLoader();
    -        if (loader != null) {
    +        if (loader !=null) {
    --- End diff --
    
    Shouldn't be allowed to stack loaders. A new loader barrier starts at a catalog item and the same loader is used for all types in the item, including specs referenced in URLs.
    A loader delegates to a single item's osgi bundles, falling back to the catalog class loader. The root application loader uses only the catalog class loader.
    I believe that even adding the thread local loader is wrong (in the sense that shouldn't be needed, but added just in case).



---
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: Flesh out type creation using the...

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/1017#discussion_r45047168
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/TypePlanTransformers.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.TreeMap;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.PropagatedRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.Beta;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.collect.ArrayListMultimap;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Iterables;
    +import com.google.common.collect.Multimap;
    +
    +public class TypePlanTransformers {
    +
    +    private static final Logger log = LoggerFactory.getLogger(TypePlanTransformers.class);
    +
    +    private static Collection<BrooklynTypePlanTransformer> getAll() {
    +        return ImmutableList.copyOf(ServiceLoader.load(BrooklynTypePlanTransformer.class));
    +    }
    +
    +    private static Collection<Class<? extends BrooklynTypePlanTransformer>> OVERRIDE;
    +    @SafeVarargs
    +    @VisibleForTesting
    +    public synchronized static void forceAvailable(Class<? extends BrooklynTypePlanTransformer> ...classes) {
    +        OVERRIDE = Arrays.asList(classes);
    +    }
    +    public synchronized static void clearForced() {
    +        OVERRIDE = null;
    +    }
    +
    +    public static Collection<BrooklynTypePlanTransformer> all(ManagementContext mgmt) {
    +        // TODO cache these in the TypeRegistry, looking for new ones periodically or supplying a way to register them
    --- End diff --
    
    +1


---
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: Flesh out type creation using the...

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/1017#discussion_r45044228
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.SPEC, symbolicName, version, plan), superType);
         }
    -    
    -    public abstract static class AbstractRegisteredType implements RegisteredType {
     
    -        final String symbolicName;
    -        final String version;
    +    /** returns the {@link Class} object corresponding to the given java type name,
    +     * using the cache on the type and the loader defined on the type
    +     * @param mgmt */
    +    @Beta
    +    // TODO should this be on the AbstractTypePlanTransformer ?
    +    public static Class<?> loadActualJavaType(String javaTypeName, ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) throws Exception {
    +        Class<?> result = ((BasicRegisteredType)type).getCache().get(ACTUAL_JAVA_TYPE);
    --- End diff --
    
    interesting idea.  have added `@Beta // TODO` on `getCache()` suggesting that, if it ends up that the cache is not wanted elsewhere


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44556979
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/AbstractTypePlanTransformer.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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.typereg;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Convenience supertype for {@link BrooklynTypePlanTransformer} instances.
    + */
    +public abstract class AbstractTypePlanTransformer implements BrooklynTypePlanTransformer {
    +
    +    private static final Logger log = LoggerFactory.getLogger(AbstractTypePlanTransformer.class);
    +    
    +    protected ManagementContext mgmt;
    +
    +    @Override
    +    public void injectManagementContext(ManagementContext mgmt) {
    +        this.mgmt = mgmt;
    +    }
    +
    +    private final String format;
    +    private final String formatName;
    +    private final String formatDescription;
    +    
    +    protected AbstractTypePlanTransformer(String format, String formatName, String formatDescription) {
    +        this.format = format;
    +        this.formatName = formatName;
    +        this.formatDescription = formatDescription;
    +    }
    +    
    +    @Override
    +    public String getFormatCode() {
    +        return format;
    +    }
    +
    +    @Override
    +    public String getFormatName() {
    +        return formatName;
    +    }
    +
    +    @Override
    +    public String getFormatDescription() {
    +        return formatDescription;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return getFormatCode()+":"+JavaClassNames.simpleClassName(this);
    +    }
    +    
    +    @Override
    +    public double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context) {
    +        if (getFormatCode().equals(type.getPlan().getPlanFormat())) return 1;
    +        if (type.getPlan().getPlanFormat()==null)
    +            return scoreForNullFormat(type.getPlan().getPlanData(), type, context);
    +        else
    +            return scoreForNonmatchingNonnullFormat(type.getPlan().getPlanFormat(), type.getPlan().getPlanData(), type, context);
    +    }
    +
    +    protected abstract double scoreForNullFormat(Object planData, RegisteredType type, RegisteredTypeLoadingContext context);
    +    protected abstract double scoreForNonmatchingNonnullFormat(String planFormat, Object planData, RegisteredType type, RegisteredTypeLoadingContext context);
    +
    +    /** delegates to more specific abstract create methods,
    +     * and performs common validation and customisation of the items created.
    +     * <p>
    +     * this includes:
    +     * <li> setting the {@link AbstractBrooklynObjectSpec#catalogItemId(String)}
    +     */
    +    @Override
    +    public Object create(final RegisteredType type, final RegisteredTypeLoadingContext context) {
    +        try {
    +            return validate(new RegisteredTypeKindVisitor<Object>() {
    +                @Override protected Object visitSpec(RegisteredType type) {
    +                    try { 
    +                        AbstractBrooklynObjectSpec<?, ?> result = createSpec(type, context);
    +                        result.catalogItemId(type.getId());
    +                        return result;
    +                    } catch (Exception e) { throw Exceptions.propagate(e); }
    +                }
    +                @Override protected Object visitBean(RegisteredType type) {
    +                    try { 
    +                        return createBean(type, context);
    +                    } catch (Exception e) { throw Exceptions.propagate(e); }
    +                }
    +                
    +            }.visit(type), type, context);
    +        } catch (UnsupportedTypePlanException e) {
    +            // no logging
    +            throw Exceptions.propagate(e);
    --- End diff --
    
    My understanding is that this is just to prevent the next catch from taking over - worth a comment. May be even throwing the same exception instead of calling in to `Exceptions.propagate` to make it more visible (even if the end result is the same).


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r45060681
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/SpecParameterInMetaTest.java ---
    @@ -43,18 +49,40 @@
     public class SpecParameterInMetaTest {
         private ManagementContext mgmt;
         private BrooklynCatalog catalog;
    -    private String spec;
    +    private String specId;
     
         @BeforeMethod(alwaysRun=true)
         public void setUp() {
             mgmt = LocalManagementContextForTests.newInstanceWithOsgi();
             catalog = mgmt.getCatalog();
    -        spec = TestToSpecTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    +        StaticTypePlanTransformer.forceInstall();
    +        PlanToSpecFactory.forceAvailable(TestToSpecTransformer.class, JavaCatalogToSpecTransformer.class);
    --- End diff --
    
    btw removing parameters in meta support made this class obsolete, see https://github.com/apache/incubator-brooklyn/pull/1036. Some other conflicts in the tests between the PRs unfortunately due to a renaming which was needed.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44557882
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/AbstractTypePlanTransformer.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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.typereg;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Convenience supertype for {@link BrooklynTypePlanTransformer} instances.
    + */
    +public abstract class AbstractTypePlanTransformer implements BrooklynTypePlanTransformer {
    +
    +    private static final Logger log = LoggerFactory.getLogger(AbstractTypePlanTransformer.class);
    +    
    +    protected ManagementContext mgmt;
    +
    +    @Override
    +    public void injectManagementContext(ManagementContext mgmt) {
    +        this.mgmt = mgmt;
    +    }
    +
    +    private final String format;
    +    private final String formatName;
    +    private final String formatDescription;
    +    
    +    protected AbstractTypePlanTransformer(String format, String formatName, String formatDescription) {
    +        this.format = format;
    +        this.formatName = formatName;
    +        this.formatDescription = formatDescription;
    +    }
    +    
    +    @Override
    +    public String getFormatCode() {
    +        return format;
    +    }
    +
    +    @Override
    +    public String getFormatName() {
    +        return formatName;
    +    }
    +
    +    @Override
    +    public String getFormatDescription() {
    +        return formatDescription;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return getFormatCode()+":"+JavaClassNames.simpleClassName(this);
    +    }
    +    
    +    @Override
    +    public double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context) {
    +        if (getFormatCode().equals(type.getPlan().getPlanFormat())) return 1;
    +        if (type.getPlan().getPlanFormat()==null)
    +            return scoreForNullFormat(type.getPlan().getPlanData(), type, context);
    +        else
    +            return scoreForNonmatchingNonnullFormat(type.getPlan().getPlanFormat(), type.getPlan().getPlanData(), type, context);
    --- End diff --
    
    (Personal preference ahead) I am on the fail-fast side of the spectrum so my thinking is that this kind of "make every last desperate attempt to produce something usable" approach just makes implementations more complex without actually working in the wild.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#issuecomment-156353953
  
    Finished review, the changes look great and functionally correct. No need to reply to the copious amount of comments, just take what you think is useful.
    The last two comments deserve special attention though as they touch on the high level interplay of the components.


---
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: Flesh out type creation using the...

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/1017#discussion_r45043059
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BasicBrooklynTypeRegistry.java ---
    @@ -90,31 +96,95 @@ public RegisteredType get(String symbolicNameWithOptionalVersion, RegisteredType
     
         @Override
         public RegisteredType get(String symbolicNameWithOptionalVersion) {
    -        return get(symbolicNameWithOptionalVersion, (RegisteredTypeConstraint)null);
    +        return get(symbolicNameWithOptionalVersion, (RegisteredTypeLoadingContext)null);
         }
     
         @SuppressWarnings({ "deprecation", "unchecked", "rawtypes" })
         @Override
    -    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint constraint, Class<SpecT> specSuperType) {
    -        if (!(type instanceof RegisteredSpecType)) { 
    -            throw new IllegalStateException("Cannot create spec from type "+type);
    +    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext constraint, Class<SpecT> specSuperType) {
    +        Preconditions.checkNotNull(type, "type");
    +        if (type.getKind()!=RegisteredTypeKind.SPEC) { 
    +            throw new IllegalStateException("Cannot create spec from type "+type+" (kind "+type.getKind()+")");
             }
             if (constraint!=null) {
    -            if (constraint.getKind()!=null && constraint.getKind()!=RegisteredTypeKind.SPEC) {
    +            if (constraint.getExpectedKind()!=null && constraint.getExpectedKind()!=RegisteredTypeKind.SPEC) {
                     throw new IllegalStateException("Cannot create spec with constraint "+constraint);
                 }
    -            if (constraint.getEncounteredTypes().contains(type.getSymbolicName())) {
    +            if (constraint.getAlreadyEncounteredTypes().contains(type.getSymbolicName())) {
                     // avoid recursive cycle
                     // TODO implement using java if permitted
                 }
             }
    -        constraint = RegisteredTypeConstraints.extendedWithSpecSuperType(constraint, specSuperType);
    +        constraint = RegisteredTypeLoadingContexts.withSpecSuperType(constraint, specSuperType);
     
    -        // TODO look up in the actual registry
    +        Maybe<Object> result = TypePlanTransformers.transform(mgmt, type, constraint);
    +        if (result.isPresent()) return (SpecT) result.get();
             
             // fallback: look up in (legacy) catalog
    +        // TODO remove once all transformers are available in the new style
    --- End diff --
    
    that's the plan.  but downstream projects may have them so keep them for at least a release or two.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44642664
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.SPEC, symbolicName, version, plan), superType);
         }
    -    
    -    public abstract static class AbstractRegisteredType implements RegisteredType {
     
    -        final String symbolicName;
    -        final String version;
    +    /** returns the {@link Class} object corresponding to the given java type name,
    +     * using the cache on the type and the loader defined on the type
    +     * @param mgmt */
    +    @Beta
    +    // TODO should this be on the AbstractTypePlanTransformer ?
    +    public static Class<?> loadActualJavaType(String javaTypeName, ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) throws Exception {
    +        Class<?> result = ((BasicRegisteredType)type).getCache().get(ACTUAL_JAVA_TYPE);
    +        if (result!=null) return result;
             
    -        List<OsgiBundleWithUrl> bundles;
    -        String displayName;
    -        String description;
    -        String iconUrl;
    -        boolean deprecated;
    -        boolean disabled;
    -
    -        // TODO ensure this is re-populated on rebind
    -        transient Class<?> javaType;
    +        result = CatalogUtils.newClassLoadingContext(mgmt, type, context==null ? null : context.getLoader()).loadClass( javaTypeName );
    +        Preconditions.checkNotNull(result, "Could not load class "+javaTypeName+"; returned null (should have thrown a different exception!)");
             
    -        public AbstractRegisteredType(String symbolicName, String version, Class<?> javaType) {
    -            this.symbolicName = symbolicName;
    -            this.version = version;
    -            this.javaType = javaType;
    -        }
    -
    -        @Override
    -        public String getId() {
    -            return symbolicName + (version!=null ? ":"+version : "");
    -        }
    +        ((BasicRegisteredType)type).getCache().put(ACTUAL_JAVA_TYPE, result);
    +        return result;
    +    }
     
    -        @Override
    -        public String getSymbolicName() {
    -            return symbolicName;
    +    @Beta
    +    public static RegisteredType addSuperType(RegisteredType type, @Nullable Class<?> superType) {
    +        if (superType!=null) {
    +            ((BasicRegisteredType)type).superTypes.add(superType);
             }
    +        return type;
    +    }
     
    -        @Override
    -        public String getVersion() {
    -            return version;
    -        }
    -        
    -        @Override
    -        public Collection<OsgiBundleWithUrl> getLibraries() {
    -            return bundles;
    +    @Beta
    +    public static RegisteredType addSuperType(RegisteredType type, @Nullable RegisteredType superType) {
    +        if (superType!=null) {
    +            if (isSubTypeOf(superType, type)) {
    +                throw new IllegalStateException(superType+" declares "+type+" as a supertype; cannot set "+superType+" as a supertype of "+type);
    +            }
    +            ((BasicRegisteredType)type).superTypes.add(superType);
             }
    +        return type;
    +    }
     
    -        @Override
    -        public String getDisplayName() {
    -            return displayName;
    -        }
    +    /** returns the implementation data for a spec if it is a string (e.g. plan yaml or java class name); else false */
    +    @Beta
    +    public static String getImplementationDataStringForSpec(RegisteredType item) {
    +        if (item==null || item.getPlan()==null) return null;
    +        Object data = item.getPlan().getPlanData();
    +        if (!(data instanceof String)) return null;
    --- End diff --
    
    Looking at the usages better throw in this case.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r45061327
  
    --- Diff: usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogParametersTest.java ---
    @@ -273,7 +273,54 @@ public void testAppSpecInheritsCatalogParameters() {
         }
     
         @Test
    -    public void testParametersCoercedOnSetAndReferences() throws Exception {
    +    public void testParametersOnItemCoercedOnSetAndReferences() throws Exception {
    +        Integer testValue = Integer.valueOf(55);
    +        addCatalogItems(
    +                "brooklyn.catalog:",
    +                "  id: " + SYMBOLIC_NAME,
    +                "  version: " + TEST_VERSION,
    +                "  item:",
    +                "    type: " + BasicApplication.class.getName(),
    +                "    brooklyn.parameters:",
    +                "    - name: num",
    +                "      type: integer",
    +                "    brooklyn.children:",
    +                "    - type: " + ConfigEntityForTest.class.getName(),
    +                "      brooklyn.config:",
    +                "        refConfig: $brooklyn:scopeRoot().config(\"num\")",
    +                "    - type: " + ConfigEntityForTest.class.getName(),
    +                "      brooklyn.config:",
    +                "        refConfig: $brooklyn:config(\"num\")"); //inherited config
    +
    +        Entity app = createAndStartApplication(
    +                "services:",
    +                "- type: " + BasicApplication.class.getName(),
    +                "  brooklyn.children:",
    +                "  - type: " + ver(SYMBOLIC_NAME),
    +                "    brooklyn.config:",
    +                "      num: \"" + testValue + "\"");
    +
    +        Entity scopeRoot = Iterables.getOnlyElement(app.getChildren());
    +
    +        ConfigKey<Object> numKey = ConfigKeys.newConfigKey(Object.class, "num");
    +        assertEquals(scopeRoot.config().get(numKey), testValue);
    +
    +        ConfigKey<Object> refConfigKey = ConfigKeys.newConfigKey(Object.class, "refConfig");
    +
    +        Iterator<Entity> childIter = scopeRoot.getChildren().iterator();
    +        Entity c1 = childIter.next();
    +        assertEquals(c1.config().get(refConfigKey), testValue);
    +        Entity c2 = childIter.next();
    +        assertEquals(c2.config().get(refConfigKey), testValue);
    +        assertFalse(childIter.hasNext());
    +    }
    +    
    +    // XXX TODO parameters on the root don't work with new type registry; 
    +    // they require the CI being able to keep them,
    +    // or else modifying the plan. TODO should they be permitted as metadata in this way?
    +    // or treaded like a declaration of config keys on the entity?  i (alex) prefer the latter.
    --- End diff --
    
    Yes, I commented before we did so :)


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44653653
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/typereg/JavaTypePlanTransformerTest.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * 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.typereg;
    +
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.core.test.BrooklynMgmtUnitTestSupport;
    +import org.testng.Assert;
    +import org.testng.annotations.BeforeMethod;
    +import org.testng.annotations.Test;
    +
    +public class JavaTypePlanTransformerTest extends BrooklynMgmtUnitTestSupport {
    --- End diff --
    
    Tiny, test class name should follow class being tested.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44643336
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.SPEC, symbolicName, version, plan), superType);
         }
    -    
    -    public abstract static class AbstractRegisteredType implements RegisteredType {
     
    -        final String symbolicName;
    -        final String version;
    +    /** returns the {@link Class} object corresponding to the given java type name,
    +     * using the cache on the type and the loader defined on the type
    +     * @param mgmt */
    +    @Beta
    +    // TODO should this be on the AbstractTypePlanTransformer ?
    +    public static Class<?> loadActualJavaType(String javaTypeName, ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) throws Exception {
    +        Class<?> result = ((BasicRegisteredType)type).getCache().get(ACTUAL_JAVA_TYPE);
    +        if (result!=null) return result;
             
    -        List<OsgiBundleWithUrl> bundles;
    -        String displayName;
    -        String description;
    -        String iconUrl;
    -        boolean deprecated;
    -        boolean disabled;
    -
    -        // TODO ensure this is re-populated on rebind
    -        transient Class<?> javaType;
    +        result = CatalogUtils.newClassLoadingContext(mgmt, type, context==null ? null : context.getLoader()).loadClass( javaTypeName );
    +        Preconditions.checkNotNull(result, "Could not load class "+javaTypeName+"; returned null (should have thrown a different exception!)");
             
    -        public AbstractRegisteredType(String symbolicName, String version, Class<?> javaType) {
    -            this.symbolicName = symbolicName;
    -            this.version = version;
    -            this.javaType = javaType;
    -        }
    -
    -        @Override
    -        public String getId() {
    -            return symbolicName + (version!=null ? ":"+version : "");
    -        }
    +        ((BasicRegisteredType)type).getCache().put(ACTUAL_JAVA_TYPE, result);
    +        return result;
    +    }
     
    -        @Override
    -        public String getSymbolicName() {
    -            return symbolicName;
    +    @Beta
    +    public static RegisteredType addSuperType(RegisteredType type, @Nullable Class<?> superType) {
    +        if (superType!=null) {
    +            ((BasicRegisteredType)type).superTypes.add(superType);
             }
    +        return type;
    +    }
     
    -        @Override
    -        public String getVersion() {
    -            return version;
    -        }
    -        
    -        @Override
    -        public Collection<OsgiBundleWithUrl> getLibraries() {
    -            return bundles;
    +    @Beta
    +    public static RegisteredType addSuperType(RegisteredType type, @Nullable RegisteredType superType) {
    +        if (superType!=null) {
    +            if (isSubTypeOf(superType, type)) {
    +                throw new IllegalStateException(superType+" declares "+type+" as a supertype; cannot set "+superType+" as a supertype of "+type);
    +            }
    +            ((BasicRegisteredType)type).superTypes.add(superType);
             }
    +        return type;
    +    }
     
    -        @Override
    -        public String getDisplayName() {
    -            return displayName;
    -        }
    +    /** returns the implementation data for a spec if it is a string (e.g. plan yaml or java class name); else false */
    +    @Beta
    +    public static String getImplementationDataStringForSpec(RegisteredType item) {
    +        if (item==null || item.getPlan()==null) return null;
    +        Object data = item.getPlan().getPlanData();
    +        if (!(data instanceof String)) return null;
    +        return (String)data;
    +    }
     
    -        @Override
    -        public String getDescription() {
    -            return description;
    -        }
    +    /** returns an implementation of the spec class corresponding to the given target type;
    +     * for use in {@link BrooklynTypePlanTransformer#create(RegisteredType, RegisteredTypeLoadingContext)} 
    +     * implementations when dealing with a spec; returns null if none found
    +     * @param mgmt */
    +    @Beta
    +    public static AbstractBrooklynObjectSpec<?,?> newSpecInstance(ManagementContext mgmt, Class<? extends BrooklynObject> targetType) throws Exception {
    +        Class<? extends AbstractBrooklynObjectSpec<?, ?>> specType = RegisteredTypeLoadingContexts.lookupSpecTypeForTarget(targetType);
    +        if (specType==null) return null;
    +        Method createMethod = specType.getMethod("create", Class.class);
    +        return (AbstractBrooklynObjectSpec<?, ?>) createMethod.invoke(null, targetType);
    +    }
     
    -        @Override
    -        public String getIconUrl() {
    -            return iconUrl;
    -        }
    -        
    -        @Override
    -        public boolean isDisabled() {
    -            return disabled;
    -        }
    -        
    -        @Override
    -        public boolean isDeprecated() {
    -            return deprecated;
    -        }
    -        
    -        @Override
    -        public Class<?> getJavaType() {
    -            return javaType;
    -        }
    -        
    -        @Override
    -        public String toString() {
    -            return JavaClassNames.simpleClassName(this)+"["+getId()+
    -                (isDisabled() ? ";DISABLED" : "")+
    -                (isDeprecated() ? ";deprecated" : "")+
    -                "]";
    -        }
    +    /** Returns a wrapped map, if the object is YAML which parses as a map; 
    +     * otherwise returns absent capable of throwing an error with more details */
    +    @SuppressWarnings("unchecked")
    +    public static Maybe<Map<Object,Object>> getAsYamlMap(Object planData) {
    --- End diff --
    
    Return type should be `Maybe<Map<?,?>>` (it's a read-only container), callers are not supposed to mutate 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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44704144
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BasicBrooklynTypeRegistry.java ---
    @@ -90,31 +96,95 @@ public RegisteredType get(String symbolicNameWithOptionalVersion, RegisteredType
     
         @Override
         public RegisteredType get(String symbolicNameWithOptionalVersion) {
    -        return get(symbolicNameWithOptionalVersion, (RegisteredTypeConstraint)null);
    +        return get(symbolicNameWithOptionalVersion, (RegisteredTypeLoadingContext)null);
         }
     
         @SuppressWarnings({ "deprecation", "unchecked", "rawtypes" })
         @Override
    -    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint constraint, Class<SpecT> specSuperType) {
    -        if (!(type instanceof RegisteredSpecType)) { 
    -            throw new IllegalStateException("Cannot create spec from type "+type);
    +    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext constraint, Class<SpecT> specSuperType) {
    +        Preconditions.checkNotNull(type, "type");
    +        if (type.getKind()!=RegisteredTypeKind.SPEC) { 
    +            throw new IllegalStateException("Cannot create spec from type "+type+" (kind "+type.getKind()+")");
             }
             if (constraint!=null) {
    -            if (constraint.getKind()!=null && constraint.getKind()!=RegisteredTypeKind.SPEC) {
    +            if (constraint.getExpectedKind()!=null && constraint.getExpectedKind()!=RegisteredTypeKind.SPEC) {
                     throw new IllegalStateException("Cannot create spec with constraint "+constraint);
                 }
    -            if (constraint.getEncounteredTypes().contains(type.getSymbolicName())) {
    +            if (constraint.getAlreadyEncounteredTypes().contains(type.getSymbolicName())) {
                     // avoid recursive cycle
                     // TODO implement using java if permitted
                 }
             }
    -        constraint = RegisteredTypeConstraints.extendedWithSpecSuperType(constraint, specSuperType);
    +        constraint = RegisteredTypeLoadingContexts.withSpecSuperType(constraint, specSuperType);
     
    -        // TODO look up in the actual registry
    +        Maybe<Object> result = TypePlanTransformers.transform(mgmt, type, constraint);
    +        if (result.isPresent()) return (SpecT) result.get();
             
             // fallback: look up in (legacy) catalog
    +        // TODO remove once all transformers are available in the new style
             CatalogItem item = (CatalogItem) mgmt.getCatalog().getCatalogItem(type.getSymbolicName(), type.getVersion());
    -        return (SpecT) BasicBrooklynCatalog.internalCreateSpecWithTransformers(mgmt, item, constraint.getEncounteredTypes());
    +        if (item==null) {
    --- End diff --
    
    You are mixing two types of transformations in this method:
      * plan (i.e. camp) to spec transformers - `BrooklynTypePlanTransformer` (former `PlanToSpecTransformer), the part up to this line
      * type (symbolicName, java identifier) to spec transformers - `EntitySpecResolver`,  the part from this line until the end. This type of transformation is not needed at this level, it shoud be nested in the plan transformers. Looks like it's needed here just to get `SpecParameterInMetaTest` from passing.
    
    Instead of this second part, `StaticTypePlanTransformer` should be taught how to handle the plans from the test (essentially same as pre-changes `TestToSpecTransformer`). The commit https://github.com/neykov/incubator-brooklyn/commit/8e9d69dbaeb6b15f8351f4bf5f2928c379a86b3d shows what I mean. The test `testOsgiClassScanned` is moved to `brooklyn-camp` as the camp parser is needed to create specs out of the catalog items.
    With these changes the legacy transformers are used only during catalog `addItems`. Should be easy to move to the new API, I suppose part of the next 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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r45057491
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/mgmt/classloading/BrooklynClassLoadingContext.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.api.mgmt.classloading;
    +
    +import java.net.URL;
    +
    +import javax.annotation.Nullable;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.util.guava.Maybe;
    +
    +/** 
    + * Provides functionality for loading classes based on the current context
    + * (e.g. the bundles of a registered type from which an entity is created)
    + */
    +public interface BrooklynClassLoadingContext {
    +
    +    public ManagementContext getManagementContext();
    +    public Class<?> loadClass(String className);
    +    public <T> Class<? extends T> loadClass(String className, @Nullable Class<T> supertype);
    +
    +    public Maybe<Class<?>> tryLoadClass(String className);
    +    public <T> Maybe<Class<? extends T>> tryLoadClass(String className, @Nullable Class<T> supertype);
    +
    +    /** As {@link ClassLoader#getResource(String)} */
    +    public URL getResource(String name);
    +
    +    /**
    +     * As {@link ClassLoader#getResources(String)} but returning an {@link Iterable} rather than
    +     * an {@link java.util.Enumeration}.
    +     */
    +    public Iterable<URL> getResources(String name);
    --- End diff --
    
    I guess it's down to personal preference. My view is take `Iterable` and return the most specific type possible, `Iterable` is for lazy containers only. What I don't like about `Iterable` is that it's not of much use outside of guava, most libraries take `Collection`s.
    The implementations are all eager in this case.


---
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: Flesh out type creation using the...

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/1017#discussion_r45042974
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/AbstractTypePlanTransformer.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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.typereg;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Convenience supertype for {@link BrooklynTypePlanTransformer} instances.
    + */
    +public abstract class AbstractTypePlanTransformer implements BrooklynTypePlanTransformer {
    +
    +    private static final Logger log = LoggerFactory.getLogger(AbstractTypePlanTransformer.class);
    +    
    +    protected ManagementContext mgmt;
    +
    +    @Override
    +    public void injectManagementContext(ManagementContext mgmt) {
    +        this.mgmt = mgmt;
    +    }
    +
    +    private final String format;
    +    private final String formatName;
    +    private final String formatDescription;
    +    
    +    protected AbstractTypePlanTransformer(String format, String formatName, String formatDescription) {
    +        this.format = format;
    +        this.formatName = formatName;
    +        this.formatDescription = formatDescription;
    +    }
    +    
    +    @Override
    +    public String getFormatCode() {
    +        return format;
    +    }
    +
    +    @Override
    +    public String getFormatName() {
    +        return formatName;
    +    }
    +
    +    @Override
    +    public String getFormatDescription() {
    +        return formatDescription;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return getFormatCode()+":"+JavaClassNames.simpleClassName(this);
    +    }
    +    
    +    @Override
    +    public double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context) {
    +        if (getFormatCode().equals(type.getPlan().getPlanFormat())) return 1;
    +        if (type.getPlan().getPlanFormat()==null)
    +            return scoreForNullFormat(type.getPlan().getPlanData(), type, context);
    +        else
    +            return scoreForNonmatchingNonnullFormat(type.getPlan().getPlanFormat(), type.getPlan().getPlanData(), type, context);
    +    }
    +
    +    protected abstract double scoreForNullFormat(Object planData, RegisteredType type, RegisteredTypeLoadingContext context);
    +    protected abstract double scoreForNonmatchingNonnullFormat(String planFormat, Object planData, RegisteredType type, RegisteredTypeLoadingContext context);
    +
    +    /** delegates to more specific abstract create methods,
    +     * and performs common validation and customisation of the items created.
    +     * <p>
    +     * this includes:
    +     * <li> setting the {@link AbstractBrooklynObjectSpec#catalogItemId(String)}
    +     */
    +    @Override
    +    public Object create(final RegisteredType type, final RegisteredTypeLoadingContext context) {
    +        try {
    +            return validate(new RegisteredTypeKindVisitor<Object>() {
    +                @Override protected Object visitSpec(RegisteredType type) {
    +                    try { 
    +                        AbstractBrooklynObjectSpec<?, ?> result = createSpec(type, context);
    +                        result.catalogItemId(type.getId());
    +                        return result;
    +                    } catch (Exception e) { throw Exceptions.propagate(e); }
    +                }
    +                @Override protected Object visitBean(RegisteredType type) {
    +                    try { 
    +                        return createBean(type, context);
    +                    } catch (Exception e) { throw Exceptions.propagate(e); }
    +                }
    +                
    +            }.visit(type), type, context);
    +        } catch (UnsupportedTypePlanException e) {
    +            // no logging
    +            throw Exceptions.propagate(e);
    +        } catch (Exception e) {
    +            Exceptions.propagateIfFatal(e);
    +            log.debug("Could not instantiate "+type+" (rethrowing): "+Exceptions.collapseText(e));
    +            throw Exceptions.propagate(e);
    +        }
    +    }
    +    
    +    protected <T> T validate(T createdObject, RegisteredType type, RegisteredTypeLoadingContext context) {
    +        if (createdObject==null) return null;
    +        // TODO validation based on the constraint, throw UnsupportedTypePlanException with details if not matched
    +        return createdObject;
    +    }
    +
    +    protected abstract AbstractBrooklynObjectSpec<?,?> createSpec(RegisteredType type, RegisteredTypeLoadingContext context) throws Exception;
    --- End diff --
    
    there are so many exceptions when working with classes and loading, i don't think that's likely.  happy to revert in time however; i figure this will be beta for a few releases.


---
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: Flesh out type creation using the...

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/1017#discussion_r45048982
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/SpecParameterInMetaTest.java ---
    @@ -43,18 +49,40 @@
     public class SpecParameterInMetaTest {
         private ManagementContext mgmt;
         private BrooklynCatalog catalog;
    -    private String spec;
    +    private String specId;
     
         @BeforeMethod(alwaysRun=true)
         public void setUp() {
             mgmt = LocalManagementContextForTests.newInstanceWithOsgi();
             catalog = mgmt.getCatalog();
    -        spec = TestToSpecTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    +        StaticTypePlanTransformer.forceInstall();
    +        PlanToSpecFactory.forceAvailable(TestToSpecTransformer.class, JavaCatalogToSpecTransformer.class);
    --- End diff --
    
    Intention is to get rid of the `TestToSpecTransformer` but we need a bit more machinery first.  Have added TODO to that effect.


---
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: Flesh out type creation using the...

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/1017#discussion_r45053285
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogUtils.java ---
    @@ -85,16 +89,23 @@ public static BrooklynClassLoadingContext getClassLoadingContext(Entity entity)
         }
     
         public static BrooklynClassLoadingContext newClassLoadingContext(@Nullable ManagementContext mgmt, String catalogItemId, Collection<? extends OsgiBundleWithUrl> libraries) {
    +        return newClassLoadingContext(mgmt, catalogItemId, libraries, null);
    +    }
    +    
    +    public static BrooklynClassLoadingContext newClassLoadingContext(@Nullable ManagementContext mgmt, String catalogItemId, Collection<? extends OsgiBundleWithUrl> libraries, BrooklynClassLoadingContext loader) {
             BrooklynClassLoadingContextSequential result = new BrooklynClassLoadingContextSequential(mgmt);
     
             if (libraries!=null && !libraries.isEmpty()) {
                 result.add(new OsgiBrooklynClassLoadingContext(mgmt, catalogItemId, libraries));
             }
     
    -        BrooklynClassLoadingContext loader = BrooklynLoaderTracker.getLoader();
    -        if (loader != null) {
    +        if (loader !=null) {
    --- End diff --
    
    +1 to tightening this; i've added some TODO comments and deprecation to move us in this direction


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44554041
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BrooklynTypePlanTransformer.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.List;
    +import java.util.ServiceLoader;
    +
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
    +
    +/**
    + * Interface for use by schemes which with to be able to transform plans.
    + * <p>
    + * To add a new plan transformation scheme, simply create an implementation and declare it
    + * as a java service (cf {@link ServiceLoader}).
    + * <p>
    + * Implementations may wish to extend {@link AbstractTypePlanTransformer} which simplifies the process.
    + */
    +public interface BrooklynTypePlanTransformer extends ManagementContextInjectable {
    +
    +    /** @return a code to identify type implementations created specifying the use of this plan transformer. */
    +    String getFormatCode();
    +    /** @return a display name for this transformer. */
    +    String getFormatName();
    +    /** @return a description for this transformer */
    +    String getFormatDescription();
    +
    +    /** @return how appropriate is this transformer for the {@link RegisteredType#getPlan()} of the type;
    +     * 0 (or less) if not, 1 for absolutely, and in some autodetect cases a value between 0 and 1 indicate a ranking.
    +     * <p>
    +     * The framework guarantees arguments are nonnull, and that the {@link RegisteredType#getPlan()} is also not-null.
    +     * However the format in that plan may be null. */
    +    double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context);
    --- End diff --
    
    Don't like this kind of non-determinism for heuristics - in particular how each implementation needs to keep an eye on the other implementations for determining the right score. Don't have a better suggestion though so will just :mute:.


---
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: Flesh out type creation using the...

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/1017#discussion_r45044766
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypePredicates.java ---
    @@ -109,36 +109,46 @@ public boolean apply(@Nullable RegisteredType item) {
             }
         }
     
    -    public static <T> Predicate<RegisteredType> javaType(final Predicate<Class<T>> filter) {
    -        return new JavaTypeMatches(filter);
    +    public static <T> Predicate<RegisteredType> anySuperType(final Predicate<Class<T>> filter) {
    --- End diff --
    
    i don't think nested generic wildcard works like that -- `Predicate<Class<?>>` does not match any `Predicate<Class<T>>`.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44641309
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypePredicates.java ---
    @@ -109,36 +109,46 @@ public boolean apply(@Nullable RegisteredType item) {
             }
         }
     
    -    public static <T> Predicate<RegisteredType> javaType(final Predicate<Class<T>> filter) {
    -        return new JavaTypeMatches(filter);
    +    public static <T> Predicate<RegisteredType> anySuperType(final Predicate<Class<T>> filter) {
    +        return new AnySuperTypeMatches(filter);
         }
         @SuppressWarnings({ "unchecked", "rawtypes" })
    -    public static Predicate<RegisteredType> javaTypeAssignableFrom(final Class<?> filter) {
    -        return javaType((Predicate)Predicates.assignableFrom(filter));
    +    public static Predicate<RegisteredType> assignableFrom(final Class<?> filter) {
    +        return anySuperType((Predicate)Predicates.assignableFrom(filter));
         }
         
    -    private static class JavaTypeMatches implements Predicate<RegisteredType> {
    +    private static class AnySuperTypeMatches implements Predicate<RegisteredType> {
             private final Predicate<Class<?>> filter;
             
             @SuppressWarnings({ "rawtypes", "unchecked" })
    -        private <T> JavaTypeMatches(Predicate filter) {
    +        private <T> AnySuperTypeMatches(Predicate filter) {
    --- End diff --
    
    Generic type not used, remove.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44534031
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/internal/AbstractBrooklynObjectSpec.java ---
    @@ -160,5 +172,9 @@ public boolean equals(Object obj) {
         public int hashCode() {
             return Objects.hashCode(getCatalogItemId(), getDisplayName(), getType(), getTags());
         }
    +
    +    /** strings inserted as flags, config keys inserted as config keys; 
    +     * if you want to force one or the other, create a ConfigBag and convert to the appropriate map type */
    +    public abstract SpecT configure(Map<?,?> val);
    --- End diff --
    
    Configuration handling between specs seems identical these days, why not promote all related implementations?
    
    But at least promote abstract `getFlags()/getConfig` as otherwise the javadoc doesn't make much sense, wdyt?


---
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: Flesh out type creation using the...

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/1017#discussion_r45043680
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BrooklynTypePlanTransformer.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.List;
    +import java.util.ServiceLoader;
    +
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
    +
    +/**
    + * Interface for use by schemes which with to be able to transform plans.
    + * <p>
    + * To add a new plan transformation scheme, simply create an implementation and declare it
    + * as a java service (cf {@link ServiceLoader}).
    + * <p>
    + * Implementations may wish to extend {@link AbstractTypePlanTransformer} which simplifies the process.
    + */
    +public interface BrooklynTypePlanTransformer extends ManagementContextInjectable {
    --- End diff --
    
    It's unfortunate we have `BrooklynType`.  I've proposed in comment there that *that* be renamed.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44542412
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/BrooklynTypeRegistry.java ---
    @@ -45,24 +45,25 @@
         /** @return The item matching the given given 
          * {@link RegisteredType#getSymbolicName() symbolicName} 
          * and optionally {@link RegisteredType#getVersion()},
    -     * filtered for the optionally supplied {@link RegisteredTypeConstraint}, 
    +     * filtered for the optionally supplied {@link RegisteredTypeLoadingContext}, 
          * taking the best version if the version is null or a default marker,
          * returning null if no matches are found. */
    -    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} with no constraints */
    +    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} with no constraints */
         RegisteredType get(String symbolicName, String version);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} but allows <code>"name:version"</code> 
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} but allows <code>"name:version"</code> 
          * (the {@link RegisteredType#getId()}) in addition to the unversioned name,
          * using a default marker if no version can be inferred */
    -    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, RegisteredTypeConstraint)} but with no constraints */
    +    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeLoadingContext constraint);
    --- End diff --
    
    It's not clear from the javadoc what does the `constraint` filter on `get` methods. Do we allow conflicting types in the registry - for example a catalog item and a bean with the same `symbolicName` (in which case `version` is not enough of a discriminator)?
      * if yes, then shouldn't the return type be a collection for the case where the `constraint` wasn't strict enough?
      * if no, then why have a constraint at all here, I see it's not used yet in the implementation?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44635434
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.SPEC, symbolicName, version, plan), superType);
         }
    -    
    -    public abstract static class AbstractRegisteredType implements RegisteredType {
     
    -        final String symbolicName;
    -        final String version;
    +    /** returns the {@link Class} object corresponding to the given java type name,
    +     * using the cache on the type and the loader defined on the type
    +     * @param mgmt */
    +    @Beta
    +    // TODO should this be on the AbstractTypePlanTransformer ?
    +    public static Class<?> loadActualJavaType(String javaTypeName, ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) throws Exception {
    --- End diff --
    
    Looks like `throws Exception` is not needed.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44645535
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/TypePlanTransformers.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.TreeMap;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.PropagatedRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.Beta;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.collect.ArrayListMultimap;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Iterables;
    +import com.google.common.collect.Multimap;
    +
    +public class TypePlanTransformers {
    +
    +    private static final Logger log = LoggerFactory.getLogger(TypePlanTransformers.class);
    +
    +    private static Collection<BrooklynTypePlanTransformer> getAll() {
    +        return ImmutableList.copyOf(ServiceLoader.load(BrooklynTypePlanTransformer.class));
    +    }
    +
    +    private static Collection<Class<? extends BrooklynTypePlanTransformer>> OVERRIDE;
    +    @SafeVarargs
    +    @VisibleForTesting
    +    public synchronized static void forceAvailable(Class<? extends BrooklynTypePlanTransformer> ...classes) {
    +        OVERRIDE = Arrays.asList(classes);
    +    }
    +    public synchronized static void clearForced() {
    +        OVERRIDE = null;
    +    }
    +
    +    public static Collection<BrooklynTypePlanTransformer> all(ManagementContext mgmt) {
    +        // TODO cache these in the TypeRegistry, looking for new ones periodically or supplying a way to register them
    +        Collection<Class<? extends BrooklynTypePlanTransformer>> override = OVERRIDE;
    +        Collection<BrooklynTypePlanTransformer> result = new ArrayList<BrooklynTypePlanTransformer>();
    +        if (override!=null) {
    +            for (Class<? extends BrooklynTypePlanTransformer> o1: override) {
    +                try {
    +                    result.add(o1.newInstance());
    +                } catch (Exception e) {
    +                    Exceptions.propagate(e);
    +                }
    +            }
    +        } else {
    +            result.addAll(getAll());
    +        }
    +        for(BrooklynTypePlanTransformer t : result) {
    +            t.injectManagementContext(mgmt);
    +        }
    +        return result;
    +    }
    +
    +    /** returns a list of {@link BrooklynTypePlanTransformer} instances for this {@link ManagementContext}
    +     * which may be able to handle the given plan; the list is sorted with highest-score transformer first */
    +    @Beta
    +    public static List<BrooklynTypePlanTransformer> forType(ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext constraint) {
    +        Multimap<Double,BrooklynTypePlanTransformer> byScoreMulti = ArrayListMultimap.create(); 
    +        Collection<BrooklynTypePlanTransformer> transformers = all(mgmt);
    +        for (BrooklynTypePlanTransformer transformer : transformers) {
    +            double score = transformer.scoreForType(type, constraint);
    +            if (score>0) byScoreMulti.put(score, transformer);
    +        }
    +        Map<Double, Collection<BrooklynTypePlanTransformer>> tree = new TreeMap<Double, Collection<BrooklynTypePlanTransformer>>(byScoreMulti.asMap());
    +        List<Collection<BrooklynTypePlanTransformer>> highestFirst = new ArrayList<Collection<BrooklynTypePlanTransformer>>(tree.values());
    +        Collections.reverse(highestFirst);
    --- End diff --
    
    Could pass `Collections.reverseOrder()` to the `TreeMap` instead.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44666355
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/BrooklynTypeRegistry.java ---
    @@ -45,24 +45,25 @@
         /** @return The item matching the given given 
          * {@link RegisteredType#getSymbolicName() symbolicName} 
          * and optionally {@link RegisteredType#getVersion()},
    -     * filtered for the optionally supplied {@link RegisteredTypeConstraint}, 
    +     * filtered for the optionally supplied {@link RegisteredTypeLoadingContext}, 
          * taking the best version if the version is null or a default marker,
          * returning null if no matches are found. */
    -    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} with no constraints */
    +    RegisteredType get(String symbolicName, String version, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} with no constraints */
         RegisteredType get(String symbolicName, String version);
    -    /** as {@link #get(String, String, RegisteredTypeConstraint)} but allows <code>"name:version"</code> 
    +    /** as {@link #get(String, String, RegisteredTypeLoadingContext)} but allows <code>"name:version"</code> 
          * (the {@link RegisteredType#getId()}) in addition to the unversioned name,
          * using a default marker if no version can be inferred */
    -    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeConstraint constraint);
    -    /** as {@link #get(String, RegisteredTypeConstraint)} but with no constraints */
    +    RegisteredType get(String symbolicNameWithOptionalVersion, @Nullable RegisteredTypeLoadingContext constraint);
    +    /** as {@link #get(String, RegisteredTypeLoadingContext)} but with no constraints */
         RegisteredType get(String symbolicNameWithOptionalVersion);
     
         // NB the seemingly more correct generics <T,SpecT extends AbstractBrooklynObjectSpec<T,SpecT>> 
         // cause compile errors, not in Eclipse, but in maven (?) 
    -    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint optionalConstraint, Class<SpecT> optionalSpecSuperType);
    +    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<SpecT> optionalSpecSuperType);
    +    <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpecFromPlan(String planFormat, Object planData, @Nullable RegisteredTypeLoadingContext optionalConstraint, Class<SpecT> optionalSpecSuperType);
    --- End diff --
    
    `planFormat` is `@Nullable`, the optional arguments obviously as well.


---
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: Flesh out type creation using the...

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/1017#discussion_r45049881
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/api/AssemblyTemplateSpecInstantiator.java ---
    @@ -23,19 +23,23 @@
     
     import org.apache.brooklyn.api.entity.Application;
     import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
     import org.apache.brooklyn.camp.CampPlatform;
     import org.apache.brooklyn.camp.spi.AssemblyTemplate;
     import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
    -import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
     
     public interface AssemblyTemplateSpecInstantiator extends AssemblyTemplateInstantiator {
     
    +    @Deprecated /** @deprecaed since 0.9.0 include encountered types */
    +    EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader);
    --- 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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44634399
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BrooklynTypePlanTransformer.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.List;
    +import java.util.ServiceLoader;
    +
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
    +
    +/**
    + * Interface for use by schemes which with to be able to transform plans.
    + * <p>
    + * To add a new plan transformation scheme, simply create an implementation and declare it
    + * as a java service (cf {@link ServiceLoader}).
    + * <p>
    + * Implementations may wish to extend {@link AbstractTypePlanTransformer} which simplifies the process.
    + */
    +public interface BrooklynTypePlanTransformer extends ManagementContextInjectable {
    +
    +    /** @return a code to identify type implementations created specifying the use of this plan transformer. */
    +    String getFormatCode();
    +    /** @return a display name for this transformer. */
    +    String getFormatName();
    +    /** @return a description for this transformer */
    +    String getFormatDescription();
    +
    +    /** @return how appropriate is this transformer for the {@link RegisteredType#getPlan()} of the type;
    +     * 0 (or less) if not, 1 for absolutely, and in some autodetect cases a value between 0 and 1 indicate a ranking.
    +     * <p>
    +     * The framework guarantees arguments are nonnull, and that the {@link RegisteredType#getPlan()} is also not-null.
    +     * However the format in that plan may be null. */
    +    double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context);
    +    /** Creates a new instance of the indicated type, or throws if not supported;
    +     * this method is used by the {@link BrooklynTypeRegistry} when it creates instances,
    +     * so implementations must respect the {@link RegisteredTypeKind} semantics and the {@link RegisteredTypeLoadingContext}
    +     * if they return an instance.
    +     * <p>
    +     * The framework guarantees this will only be invoked when {@link #scoreForType(RegisteredType, RegisteredTypeLoadingContext)} 
    +     * has returned a positive value.
    +     * <p>
    +     * Implementations should either return null or throw {@link UnsupportedTypePlanException} 
    +     * if the {@link RegisteredType#getPlan()} is not supported. */
    +    Object create(RegisteredType type, RegisteredTypeLoadingContext context);
    +    
    +    double scoreForTypeDefinition(String formatCode, Object catalogData);
    +    List<RegisteredType> createFromTypeDefinition(String formatCode, Object catalogData);
    --- End diff --
    
    Will these ever be used, the registry `xxxFromPlan` methods already wrap the plans in a type?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44645660
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/TypePlanTransformers.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.TreeMap;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.PropagatedRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.Beta;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.collect.ArrayListMultimap;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Iterables;
    +import com.google.common.collect.Multimap;
    +
    +public class TypePlanTransformers {
    +
    +    private static final Logger log = LoggerFactory.getLogger(TypePlanTransformers.class);
    +
    +    private static Collection<BrooklynTypePlanTransformer> getAll() {
    +        return ImmutableList.copyOf(ServiceLoader.load(BrooklynTypePlanTransformer.class));
    +    }
    +
    +    private static Collection<Class<? extends BrooklynTypePlanTransformer>> OVERRIDE;
    +    @SafeVarargs
    +    @VisibleForTesting
    +    public synchronized static void forceAvailable(Class<? extends BrooklynTypePlanTransformer> ...classes) {
    +        OVERRIDE = Arrays.asList(classes);
    +    }
    +    public synchronized static void clearForced() {
    +        OVERRIDE = null;
    +    }
    +
    +    public static Collection<BrooklynTypePlanTransformer> all(ManagementContext mgmt) {
    +        // TODO cache these in the TypeRegistry, looking for new ones periodically or supplying a way to register them
    +        Collection<Class<? extends BrooklynTypePlanTransformer>> override = OVERRIDE;
    +        Collection<BrooklynTypePlanTransformer> result = new ArrayList<BrooklynTypePlanTransformer>();
    +        if (override!=null) {
    +            for (Class<? extends BrooklynTypePlanTransformer> o1: override) {
    +                try {
    +                    result.add(o1.newInstance());
    +                } catch (Exception e) {
    +                    Exceptions.propagate(e);
    +                }
    +            }
    +        } else {
    +            result.addAll(getAll());
    +        }
    +        for(BrooklynTypePlanTransformer t : result) {
    +            t.injectManagementContext(mgmt);
    +        }
    +        return result;
    +    }
    +
    +    /** returns a list of {@link BrooklynTypePlanTransformer} instances for this {@link ManagementContext}
    +     * which may be able to handle the given plan; the list is sorted with highest-score transformer first */
    +    @Beta
    +    public static List<BrooklynTypePlanTransformer> forType(ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext constraint) {
    +        Multimap<Double,BrooklynTypePlanTransformer> byScoreMulti = ArrayListMultimap.create(); 
    +        Collection<BrooklynTypePlanTransformer> transformers = all(mgmt);
    +        for (BrooklynTypePlanTransformer transformer : transformers) {
    +            double score = transformer.scoreForType(type, constraint);
    +            if (score>0) byScoreMulti.put(score, transformer);
    +        }
    +        Map<Double, Collection<BrooklynTypePlanTransformer>> tree = new TreeMap<Double, Collection<BrooklynTypePlanTransformer>>(byScoreMulti.asMap());
    +        List<Collection<BrooklynTypePlanTransformer>> highestFirst = new ArrayList<Collection<BrooklynTypePlanTransformer>>(tree.values());
    +        Collections.reverse(highestFirst);
    +        return MutableList.copyOf(Iterables.concat(highestFirst)).asUnmodifiable();
    --- End diff --
    
    Could use `ImmutableList.copyOf()` instead.
    Don't understand the usage of `concat` 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] incubator-brooklyn pull request: Flesh out type creation using the...

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/1017#discussion_r45052574
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BasicBrooklynTypeRegistry.java ---
    @@ -90,31 +96,95 @@ public RegisteredType get(String symbolicNameWithOptionalVersion, RegisteredType
     
         @Override
         public RegisteredType get(String symbolicNameWithOptionalVersion) {
    -        return get(symbolicNameWithOptionalVersion, (RegisteredTypeConstraint)null);
    +        return get(symbolicNameWithOptionalVersion, (RegisteredTypeLoadingContext)null);
         }
     
         @SuppressWarnings({ "deprecation", "unchecked", "rawtypes" })
         @Override
    -    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeConstraint constraint, Class<SpecT> specSuperType) {
    -        if (!(type instanceof RegisteredSpecType)) { 
    -            throw new IllegalStateException("Cannot create spec from type "+type);
    +    public <SpecT extends AbstractBrooklynObjectSpec<?,?>> SpecT createSpec(RegisteredType type, @Nullable RegisteredTypeLoadingContext constraint, Class<SpecT> specSuperType) {
    +        Preconditions.checkNotNull(type, "type");
    +        if (type.getKind()!=RegisteredTypeKind.SPEC) { 
    +            throw new IllegalStateException("Cannot create spec from type "+type+" (kind "+type.getKind()+")");
             }
             if (constraint!=null) {
    -            if (constraint.getKind()!=null && constraint.getKind()!=RegisteredTypeKind.SPEC) {
    +            if (constraint.getExpectedKind()!=null && constraint.getExpectedKind()!=RegisteredTypeKind.SPEC) {
                     throw new IllegalStateException("Cannot create spec with constraint "+constraint);
                 }
    -            if (constraint.getEncounteredTypes().contains(type.getSymbolicName())) {
    +            if (constraint.getAlreadyEncounteredTypes().contains(type.getSymbolicName())) {
                     // avoid recursive cycle
                     // TODO implement using java if permitted
                 }
             }
    -        constraint = RegisteredTypeConstraints.extendedWithSpecSuperType(constraint, specSuperType);
    +        constraint = RegisteredTypeLoadingContexts.withSpecSuperType(constraint, specSuperType);
     
    -        // TODO look up in the actual registry
    +        Maybe<Object> result = TypePlanTransformers.transform(mgmt, type, constraint);
    +        if (result.isPresent()) return (SpecT) result.get();
             
             // fallback: look up in (legacy) catalog
    +        // TODO remove once all transformers are available in the new style
             CatalogItem item = (CatalogItem) mgmt.getCatalog().getCatalogItem(type.getSymbolicName(), type.getVersion());
    -        return (SpecT) BasicBrooklynCatalog.internalCreateSpecWithTransformers(mgmt, item, constraint.getEncounteredTypes());
    +        if (item==null) {
    --- End diff --
    
    Yes, this is what I'm trying to clear up in the next PR.  Your reference is helpful!


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44656326
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampResolver.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.camp.brooklyn.spi.creation;
    +
    +import java.util.Set;
    +
    +import org.apache.brooklyn.api.entity.Application;
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
    +import org.apache.brooklyn.api.policy.Policy;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.camp.CampPlatform;
    +import org.apache.brooklyn.camp.brooklyn.api.AssemblyTemplateSpecInstantiator;
    +import org.apache.brooklyn.camp.spi.AssemblyTemplate;
    +import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
    +import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
    +import org.apache.brooklyn.core.mgmt.EntityManagementUtils;
    +import org.apache.brooklyn.core.typereg.RegisteredTypes;
    +import org.apache.brooklyn.util.collections.MutableSet;
    +import org.apache.brooklyn.util.text.Strings;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +class CampResolver {
    +
    +    private ManagementContext mgmt;
    +    private RegisteredType type;
    +    private RegisteredTypeLoadingContext context;
    +
    +    /** whether to allow parsing of the 'full' syntax for applications,
    +     * where items are wrapped in a "services:" block, and if the wrapper is an application,
    +     * to promote it */
    +    boolean allowApplicationFullSyntax = true;
    +
    +    /** whether to allow parsing of the legacy 'full' syntax, 
    +     * where a non-application items are wrapped:
    +     * <li> in a "services:" block for entities,
    +     * <li> in a "brooklyn.locations" or "brooklyn.policies" block for locations and policies */
    +    boolean allowLegacyFullSyntax = true;
    +
    +    /** whether to allow parsing of the type syntax, where an item is a map with a "type:" field,
    +     * i.e. not wrapped in any "services:" or "brooklyn.{locations,policies}" block */
    +    boolean allowTypeSyntax = true;
    --- End diff --
    
    Are these a part of future changes, no used at the moment?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44651669
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/plan/XmlPlanToSpecTransformer.java ---
    @@ -41,6 +41,8 @@
     /** Example implementation of {@link PlanToSpecTransformer} showing 
      * how implementations are meant to be written. */
     public class XmlPlanToSpecTransformer implements PlanToSpecTransformer {
    +    
    +    // this is REPLACED by ExampleXmlTypePlanTransformer
    --- End diff --
    
    Just delete this one?


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44642260
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypes.java ---
    @@ -66,180 +94,131 @@ public static RegisteredType of(CatalogItem<?, ?> item) {
             type.deprecated = item.isDeprecated();
     
             // TODO
    -        // javaType, specType, registeredTypeName ...
    -        // tags ?
    +        // probably not: javaType, specType, registeredTypeName ...
    +        // maybe: tags ?
             return type;
         }
     
    -    /** Visitor adapter which can be used to ensure all kinds are supported */
    -    public static abstract class RegisteredTypeKindVisitor<T> {
    -        public T visit(RegisteredType type) {
    -            if (type==null) throw new NullPointerException("Registered type must not be null");
    -            if (type instanceof RegisteredSpecType) {
    -                return visitSpec((RegisteredSpecType)type);
    -            }
    -            // others go here
    -            throw new IllegalStateException("Unexpected registered type: "+type.getClass());
    -        }
    -
    -        protected abstract T visitSpec(RegisteredSpecType type);
    -        
    -        // TODO beans, others
    +    /** Preferred mechanism for defining a bean {@link RegisteredType} */
    +    public static RegisteredType bean(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.BEAN, symbolicName, version, plan), superType);
         }
         
    -    public static RegisteredTypeKind getKindOf(RegisteredType type) {
    -        return new RegisteredTypeKindVisitor<RegisteredTypeKind>() {
    -            @Override protected RegisteredTypeKind visitSpec(RegisteredSpecType type) { return RegisteredTypeKind.SPEC; }
    -        }.visit(type);
    +    public static RegisteredType spec(String symbolicName, String version, TypeImplementationPlan plan, @Nullable Class<?> superType) {
    +        return addSuperType(new BasicRegisteredType(RegisteredTypeKind.SPEC, symbolicName, version, plan), superType);
         }
    -    
    -    public abstract static class AbstractRegisteredType implements RegisteredType {
     
    -        final String symbolicName;
    -        final String version;
    +    /** returns the {@link Class} object corresponding to the given java type name,
    +     * using the cache on the type and the loader defined on the type
    +     * @param mgmt */
    +    @Beta
    +    // TODO should this be on the AbstractTypePlanTransformer ?
    +    public static Class<?> loadActualJavaType(String javaTypeName, ManagementContext mgmt, RegisteredType type, RegisteredTypeLoadingContext context) throws Exception {
    +        Class<?> result = ((BasicRegisteredType)type).getCache().get(ACTUAL_JAVA_TYPE);
    +        if (result!=null) return result;
             
    -        List<OsgiBundleWithUrl> bundles;
    -        String displayName;
    -        String description;
    -        String iconUrl;
    -        boolean deprecated;
    -        boolean disabled;
    -
    -        // TODO ensure this is re-populated on rebind
    -        transient Class<?> javaType;
    +        result = CatalogUtils.newClassLoadingContext(mgmt, type, context==null ? null : context.getLoader()).loadClass( javaTypeName );
    +        Preconditions.checkNotNull(result, "Could not load class "+javaTypeName+"; returned null (should have thrown a different exception!)");
             
    -        public AbstractRegisteredType(String symbolicName, String version, Class<?> javaType) {
    -            this.symbolicName = symbolicName;
    -            this.version = version;
    -            this.javaType = javaType;
    -        }
    -
    -        @Override
    -        public String getId() {
    -            return symbolicName + (version!=null ? ":"+version : "");
    -        }
    +        ((BasicRegisteredType)type).getCache().put(ACTUAL_JAVA_TYPE, result);
    +        return result;
    +    }
     
    -        @Override
    -        public String getSymbolicName() {
    -            return symbolicName;
    +    @Beta
    +    public static RegisteredType addSuperType(RegisteredType type, @Nullable Class<?> superType) {
    +        if (superType!=null) {
    +            ((BasicRegisteredType)type).superTypes.add(superType);
             }
    +        return type;
    +    }
     
    -        @Override
    -        public String getVersion() {
    -            return version;
    -        }
    -        
    -        @Override
    -        public Collection<OsgiBundleWithUrl> getLibraries() {
    -            return bundles;
    +    @Beta
    +    public static RegisteredType addSuperType(RegisteredType type, @Nullable RegisteredType superType) {
    +        if (superType!=null) {
    +            if (isSubTypeOf(superType, type)) {
    +                throw new IllegalStateException(superType+" declares "+type+" as a supertype; cannot set "+superType+" as a supertype of "+type);
    +            }
    +            ((BasicRegisteredType)type).superTypes.add(superType);
             }
    +        return type;
    +    }
     
    -        @Override
    -        public String getDisplayName() {
    -            return displayName;
    -        }
    +    /** returns the implementation data for a spec if it is a string (e.g. plan yaml or java class name); else false */
    --- End diff --
    
    ; else null


---
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: Flesh out type creation using the...

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/1017#discussion_r45049038
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/TestToSpecTransformer.java ---
    @@ -20,43 +20,36 @@
     
     import java.util.Map;
     import java.util.Set;
    -import java.util.concurrent.ConcurrentHashMap;
     
    +import org.apache.brooklyn.api.catalog.BrooklynCatalog;
     import org.apache.brooklyn.api.catalog.CatalogItem;
     import org.apache.brooklyn.api.entity.Application;
     import org.apache.brooklyn.api.entity.EntitySpec;
     import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
     import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
     import org.apache.brooklyn.core.plan.PlanNotRecognizedException;
     import org.apache.brooklyn.core.plan.PlanToSpecTransformer;
    -import org.apache.brooklyn.util.text.Identifiers;
    +import org.apache.brooklyn.core.typereg.JavaClassNameTypePlanTransformer;
    +import org.apache.brooklyn.core.typereg.RegisteredTypeLoadingContexts;
    +import org.apache.brooklyn.util.collections.MutableSet;
     import org.apache.brooklyn.util.yaml.Yamls;
     
    +import com.google.common.collect.Iterables;
    +
     /**
    - * Resolves previously registered specs by id.
    - * First create a spec and register it, keeping the returned ID:
    - * <pre> {@code
    - * String specId = TestToSpecTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    - * }</pre>
    - *
    - * Then build a plan to be resolved such as:
    - * <pre> {@code
    - *  brooklyn.catalog:
    - *    id: test.inputs
    - *    version: 0.0.1
    - *    item: <specId>
    - * } </pre>
    + * For use in conjunction with {@link StaticTypePlanTransformer};
    + * this will lookup an item by ID or in a map "type: id".
    + * <p>
    + * Should be removed when catalog is able to function using new-style lookups.
      */
     public class TestToSpecTransformer implements PlanToSpecTransformer {
    --- End diff --
    
    It's a good exercise to make this work.  Should be cleaner when we're completely migrated.


---
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: Flesh out type creation using the...

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/1017#discussion_r45051510
  
    --- Diff: utils/common/src/main/java/org/apache/brooklyn/util/text/Identifiers.java ---
    @@ -26,6 +26,9 @@
         
         public static final String JAVA_GOOD_START_CHARS = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz_";
         public static final String JAVA_GOOD_NONSTART_CHARS = JAVA_GOOD_START_CHARS+"1234567890";
    +    public static final String JAVA_SEGMENT_REGEX = "["+JAVA_GOOD_START_CHARS+"]"+"["+JAVA_GOOD_NONSTART_CHARS+"]*";
    +    public static final String JAVA_PACKAGE_OR_CLASS_REGEX = "("+JAVA_SEGMENT_REGEX+"\\."+")*"+JAVA_SEGMENT_REGEX;
    +    public static final String JAVA_BINARY_REGEX = JAVA_PACKAGE_OR_CLASS_REGEX+"(\\$["+JAVA_GOOD_NONSTART_CHARS+"]+)*";
    --- End diff --
    
    agree, added the word `GOOD` and 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: Flesh out type creation using the...

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/1017#discussion_r45042299
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/AbstractTypePlanTransformer.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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.typereg;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.javalang.JavaClassNames;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Convenience supertype for {@link BrooklynTypePlanTransformer} instances.
    + */
    +public abstract class AbstractTypePlanTransformer implements BrooklynTypePlanTransformer {
    +
    +    private static final Logger log = LoggerFactory.getLogger(AbstractTypePlanTransformer.class);
    +    
    +    protected ManagementContext mgmt;
    +
    +    @Override
    +    public void injectManagementContext(ManagementContext mgmt) {
    +        this.mgmt = mgmt;
    +    }
    +
    +    private final String format;
    +    private final String formatName;
    +    private final String formatDescription;
    +    
    +    protected AbstractTypePlanTransformer(String format, String formatName, String formatDescription) {
    +        this.format = format;
    +        this.formatName = formatName;
    +        this.formatDescription = formatDescription;
    +    }
    +    
    +    @Override
    +    public String getFormatCode() {
    +        return format;
    +    }
    +
    +    @Override
    +    public String getFormatName() {
    +        return formatName;
    +    }
    +
    +    @Override
    +    public String getFormatDescription() {
    +        return formatDescription;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return getFormatCode()+":"+JavaClassNames.simpleClassName(this);
    +    }
    +    
    +    @Override
    +    public double scoreForType(RegisteredType type, RegisteredTypeLoadingContext context) {
    +        if (getFormatCode().equals(type.getPlan().getPlanFormat())) return 1;
    +        if (type.getPlan().getPlanFormat()==null)
    +            return scoreForNullFormat(type.getPlan().getPlanData(), type, context);
    +        else
    +            return scoreForNonmatchingNonnullFormat(type.getPlan().getPlanFormat(), type.getPlan().getPlanData(), type, context);
    +    }
    +
    +    protected abstract double scoreForNullFormat(Object planData, RegisteredType type, RegisteredTypeLoadingContext context);
    +    protected abstract double scoreForNonmatchingNonnullFormat(String planFormat, Object planData, RegisteredType type, RegisteredTypeLoadingContext context);
    +
    +    /** delegates to more specific abstract create methods,
    +     * and performs common validation and customisation of the items created.
    +     * <p>
    +     * this includes:
    +     * <li> setting the {@link AbstractBrooklynObjectSpec#catalogItemId(String)}
    +     */
    +    @Override
    +    public Object create(final RegisteredType type, final RegisteredTypeLoadingContext context) {
    +        try {
    +            return validate(new RegisteredTypeKindVisitor<Object>() {
    +                @Override protected Object visitSpec(RegisteredType type) {
    +                    try { 
    +                        AbstractBrooklynObjectSpec<?, ?> result = createSpec(type, context);
    +                        result.catalogItemId(type.getId());
    +                        return result;
    +                    } catch (Exception e) { throw Exceptions.propagate(e); }
    +                }
    +                @Override protected Object visitBean(RegisteredType type) {
    +                    try { 
    +                        return createBean(type, context);
    +                    } catch (Exception e) { throw Exceptions.propagate(e); }
    +                }
    +                
    +            }.visit(type), type, context);
    +        } catch (UnsupportedTypePlanException e) {
    +            // no logging
    +            throw Exceptions.propagate(e);
    --- End diff --
    
    correct.  have combined, with type check to keep same logging behaviour.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44550928
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/BrooklynTypePlanTransformer.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.List;
    +import java.util.ServiceLoader;
    +
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry.RegisteredTypeKind;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
    +
    +/**
    + * Interface for use by schemes which with to be able to transform plans.
    + * <p>
    + * To add a new plan transformation scheme, simply create an implementation and declare it
    + * as a java service (cf {@link ServiceLoader}).
    + * <p>
    + * Implementations may wish to extend {@link AbstractTypePlanTransformer} which simplifies the process.
    + */
    +public interface BrooklynTypePlanTransformer extends ManagementContextInjectable {
    +
    +    /** @return a code to identify type implementations created specifying the use of this plan transformer. */
    --- End diff --
    
    Sounds too twisted to me. Maybe something like "An identifier for the transformer. Can be used by `RegisteredType` instances to target a specific transformer".


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44654399
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/api/AssemblyTemplateSpecInstantiator.java ---
    @@ -23,19 +23,23 @@
     
     import org.apache.brooklyn.api.entity.Application;
     import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
     import org.apache.brooklyn.camp.CampPlatform;
     import org.apache.brooklyn.camp.spi.AssemblyTemplate;
     import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
    -import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
     
     public interface AssemblyTemplateSpecInstantiator extends AssemblyTemplateInstantiator {
     
    +    @Deprecated /** @deprecaed since 0.9.0 include encountered types */
    +    EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader);
    +    
         /**
          * Gets the single item returned by {@link #createServiceSpecs}
          * and wraps it in an Application if needed, applying top-level
          * attributes and locations to the root entity.
          */
    -    EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader);
    +    EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader, Set<String> encounteredCatalogTypes);
    --- End diff --
    
    Doesn't look right, aren't application specs always top level?
    
    On another note, I haven't removed this method just for backwards compatibility. It belongs to an utils class with the `createServiceSpecs` doing the heavy lifting.


---
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: Flesh out type creation using the...

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/1017#discussion_r45048688
  
    --- Diff: core/src/test/java/org/apache/brooklyn/core/catalog/internal/SpecParameterInMetaTest.java ---
    @@ -43,18 +49,40 @@
     public class SpecParameterInMetaTest {
         private ManagementContext mgmt;
         private BrooklynCatalog catalog;
    -    private String spec;
    +    private String specId;
     
         @BeforeMethod(alwaysRun=true)
         public void setUp() {
             mgmt = LocalManagementContextForTests.newInstanceWithOsgi();
             catalog = mgmt.getCatalog();
    -        spec = TestToSpecTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
    +        StaticTypePlanTransformer.forceInstall();
    +        PlanToSpecFactory.forceAvailable(TestToSpecTransformer.class, JavaCatalogToSpecTransformer.class);
    +        specId = StaticTypePlanTransformer.registerSpec(EntitySpec.create(BasicEntity.class));
         }
     
    +    @AfterMethod(alwaysRun=true)
    +    public void tearDown() {
    +        StaticTypePlanTransformer.clearForced();
    +        PlanToSpecFactory.clearForced();
    +    }
    +
    +    @Test
    +    public void testCanRetrieveWithNew() {
    +        AbstractBrooklynObjectSpec<?, ?> spec = mgmt.getTypeRegistry().createSpecFromPlan(null, specId, null, null);
    +        Assert.assertNotNull(spec);
    +    }
    +
    +    // it's not actually added to the catalog; probably it would be cleaner if it is;
    --- End diff --
    
    I don't quite follow.  In general if a plan refers to a type, I'd expect that type in the catalog (or a java class).  While a transformer can define other rules for instantiating types, I'm not sure that's good practice.  (Except it's okay for tests.)
    
    (have captured this discussion in the code)


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44644972
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/TypePlanTransformers.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.typereg;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.TreeMap;
    +
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.PropagatedRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.Beta;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.collect.ArrayListMultimap;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Iterables;
    +import com.google.common.collect.Multimap;
    +
    +public class TypePlanTransformers {
    +
    +    private static final Logger log = LoggerFactory.getLogger(TypePlanTransformers.class);
    +
    +    private static Collection<BrooklynTypePlanTransformer> getAll() {
    +        return ImmutableList.copyOf(ServiceLoader.load(BrooklynTypePlanTransformer.class));
    +    }
    +
    +    private static Collection<Class<? extends BrooklynTypePlanTransformer>> OVERRIDE;
    +    @SafeVarargs
    +    @VisibleForTesting
    +    public synchronized static void forceAvailable(Class<? extends BrooklynTypePlanTransformer> ...classes) {
    --- End diff --
    
    I like the idea of tests not using the `ServiceLoader` mechanism. Suggest passing instances instead.


---
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: Flesh out type creation using the...

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/1017#discussion_r45041615
  
    --- Diff: usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampTypePlanTransformer.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.camp.brooklyn.spi.creation;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
    +import org.apache.brooklyn.api.typereg.RegisteredType;
    +import org.apache.brooklyn.api.typereg.RegisteredType.TypeImplementationPlan;
    +import org.apache.brooklyn.api.typereg.RegisteredTypeLoadingContext;
    +import org.apache.brooklyn.core.typereg.AbstractCustomImplementationPlan;
    +import org.apache.brooklyn.core.typereg.AbstractTypePlanTransformer;
    +import org.apache.brooklyn.core.typereg.BasicTypeImplementationPlan;
    +import org.apache.brooklyn.core.typereg.RegisteredTypes;
    +import org.apache.brooklyn.util.guava.Maybe;
    +
    +import com.google.common.collect.ImmutableList;
    +
    +public class CampTypePlanTransformer extends AbstractTypePlanTransformer {
    +
    +    private static final List<String> FORMATS = ImmutableList.of("brooklyn-camp", "camp", "brooklyn");
    --- End diff --
    
    agree, for 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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44548903
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/plan/PlanToSpecFactory.java ---
    @@ -36,12 +40,30 @@
     import com.google.common.base.Function;
     import com.google.common.collect.ImmutableList;
     
    +/** @deprecated since 0.9.0 use {@link TypePlanTransformers} as part of switch to {@link BrooklynTypePlanTransformer};
    + * mark transformers as deprecated if there is a preferred corresponding {@link BrooklynTypePlanTransformer} */
    +@Deprecated 
     public class PlanToSpecFactory {
         
         private static final Logger log = LoggerFactory.getLogger(PlanToSpecFactory.class);
     
    -    private static Collection<PlanToSpecTransformer> getAll() {
    -        return ImmutableList.copyOf(ServiceLoader.load(PlanToSpecTransformer.class));
    +    private static Collection<PlanToSpecTransformer> getAll(boolean includeDeprecated) {
    --- End diff --
    
    The `includeDeprecated` filter doesn't have good vibes :). It's all in deprecated functionality so no big deal, but still hope it doesn't stick for too long.


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r44641645
  
    --- Diff: core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypePredicates.java ---
    @@ -109,36 +109,46 @@ public boolean apply(@Nullable RegisteredType item) {
             }
         }
     
    -    public static <T> Predicate<RegisteredType> javaType(final Predicate<Class<T>> filter) {
    -        return new JavaTypeMatches(filter);
    +    public static <T> Predicate<RegisteredType> anySuperType(final Predicate<Class<T>> filter) {
    +        return new AnySuperTypeMatches(filter);
         }
         @SuppressWarnings({ "unchecked", "rawtypes" })
    -    public static Predicate<RegisteredType> javaTypeAssignableFrom(final Class<?> filter) {
    -        return javaType((Predicate)Predicates.assignableFrom(filter));
    +    public static Predicate<RegisteredType> assignableFrom(final Class<?> filter) {
    +        return anySuperType((Predicate)Predicates.assignableFrom(filter));
         }
         
    -    private static class JavaTypeMatches implements Predicate<RegisteredType> {
    +    private static class AnySuperTypeMatches implements Predicate<RegisteredType> {
             private final Predicate<Class<?>> filter;
             
             @SuppressWarnings({ "rawtypes", "unchecked" })
    -        private <T> JavaTypeMatches(Predicate filter) {
    +        private <T> AnySuperTypeMatches(Predicate filter) {
                 this.filter = filter;
             }
             @Override
             public boolean apply(@Nullable RegisteredType item) {
                 if (item==null) return false;
    -            return (item != null) && filter.apply(item.getJavaType());
    +            for (Object o: item.getSuperTypes()) {
    +                if (o instanceof Class) {
    +                    if (filter.apply((Class<?>)o)) return true;
    +                }
    +            }
    +            for (Object o: item.getSuperTypes()) {
    +                if (o instanceof RegisteredType) {
    +                    if (apply((RegisteredType)o)) return true;
    +                }
    +            }
    +            return false;
             }
         }
     
    -    public static final Predicate<RegisteredType> IS_APPLICATION = javaTypeAssignableFrom(Application.class);
    +    public static final Predicate<RegisteredType> IS_APPLICATION = assignableFrom(Application.class);
         // TODO do we need this?  introduced already deprecated in 0.9.0 so can be removed, or enabled
    --- End diff --
    
    +1 to remove


---
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: Flesh out type creation using the...

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

    https://github.com/apache/incubator-brooklyn/pull/1017#discussion_r45091680
  
    --- Diff: api/src/main/java/org/apache/brooklyn/api/typereg/RegisteredType.java ---
    @@ -38,19 +46,24 @@
         String getDescription();
         String getIconUrl();
     
    -    /** @return the java type or a supertype thereof that this registered type represents.
    +    /** @return all declared supertypes or super-interfaces of this registered type,
    +     * consisting of a collection of {@link Class} or {@link RegisteredType}
          * <p>
    -     * For beans, this is the type that the {@link BrooklynTypeRegistry} will create. 
    -     * For specs, this is what the spec that will be created points at 
    -     * (e.g. the concrete {@link Entity}, not the {@link EntitySpec});
    +     * This should normally include at least one {@link Class} object:
    +     * For beans, this should include the java type that the {@link BrooklynTypeRegistry} will create. 
    +     * For specs, this should refer to the {@link BrooklynObject} type that the created spec will point at 
    +     * (e.g. the concrete {@link Entity}, not the {@link EntitySpec}).
          * <p>
    -     * In some cases this may return an interface or a super-type of what will actually be created, 
    +     * This may not necessarily return the most specific java class or classes;
          * such as if the concrete type is private and callers should know only about a particular public interface,
          * or if precise type details are unavailable and all that is known at creation is some higher level interface/supertype
          * (e.g. this may return {@link Entity} even though the spec points at a specific subclass,
    -     * for instance because the YAML has not yet been parsed or OSGi bundles downloaded). 
    +     * for instance because the YAML has not yet been parsed or OSGi bundles downloaded).
    +     * <p>
    +     * This may include other registered types such as marker interfaces.
          */
    -    Class<?> getJavaType();
    +    @Beta
    +    @Nullable Set<Object> getSuperTypes();
    --- End diff --
    
    Any particular reason this is `@Nullable`? Feels better for it to be the opposite. Its callers already seem to assume otherwise (e.g. https://github.com/apache/incubator-brooklyn/pull/1017/files#diff-2c1cbcb42656df77efc28ab3153cb1e5R250).


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