You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by nf...@apache.org on 2018/09/21 13:56:55 UTC

[camel-k] branch master updated (d110a8c -> 3761557)

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

nferraro pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k.git.


    from d110a8c  Merge pull request #111 from lburgazzoli/kamel-cmd
     new 0047930  Refactored build module into assembler and publisher
     new 2c51dfd  Added incremental publisher
     new 1b361c0  bind incremental builder to standard flow
     new 3761557  Fixing artifact path and selection algorithm

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 Gopkg.lock                                         |   9 +
 pkg/{discover => build/assemble}/doc.go            |   5 +-
 pkg/build/assemble/maven_assembler.go              | 171 +++++++++++++++++
 .../maven_assembler_test.go}                       |   2 +-
 pkg/build/build_manager.go                         |  87 +++++++--
 pkg/build/build_types.go                           |  25 ++-
 pkg/{apis/camel/v1alpha1 => build/publish}/doc.go  |   6 +-
 pkg/build/publish/s2i_incremental_publisher.go     | 110 +++++++++++
 .../local_builder.go => publish/s2i_publisher.go}  | 207 ++++++++++-----------
 pkg/stub/action/context/build.go                   |  43 ++++-
 pkg/util/maven/maven.go                            | 112 +----------
 pkg/util/tar/appender.go                           | 106 +++++++++++
 test/build_manager_integration_test.go             |  15 +-
 test/local_builder_integration_test.go             | 107 -----------
 14 files changed, 649 insertions(+), 356 deletions(-)
 copy pkg/{discover => build/assemble}/doc.go (85%)
 create mode 100644 pkg/build/assemble/maven_assembler.go
 rename pkg/build/{local/local_builder_test.go => assemble/maven_assembler_test.go} (99%)
 copy pkg/{apis/camel/v1alpha1 => build/publish}/doc.go (87%)
 create mode 100644 pkg/build/publish/s2i_incremental_publisher.go
 rename pkg/build/{local/local_builder.go => publish/s2i_publisher.go} (57%)
 create mode 100644 pkg/util/tar/appender.go
 delete mode 100644 test/local_builder_integration_test.go


[camel-k] 02/04: Added incremental publisher

Posted by nf...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

nferraro pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k.git

commit 2c51dfd0d359885a8b298b35d77e50fbf0fb200f
Author: nferraro <ni...@gmail.com>
AuthorDate: Fri Sep 21 13:26:20 2018 +0200

    Added incremental publisher
---
 pkg/build/publish/s2i_incremental_publisher.go | 105 +++++++++++++++++++++++++
 pkg/build/publish/s2i_publisher.go             |  43 +++++++---
 2 files changed, 137 insertions(+), 11 deletions(-)

diff --git a/pkg/build/publish/s2i_incremental_publisher.go b/pkg/build/publish/s2i_incremental_publisher.go
new file mode 100644
index 0000000..42d7c81
--- /dev/null
+++ b/pkg/build/publish/s2i_incremental_publisher.go
@@ -0,0 +1,105 @@
+/*
+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 publish
+
+import (
+	"context"
+	"github.com/apache/camel-k/pkg/build"
+)
+
+type s2iIncrementalPublisher struct {
+	s2iPublisher *s2iPublisher
+	lister       PublishedImagesLister
+}
+
+// PublishedImage represent a base image that can be used as starting point
+type PublishedImage struct {
+	Image     string
+	Classpath []string
+}
+
+// PublishedImagesLister allows to list all images already published
+type PublishedImagesLister interface {
+	ListPublishedImages() ([]PublishedImage, error)
+}
+
+// NewS2IIncrementalPublisher creates a new publisher that is able to do a Openshift S2I binary builds on top of other builds
+func NewS2IIncrementalPublisher(ctx context.Context, namespace string, lister PublishedImagesLister) build.Publisher {
+	layeredPublisher := s2iIncrementalPublisher{
+		lister: lister,
+	}
+	layeredPublisher.s2iPublisher = newS2IPublisher(ctx, namespace, layeredPublisher.selectArtifactsToUpload)
+	return &layeredPublisher
+}
+
+func (p *s2iIncrementalPublisher) Publish(req build.Request, assembled build.AssembledOutput) <-chan build.PublishedOutput {
+	return p.s2iPublisher.Publish(req, assembled)
+}
+
+func (p *s2iIncrementalPublisher) selectArtifactsToUpload(entries []build.ClasspathEntry) (string, []build.ClasspathEntry, error) {
+	images, err := p.lister.ListPublishedImages()
+	if err != nil {
+		return "", nil, err
+	}
+
+	bestImage, commonLibs := p.findBestImage(images, entries)
+	if (bestImage != nil) {
+		selectedClasspath := make([]build.ClasspathEntry, 0)
+		for _, entry := range entries {
+			if _, isCommon := commonLibs[entry.ID]; !isCommon {
+				selectedClasspath = append(selectedClasspath, entry)
+			}
+		}
+
+		return bestImage.Image, selectedClasspath, nil
+	}
+
+	// return default selection
+	return baseImage, entries, nil
+}
+
+func (p *s2iIncrementalPublisher) findBestImage(images []PublishedImage, entries []build.ClasspathEntry) (*PublishedImage, map[string]bool) {
+	requiredLibs := make(map[string]bool, len(entries))
+	for _, entry := range entries {
+		requiredLibs[entry.ID] = true
+	}
+
+	var bestImage *PublishedImage
+	bestImageCommonLibs := make(map[string]bool, 0)
+	for _, image := range images {
+		common := make(map[string]bool)
+		for _, id := range image.Classpath {
+			if _, ok := requiredLibs[id]; ok {
+				common[id] = true
+			}
+		}
+		numCommonLibs := len(common)
+		surplus := len(image.Classpath) - numCommonLibs
+		if surplus >= numCommonLibs/3 {
+			// Heuristic approach: if there are too many unrelated libraries, just use the base image
+			continue
+		}
+
+		if (numCommonLibs > len(bestImageCommonLibs)) {
+			bestImage = &image
+			bestImageCommonLibs = common
+		}
+	}
+
+	return bestImage, bestImageCommonLibs
+}
diff --git a/pkg/build/publish/s2i_publisher.go b/pkg/build/publish/s2i_publisher.go
index 5548194..ee86a1e 100644
--- a/pkg/build/publish/s2i_publisher.go
+++ b/pkg/build/publish/s2i_publisher.go
@@ -40,11 +40,13 @@ import (
 
 const (
 	artifactDirPrefix = "s2i-"
+	baseImage         = "fabric8/s2i-java:2.3"
 )
 
 type s2iPublisher struct {
 	buffer    chan publishOperation
 	namespace string
+	uploadedArtifactsSelector
 }
 
 type publishOperation struct {
@@ -53,11 +55,20 @@ type publishOperation struct {
 	output    chan build.PublishedOutput
 }
 
+type uploadedArtifactsSelector func([]build.ClasspathEntry) (string, []build.ClasspathEntry, error)
+
 // NewS2IPublisher creates a new publisher doing a Openshift S2I binary build
 func NewS2IPublisher(ctx context.Context, namespace string) build.Publisher {
+	identitySelector := func(entries []build.ClasspathEntry) (string, []build.ClasspathEntry, error) { return baseImage, entries, nil }
+	return newS2IPublisher(ctx, namespace, identitySelector)
+}
+
+// NewS2IPublisher creates a new publisher doing a Openshift S2I binary build
+func newS2IPublisher(ctx context.Context, namespace string, uploadedArtifactsSelector uploadedArtifactsSelector) *s2iPublisher {
 	publisher := s2iPublisher{
-		buffer:    make(chan publishOperation, 100),
-		namespace: namespace,
+		buffer:                    make(chan publishOperation, 100),
+		namespace:                 namespace,
+		uploadedArtifactsSelector: uploadedArtifactsSelector,
 	}
 	go publisher.publishCycle(ctx)
 	return &publisher
@@ -98,12 +109,17 @@ func (b *s2iPublisher) publishCycle(ctx context.Context) {
 }
 
 func (b *s2iPublisher) execute(request build.Request, assembled build.AssembledOutput) build.PublishedOutput {
-	tarFile, err := b.createTar(assembled)
+	baseImageName, selectedArtifacts, err := b.uploadedArtifactsSelector(assembled.Classpath)
 	if err != nil {
 		return build.PublishedOutput{Error: err}
 	}
 
-	image, err := b.publish(tarFile, request)
+	tarFile, err := b.createTar(assembled, selectedArtifacts)
+	if err != nil {
+		return build.PublishedOutput{Error: err}
+	}
+
+	image, err := b.publish(tarFile, baseImageName, request)
 	if err != nil {
 		return build.PublishedOutput{Error: err}
 	}
@@ -111,7 +127,7 @@ func (b *s2iPublisher) execute(request build.Request, assembled build.AssembledO
 	return build.PublishedOutput{Image: image}
 }
 
-func (b *s2iPublisher) publish(tarFile string, source build.Request) (string, error) {
+func (b *s2iPublisher) publish(tarFile string, imageName string, source build.Request) (string, error) {
 
 	bc := buildv1.BuildConfig{
 		TypeMeta: metav1.TypeMeta{
@@ -131,7 +147,7 @@ func (b *s2iPublisher) publish(tarFile string, source build.Request) (string, er
 					SourceStrategy: &buildv1.SourceBuildStrategy{
 						From: v1.ObjectReference{
 							Kind: "DockerImage",
-							Name: "fabric8/s2i-java:2.3",
+							Name: imageName,
 						},
 					},
 				},
@@ -235,7 +251,7 @@ func (b *s2iPublisher) publish(tarFile string, source build.Request) (string, er
 	return is.Status.DockerImageRepository + ":" + source.Identifier.Qualifier, nil
 }
 
-func (b *s2iPublisher) createTar(assembled build.AssembledOutput) (string, error) {
+func (b *s2iPublisher) createTar(assembled build.AssembledOutput, selectedArtifacts []build.ClasspathEntry) (string, error) {
 	artifactDir, err := ioutil.TempDir("", artifactDirPrefix)
 	if err != nil {
 		return "", errors.Wrap(err, "could not create temporary dir for s2i artifacts")
@@ -248,19 +264,24 @@ func (b *s2iPublisher) createTar(assembled build.AssembledOutput) (string, error
 	}
 	defer tarAppender.Close()
 
-	cp := ""
-	for _, entry := range assembled.Classpath {
+	tarDir := "dependencies/"
+	for _, entry := range selectedArtifacts {
 		gav, err := maven.ParseGAV(entry.ID)
 		if err != nil {
 			return "", nil
 		}
 
-		tarPath := path.Join("dependencies/", gav.GroupID)
-		fileName, err := tarAppender.AddFile(entry.Location, tarPath)
+		tarPath := path.Join(tarDir, gav.GroupID)
+		_, err = tarAppender.AddFile(entry.Location, tarPath)
 		if err != nil {
 			return "", err
 		}
+	}
 
+	cp := ""
+	for _, entry := range assembled.Classpath {
+		_, fileName := path.Split(entry.Location)
+		fileName = path.Join(tarDir, fileName)
 		cp += fileName + "\n"
 	}
 


[camel-k] 04/04: Fixing artifact path and selection algorithm

Posted by nf...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

nferraro pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k.git

commit 37615577b78b19b1ea45d957b2845b28cfbddb95
Author: nferraro <ni...@gmail.com>
AuthorDate: Fri Sep 21 15:55:32 2018 +0200

    Fixing artifact path and selection algorithm
---
 pkg/build/publish/s2i_incremental_publisher.go | 13 +++++++++----
 pkg/build/publish/s2i_publisher.go             |  7 ++++++-
 pkg/stub/action/context/build.go               |  2 +-
 3 files changed, 16 insertions(+), 6 deletions(-)

diff --git a/pkg/build/publish/s2i_incremental_publisher.go b/pkg/build/publish/s2i_incremental_publisher.go
index 42d7c81..f3921cc 100644
--- a/pkg/build/publish/s2i_incremental_publisher.go
+++ b/pkg/build/publish/s2i_incremental_publisher.go
@@ -74,13 +74,17 @@ func (p *s2iIncrementalPublisher) selectArtifactsToUpload(entries []build.Classp
 }
 
 func (p *s2iIncrementalPublisher) findBestImage(images []PublishedImage, entries []build.ClasspathEntry) (*PublishedImage, map[string]bool) {
+	if len(images) == 0 {
+		return nil, nil
+	}
 	requiredLibs := make(map[string]bool, len(entries))
 	for _, entry := range entries {
 		requiredLibs[entry.ID] = true
 	}
 
-	var bestImage *PublishedImage
+	var bestImage PublishedImage
 	bestImageCommonLibs := make(map[string]bool, 0)
+	bestImageSurplusLibs := 0
 	for _, image := range images {
 		common := make(map[string]bool)
 		for _, id := range image.Classpath {
@@ -95,11 +99,12 @@ func (p *s2iIncrementalPublisher) findBestImage(images []PublishedImage, entries
 			continue
 		}
 
-		if (numCommonLibs > len(bestImageCommonLibs)) {
-			bestImage = &image
+		if numCommonLibs > len(bestImageCommonLibs) || (numCommonLibs == len(bestImageCommonLibs) && surplus < bestImageSurplusLibs) {
+			bestImage = image
 			bestImageCommonLibs = common
+			bestImageSurplusLibs = surplus
 		}
 	}
 
-	return bestImage, bestImageCommonLibs
+	return &bestImage, bestImageCommonLibs
 }
diff --git a/pkg/build/publish/s2i_publisher.go b/pkg/build/publish/s2i_publisher.go
index ee86a1e..7fd06c2 100644
--- a/pkg/build/publish/s2i_publisher.go
+++ b/pkg/build/publish/s2i_publisher.go
@@ -280,8 +280,13 @@ func (b *s2iPublisher) createTar(assembled build.AssembledOutput, selectedArtifa
 
 	cp := ""
 	for _, entry := range assembled.Classpath {
+		gav, err := maven.ParseGAV(entry.ID)
+		if err != nil {
+			return "", nil
+		}
+		tarPath := path.Join(tarDir, gav.GroupID)
 		_, fileName := path.Split(entry.Location)
-		fileName = path.Join(tarDir, fileName)
+		fileName = path.Join(tarPath, fileName)
 		cp += fileName + "\n"
 	}
 
diff --git a/pkg/stub/action/context/build.go b/pkg/stub/action/context/build.go
index 7b872f4..a0ab58f 100644
--- a/pkg/stub/action/context/build.go
+++ b/pkg/stub/action/context/build.go
@@ -136,7 +136,7 @@ func (l contextLister) ListPublishedImages() ([]publish.PublishedImage, error) {
 	}
 	images := make([]publish.PublishedImage, 0)
 	for _, ctx := range list.Items {
-		if ctx.Labels == nil {
+		if ctx.Status.Phase != v1alpha1.IntegrationContextPhaseReady || ctx.Labels == nil {
 			continue
 		}
 		if ctxType, present := ctx.Labels["camel.apache.org/context.type"]; !present || ctxType != "platform" {


[camel-k] 03/04: bind incremental builder to standard flow

Posted by nf...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

nferraro pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k.git

commit 1b361c01ae14485668e58ef12fc674f4e91daef7
Author: nferraro <ni...@gmail.com>
AuthorDate: Fri Sep 21 13:40:16 2018 +0200

    bind incremental builder to standard flow
---
 pkg/stub/action/context/build.go | 37 ++++++++++++++++++++++++++++++++++++-
 1 file changed, 36 insertions(+), 1 deletion(-)

diff --git a/pkg/stub/action/context/build.go b/pkg/stub/action/context/build.go
index a815c7c..7b872f4 100644
--- a/pkg/stub/action/context/build.go
+++ b/pkg/stub/action/context/build.go
@@ -34,7 +34,7 @@ import (
 // NewIntegrationContextBuildAction creates a new build handling action for the context
 func NewIntegrationContextBuildAction(ctx context.Context, namespace string) IntegrationContextAction {
 	assembler := assemble.NewMavenAssembler(ctx)
-	publisher := publish.NewS2IPublisher(ctx, namespace)
+	publisher := publish.NewS2IIncrementalPublisher(ctx, namespace, newContextLister(namespace))
 	manager := build.NewManager(ctx, assembler, publisher)
 
 	return &integrationContextBuildAction{
@@ -115,3 +115,38 @@ func (action *integrationContextBuildAction) informIntegrations(context *v1alpha
 	}
 	return nil
 }
+
+// =================================================================
+
+type contextLister struct {
+	namespace string
+}
+
+func newContextLister(namespace string) contextLister {
+	return contextLister{
+		namespace: namespace,
+	}
+}
+
+func (l contextLister) ListPublishedImages() ([]publish.PublishedImage, error) {
+	list := v1alpha1.NewIntegrationContextList()
+	err := sdk.List(l.namespace, &list, sdk.WithListOptions(&metav1.ListOptions{}))
+	if err != nil {
+		return nil, err
+	}
+	images := make([]publish.PublishedImage, 0)
+	for _, ctx := range list.Items {
+		if ctx.Labels == nil {
+			continue
+		}
+		if ctxType, present := ctx.Labels["camel.apache.org/context.type"]; !present || ctxType != "platform" {
+			continue
+		}
+
+		images = append(images, publish.PublishedImage{
+			Image:     ctx.Status.Image,
+			Classpath: ctx.Status.Classpath,
+		})
+	}
+	return images, nil
+}


[camel-k] 01/04: Refactored build module into assembler and publisher

Posted by nf...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

nferraro pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k.git

commit 0047930e30b2429af3b2540d70da17d99f4b09c9
Author: nferraro <ni...@gmail.com>
AuthorDate: Fri Sep 21 11:25:41 2018 +0200

    Refactored build module into assembler and publisher
---
 Gopkg.lock                                         |   9 +
 pkg/build/assemble/doc.go                          |  19 +++
 pkg/build/assemble/maven_assembler.go              | 171 +++++++++++++++++++
 .../maven_assembler_test.go}                       |   2 +-
 pkg/build/build_manager.go                         |  87 ++++++++--
 pkg/build/build_types.go                           |  25 ++-
 pkg/build/publish/doc.go                           |  19 +++
 .../local_builder.go => publish/s2i_publisher.go}  | 183 ++++++++-------------
 pkg/stub/action/context/build.go                   |   8 +-
 pkg/util/maven/maven.go                            | 112 ++-----------
 pkg/util/tar/appender.go                           | 106 ++++++++++++
 test/build_manager_integration_test.go             |  15 +-
 test/local_builder_integration_test.go             | 107 ------------
 13 files changed, 513 insertions(+), 350 deletions(-)

diff --git a/Gopkg.lock b/Gopkg.lock
index 4c8e50f..0975b14 100644
--- a/Gopkg.lock
+++ b/Gopkg.lock
@@ -493,6 +493,14 @@
   version = "v0.9.1"
 
 [[projects]]
+  branch = "v1"
+  digest = "1:52ffb9db0286de37253a5098607cfbcfcdc94e51e8c226da120513df82adab0c"
+  name = "gopkg.in/yaml.v1"
+  packages = ["."]
+  pruneopts = "NUT"
+  revision = "9f9df34309c04878acc86042b16630b0f696e1de"
+
+[[projects]]
   digest = "1:7c95b35057a0ff2e19f707173cc1a947fa43a6eb5c4d300d196ece0334046082"
   name = "gopkg.in/yaml.v2"
   packages = ["."]
@@ -733,6 +741,7 @@
     "github.com/spf13/cobra",
     "github.com/stoewer/go-strcase",
     "github.com/stretchr/testify/assert",
+    "gopkg.in/yaml.v1",
     "gopkg.in/yaml.v2",
     "k8s.io/api/apps/v1",
     "k8s.io/api/core/v1",
diff --git a/pkg/build/assemble/doc.go b/pkg/build/assemble/doc.go
new file mode 100644
index 0000000..20b1d42
--- /dev/null
+++ b/pkg/build/assemble/doc.go
@@ -0,0 +1,19 @@
+/*
+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 assemble contains tools that convert source files and dependencies into the integration classpath
+package assemble
diff --git a/pkg/build/assemble/maven_assembler.go b/pkg/build/assemble/maven_assembler.go
new file mode 100644
index 0000000..9094181
--- /dev/null
+++ b/pkg/build/assemble/maven_assembler.go
@@ -0,0 +1,171 @@
+/*
+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 assemble
+
+import (
+	"context"
+	"encoding/xml"
+	"fmt"
+	"strings"
+	"time"
+
+	"github.com/apache/camel-k/pkg/build"
+	"github.com/apache/camel-k/pkg/util/maven"
+	"github.com/sirupsen/logrus"
+
+	// import openshift utilities
+	_ "github.com/apache/camel-k/pkg/util/openshift"
+	"github.com/apache/camel-k/version"
+)
+
+type mavenAssembler struct {
+	buffer chan assembleOperation
+}
+
+type assembleOperation struct {
+	request build.Request
+	output  chan build.AssembledOutput
+}
+
+// NewMavenAssembler create a new builder
+func NewMavenAssembler(ctx context.Context) build.Assembler {
+	assembler := mavenAssembler{
+		buffer: make(chan assembleOperation, 100),
+	}
+	go assembler.assembleCycle(ctx)
+	return &assembler
+}
+
+func (b *mavenAssembler) Assemble(request build.Request) <-chan build.AssembledOutput {
+	res := make(chan build.AssembledOutput, 1)
+	op := assembleOperation{
+		request: request,
+		output:  res,
+	}
+	b.buffer <- op
+	return res
+}
+
+func (b *mavenAssembler) assembleCycle(ctx context.Context) {
+	for {
+		select {
+		case <-ctx.Done():
+			b.buffer = nil
+			return
+		case op := <-b.buffer:
+			now := time.Now()
+			logrus.Info("Starting new Maven build")
+			res := b.execute(&op.request)
+			elapsed := time.Now().Sub(now)
+
+			if res.Error != nil {
+				logrus.Error("Error during Maven build (total time ", elapsed.Seconds(), " seconds): ", res.Error)
+			} else {
+				logrus.Info("Maven build completed in ", elapsed.Seconds(), " seconds")
+			}
+
+			op.output <- res
+		}
+	}
+}
+
+func (b *mavenAssembler) execute(request *build.Request) build.AssembledOutput {
+	project, err := generateProject(request)
+	if err != nil {
+		return build.AssembledOutput{
+			Error: err,
+		}
+	}
+
+	res, err := maven.Process(project)
+	if err != nil {
+		return build.AssembledOutput{
+			Error: err,
+		}
+	}
+
+	output := build.AssembledOutput{
+		Classpath: make([]build.ClasspathEntry, 0, len(res.Classpath)),
+	}
+	for _, e := range res.Classpath {
+		output.Classpath = append(output.Classpath, build.ClasspathEntry{
+			ID:       e.ID,
+			Location: e.Location,
+		})
+	}
+
+	return output
+}
+
+func generateProject(source *build.Request) (maven.Project, error) {
+	project := maven.Project{
+		XMLName:           xml.Name{Local: "project"},
+		XMLNs:             "http://maven.apache.org/POM/4.0.0",
+		XMLNsXsi:          "http://www.w3.org/2001/XMLSchema-instance",
+		XsiSchemaLocation: "http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd",
+		ModelVersion:      "4.0.0",
+		GroupID:           "org.apache.camel.k.integration",
+		ArtifactID:        "camel-k-integration",
+		Version:           version.Version,
+		DependencyManagement: maven.DependencyManagement{
+			Dependencies: maven.Dependencies{
+				Dependencies: []maven.Dependency{
+					{
+						//TODO: camel version should be retrieved from an external request or provided as static version
+						GroupID:    "org.apache.camel",
+						ArtifactID: "camel-bom",
+						Version:    "2.22.1",
+						Type:       "pom",
+						Scope:      "import",
+					},
+				},
+			},
+		},
+		Dependencies: maven.Dependencies{
+			Dependencies: make([]maven.Dependency, 0),
+		},
+	}
+
+	//
+	// set-up dependencies
+	//
+
+	deps := &project.Dependencies
+	deps.AddGAV("org.apache.camel.k", "camel-k-runtime-jvm", version.Version)
+
+	for _, d := range source.Dependencies {
+		if strings.HasPrefix(d, "camel:") {
+			artifactID := strings.TrimPrefix(d, "camel:")
+
+			if !strings.HasPrefix(artifactID, "camel-") {
+				artifactID = "camel-" + artifactID
+			}
+
+			deps.AddGAV("org.apache.camel", artifactID, "")
+		} else if strings.HasPrefix(d, "mvn:") {
+			mid := strings.TrimPrefix(d, "mvn:")
+			gav := strings.Replace(mid, "/", ":", -1)
+
+			deps.AddEncodedGAV(gav)
+		} else {
+			return maven.Project{}, fmt.Errorf("unknown dependency type: %s", d)
+		}
+	}
+
+	return project, nil
+}
diff --git a/pkg/build/local/local_builder_test.go b/pkg/build/assemble/maven_assembler_test.go
similarity index 99%
rename from pkg/build/local/local_builder_test.go
rename to pkg/build/assemble/maven_assembler_test.go
index ddb928e..20de3d8 100644
--- a/pkg/build/local/local_builder_test.go
+++ b/pkg/build/assemble/maven_assembler_test.go
@@ -15,7 +15,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package local
+package assemble
 
 import (
 	"testing"
diff --git a/pkg/build/build_manager.go b/pkg/build/build_manager.go
index 50d1187..895575b 100644
--- a/pkg/build/build_manager.go
+++ b/pkg/build/build_manager.go
@@ -19,23 +19,24 @@ package build
 
 import (
 	"context"
+	"errors"
 	"sync"
 )
 
 // Manager represent the main facade to the image build system
 type Manager struct {
-	builds    sync.Map
 	ctx       context.Context
-	namespace string
-	builder   Builder
+	builds    sync.Map
+	assembler Assembler
+	publisher Publisher
 }
 
 // NewManager creates an instance of the build manager using the given builder
-func NewManager(ctx context.Context, namespace string, builder Builder) *Manager {
+func NewManager(ctx context.Context, assembler Assembler, publisher Publisher) *Manager {
 	return &Manager{
 		ctx:       ctx,
-		namespace: namespace,
-		builder:   builder,
+		assembler: assembler,
+		publisher: publisher,
 	}
 }
 
@@ -46,18 +47,41 @@ func (m *Manager) Get(identifier Identifier) Result {
 		return noBuildInfo()
 	}
 
-	return *info.(*Result)
+	return info.(Result)
 }
 
 // Start starts a new build
-func (m *Manager) Start(source Request) {
-	initialBuildInfo := initialBuildInfo(&source)
-	m.builds.Store(source.Identifier, &initialBuildInfo)
+func (m *Manager) Start(request Request) {
+	m.builds.Store(request.Identifier, initialBuildInfo(request))
 
-	resChannel := m.builder.Build(source)
+	assembleChannel := m.assembler.Assemble(request)
 	go func() {
-		res := <-resChannel
-		m.builds.Store(res.Request.Identifier, &res)
+		var assembled AssembledOutput
+		select {
+		case <-m.ctx.Done():
+			m.builds.Store(request.Identifier, canceledBuildInfo(request))
+			return
+		case assembled = <-assembleChannel:
+			if assembled.Error != nil {
+				m.builds.Store(request.Identifier, failedAssembleBuildInfo(request, assembled))
+				return
+			}
+		}
+
+		publishChannel := m.publisher.Publish(request, assembled)
+		var published PublishedOutput
+		select {
+		case <-m.ctx.Done():
+			m.builds.Store(request.Identifier, canceledBuildInfo(request))
+			return
+		case published = <-publishChannel:
+			if published.Error != nil {
+				m.builds.Store(request.Identifier, failedPublishBuildInfo(request, published))
+				return
+			}
+		}
+
+		m.builds.Store(request.Identifier, completeResult(request, assembled, published))
 	}()
 }
 
@@ -67,9 +91,42 @@ func noBuildInfo() Result {
 	}
 }
 
-func initialBuildInfo(source *Request) Result {
+func initialBuildInfo(request Request) Result {
 	return Result{
-		Request: source,
+		Request: request,
 		Status:  StatusStarted,
 	}
 }
+
+func canceledBuildInfo(request Request) Result {
+	return Result{
+		Request: request,
+		Error:   errors.New("build canceled"),
+		Status:  StatusError,
+	}
+}
+
+func failedAssembleBuildInfo(request Request, output AssembledOutput) Result {
+	return Result{
+		Request: request,
+		Error:   output.Error,
+		Status:  StatusError,
+	}
+}
+
+func failedPublishBuildInfo(request Request, output PublishedOutput) Result {
+	return Result{
+		Request: request,
+		Error:   output.Error,
+		Status:  StatusError,
+	}
+}
+
+func completeResult(request Request, a AssembledOutput, p PublishedOutput) Result {
+	return Result{
+		Request:   request,
+		Status:    StatusCompleted,
+		Classpath: a.Classpath,
+		Image:     p.Image,
+	}
+}
diff --git a/pkg/build/build_types.go b/pkg/build/build_types.go
index dc7c787..22ff178 100644
--- a/pkg/build/build_types.go
+++ b/pkg/build/build_types.go
@@ -39,7 +39,7 @@ type Source struct {
 
 // Result represents the result of a build
 type Result struct {
-	Request   *Request
+	Request   Request
 	Status    Status
 	Image     string
 	Error     error
@@ -52,9 +52,26 @@ type ClasspathEntry struct {
 	Location string `json:"location,omitempty" yaml:"location,omitempty"`
 }
 
-// Builder is supertype of all builders
-type Builder interface {
-	Build(Request) <-chan Result
+// AssembledOutput represents the output of the assemble phase
+type AssembledOutput struct {
+	Error     error
+	Classpath []ClasspathEntry
+}
+
+// A Assembler can be used to compute the classpath of a integration context
+type Assembler interface {
+	Assemble(Request) <-chan AssembledOutput
+}
+
+// PublishedOutput is the output of the publish phase
+type PublishedOutput struct {
+	Error error
+	Image string
+}
+
+// A Publisher publishes a docker image of a build request
+type Publisher interface {
+	Publish(Request, AssembledOutput) <-chan PublishedOutput
 }
 
 // Status --
diff --git a/pkg/build/publish/doc.go b/pkg/build/publish/doc.go
new file mode 100644
index 0000000..19343a8
--- /dev/null
+++ b/pkg/build/publish/doc.go
@@ -0,0 +1,19 @@
+/*
+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 publish contains strategies for publishing integrations into a Docker registries
+package publish
diff --git a/pkg/build/local/local_builder.go b/pkg/build/publish/s2i_publisher.go
similarity index 58%
rename from pkg/build/local/local_builder.go
rename to pkg/build/publish/s2i_publisher.go
index 232360b..5548194 100644
--- a/pkg/build/local/local_builder.go
+++ b/pkg/build/publish/s2i_publisher.go
@@ -15,67 +15,66 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package local
+package publish
 
 import (
 	"context"
-	"encoding/xml"
-	"fmt"
-	"io/ioutil"
-	"strings"
-	"time"
-
+	"github.com/apache/camel-k/pkg/build"
+	"github.com/apache/camel-k/pkg/util/kubernetes"
+	"github.com/apache/camel-k/pkg/util/kubernetes/customclient"
+	"github.com/apache/camel-k/pkg/util/maven"
+	"github.com/apache/camel-k/pkg/util/tar"
 	buildv1 "github.com/openshift/api/build/v1"
 	imagev1 "github.com/openshift/api/image/v1"
 	"github.com/operator-framework/operator-sdk/pkg/sdk"
 	"github.com/operator-framework/operator-sdk/pkg/util/k8sutil"
 	"github.com/pkg/errors"
 	"github.com/sirupsen/logrus"
+	"io/ioutil"
 	"k8s.io/api/core/v1"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
+	"path"
+	"time"
+)
 
-	"github.com/apache/camel-k/pkg/build"
-	"github.com/apache/camel-k/pkg/util/kubernetes"
-	"github.com/apache/camel-k/pkg/util/kubernetes/customclient"
-	"github.com/apache/camel-k/pkg/util/maven"
-
-	// import openshift utilities
-	_ "github.com/apache/camel-k/pkg/util/openshift"
-	"github.com/apache/camel-k/version"
+const (
+	artifactDirPrefix = "s2i-"
 )
 
-type localBuilder struct {
-	buffer    chan buildOperation
+type s2iPublisher struct {
+	buffer    chan publishOperation
 	namespace string
 }
 
-type buildOperation struct {
-	request build.Request
-	output  chan build.Result
+type publishOperation struct {
+	request   build.Request
+	assembled build.AssembledOutput
+	output    chan build.PublishedOutput
 }
 
-// NewLocalBuilder create a new builder
-func NewLocalBuilder(ctx context.Context, namespace string) build.Builder {
-	builder := localBuilder{
-		buffer:    make(chan buildOperation, 100),
+// NewS2IPublisher creates a new publisher doing a Openshift S2I binary build
+func NewS2IPublisher(ctx context.Context, namespace string) build.Publisher {
+	publisher := s2iPublisher{
+		buffer:    make(chan publishOperation, 100),
 		namespace: namespace,
 	}
-	go builder.buildCycle(ctx)
-	return &builder
+	go publisher.publishCycle(ctx)
+	return &publisher
 }
 
-func (b *localBuilder) Build(request build.Request) <-chan build.Result {
-	res := make(chan build.Result, 1)
-	op := buildOperation{
-		request: request,
-		output:  res,
+func (b *s2iPublisher) Publish(request build.Request, assembled build.AssembledOutput) <-chan build.PublishedOutput {
+	res := make(chan build.PublishedOutput, 1)
+	op := publishOperation{
+		request:   request,
+		assembled: assembled,
+		output:    res,
 	}
 	b.buffer <- op
 	return res
 }
 
-func (b *localBuilder) buildCycle(ctx context.Context) {
+func (b *s2iPublisher) publishCycle(ctx context.Context) {
 	for {
 		select {
 		case <-ctx.Done():
@@ -83,14 +82,14 @@ func (b *localBuilder) buildCycle(ctx context.Context) {
 			return
 		case op := <-b.buffer:
 			now := time.Now()
-			logrus.Info("Starting new build")
-			res := b.execute(&op.request)
+			logrus.Info("Starting a new image publication")
+			res := b.execute(op.request, op.assembled)
 			elapsed := time.Now().Sub(now)
 
 			if res.Error != nil {
-				logrus.Error("Error during build (total time ", elapsed.Seconds(), " seconds): ", res.Error)
+				logrus.Error("Error during publication (total time ", elapsed.Seconds(), " seconds): ", res.Error)
 			} else {
-				logrus.Info("Process completed in ", elapsed.Seconds(), " seconds")
+				logrus.Info("Publication completed in ", elapsed.Seconds(), " seconds")
 			}
 
 			op.output <- res
@@ -98,43 +97,22 @@ func (b *localBuilder) buildCycle(ctx context.Context) {
 	}
 }
 
-func (b *localBuilder) execute(request *build.Request) build.Result {
-	project, err := generateProject(request)
+func (b *s2iPublisher) execute(request build.Request, assembled build.AssembledOutput) build.PublishedOutput {
+	tarFile, err := b.createTar(assembled)
 	if err != nil {
-		return build.Result{
-			Error:  err,
-			Status: build.StatusError,
-		}
+		return build.PublishedOutput{Error: err}
 	}
 
-	res, err := maven.Process(project)
+	image, err := b.publish(tarFile, request)
 	if err != nil {
-		return build.Result{
-			Error:  err,
-			Status: build.StatusError,
-		}
+		return build.PublishedOutput{Error: err}
 	}
 
-	logrus.Info("Created tar file ", res.TarFilePath)
-
-	image, err := b.publish(res.TarFilePath, request)
-	if err != nil {
-		return build.Result{
-			Error:  errors.Wrap(err, "could not publish docker image"),
-			Status: build.StatusError,
-		}
-	}
-
-	return build.Result{
-		Request:   request,
-		Image:     image,
-		Error:     nil,
-		Status:    build.StatusCompleted,
-		Classpath: res.Classpath,
-	}
+	return build.PublishedOutput{Image: image}
 }
 
-func (b *localBuilder) publish(tarFile string, source *build.Request) (string, error) {
+func (b *s2iPublisher) publish(tarFile string, source build.Request) (string, error) {
+
 	bc := buildv1.BuildConfig{
 		TypeMeta: metav1.TypeMeta{
 			APIVersion: buildv1.SchemeGroupVersion.String(),
@@ -257,60 +235,39 @@ func (b *localBuilder) publish(tarFile string, source *build.Request) (string, e
 	return is.Status.DockerImageRepository + ":" + source.Identifier.Qualifier, nil
 }
 
-func generateProject(source *build.Request) (maven.Project, error) {
-	project := maven.Project{
-		XMLName:           xml.Name{Local: "project"},
-		XMLNs:             "http://maven.apache.org/POM/4.0.0",
-		XMLNsXsi:          "http://www.w3.org/2001/XMLSchema-instance",
-		XsiSchemaLocation: "http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd",
-		ModelVersion:      "4.0.0",
-		GroupID:           "org.apache.camel.k.integration",
-		ArtifactID:        "camel-k-integration",
-		Version:           version.Version,
-		DependencyManagement: maven.DependencyManagement{
-			Dependencies: maven.Dependencies{
-				Dependencies: []maven.Dependency{
-					{
-						//TODO: camel version should be retrieved from an external request or provided as static version
-						GroupID:    "org.apache.camel",
-						ArtifactID: "camel-bom",
-						Version:    "2.22.1",
-						Type:       "pom",
-						Scope:      "import",
-					},
-				},
-			},
-		},
-		Dependencies: maven.Dependencies{
-			Dependencies: make([]maven.Dependency, 0),
-		},
+func (b *s2iPublisher) createTar(assembled build.AssembledOutput) (string, error) {
+	artifactDir, err := ioutil.TempDir("", artifactDirPrefix)
+	if err != nil {
+		return "", errors.Wrap(err, "could not create temporary dir for s2i artifacts")
 	}
 
-	//
-	// set-up dependencies
-	//
-
-	deps := &project.Dependencies
-	deps.AddGAV("org.apache.camel.k", "camel-k-runtime-jvm", version.Version)
+	tarFileName := path.Join(artifactDir, "occi.tar")
+	tarAppender, err := tar.NewAppender(tarFileName)
+	if err != nil {
+		return "", err
+	}
+	defer tarAppender.Close()
 
-	for _, d := range source.Dependencies {
-		if strings.HasPrefix(d, "camel:") {
-			artifactID := strings.TrimPrefix(d, "camel:")
+	cp := ""
+	for _, entry := range assembled.Classpath {
+		gav, err := maven.ParseGAV(entry.ID)
+		if err != nil {
+			return "", nil
+		}
 
-			if !strings.HasPrefix(artifactID, "camel-") {
-				artifactID = "camel-" + artifactID
-			}
+		tarPath := path.Join("dependencies/", gav.GroupID)
+		fileName, err := tarAppender.AddFile(entry.Location, tarPath)
+		if err != nil {
+			return "", err
+		}
 
-			deps.AddGAV("org.apache.camel", artifactID, "")
-		} else if strings.HasPrefix(d, "mvn:") {
-			mid := strings.TrimPrefix(d, "mvn:")
-			gav := strings.Replace(mid, "/", ":", -1)
+		cp += fileName + "\n"
+	}
 
-			deps.AddEncodedGAV(gav)
-		} else {
-			return maven.Project{}, fmt.Errorf("unknown dependency type: %s", d)
-		}
+	err = tarAppender.AppendData([]byte(cp), "classpath")
+	if err != nil {
+		return "", err
 	}
 
-	return project, nil
+	return tarFileName, nil
 }
diff --git a/pkg/stub/action/context/build.go b/pkg/stub/action/context/build.go
index 9d5d3b1..a815c7c 100644
--- a/pkg/stub/action/context/build.go
+++ b/pkg/stub/action/context/build.go
@@ -19,6 +19,8 @@ package action
 
 import (
 	"context"
+	"github.com/apache/camel-k/pkg/build/assemble"
+	"github.com/apache/camel-k/pkg/build/publish"
 
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 
@@ -27,13 +29,13 @@ import (
 
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/build"
-	"github.com/apache/camel-k/pkg/build/local"
 )
 
 // NewIntegrationContextBuildAction creates a new build handling action for the context
 func NewIntegrationContextBuildAction(ctx context.Context, namespace string) IntegrationContextAction {
-	builder := local.NewLocalBuilder(ctx, namespace)
-	manager := build.NewManager(ctx, namespace, builder)
+	assembler := assemble.NewMavenAssembler(ctx)
+	publisher := publish.NewS2IPublisher(ctx, namespace)
+	manager := build.NewManager(ctx, assembler, publisher)
 
 	return &integrationContextBuildAction{
 		buildManager: manager,
diff --git a/pkg/util/maven/maven.go b/pkg/util/maven/maven.go
index 1920be8..e2a9b35 100644
--- a/pkg/util/maven/maven.go
+++ b/pkg/util/maven/maven.go
@@ -18,11 +18,9 @@ limitations under the License.
 package maven
 
 import (
-	"archive/tar"
 	"bytes"
 	"encoding/xml"
 	"fmt"
-	"io"
 	"io/ioutil"
 	"os"
 	"os/exec"
@@ -32,7 +30,6 @@ import (
 
 	"github.com/apache/camel-k/version"
 
-	"github.com/apache/camel-k/pkg/build"
 	"gopkg.in/yaml.v2"
 
 	"github.com/pkg/errors"
@@ -41,13 +38,18 @@ import (
 
 const (
 	buildDirPrefix    = "maven-"
-	artifactDirPrefix = "maven-bin-"
+
 )
 
 // BuildResult --
 type BuildResult struct {
-	TarFilePath string
-	Classpath   []build.ClasspathEntry
+	Classpath []ClasspathLibrary
+}
+
+// ClasspathLibrary --
+type ClasspathLibrary struct {
+	ID       string `json:"id" yaml:"id"`
+	Location string `json:"location,omitempty" yaml:"location,omitempty"`
 }
 
 // Process takes a project description and returns a binary tar with the built artifacts
@@ -65,16 +67,7 @@ func Process(project Project) (BuildResult, error) {
 		return res, errors.Wrap(err, "could not write maven source files")
 	}
 	err = runMavenBuild(buildDir, &res)
-	if err != nil {
-		return res, err
-	}
-
-	res.TarFilePath, err = createTar(project, &res)
-	if err != nil {
-		return res, err
-	}
-
-	return res, nil
+	return res, err
 }
 
 func runMavenBuild(buildDir string, result *BuildResult) error {
@@ -95,7 +88,7 @@ func runMavenBuild(buildDir string, result *BuildResult) error {
 		return err
 	}
 
-	cp := make(map[string][]build.ClasspathEntry)
+	cp := make(map[string][]ClasspathLibrary)
 	if err := yaml.Unmarshal(content, &cp); err != nil {
 		return err
 	}
@@ -113,91 +106,6 @@ func mavenExtraOptions() string {
 	return "-Dcamel.noop=true"
 }
 
-func createTar(project Project, result *BuildResult) (string, error) {
-	artifactDir, err := ioutil.TempDir("", artifactDirPrefix)
-	if err != nil {
-		return "", errors.Wrap(err, "could not create temporary dir for maven artifacts")
-	}
-
-	tarFileName := path.Join(artifactDir, project.ArtifactID+".tar")
-	tarFile, err := os.Create(tarFileName)
-	if err != nil {
-		return "", errors.Wrap(err, "cannot create tar file "+tarFileName)
-	}
-	defer tarFile.Close()
-
-	writer := tar.NewWriter(tarFile)
-	defer writer.Close()
-
-	cp := ""
-	for _, entry := range result.Classpath {
-		gav, err := ParseGAV(entry.ID)
-		if err != nil {
-			return "", nil
-		}
-
-		tarPath := path.Join("dependencies/", gav.GroupID)
-		fileName, err := appendFileToTar(entry.Location, tarPath, writer)
-		if err != nil {
-			return "", err
-		}
-
-		cp += fileName + "\n"
-	}
-
-	err = appendDataToTar([]byte(cp), "classpath", writer)
-	if err != nil {
-		return "", err
-	}
-
-	return tarFileName, nil
-}
-
-func appendFileToTar(filePath string, tarPath string, writer *tar.Writer) (string, error) {
-	info, err := os.Stat(filePath)
-	if err != nil {
-		return "", err
-	}
-	_, fileName := path.Split(filePath)
-	if tarPath != "" {
-		fileName = path.Join(tarPath, fileName)
-	}
-
-	writer.WriteHeader(&tar.Header{
-		Name:    fileName,
-		Size:    info.Size(),
-		Mode:    int64(info.Mode()),
-		ModTime: info.ModTime(),
-	})
-
-	file, err := os.Open(filePath)
-	if err != nil {
-		return "", err
-	}
-	defer file.Close()
-
-	_, err = io.Copy(writer, file)
-	if err != nil {
-		return "", errors.Wrap(err, "cannot add file to the tar archive")
-	}
-
-	return fileName, nil
-}
-
-func appendDataToTar(data []byte, tarPath string, writer *tar.Writer) error {
-	writer.WriteHeader(&tar.Header{
-		Name: tarPath,
-		Size: int64(len(data)),
-		Mode: 0644,
-	})
-
-	_, err := writer.Write(data)
-	if err != nil {
-		return errors.Wrap(err, "cannot add data to the tar archive")
-	}
-	return nil
-}
-
 func createMavenStructure(buildDir string, project Project) error {
 	pom, err := GeneratePomFileContent(project)
 	if err != nil {
diff --git a/pkg/util/tar/appender.go b/pkg/util/tar/appender.go
new file mode 100644
index 0000000..3e4aebf
--- /dev/null
+++ b/pkg/util/tar/appender.go
@@ -0,0 +1,106 @@
+/*
+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 tar
+
+import (
+	atar "archive/tar"
+	"github.com/pkg/errors"
+	"io"
+	"os"
+	"path"
+)
+
+// Appender provides a high level abstraction over writing tar files
+type Appender struct {
+	tarFile *os.File
+	writer  *atar.Writer
+}
+
+// NewAppender creates a new tar appender
+func NewAppender(fileName string) (*Appender, error) {
+	tarFile, err := os.Create(fileName)
+	if err != nil {
+		return nil, errors.Wrap(err, "cannot create tar file "+fileName)
+	}
+
+	writer := atar.NewWriter(tarFile)
+	appender := Appender{
+		tarFile: tarFile,
+		writer:  writer,
+	}
+	return &appender, nil
+}
+
+// Close closes all handles managed by the appender
+func (t *Appender) Close() error {
+	if err := t.writer.Close(); err != nil {
+		return err
+	}
+	if err := t.tarFile.Close(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// AddFile adds a file content to the tarDir, using the original file name.
+// It returns the full path of the file inside the tar.
+func (t *Appender) AddFile(filePath string, tarDir string) (string, error) {
+	info, err := os.Stat(filePath)
+	if err != nil {
+		return "", err
+	}
+	_, fileName := path.Split(filePath)
+	if tarDir != "" {
+		fileName = path.Join(tarDir, fileName)
+	}
+
+	t.writer.WriteHeader(&atar.Header{
+		Name:    fileName,
+		Size:    info.Size(),
+		Mode:    int64(info.Mode()),
+		ModTime: info.ModTime(),
+	})
+
+	file, err := os.Open(filePath)
+	if err != nil {
+		return "", err
+	}
+	defer file.Close()
+
+	_, err = io.Copy(t.writer, file)
+	if err != nil {
+		return "", errors.Wrap(err, "cannot add file to the tar archive")
+	}
+
+	return fileName, nil
+}
+
+// AppendData appends the given content to a file inside the tar, creating it if it does not exist
+func (t *Appender) AppendData(data []byte, tarPath string) error {
+	t.writer.WriteHeader(&atar.Header{
+		Name: tarPath,
+		Size: int64(len(data)),
+		Mode: 0644,
+	})
+
+	_, err := t.writer.Write(data)
+	if err != nil {
+		return errors.Wrap(err, "cannot add data to the tar archive")
+	}
+	return nil
+}
diff --git a/test/build_manager_integration_test.go b/test/build_manager_integration_test.go
index 9ba22f0..fa7d8f8 100644
--- a/test/build_manager_integration_test.go
+++ b/test/build_manager_integration_test.go
@@ -23,7 +23,8 @@ package test
 
 import (
 	"context"
-	"github.com/apache/camel-k/pkg/build/local"
+	"github.com/apache/camel-k/pkg/build/assemble"
+	"github.com/apache/camel-k/pkg/build/publish"
 	"testing"
 	"time"
 
@@ -35,9 +36,11 @@ import (
 func TestBuildManagerBuild(t *testing.T) {
 	ctx := context.TODO()
 	namespace := getTargetNamespace()
-	buildManager := build.NewManager(ctx, namespace, local.NewLocalBuilder(ctx, namespace))
+	assembler := assemble.NewMavenAssembler(ctx)
+	publisher := publish.NewS2IPublisher(ctx, namespace)
+	buildManager := build.NewManager(ctx, assembler, publisher)
 	identifier := build.Identifier{
-		Name:   "man-test",
+		Name:      "man-test",
 		Qualifier: digest.Random(),
 	}
 	buildManager.Start(build.Request{
@@ -70,9 +73,11 @@ func TestBuildManagerFailedBuild(t *testing.T) {
 
 	ctx := context.TODO()
 	namespace := getTargetNamespace()
-	buildManager := build.NewManager(ctx, namespace, local.NewLocalBuilder(ctx, namespace))
+	assembler := assemble.NewMavenAssembler(ctx)
+	publisher := publish.NewS2IPublisher(ctx, namespace)
+	buildManager := build.NewManager(ctx, assembler, publisher)
 	identifier := build.Identifier{
-		Name:   "man-test-2",
+		Name:      "man-test-2",
 		Qualifier: digest.Random(),
 	}
 	buildManager.Start(build.Request{
diff --git a/test/local_builder_integration_test.go b/test/local_builder_integration_test.go
deleted file mode 100644
index c3f35a2..0000000
--- a/test/local_builder_integration_test.go
+++ /dev/null
@@ -1,107 +0,0 @@
-// +build integration
-
-// To enable compilation of this file in Goland, go to "Settings -> Go -> Vendoring & Build Tags -> Custom Tags" and add "integration"
-
-/*
-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 test
-
-import (
-	"context"
-	"testing"
-
-	"github.com/apache/camel-k/pkg/build"
-	"github.com/apache/camel-k/pkg/util/digest"
-	"github.com/stretchr/testify/assert"
-	"github.com/apache/camel-k/pkg/build/local"
-)
-
-func TestLocalBuild(t *testing.T) {
-
-	ctx := context.TODO()
-	builder := local.NewLocalBuilder(ctx, getTargetNamespace())
-
-	execution := builder.Build(build.Request{
-		Identifier: build.Identifier{
-			Name:      "test0",
-			Qualifier: digest.Random(),
-		},
-		Code: build.Source{
-			Content: createTimerToLogIntegrationCode(),
-		},
-	})
-
-	res := <-execution
-
-	assert.Nil(t, res.Error, "Build failed")
-}
-
-func TestLocalDoubleBuild(t *testing.T) {
-
-	ctx := context.TODO()
-	builder := local.NewLocalBuilder(ctx, getTargetNamespace())
-
-	execution1 := builder.Build(build.Request{
-		Identifier: build.Identifier{
-			Name:      "test1",
-			Qualifier: digest.Random(),
-		},
-		Code: build.Source{
-			Content: createTimerToLogIntegrationCode(),
-		},
-	})
-
-	execution2 := builder.Build(build.Request{
-		Identifier: build.Identifier{
-			Name:      "test2",
-			Qualifier: digest.Random(),
-		},
-		Code: build.Source{
-			Content: createTimerToLogIntegrationCode(),
-		},
-	})
-
-	res1 := <-execution1
-	res2 := <-execution2
-
-	assert.Nil(t, res1.Error, "Build failed")
-	assert.Nil(t, res2.Error, "Build failed")
-}
-
-func TestLocalFailedBuild(t *testing.T) {
-
-	ctx := context.TODO()
-	builder := local.NewLocalBuilder(ctx, getTargetNamespace())
-
-	execution := builder.Build(build.Request{
-		Identifier: build.Identifier{
-			Name:      "test3",
-			Qualifier: digest.Random(),
-		},
-		Code: build.Source{
-			Content: createTimerToLogIntegrationCode(),
-		},
-		Dependencies: []string{
-			"camel:cippalippa",
-		},
-	})
-
-	res := <-execution
-
-	assert.NotNil(t, res.Error, "Build should fail")
-}