You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by he...@apache.org on 2014/08/05 07:24:48 UTC

[04/10] git commit: Resolve classloading issues related to OSGi.

Resolve classloading issues related to OSGi.

The changes should make it impossible to have the same
class loaded from different classloaders resulting in linker
errors. For example a catalog item could load a class from
one of its linked bundles, while the same class would be available
on the app classpath where it was already loaded.

The solution is to make sure that a bundle is loaded exactly once.
While this is normally the case we were having issues with bundles
included in the boot classpath. We were exporting their packages
but the framework didn't know which bundle name:version they
belonged to, thus making it possible to load the same bundle from
external source.

Instead of including the exported packages of bundles on the
classpath to the system.packages.extra config, now for each bundle
on the classpath an extension bundle will be created and installed.


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

Branch: refs/heads/master
Commit: b2da61ebbf32cf0edfda70d507a9b847f98ca674
Parents: 9980803
Author: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Authored: Wed Jul 23 18:48:58 2014 +0300
Committer: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Committed: Tue Jul 29 10:56:42 2014 +0300

----------------------------------------------------------------------
 .../entity/proxying/InternalEntityFactory.java  |  14 +-
 .../brooklyn/management/ha/OsgiManager.java     |  23 +-
 .../main/java/brooklyn/util/ResourceUtils.java  |  37 ++--
 .../src/main/java/brooklyn/util/osgi/Osgis.java | 221 ++++++++++++++++---
 .../management/osgi/OsgiStandaloneTest.java     |  70 +++++-
 pom.xml                                         |   9 +
 .../brooklyn/util/maven/MavenRetriever.java     |   2 +-
 7 files changed, 312 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b2da61eb/core/src/main/java/brooklyn/entity/proxying/InternalEntityFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/proxying/InternalEntityFactory.java b/core/src/main/java/brooklyn/entity/proxying/InternalEntityFactory.java
index df73423..2081780 100644
--- a/core/src/main/java/brooklyn/entity/proxying/InternalEntityFactory.java
+++ b/core/src/main/java/brooklyn/entity/proxying/InternalEntityFactory.java
@@ -129,7 +129,6 @@ public class InternalEntityFactory {
     public <T extends Entity> T createEntityProxy(EntitySpec<T> spec, T entity) {
         // TODO Don't want the proxy to have to implement EntityLocal, but required by how 
         // AbstractEntity.parent is used (e.g. parent.getAllConfig)
-        ClassLoader classloader = (spec.getImplementation() != null ? spec.getImplementation() : spec.getType()).getClassLoader();
         MutableSet.Builder<Class<?>> builder = MutableSet.<Class<?>>builder()
                 .add(EntityProxy.class, Entity.class, EntityLocal.class, EntityInternal.class);
         if (spec.getType().isInterface()) {
@@ -139,15 +138,16 @@ public class InternalEntityFactory {
         }
         builder.addAll(spec.getAdditionalInterfaces());
         Set<Class<?>> interfaces = builder.build();
-        
-        // TODO OSGi strangeness! The classloader obtained from the type should be enough.
-        // If an OSGi class loader, it should delegate to find things like Entity.class etc.
-        // However, we get errors such as:
+
+        // When using the entity's classloader, we get errors such as:
         //    NoClassDefFoundError: brooklyn.event.AttributeSensor not found by io.brooklyn.brooklyn-test-osgi-entities
         // Building our own aggregating class loader gets around this.
-        // But we really should not have to do this! What are the consequences?
+        //
+        // The reason for the error is that the proxy tries to load all classes
+        // referenced from the entity and its interfaces with the single passed loader
+        // while a normal class loading would nest the class loaders (loading interfaces'
+        // references with their own class loaders which in our case are different).
         AggregateClassLoader aggregateClassLoader =  AggregateClassLoader.newInstanceWithNoLoaders();
-        aggregateClassLoader.addFirst(classloader);
         aggregateClassLoader.addFirst(entity.getClass().getClassLoader());
         for(Class<?> iface : interfaces) {
             aggregateClassLoader.addLast(iface.getClassLoader());

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b2da61eb/core/src/main/java/brooklyn/management/ha/OsgiManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/management/ha/OsgiManager.java b/core/src/main/java/brooklyn/management/ha/OsgiManager.java
index c672969..bfb2801 100644
--- a/core/src/main/java/brooklyn/management/ha/OsgiManager.java
+++ b/core/src/main/java/brooklyn/management/ha/OsgiManager.java
@@ -108,11 +108,22 @@ public class OsgiManager {
                 if (bundleNameVersion==null) {
                     bundleNameVersion = bundleUrlOrNameVersionString;
                 }
-                
+
                 Maybe<Bundle> bundle = Osgis.getBundle(framework, bundleNameVersion);
                 if (bundle.isPresent()) {
-                    @SuppressWarnings("unchecked")
-                    Class<T> clazz = (Class<T>) bundle.get().loadClass(type);
+                    Bundle b = bundle.get();
+                    Class<T> clazz;
+                    //Extension bundles don't support loadClass.
+                    //Instead load from the app classpath.
+                    if (Osgis.isExtensionBundle(b)) {
+                        @SuppressWarnings("unchecked")
+                        Class<T> c = (Class<T>)Class.forName(type);
+                        clazz = c;
+                    } else {
+                        @SuppressWarnings("unchecked")
+                        Class<T> c = (Class<T>)b.loadClass(type);
+                        clazz = c;
+                    }
                     return Maybe.of(clazz);
                 } else {
                     bundleProblems.put(bundleUrlOrNameVersionString, new IllegalStateException("Unable to find bundle "+bundleUrlOrNameVersionString));
@@ -120,6 +131,12 @@ public class OsgiManager {
             } catch (Exception e) {
                 Exceptions.propagateIfFatal(e);
                 bundleProblems.put(bundleUrlOrNameVersionString, e);
+
+                Throwable cause = e.getCause();
+                if (cause != null && cause.getMessage().contains("Unresolved constraint in bundle")) {
+                    log.warn("Unresolved constraint resolving OSGi bundle "+bundleUrlOrNameVersionString+" to load "+type+": "+cause.getMessage());
+                    if (log.isDebugEnabled()) log.debug("Trace for OSGi resolution failure", e);
+                }
             }
         }
         return Maybe.absent("Unable to resolve class "+type+": "+bundleProblems);

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b2da61eb/core/src/main/java/brooklyn/util/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/util/ResourceUtils.java b/core/src/main/java/brooklyn/util/ResourceUtils.java
index 5aacfb3..02526c7 100644
--- a/core/src/main/java/brooklyn/util/ResourceUtils.java
+++ b/core/src/main/java/brooklyn/util/ResourceUtils.java
@@ -440,9 +440,27 @@ public class ResourceUtils {
     
     public String getClassLoaderDir(String resourceInThatDir) {
         resourceInThatDir = Strings.removeFromStart(resourceInThatDir, "/");
-        URL url = getLoader().getResource(resourceInThatDir);
-        if (url==null) throw new NoSuchElementException("Resource ("+resourceInThatDir+") not found");
+        URL resourceUrl = getLoader().getResource(resourceInThatDir);
+        if (resourceUrl==null) throw new NoSuchElementException("Resource ("+resourceInThatDir+") not found");
 
+        URL containerUrl = getContainerUrl(resourceUrl, resourceInThatDir);
+
+        if (!"file".equals(containerUrl.getProtocol())) throw new IllegalStateException("Resource ("+resourceInThatDir+") not on file system (at "+containerUrl+")");
+
+        //convert from file: URL to File
+        File file;
+        try {
+            file = new File(containerUrl.toURI());
+        } catch (URISyntaxException e) {
+            throw new IllegalStateException("Resource ("+resourceInThatDir+") found at invalid URI (" + containerUrl + ")", e);
+        }
+        
+        if (!file.exists()) throw new IllegalStateException("Context class url substring ("+containerUrl+") not found on filesystem");
+        return file.getPath();
+        
+    }
+
+    public static URL getContainerUrl(URL url, String resourceInThatDir) {
         //Switching from manual parsing of jar: and file: URLs to java provided functionality.
         //The old code was breaking on any Windows path and instead of fixing it, using
         //the provided Java APIs seemed like the better option since they are already tested
@@ -472,20 +490,7 @@ public class ResourceUtils {
                 throw new IllegalStateException("Resource ("+resourceInThatDir+") found at invalid URL parent (" + parent + ")", e);
             }
         }
-        
-        if (!"file".equals(url.getProtocol())) throw new IllegalStateException("Resource ("+resourceInThatDir+") not on file system (at "+url+")");
-
-        //convert from file: URL to File
-        File file;
-        try {
-            file = new File(url.toURI());
-        } catch (URISyntaxException e) {
-            throw new IllegalStateException("Resource ("+resourceInThatDir+") found at invalid URI (" + url + ")", e);
-        }
-        
-        if (!file.exists()) throw new IllegalStateException("Context class url substring ("+url+") not found on filesystem");
-        return file.getPath();
-        
+        return url;
     }
     
     /** @deprecated since 0.7.0 use {@link Streams#readFullyString(InputStream) */ @Deprecated

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b2da61eb/core/src/main/java/brooklyn/util/osgi/Osgis.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/util/osgi/Osgis.java b/core/src/main/java/brooklyn/util/osgi/Osgis.java
index 24a1360..c89ae54 100644
--- a/core/src/main/java/brooklyn/util/osgi/Osgis.java
+++ b/core/src/main/java/brooklyn/util/osgi/Osgis.java
@@ -19,17 +19,22 @@
 package brooklyn.util.osgi;
 
 import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.Enumeration;
-import java.util.Iterator;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.jar.Attributes;
+import java.util.jar.JarInputStream;
+import java.util.jar.JarOutputStream;
 import java.util.jar.Manifest;
 
 import javax.annotation.Nullable;
@@ -38,6 +43,7 @@ import org.apache.felix.framework.FrameworkFactory;
 import org.apache.felix.framework.util.StringMap;
 import org.apache.felix.framework.util.manifestparser.ManifestParser;
 import org.osgi.framework.Bundle;
+import org.osgi.framework.BundleContext;
 import org.osgi.framework.BundleException;
 import org.osgi.framework.Constants;
 import org.osgi.framework.Version;
@@ -52,13 +58,15 @@ import brooklyn.util.collections.MutableList;
 import brooklyn.util.collections.MutableMap;
 import brooklyn.util.exceptions.Exceptions;
 import brooklyn.util.guava.Maybe;
+import brooklyn.util.net.Urls;
+import brooklyn.util.os.Os;
 import brooklyn.util.stream.Streams;
+import brooklyn.util.text.Strings;
 
 import com.google.common.annotations.Beta;
 import com.google.common.base.Joiner;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
-import com.google.common.collect.Iterators;
 
 /** 
  * utilities for working with osgi.
@@ -68,10 +76,11 @@ import com.google.common.collect.Iterators;
  * @since 0.7.0  */
 @Beta
 public class Osgis {
-
-    private static final String BROOKLYN_PACKAGE_PREFIX = "brooklyn.";
     private static final Logger LOG = LoggerFactory.getLogger(Osgis.class);
 
+    private static final String EXTENSION_PROTOCOL = "system";
+    private static final String MANIFEST_PATH = "META-INF/MANIFEST.MF";
+
     public static List<Bundle> getBundlesByName(Framework framework, String symbolicName, Predicate<Version> versionMatcher) {
         List<Bundle> result = MutableList.of();
         for (Bundle b: framework.getBundleContext().getBundles()) {
@@ -157,77 +166,221 @@ public class Osgis {
         Map<Object,Object> cfg = MutableMap.copyOf(extraStartupConfig);
         if (clean) cfg.put(Constants.FRAMEWORK_STORAGE_CLEAN, "onFirstInit");
         if (felixCacheDir!=null) cfg.put(Constants.FRAMEWORK_STORAGE, felixCacheDir);
-        cfg.put(Constants.FRAMEWORK_SYSTEMPACKAGES_EXTRA, getBrooklynBootBundles());
         FrameworkFactory factory = newFrameworkFactory();
-        
+
+        long frameworkInitStart = System.currentTimeMillis();
         Framework framework = factory.newFramework(cfg);
         try {
             framework.init();
-            // nothing needs auto-loading, currently (and this needs a new dependency)
-            // AutoProcessor.process(configProps, m_fwk.getBundleContext());
+            installBootBundles(framework);
             framework.start();
         } catch (Exception e) {
             // framework bundle start exceptions are not interesting to caller...
             throw Exceptions.propagate(e);
         }
+        long frameworkInitEnd = System.currentTimeMillis();
+        double frameworkInitSeconds = ((double)frameworkInitEnd - frameworkInitStart) / 1000;
+        LOG.info("OSGi framework started in " + Strings.makeRealString(frameworkInitSeconds, -1, 2, -1) + " seconds.");
+
         return framework;
     }
 
-    private static String getBrooklynBootBundles() {
+    private static void installBootBundles(Framework framework) {
         Enumeration<URL> resources;
         try {
-            resources = Osgis.class.getClassLoader().getResources("META-INF/MANIFEST.MF");
+            resources = Osgis.class.getClassLoader().getResources(MANIFEST_PATH);
         } catch (IOException e) {
             throw Exceptions.propagate(e);
         }
-        
-        Collection<String> exportPackages = new ArrayList<String>();
+        BundleContext bundleContext = framework.getBundleContext();
+        Map<String, Bundle> installedBundles = getInstalledBundles(bundleContext);
         while(resources.hasMoreElements()) {
             URL url = resources.nextElement();
-            exportPackages.addAll(getBundleExportedPackages(url));
+            installExtensionBundle(bundleContext, url, installedBundles, getVersionedId(framework));
         }
+    }
 
-        Iterator<String> brooklynPackages = Iterators.filter(exportPackages.iterator(), new Predicate<String>() {
-            @Override
-            public boolean apply(String input) {
-                return input.startsWith(BROOKLYN_PACKAGE_PREFIX);
-            }
-        });
-        
-        String bootBundles = Joiner.on(",").join(brooklynPackages);
-        LOG.debug("Found the following boot OSGi packages: " + bootBundles);
-        return bootBundles;
+    private static Map<String, Bundle> getInstalledBundles(BundleContext bundleContext) {
+        Map<String, Bundle> installedBundles = new HashMap<String, Bundle>();
+        Bundle[] bundles = bundleContext.getBundles();
+        for (Bundle b : bundles) {
+            installedBundles.put(getVersionedId(b), b);
+        }
+        return installedBundles;
     }
 
-    private static Collection<String> getBundleExportedPackages(URL manifestUrl) {
+    private static void installExtensionBundle(BundleContext bundleContext, URL manifestUrl, Map<String, Bundle> installedBundles, String frameworkVersionedId) {
+        //ignore http://felix.extensions:9/ system entry
+        if("felix.extensions".equals(manifestUrl.getHost())) return;
+
         try {
-            ManifestHelper helper = ManifestHelper.forManifest(manifestUrl);
-            return helper.getExportedPackages();
+            Manifest manifest = readManifest(manifestUrl);
+            if (!isValidBundle(manifest)) return;
+            String versionedId = getVersionedId(manifest);
+            URL bundleUrl = ResourceUtils.getContainerUrl(manifestUrl, MANIFEST_PATH);
+
+            Bundle existingBundle = installedBundles.get(versionedId);
+            if (existingBundle != null) {
+                if (!bundleUrl.equals(existingBundle.getLocation()) &&
+                        //the framework bundle is always pre-installed, don't display duplicate info
+                        !versionedId.equals(frameworkVersionedId)) {
+                    LOG.info("Ignoring duplicate " + versionedId + " from manifest " + manifestUrl + ", already loaded from " + existingBundle.getLocation());
+                }
+                return;
+            }
+            
+            byte[] jar = buildExtensionBundle(manifest);
+            LOG.debug("Installing boot bundle " + bundleUrl);
+            //mark the bundle as extension so we can detect it later using the "system:" protocol
+            //(since we cannot access BundleImpl.isExtension)
+            Bundle newBundle = bundleContext.installBundle(EXTENSION_PROTOCOL + ":" + bundleUrl.toString(), new ByteArrayInputStream(jar));
+            installedBundles.put(versionedId, newBundle);
         } catch (IOException e) {
             LOG.warn("Unable to load manifest from " + manifestUrl + ", ignoring.", e);
         } catch (BundleException e) {
             LOG.warn("Unable to load manifest from " + manifestUrl + ", ignoring.", e);
         }
-        return Collections.emptyList();
     }
 
+    private static Manifest readManifest(URL manifestUrl) throws IOException {
+        Manifest manifest;
+        InputStream in = null;
+        try {
+            in = manifestUrl.openStream();
+            manifest = new Manifest(in);
+        } finally {
+            if (in != null) {
+                try {in.close();} 
+                catch (Exception e) {};
+            }
+        }
+        return manifest;
+    }
+
+    private static byte[] buildExtensionBundle(Manifest manifest) throws IOException {
+        Attributes atts = manifest.getMainAttributes();
+
+        //the following properties are invalid in extension bundles
+        atts.remove(new Attributes.Name(Constants.IMPORT_PACKAGE));
+        atts.remove(new Attributes.Name(Constants.REQUIRE_BUNDLE));
+        atts.remove(new Attributes.Name(Constants.BUNDLE_NATIVECODE));
+        atts.remove(new Attributes.Name(Constants.DYNAMICIMPORT_PACKAGE));
+        atts.remove(new Attributes.Name(Constants.BUNDLE_ACTIVATOR));
+        
+        //mark as extension bundle
+        atts.putValue(Constants.FRAGMENT_HOST, "system.bundle; extension:=framework");
+
+        //create the jar containing the manifest
+        ByteArrayOutputStream jar = new ByteArrayOutputStream();
+        JarOutputStream out = new JarOutputStream(jar, manifest);
+        out.close();
+        return jar.toByteArray();
+    }
+
+    private static boolean isValidBundle(Manifest manifest) {
+        Attributes atts = manifest.getMainAttributes();
+        return atts.containsKey(new Attributes.Name(Constants.BUNDLE_MANIFESTVERSION));
+    }
+
+    private static String getVersionedId(Bundle b) {
+        return b.getSymbolicName() + ":" + b.getVersion();
+    }
+
+    private static String getVersionedId(Manifest manifest) {
+        Attributes atts = manifest.getMainAttributes();
+        return atts.getValue(Constants.BUNDLE_SYMBOLICNAME) + ":" +
+            atts.getValue(Constants.BUNDLE_VERSION);
+    }
 
     /**
      * Installs a bundle from the given URL, doing a check if already installed, and
      * using the {@link ResourceUtils} loader for this project (brooklyn core)
      */
     public static Bundle install(Framework framework, String url) throws BundleException {
+        boolean isLocal = isLocalUrl(url);
+        String localUrl = url;
+        if (!isLocal) {
+            localUrl = cacheFile(url);
+        }
+
+        try {
+            Bundle bundle = getInstalledBundle(framework, localUrl);
+            if (bundle != null) {
+                return bundle;
+            }
+    
+            // use our URL resolution so we get classpath items
+            LOG.debug("Installing bundle into {} from url: {}", framework, url);
+            InputStream stream = getUrlStream(localUrl);
+            Bundle installedBundle = framework.getBundleContext().installBundle(url, stream);
+            
+            return installedBundle;
+        } finally {
+            if (!isLocal) {
+                try {
+                    new File(new URI(localUrl)).delete();
+                } catch (URISyntaxException e) {
+                    throw Exceptions.propagate(e);
+                }
+            }
+        }
+    }
+
+    private static String cacheFile(String url) {
+        InputStream in = getUrlStream(url);
+        File cache = Os.writeToTempFile(in, "bundle-cache", "jar");
+        return cache.toURI().toString();
+    }
+
+    private static boolean isLocalUrl(String url) {
+        String protocol = Urls.getProtocol(url);
+        return "file".equals(protocol) ||
+                "classpath".equals(protocol) ||
+                "jar".equals(protocol);
+    }
+
+    private static Bundle getInstalledBundle(Framework framework, String url) {
         Bundle bundle = framework.getBundleContext().getBundle(url);
         if (bundle != null) {
             return bundle;
         }
 
-        // use our URL resolution so we get classpath items
-        LOG.debug("Installing bundle into {} from url: {}", framework, url);
-        InputStream stream = ResourceUtils.create(Osgis.class).getResourceFromUrl(url);
-        return framework.getBundleContext().installBundle(url, stream);
+        //Note that in OSGi 4.3+ it could be possible to have the same version installed
+        //multiple times in more advanced scenarios. In our case we don't support it.
+        
+        //Felix already assumes the stream is pointing to a Jar
+        JarInputStream stream;
+        try {
+            stream = new JarInputStream(getUrlStream(url));
+        } catch (IOException e) {
+            throw Exceptions.propagate(e);
+        }
+        Manifest manifest = stream.getManifest();
+        Streams.closeQuietly(stream);
+        String versionedId = getVersionedId(manifest);
+        for (Bundle installedBundle : framework.getBundleContext().getBundles()) {
+            if (versionedId.equals(getVersionedId(installedBundle))) {
+                return installedBundle;
+            }
+        }
+        return null;
     }
 
+    private static InputStream getUrlStream(String url) {
+        return ResourceUtils.create(Osgis.class).getResourceFromUrl(url);
+    }
+    
+    public static boolean isExtensionBundle(Bundle bundle) {
+        String location = bundle.getLocation();
+        return location != null && 
+                EXTENSION_PROTOCOL.equals(Urls.getProtocol(location));
+    }
+
+
+    /**
+     * The class is not used, staying for future reference.
+     * Remove after OSGi transition is completed.
+     */
     public static class ManifestHelper {
         
         private static ManifestParser parse;

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b2da61eb/core/src/test/java/brooklyn/management/osgi/OsgiStandaloneTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/brooklyn/management/osgi/OsgiStandaloneTest.java b/core/src/test/java/brooklyn/management/osgi/OsgiStandaloneTest.java
index 1f55e26..6268c93 100644
--- a/core/src/test/java/brooklyn/management/osgi/OsgiStandaloneTest.java
+++ b/core/src/test/java/brooklyn/management/osgi/OsgiStandaloneTest.java
@@ -38,8 +38,18 @@ import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import brooklyn.entity.Entity;
+import brooklyn.entity.basic.Entities;
+import brooklyn.entity.proxying.EntitySpec;
+import brooklyn.entity.proxying.InternalEntityFactory;
+import brooklyn.entity.proxying.InternalPolicyFactory;
+import brooklyn.management.internal.ManagementContextInternal;
+import brooklyn.test.entity.LocalManagementContextForTests;
+import brooklyn.test.entity.TestApplicationImpl;
 import brooklyn.util.ResourceUtils;
 import brooklyn.util.collections.MutableSet;
+import brooklyn.util.maven.MavenArtifact;
+import brooklyn.util.maven.MavenRetriever;
 import brooklyn.util.os.Os;
 import brooklyn.util.osgi.Osgis;
 import brooklyn.util.osgi.Osgis.ManifestHelper;
@@ -56,9 +66,8 @@ import brooklyn.util.stream.Streams;
  *  */
 public class OsgiStandaloneTest {
 
-
     private static final Logger log = LoggerFactory.getLogger(OsgiStandaloneTest.class);
-    
+
     public static final String BROOKLYN_OSGI_TEST_A_0_1_0_URL = "classpath:/brooklyn/osgi/brooklyn-osgi-test-a_0.1.0.jar";
     
     public static final String BROOKLYN_TEST_OSGI_ENTITIES_PATH = "/brooklyn/osgi/brooklyn-test-osgi-entities.jar";
@@ -93,7 +102,7 @@ public class OsgiStandaloneTest {
             throw new IllegalStateException("test resources not available; may be an IDE issue, so try a mvn rebuild of this project", e);
         }
     }
-    
+
     @Test
     public void testInstallBundle() throws Exception {
         Bundle bundle = install(BROOKLYN_OSGI_TEST_A_0_1_0_URL);
@@ -101,6 +110,61 @@ public class OsgiStandaloneTest {
     }
 
     @Test
+    public void testBootBundle() throws Exception {
+        Bundle bundle = install(BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+        Class<?> bundleCls = bundle.loadClass("brooklyn.osgi.tests.SimpleEntity");
+        Assert.assertEquals(Entity.class,  bundle.loadClass(Entity.class.getName()));
+        Assert.assertEquals(Entity.class, bundleCls.getClassLoader().loadClass(Entity.class.getName()));
+    }
+
+    @Test
+    public void testDuplicateBundle() throws Exception {
+        helperDuplicateBundle(MavenRetriever.localUrl(new MavenArtifact("io.brooklyn", "brooklyn-api", "jar", "0.7.0-SNAPSHOT")));
+    }
+
+    @Test(groups="Integration")
+    public void testRemoteDuplicateBundle() throws Exception {
+        helperDuplicateBundle(MavenRetriever.hostedUrl(new MavenArtifact("io.brooklyn", "brooklyn-api", "jar", "0.7.0-SNAPSHOT")));
+    }
+
+    public void helperDuplicateBundle(String url) throws Exception {
+        //The bundle is already installed from the boot path.
+        //Make sure that we still get the initially loaded
+        //bundle after trying to install the same version.
+        Bundle bundle = install(url);
+        Assert.assertTrue(Osgis.isExtensionBundle(bundle));
+    }
+
+
+    /**
+     * Test fix for
+     * java.lang.NoClassDefFoundError: brooklyn.event.AttributeSensor not found by io.brooklyn.brooklyn-test-osgi-entities [41]
+     */
+    @Test
+    public void testEntityProxy() throws Exception {
+        ManagementContextInternal managementContext;
+        InternalEntityFactory factory;
+
+        managementContext = new LocalManagementContextForTests();
+        InternalPolicyFactory policyFactory = new InternalPolicyFactory(managementContext);
+        factory = new InternalEntityFactory(managementContext, managementContext.getEntityManager().getEntityTypeRegistry(), policyFactory);
+
+        Bundle bundle = install(BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+        @SuppressWarnings("unchecked")
+        Class<? extends Entity> bundleCls = (Class<? extends Entity>) bundle.loadClass("brooklyn.osgi.tests.SimpleEntityImpl");
+        @SuppressWarnings("unchecked")
+        Class<? extends Entity> bundleInterface = (Class<? extends Entity>) bundle.loadClass("brooklyn.osgi.tests.SimpleEntity");
+
+        TestApplicationImpl app = new TestApplicationImpl();
+        @SuppressWarnings("unchecked")
+        EntitySpec<Entity> spec = (EntitySpec<Entity>) (((EntitySpec<Entity>)EntitySpec.create(bundleInterface))).impl(bundleCls);
+        Entity entity = bundleCls.newInstance();
+        factory.createEntityProxy(spec, entity);
+
+        if (managementContext != null) Entities.destroyAll(managementContext);
+    }
+
+    @Test
     public void testAMultiplier() throws Exception {
         Bundle bundle = install(BROOKLYN_OSGI_TEST_A_0_1_0_URL);
         checkMath(bundle, 3, 6);

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b2da61eb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 19b3259..13855c6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -937,6 +937,15 @@
                             </supportedProjectTypes>
                             <instructions>
                                 <!-- OSGi specific instruction -->
+                                <!--
+                                    By default packages containing impl and internal
+                                    are not included in the export list. Setting an
+                                    explicit wildcard will include all packages
+                                    regardless of name.
+                                    In time we should minimize our export lists to
+                                    what is really needed.
+                                -->
+                                <Export-Package>brooklyn.*</Export-Package>
                             </instructions>
                         </configuration>
                     </plugin>

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b2da61eb/utils/common/src/main/java/brooklyn/util/maven/MavenRetriever.java
----------------------------------------------------------------------
diff --git a/utils/common/src/main/java/brooklyn/util/maven/MavenRetriever.java b/utils/common/src/main/java/brooklyn/util/maven/MavenRetriever.java
index 5092285..9753b19 100644
--- a/utils/common/src/main/java/brooklyn/util/maven/MavenRetriever.java
+++ b/utils/common/src/main/java/brooklyn/util/maven/MavenRetriever.java
@@ -86,7 +86,7 @@ public class MavenRetriever {
     /** returns a URL for accessing the given artifact, preferring a local file if available,
      * else generating a hosted URL (but not checking) */
     public String getLocalUrl(MavenArtifact artifact) {
-        if (isInstalledLocally(artifact)) return "file://"+getLocalPath(artifact);
+        if (isInstalledLocally(artifact)) return new File(getLocalPath(artifact)).toURI().toString();
         if (artifact.isSnapshot()) return snapshotUrlGenerator.apply(artifact);
         else return releaseUrlGenerator.apply(artifact);
     }