You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sling.apache.org by da...@apache.org on 2020/09/01 16:33:20 UTC

[sling-org-apache-sling-installer-factory-feature] 29/32: SLING-9658 Support binary archives in .far files

This is an automated email from the ASF dual-hosted git repository.

davidb pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/sling-org-apache-sling-installer-factory-feature.git

commit 34e639d625746627ec81e34877e5cf8b36317841
Author: David Bosschaert <da...@apache.org>
AuthorDate: Tue Sep 1 14:52:41 2020 +0100

    SLING-9658 Support binary archives in .far files
    
    New extension handler service API added.
    InstallFeatureModelTask that uses these services to handle extensions.
    Additionalservices are provided by the unpack extension to provide binary support.
---
 bnd.bnd                                            |   2 +-
 pom.xml                                            |  31 +++-
 .../feature/spi/context/ExtensionHandler.java      |  37 +++++
 .../spi/context/ExtensionHandlerContext.java       |  61 +++++++
 .../sling/feature/spi/context/package-info.java    |  22 +++
 .../model/impl/APIRegionsExtensionHandler.java     |  74 +++++++++
 .../model/impl/FeatureModelInstallerPlugin.java    |  10 +-
 .../model/impl/InstallFeatureModelTask.java        | 164 ++++++++++++-------
 .../model/impl/RepoinitExtensionHandler.java       |  51 ++++++
 .../model/impl/APIRegionsExtensionHandlerTest.java |  71 +++++++++
 .../model/impl/InstallFeatureModelTaskTest.java    | 177 +++++++++++++++++++++
 .../model/impl/RepoinitExtensionHandlerTest.java   |  68 ++++++++
 src/test/resources/test2/test2.far                 | Bin 0 -> 103630 bytes
 src/test/resources/test2/test2.slingosgifeature    |  10 ++
 14 files changed, 714 insertions(+), 64 deletions(-)

diff --git a/bnd.bnd b/bnd.bnd
index d2c832f..3e6776b 100644
--- a/bnd.bnd
+++ b/bnd.bnd
@@ -1,2 +1,2 @@
 -includeresource: @org.apache.sling.feature.extension.apiregions-[0-9.]*.jar!/org/apache/sling/feature/extension/apiregions/launcher/LauncherProperties.*
-Conditional-Package: org.apache.sling.feature.extension.apiregions.api
+Conditional-Package: org.apache.sling.feature.extension.apiregions.api,org.apache.felix.utils.manifest
diff --git a/pom.xml b/pom.xml
index 507b9fa..ff8c46a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -50,6 +50,16 @@
                     <skip>true</skip>
                 </configuration>
             </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes>
+                        <exclude>src/test/resources/**/*.slingosgifeature</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+            
         </plugins>
     </build>
 
@@ -93,7 +103,7 @@
         <dependency>
             <groupId>org.apache.sling</groupId>
             <artifactId>org.apache.sling.feature</artifactId>
-            <version>1.2.0</version>
+            <version>1.2.4</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -102,6 +112,13 @@
             <version>1.1.4</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.felix</groupId>
+            <artifactId>org.apache.felix.utils</artifactId>
+            <version>1.11.4</version>
+            <scope>provided</scope>
+        </dependency>
+        
    <!-- Testing -->
         <dependency>
             <groupId>junit</groupId>
@@ -113,6 +130,18 @@
             <version>3.3.0</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.johnzon</groupId>
+            <artifactId>johnzon-core</artifactId>
+            <version>1.2.8</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.felix</groupId>
+            <artifactId>org.apache.felix.cm.json</artifactId>
+            <version>1.0.2</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
 </project>
diff --git a/src/main/java/org/apache/sling/feature/spi/context/ExtensionHandler.java b/src/main/java/org/apache/sling/feature/spi/context/ExtensionHandler.java
new file mode 100644
index 0000000..da4ad62
--- /dev/null
+++ b/src/main/java/org/apache/sling/feature/spi/context/ExtensionHandler.java
@@ -0,0 +1,37 @@
+/*
+ * 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.sling.feature.spi.context;
+
+import org.apache.sling.feature.Extension;
+import org.apache.sling.feature.Feature;
+
+/**
+ * Service interface to handle feature model extensions in the feature installer
+ */
+public interface ExtensionHandler {
+    /**
+     * Called on registered services when an extension is encountered
+     * @param context The extension context
+     * @param extension The extension to be handled
+     * @param feature The feature that contains the extension
+     * @return Returns {@code true} if this extension handler can handle the presented extension.
+     * @throws Exception If something goes wrong.
+     */
+    public boolean handle(ExtensionHandlerContext context, Extension extension, Feature feature) throws Exception;
+}
diff --git a/src/main/java/org/apache/sling/feature/spi/context/ExtensionHandlerContext.java b/src/main/java/org/apache/sling/feature/spi/context/ExtensionHandlerContext.java
new file mode 100644
index 0000000..384279e
--- /dev/null
+++ b/src/main/java/org/apache/sling/feature/spi/context/ExtensionHandlerContext.java
@@ -0,0 +1,61 @@
+/*
+ * 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.sling.feature.spi.context;
+
+import org.apache.sling.feature.ArtifactId;
+import org.apache.sling.feature.builder.ArtifactProvider;
+
+import java.net.URL;
+import java.util.Dictionary;
+import java.util.Map;
+
+/**
+ * This context is provided with calls to {@link ExtensionHandler} services.
+ */
+public interface ExtensionHandlerContext {
+    /**
+     * Add a bundle to be installed by the launcher.
+     * @param id The bundle's artifact ID
+     * @param startLevel The start level for the bundle.
+     * @param file The file with the bundle.
+     */
+    public void addBundle(ArtifactId id, URL file, Integer startLevel);
+
+    /**
+     * Add an artifact to be installed by the launcher
+     * @param id The artifact's ID
+     * @param url The url to the Artifact resource
+     * @param props Additional installation metadata
+     */
+    public void addInstallableArtifact(ArtifactId id, final URL url, final Map<String,Object> props);
+
+    /**
+     * Add a configuration to be installed by the launcher
+     * @param pid The pid
+     * @param factoryPid The factory pid
+     * @param properties The propertis
+     */
+    public void addConfiguration(final String pid, final String factoryPid, final Dictionary<String, Object> properties);
+
+    /**
+     * Obtain the artifact provider.
+     * @return The artifact provider.
+     */
+    public ArtifactProvider getArtifactProvider();
+}
diff --git a/src/main/java/org/apache/sling/feature/spi/context/package-info.java b/src/main/java/org/apache/sling/feature/spi/context/package-info.java
new file mode 100644
index 0000000..8d0d1e1
--- /dev/null
+++ b/src/main/java/org/apache/sling/feature/spi/context/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@org.osgi.annotation.versioning.ProviderType
+@org.osgi.annotation.versioning.Version("1.0.0")
+package org.apache.sling.feature.spi.context;
+
diff --git a/src/main/java/org/apache/sling/installer/factory/model/impl/APIRegionsExtensionHandler.java b/src/main/java/org/apache/sling/installer/factory/model/impl/APIRegionsExtensionHandler.java
new file mode 100644
index 0000000..619540d
--- /dev/null
+++ b/src/main/java/org/apache/sling/installer/factory/model/impl/APIRegionsExtensionHandler.java
@@ -0,0 +1,74 @@
+/*
+ * 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.sling.installer.factory.model.impl;
+
+import org.apache.sling.feature.Extension;
+import org.apache.sling.feature.ExtensionType;
+import org.apache.sling.feature.Feature;
+import org.apache.sling.feature.extension.apiregions.api.ApiRegions;
+import org.apache.sling.feature.extension.apiregions.launcher.LauncherProperties;
+import org.apache.sling.feature.spi.context.ExtensionHandler;
+import org.apache.sling.feature.spi.context.ExtensionHandlerContext;
+import org.osgi.service.component.annotations.Component;
+
+import java.util.ArrayList;
+import java.util.Dictionary;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+@Component
+public class APIRegionsExtensionHandler implements ExtensionHandler {
+    private static final String REGION_FACTORY_PID = "org.apache.sling.feature.apiregions.factory~";
+
+    private static final String PROP_idbsnver = "mapping.bundleid.bsnver";
+    private static final String PROP_bundleFeatures = "mapping.bundleid.features";
+    private static final String PROP_featureRegions = "mapping.featureid.regions";
+    private static final String PROP_regionPackage = "mapping.region.packages";
+
+    @Override
+    public boolean handle(ExtensionHandlerContext context, Extension extension, Feature feature) throws Exception {
+        if (!extension.getName().equals(ApiRegions.EXTENSION_NAME) ||
+                extension.getType() != ExtensionType.JSON) {
+            return false;
+        }
+
+        final ApiRegions regions = ApiRegions.parse(extension.getJSONStructure().asJsonArray());
+
+        final String configPid = REGION_FACTORY_PID.concat(feature.getId().toMvnName().replace('-', '_'));
+        final Dictionary<String, Object> props = new Hashtable<>();
+        props.put(PROP_idbsnver, convert(LauncherProperties.getBundleIDtoBSNandVersionMap(feature, context.getArtifactProvider())));
+        props.put(PROP_bundleFeatures, convert(LauncherProperties.getBundleIDtoFeaturesMap(feature)));
+        props.put(PROP_featureRegions, convert(LauncherProperties.getFeatureIDtoRegionsMap(regions)));
+        props.put(PROP_regionPackage, convert(LauncherProperties.getRegionNametoPackagesMap(regions)));
+
+        context.addConfiguration(null, configPid, props);
+        return true;
+    }
+
+    private String[] convert(final Properties props) {
+        final List<String> result = new ArrayList<>();
+
+        for(final Map.Entry<Object, Object> entry : props.entrySet()) {
+            result.add(entry.getKey().toString().concat("=").concat(entry.getValue().toString()));
+        }
+        return result.toArray(new String[result.size()]);
+    }
+}
diff --git a/src/main/java/org/apache/sling/installer/factory/model/impl/FeatureModelInstallerPlugin.java b/src/main/java/org/apache/sling/installer/factory/model/impl/FeatureModelInstallerPlugin.java
index eaec3c5..a3f5039 100644
--- a/src/main/java/org/apache/sling/installer/factory/model/impl/FeatureModelInstallerPlugin.java
+++ b/src/main/java/org/apache/sling/installer/factory/model/impl/FeatureModelInstallerPlugin.java
@@ -41,6 +41,7 @@ import org.apache.sling.feature.io.artifacts.ArtifactManager;
 import org.apache.sling.feature.io.artifacts.ArtifactManagerConfig;
 import org.apache.sling.feature.io.json.FeatureJSONReader;
 import org.apache.sling.feature.io.json.FeatureJSONWriter;
+import org.apache.sling.feature.spi.context.ExtensionHandler;
 import org.apache.sling.installer.api.InstallableResource;
 import org.apache.sling.installer.api.tasks.InstallTask;
 import org.apache.sling.installer.api.tasks.InstallTaskFactory;
@@ -53,6 +54,7 @@ import org.apache.sling.installer.api.tasks.TransformationResult;
 import org.osgi.framework.BundleContext;
 import org.osgi.service.component.annotations.Activate;
 import org.osgi.service.component.annotations.Component;
+import org.osgi.service.component.annotations.Reference;
 import org.osgi.service.metatype.annotations.AttributeDefinition;
 import org.osgi.service.metatype.annotations.Designate;
 import org.osgi.service.metatype.annotations.ObjectClassDefinition;
@@ -102,6 +104,9 @@ public class FeatureModelInstallerPlugin implements InstallTaskFactory, Resource
 
     private final File storageDirectory;
 
+    @Reference
+    private volatile List<ExtensionHandler> extensionHandlers;
+
     @Activate
     public FeatureModelInstallerPlugin(final BundleContext ctx, final Config config) throws IOException {
         this.bundleContext = ctx;
@@ -180,6 +185,9 @@ public class FeatureModelInstallerPlugin implements InstallTaskFactory, Resource
                 if (!feature.isAssembled()) {
                     final BuilderContext ctx = new BuilderContext(this.artifactManager.toFeatureProvider());
                     ctx.setArtifactProvider(this.artifactManager);
+
+                    // Set all merge extensions here from the service registry?
+
                     feature = FeatureBuilder.assemble(feature, ctx);
                 }
 
@@ -229,7 +237,7 @@ public class FeatureModelInstallerPlugin implements InstallTaskFactory, Resource
         }
         final InstallContext ctx = new InstallContext(this.artifactManager, this.storageDirectory);
         return new InstallFeatureModelTask(group,
-                ctx, this.bundleContext);
+                ctx, this.bundleContext, this.extensionHandlers);
     }
 
     boolean classifierMatches(String classifier) {
diff --git a/src/main/java/org/apache/sling/installer/factory/model/impl/InstallFeatureModelTask.java b/src/main/java/org/apache/sling/installer/factory/model/impl/InstallFeatureModelTask.java
index a63ff6b..1d7c925 100644
--- a/src/main/java/org/apache/sling/installer/factory/model/impl/InstallFeatureModelTask.java
+++ b/src/main/java/org/apache/sling/installer/factory/model/impl/InstallFeatureModelTask.java
@@ -26,13 +26,13 @@ import java.io.OutputStream;
 import java.io.Reader;
 import java.io.StringReader;
 import java.net.MalformedURLException;
+import java.net.URL;
 import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.Dictionary;
 import java.util.Hashtable;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 
 import org.apache.sling.feature.Artifact;
 import org.apache.sling.feature.ArtifactId;
@@ -40,11 +40,12 @@ import org.apache.sling.feature.Configuration;
 import org.apache.sling.feature.Extension;
 import org.apache.sling.feature.ExtensionType;
 import org.apache.sling.feature.Feature;
-import org.apache.sling.feature.extension.apiregions.api.ApiRegions;
-import org.apache.sling.feature.extension.apiregions.launcher.LauncherProperties;
+import org.apache.sling.feature.builder.ArtifactProvider;
 import org.apache.sling.feature.io.archive.ArchiveReader;
 import org.apache.sling.feature.io.artifacts.ArtifactHandler;
 import org.apache.sling.feature.io.json.FeatureJSONReader;
+import org.apache.sling.feature.spi.context.ExtensionHandler;
+import org.apache.sling.feature.spi.context.ExtensionHandlerContext;
 import org.apache.sling.installer.api.InstallableResource;
 import org.apache.sling.installer.api.OsgiInstaller;
 import org.apache.sling.installer.api.tasks.InstallationContext;
@@ -52,27 +53,20 @@ import org.apache.sling.installer.api.tasks.ResourceState;
 import org.apache.sling.installer.api.tasks.TaskResource;
 import org.apache.sling.installer.api.tasks.TaskResourceGroup;
 import org.osgi.framework.BundleContext;
-import org.osgi.framework.Constants;
 
 /**
  * This task installs a feature model resources.
  */
 public class InstallFeatureModelTask extends AbstractFeatureModelTask {
-
-    private static final String PROP_idbsnver = "mapping.bundleid.bsnver";
-    private static final String PROP_bundleFeatures = "mapping.bundleid.features";
-    private static final String PROP_featureRegions = "mapping.featureid.regions";
-    private static final String PROP_regionPackage = "mapping.region.packages";
-
-    private static final String REGION_FACTORY_PID = "org.apache.sling.feature.apiregions.factory~";
-    private static final String REPOINIT_FACTORY_PID = "org.apache.sling.jcr.repoinit.RepositoryInitializer~";
-
     private final InstallContext installContext;
+    private final List<ExtensionHandler> extensionHandlers;
 
     public InstallFeatureModelTask(final TaskResourceGroup group,
-            final InstallContext installContext, final BundleContext bundleContext) {
+            final InstallContext installContext, final BundleContext bundleContext,
+            final List<ExtensionHandler> extensionHandlers) {
         super(group, bundleContext);
         this.installContext = installContext;
+        this.extensionHandlers = extensionHandlers;
     }
 
     @Override
@@ -139,18 +133,6 @@ public class InstallFeatureModelTask extends AbstractFeatureModelTask {
                     cfg.getConfigurationProperties(), null, InstallableResource.TYPE_CONFIG, null));
         }
 
-        // repoinit
-        final Extension repoInit = feature.getExtensions().getByName(Extension.EXTENSION_NAME_REPOINIT);
-        if (repoInit != null && repoInit.getType() == ExtensionType.TEXT) {
-            final String configPid = REPOINIT_FACTORY_PID.concat(feature.getId().toMvnName().replace('-', '_'));
-            final Dictionary<String, Object> props = new Hashtable<>();
-            props.put("scripts", repoInit.getText());
-            props.put(Constants.SERVICE_RANKING, 200);
-
-            result.add(new InstallableResource("/".concat(configPid).concat(".config"), null,
-                    props, null, InstallableResource.TYPE_CONFIG, null));
-        }
-
         // extract artifacts
         if (this.installContext.storageDirectory != null) {
             final byte[] buffer = new byte[1024*1024*256];
@@ -178,24 +160,27 @@ public class InstallFeatureModelTask extends AbstractFeatureModelTask {
             }
         }
 
-        // api regions
-        final Extension regionExt = feature.getExtensions().getByName(ApiRegions.EXTENSION_NAME);
-        if ( regionExt != null ) {
-            try {
-                final ApiRegions regions = ApiRegions.parse(regionExt.getJSONStructure().asJsonArray());
-
-                final String configPid = REGION_FACTORY_PID.concat(feature.getId().toMvnName().replace('-', '_'));
-                final Dictionary<String, Object> props = new Hashtable<>();
-                props.put(PROP_idbsnver, convert(LauncherProperties.getBundleIDtoBSNandVersionMap(feature, this.installContext.artifactManager)));
-                props.put(PROP_bundleFeatures, convert(LauncherProperties.getBundleIDtoFeaturesMap(feature)));
-                props.put(PROP_featureRegions, convert(LauncherProperties.getFeatureIDtoRegionsMap(regions)));
-                props.put(PROP_regionPackage, convert(LauncherProperties.getRegionNametoPackagesMap(regions)));
+        ExtensionHandlerContext context = new ContextImpl(result);
 
-                result.add(new InstallableResource("/".concat(configPid).concat(".config"), null,
-                        props, null, InstallableResource.TYPE_CONFIG, null));
-            } catch (final IOException ioe) {
-                logger.warn("Unable to parse region information " + feature.getId().toMvnId(), ioe);
-                return null;
+        for (Extension ext : feature.getExtensions()) {
+            boolean handlerFound = false;
+            for (ExtensionHandler eh : extensionHandlers) {
+                try {
+                    handlerFound |= eh.handle(context, ext, feature);
+                } catch (Exception e) {
+                    logger.error("Exception while processing extension {} with handler {}", ext, eh, e);
+                }
+            }
+            if (!handlerFound) {
+                if (ExtensionType.ARTIFACTS == ext.getType()) {
+                    // Unhandled ARTIFACTS extensions get stored
+                    for (final Artifact artifact : ext.getArtifacts()) {
+                        addArtifact(artifact, result);
+                    }
+                } else {
+                    // should this be an error?
+                    logger.warn("No extension handler found for mandartory extension " + ext);
+                }
             }
         }
 
@@ -206,27 +191,9 @@ public class InstallFeatureModelTask extends AbstractFeatureModelTask {
             }
         }
 
-        // artifact extensions
-        for(final Extension ext : feature.getExtensions()) {
-            if ( ext.getType() == ExtensionType.ARTIFACTS ) {
-                for (final Artifact artifact : ext.getArtifacts()) {
-                    addArtifact(artifact, result);
-                }
-            }
-        }
-
         return result;
     }
 
-    private String[] convert(final Properties props) {
-        final List<String> result = new ArrayList<>();
-
-        for(final Map.Entry<Object, Object> entry : props.entrySet()) {
-            result.add(entry.getKey().toString().concat("=").concat(entry.getValue().toString()));
-        }
-        return result.toArray(new String[result.size()]);
-    }
-
     private boolean addArtifact(final Artifact artifact,
             final List<InstallableResource> result) {
         File artifactFile = (this.installContext.storageDirectory == null ? null
@@ -274,4 +241,79 @@ public class InstallFeatureModelTask extends AbstractFeatureModelTask {
     public String getSortKey() {
         return "30-" + getResource().getAttribute(FeatureModelInstallerPlugin.ATTR_ID);
     }
+
+    private ArtifactProvider getLocalArtifactProvider() {
+        // TODO share with addArtifact()
+        return new ArtifactProvider() {
+            @Override
+            public URL provide(ArtifactId id) {
+                File artifactFile = (installContext.storageDirectory == null ? null
+                        : getArtifactFile(installContext.storageDirectory, id));
+                ArtifactHandler handler;
+                if (artifactFile == null || !artifactFile.exists()) {
+                    try {
+                        handler = installContext.artifactManager.getArtifactHandler(id.toMvnUrl());
+                    } catch (final IOException ignore) {
+                        return null;
+                    }
+                } else {
+                    try {
+                        handler = new ArtifactHandler(artifactFile);
+                    } catch (final MalformedURLException e) {
+                        return null;
+                    }
+                }
+                if (handler == null) {
+                    return null;
+                }
+                return handler.getLocalURL();
+            }
+        };
+    }
+
+    private class ContextImpl implements ExtensionHandlerContext {
+        private final List<InstallableResource> results;
+
+        public ContextImpl(List<InstallableResource> results) {
+            this.results = results;
+        }
+
+        @Override
+        public void addBundle(ArtifactId id, URL file, Integer startLevel) {
+            // TODO Auto-generated method stub
+        }
+
+        @Override
+        public void addInstallableArtifact(ArtifactId id, URL url, Map<String,Object> props) {
+            try {
+                Dictionary <String,Object> dict = new Hashtable<>();
+                props.entrySet().stream()
+                    .filter(e -> e.getValue() != null)
+                    .forEach(e -> dict.put(e.getKey(), e.getValue()));
+
+                InputStream is = url.openStream();
+                results.add(new InstallableResource("/".concat(id.toMvnName()), is, dict, null /* TODO digest? */,
+                        InstallableResource.TYPE_FILE, null));
+            } catch (IOException e) {
+                logger.warn("Unable to read artifact " + id + " from url " + url, e);
+            }
+        }
+
+        @Override
+        public void addConfiguration(String pid, String factoryPid, Dictionary<String, Object> properties) {
+            // TODO handler factoryPid, is this ok?
+            String cfgPid = pid;
+            if (factoryPid != null) {
+                cfgPid = factoryPid;
+            }
+
+            results.add(new InstallableResource("/".concat(cfgPid).concat(".config"), null,
+                    properties, null, InstallableResource.TYPE_CONFIG, null));
+        }
+
+        @Override
+        public ArtifactProvider getArtifactProvider() {
+            return getLocalArtifactProvider();
+        }
+    }
 }
diff --git a/src/main/java/org/apache/sling/installer/factory/model/impl/RepoinitExtensionHandler.java b/src/main/java/org/apache/sling/installer/factory/model/impl/RepoinitExtensionHandler.java
new file mode 100644
index 0000000..8440df7
--- /dev/null
+++ b/src/main/java/org/apache/sling/installer/factory/model/impl/RepoinitExtensionHandler.java
@@ -0,0 +1,51 @@
+/*
+ * 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.sling.installer.factory.model.impl;
+
+import org.apache.sling.feature.Extension;
+import org.apache.sling.feature.ExtensionType;
+import org.apache.sling.feature.Feature;
+import org.apache.sling.feature.spi.context.ExtensionHandler;
+import org.apache.sling.feature.spi.context.ExtensionHandlerContext;
+import org.osgi.framework.Constants;
+import org.osgi.service.component.annotations.Component;
+
+import java.util.Dictionary;
+import java.util.Hashtable;
+
+@Component
+public class RepoinitExtensionHandler implements ExtensionHandler {
+    private static final String REPOINIT_FACTORY_PID = "org.apache.sling.jcr.repoinit.RepositoryInitializer~";
+
+    @Override
+    public boolean handle(ExtensionHandlerContext context, Extension extension, Feature feature) throws Exception {
+        if (!extension.getName().equals(Extension.EXTENSION_NAME_REPOINIT) ||
+                extension.getType() != ExtensionType.TEXT) {
+            return false;
+        }
+
+        final String configPid = REPOINIT_FACTORY_PID.concat(feature.getId().toMvnName().replace('-', '_'));
+        final Dictionary<String, Object> props = new Hashtable<>();
+        props.put("scripts", extension.getText());
+        props.put(Constants.SERVICE_RANKING, 200);
+
+        context.addConfiguration(null, configPid, props);
+        return true;
+    }
+}
diff --git a/src/test/java/org/apache/sling/installer/factory/model/impl/APIRegionsExtensionHandlerTest.java b/src/test/java/org/apache/sling/installer/factory/model/impl/APIRegionsExtensionHandlerTest.java
new file mode 100644
index 0000000..6ffb93d
--- /dev/null
+++ b/src/test/java/org/apache/sling/installer/factory/model/impl/APIRegionsExtensionHandlerTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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.sling.installer.factory.model.impl;
+
+import org.apache.sling.feature.ArtifactId;
+import org.apache.sling.feature.Extension;
+import org.apache.sling.feature.ExtensionState;
+import org.apache.sling.feature.ExtensionType;
+import org.apache.sling.feature.Feature;
+import org.apache.sling.feature.spi.context.ExtensionHandlerContext;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Arrays;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class APIRegionsExtensionHandlerTest {
+    private static String FEATURE_EXT_1 =
+            "[" +
+            "   {" +
+            "     \"name\": \"my-region\"," +
+            "     \"exports\": [\"org.foo.bar\", \"la.di.da\"]" +
+            "   }" +
+            "]";
+
+    @Test
+    public void testHandle() throws Exception {
+        APIRegionsExtensionHandler areh = new APIRegionsExtensionHandler();
+
+        ExtensionHandlerContext ctx = Mockito.mock(ExtensionHandlerContext.class);
+
+        Extension ext = new Extension(ExtensionType.JSON, "api-regions", ExtensionState.REQUIRED);
+        ext.setJSON(FEATURE_EXT_1);
+
+        Feature feat = new Feature(ArtifactId.fromMvnId("x:y:8"));
+        assertTrue(areh.handle(ctx, ext, feat));
+
+        Mockito.verify(ctx).addConfiguration(Mockito.isNull(),
+            Mockito.eq("org.apache.sling.feature.apiregions.factory~y_8.jar"),
+            Mockito.argThat(p -> {
+                String[] pkgs = (String[]) p.get("mapping.region.packages");
+                return Arrays.deepEquals(new String [] {"my-region=org.foo.bar,la.di.da"}, pkgs);
+            }));
+    }
+
+    @Test
+    public void testDoesNotHandle() throws Exception {
+        APIRegionsExtensionHandler areh = new APIRegionsExtensionHandler();
+
+        Extension ext = new Extension(ExtensionType.TEXT, "api-regions", ExtensionState.REQUIRED);
+        assertFalse(areh.handle(null, ext, null));
+    }
+}
diff --git a/src/test/java/org/apache/sling/installer/factory/model/impl/InstallFeatureModelTaskTest.java b/src/test/java/org/apache/sling/installer/factory/model/impl/InstallFeatureModelTaskTest.java
new file mode 100644
index 0000000..b3afe1a
--- /dev/null
+++ b/src/test/java/org/apache/sling/installer/factory/model/impl/InstallFeatureModelTaskTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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.sling.installer.factory.model.impl;
+
+import org.apache.sling.feature.Extension;
+import org.apache.sling.feature.ExtensionType;
+import org.apache.sling.feature.Feature;
+import org.apache.sling.feature.io.artifacts.ArtifactManager;
+import org.apache.sling.feature.spi.context.ExtensionHandler;
+import org.apache.sling.feature.spi.context.ExtensionHandlerContext;
+import org.apache.sling.installer.api.tasks.InstallationContext;
+import org.apache.sling.installer.api.tasks.ResourceState;
+import org.apache.sling.installer.api.tasks.TaskResource;
+import org.apache.sling.installer.api.tasks.TaskResourceGroup;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.osgi.framework.BundleContext;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class InstallFeatureModelTaskTest {
+    private static String FEATURE_JSON_1 = "{" +
+            "  \"id\":\"org.apache.sling.someproj:feature:7.0.126\"," +
+            "  \"system-fonts:ARTIFACTS|true\": [" +
+            "    \"org.apache.sling.feature:my-fonts:jar:0.7.1\"" +
+            "  ]," +
+            "  \"blahblahblah:ARTIFACTS|true\": [" +
+            "  ]," +
+            "  \"framework-properties\": {" +
+            "    \"foo\": \"bar\"" + "  }" +
+            "}";
+
+
+    private Path tempDir;
+
+    @Before
+    public void setUp() throws IOException {
+        tempDir = Files.createTempDirectory(getClass().getSimpleName());
+    }
+
+    @After
+    public void tearDown() throws IOException {
+        // Delete the temp dir again
+        Files.walk(tempDir).sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete);
+    }
+
+    @Test
+    public void testTransform() {
+        TaskResource resource = Mockito.mock(TaskResource.class);
+        Mockito.when(resource.getAttribute(FeatureModelInstallerPlugin.ATTR_MODEL)).thenReturn(FEATURE_JSON_1);
+
+        TaskResourceGroup group = Mockito.mock(TaskResourceGroup.class);
+        Mockito.when(group.getActiveResource()).thenReturn(resource);
+
+        InstallContext installContext = new InstallContext(null, null);
+        BundleContext bundleContext = Mockito.mock(BundleContext.class);
+
+        TestExtensionHandler testEH = new TestExtensionHandler("system-fonts", ExtensionType.ARTIFACTS);
+        List<ExtensionHandler> extensionHandlers = Arrays.asList(testEH);
+
+        InstallFeatureModelTask ifmt = new InstallFeatureModelTask(group, installContext, bundleContext, extensionHandlers);
+
+        InstallationContext ctx = Mockito.mock(InstallationContext.class);
+
+        assertEquals("Precondition", 0, testEH.handled.size());
+        assertEquals("Precondition", 0, testEH.notHandled.size());
+
+        ifmt.execute(ctx);
+
+        assertEquals(1, testEH.handled.size());
+        assertEquals("system-fonts", testEH.handled.get(0).getName());
+
+        assertEquals(1, testEH.notHandled.size());
+        assertEquals("blahblahblah", testEH.notHandled.get(0).getName());
+
+        Mockito.verify(group).setFinishState(ResourceState.INSTALLED);
+    }
+
+    @Test
+    public void testTransform2() throws IOException {
+        URL fmRes = getClass().getResource("/test2/test2.slingosgifeature");
+        URL farRes = getClass().getResource("/test2/test2.far");
+
+        String fm = new BufferedReader(new InputStreamReader(fmRes.openStream()))
+                .lines().collect(Collectors.joining("\n"));
+
+        TaskResource resource = Mockito.mock(TaskResource.class);
+        Mockito.when(resource.getAttribute(FeatureModelInstallerPlugin.ATTR_MODEL))
+            .thenReturn(fm);
+        Mockito.when(resource.getInputStream())
+            .thenReturn(farRes.openStream());
+
+        TaskResourceGroup group = Mockito.mock(TaskResourceGroup.class);
+        Mockito.when(group.getActiveResource()).thenReturn(resource);
+
+        ArtifactManager am = Mockito.mock(ArtifactManager.class);
+        InstallContext installContext = new InstallContext(am, tempDir.toFile());
+        BundleContext bundleContext = Mockito.mock(BundleContext.class);
+
+        List<ExtensionHandler> extensionHandlers = Arrays.asList();
+
+        InstallFeatureModelTask ifmt = new InstallFeatureModelTask(group, installContext,
+                bundleContext, extensionHandlers );
+
+        InstallationContext ctx = Mockito.mock(InstallationContext.class);
+
+        ifmt.execute(ctx);
+
+        Path actualJar = tempDir.resolve(
+                "org/apache/felix/org.apache.felix.converter/1.0.14/org.apache.felix.converter-1.0.14.jar");
+        assertTrue(Files.exists(actualJar));
+        assertEquals(116605, Files.size(actualJar));
+
+        Path actualFeatureModel = tempDir.resolve(
+                "org/apache/sling/someproj/feature/3.0.2/feature-3.0.2.slingosgifeature");
+        assertTrue(Files.exists(actualFeatureModel));
+
+        String expectedFM = new String(Files.readAllBytes(new File(fmRes.getFile()).toPath()));
+        assertEquals(expectedFM, new String(Files.readAllBytes(actualFeatureModel)));
+    }
+
+    private static class TestExtensionHandler implements ExtensionHandler {
+        private final String extensionName;
+        private final ExtensionType extensionType;
+        private final List<Extension> handled = new ArrayList<>();
+        private final List<Extension> notHandled = new ArrayList<>();
+
+        private TestExtensionHandler(String name, ExtensionType type) {
+            extensionName = name;
+            extensionType = type;
+        }
+
+        @Override
+        public boolean handle(ExtensionHandlerContext context, Extension extension, Feature feature) throws Exception {
+            if (extensionName.equals(extension.getName()) && extensionType == extension.getType()) {
+
+                handled.add(extension);
+                return true;
+            }
+
+            notHandled.add(extension);
+            return false;
+        }
+    }
+}
diff --git a/src/test/java/org/apache/sling/installer/factory/model/impl/RepoinitExtensionHandlerTest.java b/src/test/java/org/apache/sling/installer/factory/model/impl/RepoinitExtensionHandlerTest.java
new file mode 100644
index 0000000..b557e9b
--- /dev/null
+++ b/src/test/java/org/apache/sling/installer/factory/model/impl/RepoinitExtensionHandlerTest.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.sling.installer.factory.model.impl;
+
+import org.apache.sling.feature.ArtifactId;
+import org.apache.sling.feature.Extension;
+import org.apache.sling.feature.ExtensionState;
+import org.apache.sling.feature.ExtensionType;
+import org.apache.sling.feature.Feature;
+import org.apache.sling.feature.spi.context.ExtensionHandlerContext;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.osgi.framework.Constants;
+
+import java.util.Dictionary;
+import java.util.Hashtable;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class RepoinitExtensionHandlerTest {
+    @Test
+    public void testExtensionHandler() throws Exception {
+        RepoinitExtensionHandler rieh = new RepoinitExtensionHandler();
+
+        ExtensionHandlerContext ctx = Mockito.mock(ExtensionHandlerContext.class);
+        Extension ex = new Extension(ExtensionType.TEXT,
+                Extension.EXTENSION_NAME_REPOINIT, ExtensionState.REQUIRED);
+        ex.setText("hello 123");
+
+        Feature feat = new Feature(ArtifactId.fromMvnId("a:b:1"));
+
+        assertTrue(rieh.handle(ctx, ex, feat));
+
+        Dictionary<String,Object> props = new Hashtable<>();
+        props.put("scripts", "hello 123");
+        props.put(Constants.SERVICE_RANKING, 200);
+
+        Mockito.verify(ctx).addConfiguration(null, "org.apache.sling.jcr.repoinit.RepositoryInitializer~b_1.jar", props);
+    }
+
+    @Test
+    public void testSkipUnrelatedExtensions() throws Exception {
+        RepoinitExtensionHandler rieh = new RepoinitExtensionHandler();
+
+        ExtensionHandlerContext ctx = Mockito.mock(ExtensionHandlerContext.class);
+        assertFalse(rieh.handle(ctx, new Extension(ExtensionType.TEXT,
+                "someother-extension", ExtensionState.REQUIRED), null));
+        assertFalse(rieh.handle(ctx, new Extension(ExtensionType.JSON,
+                Extension.EXTENSION_NAME_REPOINIT, ExtensionState.REQUIRED), null));
+    }
+}
diff --git a/src/test/resources/test2/test2.far b/src/test/resources/test2/test2.far
new file mode 100644
index 0000000..ca8c357
Binary files /dev/null and b/src/test/resources/test2/test2.far differ
diff --git a/src/test/resources/test2/test2.slingosgifeature b/src/test/resources/test2/test2.slingosgifeature
new file mode 100644
index 0000000..9442e7f
--- /dev/null
+++ b/src/test/resources/test2/test2.slingosgifeature
@@ -0,0 +1,10 @@
+{
+  "id":"org.apache.sling.someproj:feature:3.0.2",
+  "bundles": [
+    "org.apache.felix:org.apache.felix.converter:1.0.14"
+  ],
+  "framework-properties": {
+    "foo": "bar"
+  }
+}
+