You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@streampipes.apache.org by ri...@apache.org on 2022/08/01 11:52:36 UTC

[incubator-streampipes] branch STREAMPIPES-545 updated: [STREAMPIPES-565] Add import feature to configuration

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

riemer pushed a commit to branch STREAMPIPES-545
in repository https://gitbox.apache.org/repos/asf/incubator-streampipes.git


The following commit(s) were added to refs/heads/STREAMPIPES-545 by this push:
     new 59129d3cd [STREAMPIPES-565] Add import feature to configuration
59129d3cd is described below

commit 59129d3cd32da5b7a68bf3f93abfe3d3c5968e85
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Mon Aug 1 13:52:23 2022 +0200

    [STREAMPIPES-565] Add import feature to configuration
---
 .../backend/StreamPipesResourceConfig.java         |   1 +
 .../streampipes/commons/zip/ZipFileExtractor.java  |  34 ++++-
 streampipes-data-export/pom.xml                    |   5 +
 .../apache/streampipes/export/ImportManager.java   |  39 ++++++
 .../export/constants/ExportConstants.java          |  14 +-
 .../export/dataimport/ImportGenerator.java         | 106 +++++++++++++++
 .../export/dataimport/PerformImportGenerator.java  | 142 +++++++++++++++++++++
 .../export/dataimport/PreviewImportGenerator.java  | 102 +++++++++++++++
 .../export/generator/ExportPackageGenerator.java   |   7 +-
 .../export/generator/ZipFileBuilder.java           |   4 +-
 .../PermissionInfo.java}                           |  30 +++--
 .../export/resolver/AbstractResolver.java          |  12 ++
 .../export/resolver/AdapterResolver.java           |  25 +++-
 .../export/resolver/DashboardResolver.java         |  21 ++-
 .../export/resolver/DashboardWidgetResolver.java   |  21 ++-
 .../export/resolver/DataSourceResolver.java        |  21 ++-
 .../export/resolver/DataViewResolver.java          |  21 ++-
 .../export/resolver/DataViewWidgetResolver.java    |  21 ++-
 .../export/resolver/MeasurementResolver.java       |  21 ++-
 .../export/resolver/PipelineResolver.java          |  23 +++-
 .../model/datalake/DataLakeMeasure.java            |  12 ++
 .../model/export/AssetExportConfiguration.java     |  40 ++++++
 .../export/StreamPipesApplicationPackage.java      |  14 ++
 .../rest/impl/admin/DataImportResource.java        |  63 +++++++++
 .../src/lib/model/gen/streampipes-model.ts         |  38 ++++--
 ui/src/app/assets/assets.module.ts                 |   2 +
 .../asset-overview/asset-overview.component.ts     |  38 +++++-
 .../create-asset-dialog.component.html             |  50 ++++++++
 .../create-asset-dialog.component.scss}            |  15 +--
 .../create-asset/create-asset-dialog.component.ts  |  59 +++++++++
 .../export/data-export-import.component.html       |   6 +-
 .../export/data-export-import.component.ts         |  17 ++-
 .../configuration/export/data-export.service.ts    |  45 ++++++-
 .../export-dialog/data-export-dialog.component.ts  |   1 +
 .../data-import-dialog.component.html              |  76 +++++++++++
 .../import-dialog/data-import-dialog.component.ts  |  62 +++++++++
 36 files changed, 1138 insertions(+), 70 deletions(-)

diff --git a/streampipes-backend/src/main/java/org/apache/streampipes/backend/StreamPipesResourceConfig.java b/streampipes-backend/src/main/java/org/apache/streampipes/backend/StreamPipesResourceConfig.java
index 4924a5352..69df336fb 100644
--- a/streampipes-backend/src/main/java/org/apache/streampipes/backend/StreamPipesResourceConfig.java
+++ b/streampipes-backend/src/main/java/org/apache/streampipes/backend/StreamPipesResourceConfig.java
@@ -63,6 +63,7 @@ public class StreamPipesResourceConfig extends ResourceConfig {
         register(DashboardWidget.class);
         register(Dashboard.class);
         register(DataExportResource.class);
+        register(DataImportResource.class);
         register(DataLakeImageResource.class);
         register(DataLakeResourceV3.class);
         register(DataLakeMeasureResourceV3.class);
diff --git a/streampipes-commons/src/main/java/org/apache/streampipes/commons/zip/ZipFileExtractor.java b/streampipes-commons/src/main/java/org/apache/streampipes/commons/zip/ZipFileExtractor.java
index 5ada0f4f0..7fdaae804 100644
--- a/streampipes-commons/src/main/java/org/apache/streampipes/commons/zip/ZipFileExtractor.java
+++ b/streampipes-commons/src/main/java/org/apache/streampipes/commons/zip/ZipFileExtractor.java
@@ -17,10 +17,10 @@
  */
 package org.apache.streampipes.commons.zip;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
+import java.io.*;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipInputStream;
 
@@ -32,6 +32,32 @@ public class ZipFileExtractor {
     this.zipInputStream = zipInputStream;
   }
 
+  // TODO used by export feature - extend this to support binaries
+  public Map<String, String> extractZipToStringMap() throws IOException {
+    byte[] buffer = new byte[1024];
+    Map<String, String> entries = new HashMap<>();
+    ZipInputStream zis = new ZipInputStream(zipInputStream);
+    ZipEntry zipEntry = zis.getNextEntry();
+    while (zipEntry != null) {
+      ByteArrayOutputStream fos = new ByteArrayOutputStream();
+      int len;
+      while ((len = zis.read(buffer)) > 0) {
+        fos.write(buffer, 0, len);
+      }
+      entries.put(sanitizeName(zipEntry.getName()), fos.toString(StandardCharsets.UTF_8));
+      fos.close();
+      zipEntry = zis.getNextEntry();
+    }
+    zis.closeEntry();
+    zis.close();
+
+    return entries;
+  }
+
+  private String sanitizeName(String name) {
+    return name.split("\\.")[0];
+  }
+
   public void extractZipToFile(String targetFolder) throws IOException {
     File destDir = new File(targetFolder);
     if (!destDir.exists()) {
diff --git a/streampipes-data-export/pom.xml b/streampipes-data-export/pom.xml
index 216981d8b..ee886975d 100644
--- a/streampipes-data-export/pom.xml
+++ b/streampipes-data-export/pom.xml
@@ -34,6 +34,11 @@
             <artifactId>streampipes-model</artifactId>
             <version>0.70.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.streampipes</groupId>
+            <artifactId>streampipes-resource-management</artifactId>
+            <version>0.70.0-SNAPSHOT</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.streampipes</groupId>
             <artifactId>streampipes-storage-management</artifactId>
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/ImportManager.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/ImportManager.java
new file mode 100644
index 000000000..58bfe9582
--- /dev/null
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/ImportManager.java
@@ -0,0 +1,39 @@
+/*
+ * 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.streampipes.export;
+
+import org.apache.streampipes.export.dataimport.PerformImportGenerator;
+import org.apache.streampipes.export.dataimport.PreviewImportGenerator;
+import org.apache.streampipes.model.export.AssetExportConfiguration;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class ImportManager {
+
+  public static AssetExportConfiguration getImportPreview(InputStream packageZipStream) throws IOException {
+    return new PreviewImportGenerator().generate(packageZipStream);
+  }
+
+  public static void performImport(InputStream packageZipStream,
+                                   AssetExportConfiguration exportConfiguration,
+                                   String ownerSid) throws IOException {
+    new PerformImportGenerator(exportConfiguration, ownerSid).generate(packageZipStream);
+  }
+}
diff --git a/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts b/streampipes-data-export/src/main/java/org/apache/streampipes/export/constants/ExportConstants.java
similarity index 72%
copy from ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts
copy to streampipes-data-export/src/main/java/org/apache/streampipes/export/constants/ExportConstants.java
index 5ef2a7fb6..80ec171e1 100644
--- a/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/constants/ExportConstants.java
@@ -16,17 +16,9 @@
  *
  */
 
-import { Component, OnInit } from '@angular/core';
-
-@Component({
-  selector: 'sp-data-import-dialog',
-  templateUrl: './data-import-dialog.component.html',
-  styleUrls: ['./data-import-dialog.component.scss'],
-})
-export class SpDataImportDialogComponent implements OnInit {
-
-  ngOnInit(): void {
-  }
+package org.apache.streampipes.export.constants;
 
+public class ExportConstants {
 
+  public static final String MANIFEST = "manifest";
 }
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/dataimport/ImportGenerator.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/dataimport/ImportGenerator.java
new file mode 100644
index 000000000..105f99fc7
--- /dev/null
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/dataimport/ImportGenerator.java
@@ -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 org.apache.streampipes.export.dataimport;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.streampipes.commons.zip.ZipFileExtractor;
+import org.apache.streampipes.export.constants.ExportConstants;
+import org.apache.streampipes.export.utils.SerializationUtils;
+import org.apache.streampipes.model.export.StreamPipesApplicationPackage;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+public abstract class ImportGenerator<T> {
+
+  protected ObjectMapper spMapper;
+  protected ObjectMapper defaultMapper;
+
+  public ImportGenerator() {
+    this.spMapper = SerializationUtils.getSpObjectMapper();
+    this.defaultMapper = SerializationUtils.getDefaultObjectMapper();
+  }
+
+  public T generate(InputStream inputStream) throws IOException {
+    Map<String, String> previewFiles = new ZipFileExtractor(inputStream).extractZipToStringMap();
+
+    var manifest = getManifest(previewFiles);
+
+    for (String assetId: manifest.getAssets()) {
+      handleAsset(previewFiles, assetId);
+    }
+
+    for (String adapterId: manifest.getAdapters()) {
+      handleAdapter(previewFiles.get(adapterId), adapterId);
+    }
+
+    for(String dashboardId: manifest.getDashboards()) {
+      handleDashboard(previewFiles.get(dashboardId), dashboardId);
+    }
+
+    for (String dataViewId: manifest.getDataViews()) {
+      handleDataView(previewFiles.get(dataViewId), dataViewId);
+    }
+
+    for (String dataSourceId: manifest.getDataSources()) {
+      handleDataSource(previewFiles.get(dataSourceId), dataSourceId);
+    }
+
+    for (String pipelineId: manifest.getPipelines()) {
+      handlePipeline(previewFiles.get(pipelineId), pipelineId);
+    }
+
+    for (String measurementId: manifest.getDataLakeMeasures()) {
+      handleDataLakeMeasure(previewFiles.get(measurementId), measurementId);
+    }
+
+    for (String dashboardWidgetId: manifest.getDashboardWidgets()) {
+      handleDashboardWidget(previewFiles.get(dashboardWidgetId), dashboardWidgetId);
+    }
+
+    for (String dataViewWidgetId: manifest.getDataViewWidgets()) {
+      handleDataViewWidget(previewFiles.get(dataViewWidgetId), dataViewWidgetId);
+    }
+
+    afterResourcesCreated();
+
+    return getReturnObject();
+  }
+
+  private StreamPipesApplicationPackage getManifest(Map<String, String> previewFiles) throws JsonProcessingException {
+    return this.defaultMapper.readValue(previewFiles.get(ExportConstants.MANIFEST), StreamPipesApplicationPackage.class);
+  }
+
+  protected abstract void handleAsset(Map<String, String> previewFiles, String assetId) throws IOException;
+  protected abstract void handleAdapter(String document, String adapterId) throws JsonProcessingException;
+  protected abstract void handleDashboard(String document, String dashboardId) throws JsonProcessingException;
+  protected abstract void handleDataView(String document, String dataViewId) throws JsonProcessingException;
+  protected abstract void handleDataSource(String document, String dataSourceId) throws JsonProcessingException;
+  protected abstract void handlePipeline(String document, String pipelineId) throws JsonProcessingException;
+  protected abstract void handleDataLakeMeasure(String document, String dataLakeMeasureId) throws JsonProcessingException;
+  protected abstract void handleDashboardWidget(String document, String dashboardWidgetId) throws JsonProcessingException;
+  protected abstract void handleDataViewWidget(String document, String dataViewWidget) throws JsonProcessingException;
+
+  protected abstract T getReturnObject();
+
+  protected abstract void afterResourcesCreated();
+
+}
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/dataimport/PerformImportGenerator.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/dataimport/PerformImportGenerator.java
new file mode 100644
index 000000000..cf4769aae
--- /dev/null
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/dataimport/PerformImportGenerator.java
@@ -0,0 +1,142 @@
+/*
+ * 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.streampipes.export.dataimport;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.streampipes.export.model.PermissionInfo;
+import org.apache.streampipes.export.resolver.*;
+import org.apache.streampipes.model.SpDataStream;
+import org.apache.streampipes.model.connect.adapter.AdapterDescription;
+import org.apache.streampipes.model.dashboard.DashboardModel;
+import org.apache.streampipes.model.datalake.DataLakeMeasure;
+import org.apache.streampipes.model.export.AssetExportConfiguration;
+import org.apache.streampipes.model.export.ExportItem;
+import org.apache.streampipes.model.pipeline.Pipeline;
+import org.apache.streampipes.resource.management.PermissionResourceManager;
+import org.apache.streampipes.storage.api.INoSqlStorage;
+import org.apache.streampipes.storage.management.StorageDispatcher;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class PerformImportGenerator extends ImportGenerator<Void> {
+
+  private AssetExportConfiguration config;
+  private INoSqlStorage storage;
+  private Set<PermissionInfo> permissionsToStore = new HashSet<>();
+  private String ownerSid;
+
+  public PerformImportGenerator(AssetExportConfiguration config,
+                                String ownerSid) {
+    this.config = config;
+    this.storage = StorageDispatcher.INSTANCE.getNoSqlStore();
+    this.ownerSid = ownerSid;
+  }
+
+
+  @Override
+  protected void handleAsset(Map<String, String> previewFiles, String assetId) throws IOException {
+    storage.getGenericStorage().create(previewFiles.get(assetId));
+  }
+
+  @Override
+  protected void handleAdapter(String document, String adapterId) throws JsonProcessingException {
+    if (shouldStore(adapterId, config.getAdapters())) {
+      new AdapterResolver().writeDocument(document);
+      permissionsToStore.add(new PermissionInfo(adapterId, AdapterDescription.class));
+    }
+  }
+
+  @Override
+  protected void handleDashboard(String document, String dashboardId) throws JsonProcessingException {
+    if (shouldStore(dashboardId, config.getDashboards())) {
+      new DashboardResolver().writeDocument(document);
+      permissionsToStore.add(new PermissionInfo(dashboardId, DashboardModel.class));
+    }
+  }
+
+  @Override
+  protected void handleDataView(String document, String dataViewId) throws JsonProcessingException {
+    if (shouldStore(dataViewId, config.getDataViews())) {
+      new DataViewResolver().writeDocument(document);
+      permissionsToStore.add(new PermissionInfo(dataViewId, DashboardModel.class));
+    }
+  }
+
+  @Override
+  protected void handleDataSource(String document, String dataSourceId) throws JsonProcessingException {
+    if (shouldStore(dataSourceId, config.getDataSources())) {
+      new DataSourceResolver().writeDocument(document);
+      permissionsToStore.add(new PermissionInfo(dataSourceId, SpDataStream.class));
+    }
+  }
+
+  @Override
+  protected void handlePipeline(String document, String pipelineId) throws JsonProcessingException {
+    if (shouldStore(pipelineId, config.getPipelines())) {
+      new PipelineResolver().writeDocument(document);
+      permissionsToStore.add(new PermissionInfo(pipelineId, Pipeline.class));
+    }
+  }
+
+  @Override
+  protected void handleDataLakeMeasure(String document, String dataLakeMeasureId) throws JsonProcessingException {
+    if (shouldStore(dataLakeMeasureId, config.getDataLakeMeasures())) {
+      new MeasurementResolver().writeDocument(document);
+      permissionsToStore.add(new PermissionInfo(dataLakeMeasureId, DataLakeMeasure.class));
+    }
+  }
+
+  @Override
+  protected void handleDashboardWidget(String document, String dashboardWidgetId) throws JsonProcessingException {
+    new DashboardWidgetResolver().writeDocument(document);
+  }
+
+  @Override
+  protected void handleDataViewWidget(String document, String dataViewWidget) throws JsonProcessingException {
+    new DataViewWidgetResolver().writeDocument(document);
+  }
+
+  @Override
+  protected Void getReturnObject() {
+    return null;
+  }
+
+  @Override
+  protected void afterResourcesCreated() {
+    var resourceManager = new PermissionResourceManager();
+    this.permissionsToStore
+      .forEach(info -> resourceManager.createDefault(
+        info.getInstanceId(),
+        info.getInstanceClass(),
+        this.ownerSid,
+        true));
+  }
+
+  private boolean shouldStore(String adapterId,
+                              Set<ExportItem> adapters) {
+    return adapters
+      .stream()
+      .filter(item -> item.getResourceId().equals(adapterId))
+      .allMatch(ExportItem::isSelected);
+  }
+
+}
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/dataimport/PreviewImportGenerator.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/dataimport/PreviewImportGenerator.java
new file mode 100644
index 000000000..a4ace764a
--- /dev/null
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/dataimport/PreviewImportGenerator.java
@@ -0,0 +1,102 @@
+/*
+ * 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.streampipes.export.dataimport;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import org.apache.streampipes.export.resolver.*;
+import org.apache.streampipes.model.export.AssetExportConfiguration;
+import org.apache.streampipes.model.export.ExportItem;
+
+import java.util.Map;
+import java.util.function.Consumer;
+
+public class PreviewImportGenerator extends ImportGenerator<AssetExportConfiguration> {
+
+  private AssetExportConfiguration importConfig;
+
+  public PreviewImportGenerator() {
+    super();
+    this.importConfig = new AssetExportConfiguration();
+
+  }
+
+  private void addExportItem(String id,
+                             String name,
+                             Consumer<ExportItem> addAdapter) {
+    var item = new ExportItem(id, name, true);
+    addAdapter.accept(item);
+  }
+
+
+  @Override
+  protected void handleAsset(Map<String, String> previewFiles, String assetId) throws JsonProcessingException {
+    Map<String, Object> assetDescription = this.defaultMapper.readValue(previewFiles.get(assetId), new TypeReference<Map<String, Object>>() {});
+    importConfig.addAsset(new ExportItem(assetId, String.valueOf(assetDescription.get("assetName")), true));
+  }
+
+  @Override
+  protected void handleAdapter(String document, String adapterId) throws JsonProcessingException {
+    addExportItem(adapterId, new AdapterResolver().readDocument(document).getName(), importConfig::addAdapter);
+  }
+
+  @Override
+  protected void handleDashboard(String document, String dashboardId) throws JsonProcessingException {
+    addExportItem(dashboardId, new DashboardResolver().readDocument(document).getName(), importConfig::addDashboard);
+  }
+
+  @Override
+  protected void handleDataView(String document, String dataViewId) throws JsonProcessingException {
+    addExportItem(dataViewId, new DataViewResolver().readDocument(document).getName(), importConfig::addDataView);
+  }
+
+  @Override
+  protected void handleDataSource(String document, String dataSourceId) throws JsonProcessingException {
+    addExportItem(dataSourceId, new DataSourceResolver().readDocument(document).getName(), importConfig::addDataSource);
+  }
+
+  @Override
+  protected void handlePipeline(String document, String pipelineId) throws JsonProcessingException {
+    addExportItem(pipelineId, new PipelineResolver().readDocument(document).getName(), importConfig::addPipeline);
+  }
+
+  @Override
+  protected void handleDataLakeMeasure(String document, String measurementId) throws JsonProcessingException {
+    addExportItem(measurementId, new MeasurementResolver().readDocument(document).getMeasureName(), importConfig::addDataLakeMeasure);
+  }
+
+  @Override
+  protected void handleDashboardWidget(String document, String dashboardWidgetId) {
+
+  }
+
+  @Override
+  protected void handleDataViewWidget(String document, String dataViewWidget) {
+
+  }
+
+  @Override
+  protected AssetExportConfiguration getReturnObject() {
+    return this.importConfig;
+  }
+
+  @Override
+  protected void afterResourcesCreated() {
+  }
+}
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/generator/ExportPackageGenerator.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/generator/ExportPackageGenerator.java
index 832ae826b..5070423ba 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/generator/ExportPackageGenerator.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/generator/ExportPackageGenerator.java
@@ -54,7 +54,7 @@ public class ExportPackageGenerator {
       .getAssetExportConfiguration()
       .stream()
       .map(AssetExportConfiguration::getAssetId)
-      .collect(Collectors.toList()));
+      .collect(Collectors.toList()), manifest);
 
     this.exportConfiguration.getAssetExportConfiguration().forEach(config -> {
 
@@ -135,11 +135,14 @@ public class ExportPackageGenerator {
   }
 
   private void addAssets(ZipFileBuilder builder,
-                         List<String> assetIds) {
+                         List<String> assetIds,
+                         StreamPipesApplicationPackage manifest) {
     assetIds.forEach(assetId -> {
       try {
         var asset = getAsset(assetId);
+        asset.remove("_rev");
         builder.addText(String.valueOf(asset.get("_id")), this.defaultMapper.writeValueAsString(asset));
+        manifest.addAsset(String.valueOf(asset.get("_id")));
       } catch (IOException e) {
         e.printStackTrace();
       }
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/generator/ZipFileBuilder.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/generator/ZipFileBuilder.java
index ea93d03e6..58e8948bb 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/generator/ZipFileBuilder.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/generator/ZipFileBuilder.java
@@ -18,6 +18,8 @@
 
 package org.apache.streampipes.export.generator;
 
+import org.apache.streampipes.export.constants.ExportConstants;
+
 import java.io.*;
 import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
@@ -84,7 +86,7 @@ public class ZipFileBuilder {
       addZipEntry(documentKey + ".json", document, out, buffer);
     }
 
-    addZipEntry("manifest.json", asBytes(manifest), out, buffer);
+    addZipEntry(ExportConstants.MANIFEST + ".json", asBytes(manifest), out, buffer);
     out.closeEntry();
     out.close();
     return outputStream.toByteArray();
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AdapterResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/model/PermissionInfo.java
similarity index 58%
copy from streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AdapterResolver.java
copy to streampipes-data-export/src/main/java/org/apache/streampipes/export/model/PermissionInfo.java
index 45ae530c4..5bf2c8bd8 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AdapterResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/model/PermissionInfo.java
@@ -16,21 +16,31 @@
  *
  */
 
+package org.apache.streampipes.export.model;
 
-package org.apache.streampipes.export.resolver;
+public class PermissionInfo {
 
-import org.apache.streampipes.model.connect.adapter.AdapterDescription;
-import org.apache.streampipes.model.export.ExportItem;
+  private String instanceId;
+  private Class<?> instanceClass;
 
-public class AdapterResolver extends AbstractResolver<AdapterDescription> {
+  public PermissionInfo(String instanceId, Class<?> className) {
+    this.instanceId = instanceId;
+    this.instanceClass = className;
+  }
+
+  public String getInstanceId() {
+    return instanceId;
+  }
+
+  public void setInstanceId(String instanceId) {
+    this.instanceId = instanceId;
+  }
 
-  @Override
-  public AdapterDescription findDocument(String resourceId) {
-    return getNoSqlStore().getAdapterInstanceStorage().getAdapter(resourceId);
+  public Class<?> getInstanceClass() {
+    return instanceClass;
   }
 
-  @Override
-  public ExportItem convert(AdapterDescription document) {
-    return new ExportItem(document.getElementId(), document.getName(), true);
+  public void setInstanceClass(Class<?> instanceClass) {
+    this.instanceClass = instanceClass;
   }
 }
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AbstractResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AbstractResolver.java
index be8695691..6b9689abd 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AbstractResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AbstractResolver.java
@@ -19,6 +19,7 @@
 package org.apache.streampipes.export.resolver;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.streampipes.export.utils.SerializationUtils;
 import org.apache.streampipes.model.assets.AssetLink;
 import org.apache.streampipes.model.export.ExportItem;
@@ -30,6 +31,11 @@ import java.util.stream.Collectors;
 
 public abstract class AbstractResolver<T> {
 
+  protected ObjectMapper spMapper;
+
+  public AbstractResolver() {
+    this.spMapper = SerializationUtils.getSpObjectMapper();
+  }
 
   public Set<ExportItem> resolve(Set<AssetLink> assetLinks) {
     return assetLinks
@@ -49,5 +55,11 @@ public abstract class AbstractResolver<T> {
 
   public abstract T findDocument(String resourceId);
 
+  public abstract T readDocument(String serializedDoc) throws JsonProcessingException;
+
   public abstract ExportItem convert(T document);
+
+  public abstract void writeDocument(String document) throws JsonProcessingException;
+
+  protected abstract T serializeDocument(String document) throws JsonProcessingException;
 }
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AdapterResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AdapterResolver.java
index 45ae530c4..2ad74c035 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AdapterResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/AdapterResolver.java
@@ -19,18 +19,41 @@
 
 package org.apache.streampipes.export.resolver;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.streampipes.export.utils.SerializationUtils;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
+import org.apache.streampipes.model.connect.adapter.AdapterStreamDescription;
 import org.apache.streampipes.model.export.ExportItem;
 
 public class AdapterResolver extends AbstractResolver<AdapterDescription> {
 
   @Override
   public AdapterDescription findDocument(String resourceId) {
-    return getNoSqlStore().getAdapterInstanceStorage().getAdapter(resourceId);
+    var doc =  getNoSqlStore().getAdapterInstanceStorage().getAdapter(resourceId);
+    doc.setRev(null);
+    if (doc instanceof AdapterStreamDescription) {
+      ((AdapterStreamDescription) doc).setRunning(false);
+    }
+    return doc;
+  }
+
+  @Override
+  public AdapterDescription readDocument(String serializedDoc) throws JsonProcessingException {
+    return SerializationUtils.getSpObjectMapper().readValue(serializedDoc, AdapterDescription.class);
   }
 
   @Override
   public ExportItem convert(AdapterDescription document) {
     return new ExportItem(document.getElementId(), document.getName(), true);
   }
+
+  @Override
+  public void writeDocument(String document) throws JsonProcessingException {
+    getNoSqlStore().getAdapterInstanceStorage().storeAdapter(serializeDocument(document));
+  }
+
+  @Override
+  protected AdapterDescription serializeDocument(String document) throws JsonProcessingException {
+    return this.spMapper.readValue(document, AdapterDescription.class);
+  }
 }
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DashboardResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DashboardResolver.java
index 3c5c1ffa6..1b3ea1348 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DashboardResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DashboardResolver.java
@@ -18,6 +18,8 @@
 
 package org.apache.streampipes.export.resolver;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.streampipes.export.utils.SerializationUtils;
 import org.apache.streampipes.model.dashboard.DashboardItem;
 import org.apache.streampipes.model.dashboard.DashboardModel;
 import org.apache.streampipes.model.export.ExportItem;
@@ -29,7 +31,14 @@ public class DashboardResolver extends AbstractResolver<DashboardModel> {
 
   @Override
   public DashboardModel findDocument(String resourceId) {
-    return getNoSqlStore().getDashboardStorage().getDashboard(resourceId);
+    var doc = getNoSqlStore().getDashboardStorage().getDashboard(resourceId);
+    doc.setCouchDbRev(null);
+    return doc;
+  }
+
+  @Override
+  public DashboardModel readDocument(String serializedDoc) throws JsonProcessingException {
+    return SerializationUtils.getSpObjectMapper().readValue(serializedDoc, DashboardModel.class);
   }
 
   @Override
@@ -37,6 +46,16 @@ public class DashboardResolver extends AbstractResolver<DashboardModel> {
     return new ExportItem(document.getCouchDbId(), document.getName(), true);
   }
 
+  @Override
+  public void writeDocument(String document) throws JsonProcessingException {
+    getNoSqlStore().getDashboardStorage().storeDashboard(serializeDocument(document));
+  }
+
+  @Override
+  protected DashboardModel serializeDocument(String document) throws JsonProcessingException {
+    return this.spMapper.readValue(document, DashboardModel.class);
+  }
+
   public List<String> getWidgets(String resourceId) {
     var document = findDocument(resourceId);
     return document.getWidgets().stream().map(DashboardItem::getId).collect(Collectors.toList());
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DashboardWidgetResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DashboardWidgetResolver.java
index 7cf8c7381..5740b6230 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DashboardWidgetResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DashboardWidgetResolver.java
@@ -18,6 +18,8 @@
 
 package org.apache.streampipes.export.resolver;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.streampipes.export.utils.SerializationUtils;
 import org.apache.streampipes.model.dashboard.DashboardWidgetModel;
 import org.apache.streampipes.model.export.ExportItem;
 
@@ -25,11 +27,28 @@ public class DashboardWidgetResolver extends AbstractResolver<DashboardWidgetMod
 
   @Override
   public DashboardWidgetModel findDocument(String resourceId) {
-    return getNoSqlStore().getDashboardWidgetStorage().getDashboardWidget(resourceId);
+    var doc = getNoSqlStore().getDashboardWidgetStorage().getDashboardWidget(resourceId);
+    doc.setRev(null);
+    return doc;
+  }
+
+  @Override
+  public DashboardWidgetModel readDocument(String serializedDoc) throws JsonProcessingException {
+    return SerializationUtils.getSpObjectMapper().readValue(serializedDoc, DashboardWidgetModel.class);
   }
 
   @Override
   public ExportItem convert(DashboardWidgetModel document) {
     return new ExportItem(document.getId(), document.getVisualizationName(), true);
   }
+
+  @Override
+  public void writeDocument(String document) throws JsonProcessingException {
+    getNoSqlStore().getDashboardWidgetStorage().storeDashboardWidget(serializeDocument(document));
+  }
+
+  @Override
+  protected DashboardWidgetModel serializeDocument(String document) throws JsonProcessingException {
+    return this.spMapper.readValue(document, DashboardWidgetModel.class);
+  }
 }
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataSourceResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataSourceResolver.java
index 6f8083d96..a218e1d93 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataSourceResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataSourceResolver.java
@@ -18,6 +18,8 @@
 
 package org.apache.streampipes.export.resolver;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.streampipes.export.utils.SerializationUtils;
 import org.apache.streampipes.model.SpDataStream;
 import org.apache.streampipes.model.export.ExportItem;
 
@@ -25,11 +27,28 @@ public class DataSourceResolver extends AbstractResolver<SpDataStream> {
 
   @Override
   public SpDataStream findDocument(String resourceId) {
-    return getNoSqlStore().getDataStreamStorage().getElementById(resourceId);
+    var doc = getNoSqlStore().getDataStreamStorage().getElementById(resourceId);
+    doc.setRev(null);
+    return doc;
+  }
+
+  @Override
+  public SpDataStream readDocument(String serializedDoc) throws JsonProcessingException {
+    return SerializationUtils.getSpObjectMapper().readValue(serializedDoc, SpDataStream.class);
   }
 
   @Override
   public ExportItem convert(SpDataStream document) {
     return new ExportItem(document.getElementId(), document.getName(), true);
   }
+
+  @Override
+  public void writeDocument(String document) throws JsonProcessingException {
+    getNoSqlStore().getDataStreamStorage().createElement(serializeDocument(document));
+  }
+
+  @Override
+  protected SpDataStream serializeDocument(String document) throws JsonProcessingException {
+    return this.spMapper.readValue(document, SpDataStream.class);
+  }
 }
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataViewResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataViewResolver.java
index d5b501502..ecaaa3e2e 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataViewResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataViewResolver.java
@@ -19,6 +19,8 @@
 
 package org.apache.streampipes.export.resolver;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.streampipes.export.utils.SerializationUtils;
 import org.apache.streampipes.model.dashboard.DashboardItem;
 import org.apache.streampipes.model.dashboard.DashboardModel;
 import org.apache.streampipes.model.export.ExportItem;
@@ -30,7 +32,14 @@ public class DataViewResolver extends AbstractResolver<DashboardModel> {
 
   @Override
   public DashboardModel findDocument(String resourceId) {
-    return getNoSqlStore().getDataExplorerDashboardStorage().getDashboard(resourceId);
+    var doc = getNoSqlStore().getDataExplorerDashboardStorage().getDashboard(resourceId);
+    doc.setCouchDbRev(null);
+    return doc;
+  }
+
+  @Override
+  public DashboardModel readDocument(String serializedDoc) throws JsonProcessingException {
+    return SerializationUtils.getSpObjectMapper().readValue(serializedDoc, DashboardModel.class);
   }
 
   @Override
@@ -38,6 +47,16 @@ public class DataViewResolver extends AbstractResolver<DashboardModel> {
     return new ExportItem(document.getCouchDbId(), document.getName(), true);
   }
 
+  @Override
+  public void writeDocument(String document) throws JsonProcessingException {
+    getNoSqlStore().getDataExplorerDashboardStorage().storeDashboard(serializeDocument(document));
+  }
+
+  @Override
+  protected DashboardModel serializeDocument(String document) throws JsonProcessingException {
+    return this.spMapper.readValue(document, DashboardModel.class);
+  }
+
   public List<String> getWidgets(String resourceId) {
     var document = findDocument(resourceId);
     return document.getWidgets().stream().map(DashboardItem::getId).collect(Collectors.toList());
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataViewWidgetResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataViewWidgetResolver.java
index bdeb55031..bf46c0e3f 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataViewWidgetResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/DataViewWidgetResolver.java
@@ -18,6 +18,8 @@
 
 package org.apache.streampipes.export.resolver;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.streampipes.export.utils.SerializationUtils;
 import org.apache.streampipes.model.datalake.DataExplorerWidgetModel;
 import org.apache.streampipes.model.export.ExportItem;
 
@@ -25,11 +27,28 @@ public class DataViewWidgetResolver extends AbstractResolver<DataExplorerWidgetM
 
   @Override
   public DataExplorerWidgetModel findDocument(String resourceId) {
-    return getNoSqlStore().getDataExplorerWidgetStorage().getDataExplorerWidget(resourceId);
+    var doc = getNoSqlStore().getDataExplorerWidgetStorage().getDataExplorerWidget(resourceId);
+    doc.setRev(null);
+    return doc;
+  }
+
+  @Override
+  public DataExplorerWidgetModel readDocument(String serializedDoc) throws JsonProcessingException {
+   return SerializationUtils.getSpObjectMapper().readValue(serializedDoc, DataExplorerWidgetModel.class);
   }
 
   @Override
   public ExportItem convert(DataExplorerWidgetModel document) {
     return new ExportItem(document.getId(), document.getWidgetId(), true);
   }
+
+  @Override
+  public void writeDocument(String document) throws JsonProcessingException {
+    getNoSqlStore().getDataExplorerWidgetStorage().storeDataExplorerWidget(serializeDocument(document));
+  }
+
+  @Override
+  protected DataExplorerWidgetModel serializeDocument(String document) throws JsonProcessingException {
+    return this.spMapper.readValue(document, DataExplorerWidgetModel.class);
+  }
 }
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/MeasurementResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/MeasurementResolver.java
index da155d86c..f44b21741 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/MeasurementResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/MeasurementResolver.java
@@ -18,6 +18,8 @@
 
 package org.apache.streampipes.export.resolver;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.streampipes.export.utils.SerializationUtils;
 import org.apache.streampipes.model.datalake.DataLakeMeasure;
 import org.apache.streampipes.model.export.ExportItem;
 
@@ -25,11 +27,28 @@ public class MeasurementResolver extends AbstractResolver<DataLakeMeasure> {
 
   @Override
   public DataLakeMeasure findDocument(String resourceId) {
-    return getNoSqlStore().getDataLakeStorage().findOne(resourceId);
+    var doc = getNoSqlStore().getDataLakeStorage().findOne(resourceId);
+    doc.setRev(null);
+    return doc;
+  }
+
+  @Override
+  public DataLakeMeasure readDocument(String serializedDoc) throws JsonProcessingException {
+    return SerializationUtils.getSpObjectMapper().readValue(serializedDoc, DataLakeMeasure.class);
   }
 
   @Override
   public ExportItem convert(DataLakeMeasure document) {
     return new ExportItem(document.getElementId(), document.getMeasureName(), true);
   }
+
+  @Override
+  public void writeDocument(String document) throws JsonProcessingException {
+    getNoSqlStore().getDataLakeStorage().storeDataLakeMeasure(serializeDocument(document));
+  }
+
+  @Override
+  protected DataLakeMeasure serializeDocument(String document) throws JsonProcessingException {
+    return this.spMapper.readValue(document, DataLakeMeasure.class);
+  }
 }
diff --git a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/PipelineResolver.java b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/PipelineResolver.java
index 93a77acf8..326278dcc 100644
--- a/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/PipelineResolver.java
+++ b/streampipes-data-export/src/main/java/org/apache/streampipes/export/resolver/PipelineResolver.java
@@ -18,6 +18,8 @@
 
 package org.apache.streampipes.export.resolver;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.streampipes.export.utils.SerializationUtils;
 import org.apache.streampipes.model.export.ExportItem;
 import org.apache.streampipes.model.pipeline.Pipeline;
 
@@ -25,11 +27,30 @@ public class PipelineResolver extends AbstractResolver<Pipeline> {
 
   @Override
   public Pipeline findDocument(String resourceId) {
-    return getNoSqlStore().getPipelineStorageAPI().getPipeline(resourceId);
+    var doc = getNoSqlStore().getPipelineStorageAPI().getPipeline(resourceId);
+    doc.setRev(null);
+    doc.setRestartOnSystemReboot(false);
+    doc.setRunning(false);
+    return doc;
+  }
+
+  @Override
+  public Pipeline readDocument(String serializedDoc) throws JsonProcessingException {
+    return SerializationUtils.getSpObjectMapper().readValue(serializedDoc, Pipeline.class);
   }
 
   @Override
   public ExportItem convert(Pipeline document) {
     return new ExportItem(document.getPipelineId(), document.getName(), true);
   }
+
+  @Override
+  public void writeDocument(String document) throws JsonProcessingException {
+    getNoSqlStore().getPipelineStorageAPI().storePipeline(serializeDocument(document));
+  }
+
+  @Override
+  protected Pipeline serializeDocument(String document) throws JsonProcessingException {
+    return this.spMapper.readValue(document, Pipeline.class);
+  }
 }
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/datalake/DataLakeMeasure.java b/streampipes-model/src/main/java/org/apache/streampipes/model/datalake/DataLakeMeasure.java
index 7f1df47a9..b8b259a62 100644
--- a/streampipes-model/src/main/java/org/apache/streampipes/model/datalake/DataLakeMeasure.java
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/datalake/DataLakeMeasure.java
@@ -18,6 +18,8 @@
 
 package org.apache.streampipes.model.datalake;
 
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.gson.annotations.SerializedName;
 import org.apache.streampipes.model.base.UnnamedStreamPipesEntity;
 import org.apache.streampipes.model.schema.EventSchema;
 import org.apache.streampipes.model.shared.annotation.TsModel;
@@ -25,6 +27,9 @@ import org.apache.streampipes.model.shared.annotation.TsModel;
 @TsModel
 public class DataLakeMeasure extends UnnamedStreamPipesEntity {
 
+    @JsonProperty("_rev")
+    private @SerializedName("_rev") String rev;
+
     private String measureName;
     private EventSchema eventSchema;
     private String pipelineId;
@@ -87,4 +92,11 @@ public class DataLakeMeasure extends UnnamedStreamPipesEntity {
         this.pipelineIsRunning = pipelineIsRunning;
     }
 
+    public String getRev() {
+        return rev;
+    }
+
+    public void setRev(String rev) {
+        this.rev = rev;
+    }
 }
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/export/AssetExportConfiguration.java b/streampipes-model/src/main/java/org/apache/streampipes/model/export/AssetExportConfiguration.java
index 9e17d42be..525bfae1d 100644
--- a/streampipes-model/src/main/java/org/apache/streampipes/model/export/AssetExportConfiguration.java
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/export/AssetExportConfiguration.java
@@ -26,6 +26,7 @@ public class AssetExportConfiguration {
   private String assetId;
   private String assetName;
 
+  private Set<ExportItem> assets;
   private Set<ExportItem> adapters;
   private Set<ExportItem> dashboards;
   private Set<ExportItem> dataViews;
@@ -42,6 +43,7 @@ public class AssetExportConfiguration {
     this.dataSources = new HashSet<>();
     this.pipelines = new HashSet<>();
     this.files = new HashSet<>();
+    this.assets = new HashSet<>();
   }
 
   public Set<ExportItem> getAdapters() {
@@ -52,6 +54,10 @@ public class AssetExportConfiguration {
     this.adapters = adapters;
   }
 
+  public void addAdapter(ExportItem item) {
+    this.adapters.add(item);
+  }
+
   public Set<ExportItem> getDashboards() {
     return dashboards;
   }
@@ -60,6 +66,10 @@ public class AssetExportConfiguration {
     this.dashboards = dashboards;
   }
 
+  public void addDashboard(ExportItem item) {
+    this.dashboards.add(item);
+  }
+
   public Set<ExportItem> getDataViews() {
     return dataViews;
   }
@@ -68,6 +78,10 @@ public class AssetExportConfiguration {
     this.dataViews = dataViews;
   }
 
+  public void addDataView(ExportItem item) {
+    this.dataViews.add(item);
+  }
+
   public Set<ExportItem> getDataLakeMeasures() {
     return dataLakeMeasures;
   }
@@ -76,6 +90,10 @@ public class AssetExportConfiguration {
     this.dataLakeMeasures = dataLakeMeasures;
   }
 
+  public void addDataLakeMeasure(ExportItem item) {
+    this.dataLakeMeasures.add(item);
+  }
+
   public Set<ExportItem> getDataSources() {
     return dataSources;
   }
@@ -84,6 +102,10 @@ public class AssetExportConfiguration {
     this.dataSources = dataSources;
   }
 
+  public void addDataSource(ExportItem item) {
+    this.dataSources.add(item);
+  }
+
   public String getAssetId() {
     return assetId;
   }
@@ -100,6 +122,10 @@ public class AssetExportConfiguration {
     this.pipelines = pipelines;
   }
 
+  public void addPipeline(ExportItem item) {
+    this.pipelines.add(item);
+  }
+
   public Set<ExportItem> getFiles() {
     return files;
   }
@@ -108,6 +134,10 @@ public class AssetExportConfiguration {
     this.files = files;
   }
 
+  public void addFile(ExportItem item) {
+    this.files.add(item);
+  }
+
   public String getAssetName() {
     return assetName;
   }
@@ -116,5 +146,15 @@ public class AssetExportConfiguration {
     this.assetName = assetName;
   }
 
+  public Set<ExportItem> getAssets() {
+    return assets;
+  }
+
+  public void setAssets(Set<ExportItem> assets) {
+    this.assets = assets;
+  }
 
+  public void addAsset(ExportItem asset) {
+    this.assets.add(asset);
+  }
 }
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/export/StreamPipesApplicationPackage.java b/streampipes-model/src/main/java/org/apache/streampipes/model/export/StreamPipesApplicationPackage.java
index c0b3f9e1b..a95dc6344 100644
--- a/streampipes-model/src/main/java/org/apache/streampipes/model/export/StreamPipesApplicationPackage.java
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/export/StreamPipesApplicationPackage.java
@@ -30,6 +30,7 @@ public class StreamPipesApplicationPackage {
   private Set<String> requiredDataSinkAppIds;
   private Set<String> requiredAdapterAppIds;
 
+  private Set<String> assets;
   private Set<String> adapters;
   private Set<String> dashboards;
   private Set<String> dashboardWidgets;
@@ -46,6 +47,7 @@ public class StreamPipesApplicationPackage {
     this.requiredAdapterAppIds = new HashSet<>();
 
     this.adapters = new HashSet<>();
+    this.assets = new HashSet<>();
     this.dashboards = new HashSet<>();
     this.dashboardWidgets = new HashSet<>();
     this.dataViews = new HashSet<>();
@@ -187,4 +189,16 @@ public class StreamPipesApplicationPackage {
   public void addFile(String file) {
     this.files.add(file);
   }
+
+  public Set<String> getAssets() {
+    return assets;
+  }
+
+  public void setAssets(Set<String> assets) {
+    this.assets = assets;
+  }
+
+  public void addAsset(String asset) {
+    this.assets.add(asset);
+  }
 }
diff --git a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/admin/DataImportResource.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/admin/DataImportResource.java
new file mode 100644
index 000000000..72feeb8d6
--- /dev/null
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/admin/DataImportResource.java
@@ -0,0 +1,63 @@
+/*
+ * 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.streampipes.rest.impl.admin;
+
+import org.apache.streampipes.export.ImportManager;
+import org.apache.streampipes.model.export.AssetExportConfiguration;
+import org.apache.streampipes.rest.core.base.impl.AbstractAuthGuardedRestResource;
+import org.apache.streampipes.rest.security.AuthConstants;
+import org.glassfish.jersey.media.multipart.FormDataContentDisposition;
+import org.glassfish.jersey.media.multipart.FormDataParam;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.stereotype.Component;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.io.InputStream;
+
+@Path("/v2/import")
+@Component
+@PreAuthorize(AuthConstants.IS_ADMIN_ROLE)
+public class DataImportResource extends AbstractAuthGuardedRestResource {
+
+  @Path("/preview")
+  @POST
+  @Consumes(MediaType.MULTIPART_FORM_DATA)
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getImportPreview(@FormDataParam("file_upload") InputStream uploadedInputStream,
+                                   @FormDataParam("file_upload") FormDataContentDisposition fileDetail) throws IOException {
+    var importConfig =  ImportManager.getImportPreview(uploadedInputStream);
+    return ok(importConfig);
+  }
+
+  @POST
+  @Consumes(MediaType.MULTIPART_FORM_DATA)
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response importData(@FormDataParam("file_upload") InputStream uploadedInputStream,
+                             @FormDataParam("file_upload") FormDataContentDisposition fileDetail,
+                             @FormDataParam("configuration") AssetExportConfiguration exportConfiguration) throws IOException {
+    ImportManager.performImport(uploadedInputStream, exportConfiguration, getAuthenticatedUserSid());
+    return ok();
+  }
+}
diff --git a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
index 9a1a1d9b2..c4f9a582e 100644
--- a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
+++ b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
@@ -18,7 +18,7 @@
 /* tslint:disable */
 /* eslint-disable */
 // @ts-nocheck
-// Generated using typescript-generator version 2.27.744 on 2022-07-31 20:23:49.
+// Generated using typescript-generator version 2.27.744 on 2022-08-01 10:55:40.
 
 export class AbstractStreamPipesEntity {
     "@class": "org.apache.streampipes.model.base.AbstractStreamPipesEntity" | "org.apache.streampipes.model.base.NamedStreamPipesEntity" | "org.apache.streampipes.model.connect.adapter.AdapterDescription" | "org.apache.streampipes.model.connect.adapter.AdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.GenericAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.SpecificAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.AdapterStre [...]
@@ -192,9 +192,9 @@ export class AdapterDescription extends NamedStreamPipesEntity {
         instance.selectedEndpointUrl = data.selectedEndpointUrl;
         instance.correspondingServiceGroup = data.correspondingServiceGroup;
         instance.correspondingDataStreamElementId = data.correspondingDataStreamElementId;
-        instance.streamRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.streamRules);
-        instance.valueRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.valueRules);
         instance.schemaRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.schemaRules);
+        instance.valueRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.valueRules);
+        instance.streamRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.streamRules);
         return instance;
     }
 
@@ -592,6 +592,7 @@ export class AssetExportConfiguration {
     adapters: ExportItem[];
     assetId: string;
     assetName: string;
+    assets: ExportItem[];
     dashboards: ExportItem[];
     dataLakeMeasures: ExportItem[];
     dataSources: ExportItem[];
@@ -606,6 +607,7 @@ export class AssetExportConfiguration {
         const instance = target || new AssetExportConfiguration();
         instance.assetId = data.assetId;
         instance.assetName = data.assetName;
+        instance.assets = __getCopyArrayFn(ExportItem.fromData)(data.assets);
         instance.adapters = __getCopyArrayFn(ExportItem.fromData)(data.adapters);
         instance.dashboards = __getCopyArrayFn(ExportItem.fromData)(data.dashboards);
         instance.dataViews = __getCopyArrayFn(ExportItem.fromData)(data.dataViews);
@@ -1728,8 +1730,8 @@ export class GenericAdapterSetDescription extends AdapterSetDescription implemen
         const instance = target || new GenericAdapterSetDescription();
         super.fromData(data, instance);
         instance.eventSchema = EventSchema.fromData(data.eventSchema);
-        instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
         instance.formatDescription = FormatDescription.fromData(data.formatDescription);
+        instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
         return instance;
     }
 }
@@ -1747,8 +1749,8 @@ export class GenericAdapterStreamDescription extends AdapterStreamDescription im
         const instance = target || new GenericAdapterStreamDescription();
         super.fromData(data, instance);
         instance.eventSchema = EventSchema.fromData(data.eventSchema);
-        instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
         instance.formatDescription = FormatDescription.fromData(data.formatDescription);
+        instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
         return instance;
     }
 }
@@ -2560,8 +2562,8 @@ export class PipelineTemplateDescription extends NamedStreamPipesEntity {
         super.fromData(data, instance);
         instance.boundTo = __getCopyArrayFn(BoundPipelineElement.fromData)(data.boundTo);
         instance.pipelineTemplateName = data.pipelineTemplateName;
-        instance.pipelineTemplateDescription = data.pipelineTemplateDescription;
         instance.pipelineTemplateId = data.pipelineTemplateId;
+        instance.pipelineTemplateDescription = data.pipelineTemplateDescription;
         return instance;
     }
 }
@@ -2962,8 +2964,8 @@ export class SpDataSet extends SpDataStream {
         instance.datasetInvocationId = data.datasetInvocationId;
         instance.correspondingPipeline = data.correspondingPipeline;
         instance.selectedEndpointUrl = data.selectedEndpointUrl;
-        instance.actualTopicName = data.actualTopicName;
         instance.brokerHostname = data.brokerHostname;
+        instance.actualTopicName = data.actualTopicName;
         return instance;
     }
 }
@@ -3067,7 +3069,16 @@ export class StaticPropertyGroup extends StaticProperty {
 }
 
 export class StreamPipesApplicationPackage {
-    assetExportConfigurations: AssetExportConfiguration[];
+    adapters: string[];
+    assets: string[];
+    dashboardWidgets: string[];
+    dashboards: string[];
+    dataLakeMeasures: string[];
+    dataSources: string[];
+    dataViewWidgets: string[];
+    dataViews: string[];
+    files: string[];
+    pipelines: string[];
     requiredAdapterAppIds: string[];
     requiredDataSinkAppIds: string[];
     requiredProcessorAppIds: string[];
@@ -3080,7 +3091,16 @@ export class StreamPipesApplicationPackage {
         instance.requiredProcessorAppIds = __getCopyArrayFn(__identity<string>())(data.requiredProcessorAppIds);
         instance.requiredDataSinkAppIds = __getCopyArrayFn(__identity<string>())(data.requiredDataSinkAppIds);
         instance.requiredAdapterAppIds = __getCopyArrayFn(__identity<string>())(data.requiredAdapterAppIds);
-        instance.assetExportConfigurations = __getCopyArrayFn(AssetExportConfiguration.fromData)(data.assetExportConfigurations);
+        instance.assets = __getCopyArrayFn(__identity<string>())(data.assets);
+        instance.adapters = __getCopyArrayFn(__identity<string>())(data.adapters);
+        instance.dashboards = __getCopyArrayFn(__identity<string>())(data.dashboards);
+        instance.dashboardWidgets = __getCopyArrayFn(__identity<string>())(data.dashboardWidgets);
+        instance.dataViews = __getCopyArrayFn(__identity<string>())(data.dataViews);
+        instance.dataViewWidgets = __getCopyArrayFn(__identity<string>())(data.dataViewWidgets);
+        instance.dataLakeMeasures = __getCopyArrayFn(__identity<string>())(data.dataLakeMeasures);
+        instance.dataSources = __getCopyArrayFn(__identity<string>())(data.dataSources);
+        instance.pipelines = __getCopyArrayFn(__identity<string>())(data.pipelines);
+        instance.files = __getCopyArrayFn(__identity<string>())(data.files);
         return instance;
     }
 }
diff --git a/ui/src/app/assets/assets.module.ts b/ui/src/app/assets/assets.module.ts
index 3bcaa5d4c..f3417b778 100644
--- a/ui/src/app/assets/assets.module.ts
+++ b/ui/src/app/assets/assets.module.ts
@@ -42,6 +42,7 @@ import { SpAssetDetailsPanelComponent } from './components/asset-details/asset-d
 import { MatTreeModule } from '@angular/material/tree';
 import { SpAssetLinkItemComponent } from './components/asset-details/asset-details-panel/asset-link-item/asset-link-item.component';
 import { EditAssetLinkDialogComponent } from './dialog/edit-asset-link/edit-asset-link-dialog.component';
+import { SpCreateAssetDialogComponent } from './dialog/create-asset/create-asset-dialog.component';
 
 @NgModule({
   imports: [
@@ -92,6 +93,7 @@ import { EditAssetLinkDialogComponent } from './dialog/edit-asset-link/edit-asse
     SpAssetLinkItemComponent,
     SpAssetOverviewComponent,
     SpAssetSelectionPanelComponent,
+    SpCreateAssetDialogComponent
   ],
   providers: [],
 })
diff --git a/ui/src/app/assets/components/asset-overview/asset-overview.component.ts b/ui/src/app/assets/components/asset-overview/asset-overview.component.ts
index 1d6037ad0..1800e33ff 100644
--- a/ui/src/app/assets/components/asset-overview/asset-overview.component.ts
+++ b/ui/src/app/assets/components/asset-overview/asset-overview.component.ts
@@ -24,6 +24,7 @@ import { DialogService, PanelType, SpBreadcrumbService } from '@streampipes/shar
 import { SpAssetRoutes } from '../../assets.routes';
 import { AssetUploadDialogComponent } from '../../dialog/asset-upload/asset-upload-dialog.component';
 import { Router } from '@angular/router';
+import { SpCreateAssetDialogComponent } from '../../dialog/create-asset/create-asset-dialog.component';
 
 @Component({
   selector: 'sp-asset-overview-component',
@@ -57,8 +58,43 @@ export class SpAssetOverviewComponent implements OnInit {
     });
   }
 
-  createNewAsset() {
+  createNewAsset(assetModel?: SpAssetModel) {
+    if (!assetModel) {
+      assetModel = {
+        assetName: 'New Asset',
+        assetDescription: '',
+        assetLinks: [],
+        assetId: this.generateId(6),
+        _id: this.generateId(24),
+        appDocType: 'asset-management',
+        removable: true,
+        _rev: undefined,
+        assets: [],
+        assetType: undefined
+      };
+    }
+    const dialogRef = this.dialogService.open(SpCreateAssetDialogComponent, {
+      panelType: PanelType.STANDARD_PANEL,
+      title: 'Create asset',
+      width: '40vw',
+      data: {
+        'createMode': true,
+        'assetModel': assetModel
+      }
+    });
 
+    dialogRef.afterClosed().subscribe(() => {
+      this.loadAssets();
+    });
+  }
+
+  private generateId(length): string {
+    const chars = 'abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ';
+    let result = '';
+    for (let i = length; i > 0; --i) {
+      result += chars[Math.round(Math.random() * (chars.length - 1))];
+    }
+    return result;
   }
 
   uploadAsset() {
diff --git a/ui/src/app/assets/dialog/create-asset/create-asset-dialog.component.html b/ui/src/app/assets/dialog/create-asset/create-asset-dialog.component.html
new file mode 100644
index 000000000..2fe816d78
--- /dev/null
+++ b/ui/src/app/assets/dialog/create-asset/create-asset-dialog.component.html
@@ -0,0 +1,50 @@
+<!--
+~ Licensed to the Apache Software Foundation (ASF) under one or more
+~ contributor license agreements.  See the NOTICE file distributed with
+~ this work for additional information regarding copyright ownership.
+~ The ASF licenses this file to You under the Apache License, Version 2.0
+~ (the "License"); you may not use this file except in compliance with
+~ the License.  You may obtain a copy of the License at
+~
+~    http://www.apache.org/licenses/LICENSE-2.0
+~
+~ Unless required by applicable law or agreed to in writing, software
+~ distributed under the License is distributed on an "AS IS" BASIS,
+~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+~ See the License for the specific language governing permissions and
+~ limitations under the License.
+~
+-->
+
+<div class="sp-dialog-container">
+    <div class="sp-dialog-content p-15">
+        <div fxFlex="100">
+            <div fxFlex="100" fxLayout="column" style="margin:5px;width:100%">
+                <mat-form-field class="full-width" color="accent">
+                    <mat-label>Data View Name</mat-label>
+                    <input matInput
+                           data-cy="data-view-name"
+                           [(ngModel)]="assetModel.assetName">
+                </mat-form-field>
+                <mat-form-field class="full-width" color="accent">
+                    <mat-label>Description</mat-label>
+                    <input matInput [(ngModel)]="assetModel.assetDescription">
+                </mat-form-field>
+            </div>
+        </div>
+    </div>
+    <mat-divider></mat-divider>
+    <div class="sp-dialog-actions actions-align-right">
+        <button mat-button mat-raised-button class="mat-basic mr-10"
+                (click)="onCancel()" style="margin-right: 10px;">
+            Close
+        </button>
+        <button mat-button mat-raised-button color="accent"
+                data-cy="save-data-view"
+                (click)="onSave()">
+            {{createMode ? 'Create' : 'Save'}}
+        </button>
+    </div>
+</div>
+
+
diff --git a/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts b/ui/src/app/assets/dialog/create-asset/create-asset-dialog.component.scss
similarity index 72%
copy from ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts
copy to ui/src/app/assets/dialog/create-asset/create-asset-dialog.component.scss
index 5ef2a7fb6..704f843e4 100644
--- a/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts
+++ b/ui/src/app/assets/dialog/create-asset/create-asset-dialog.component.scss
@@ -16,17 +16,4 @@
  *
  */
 
-import { Component, OnInit } from '@angular/core';
-
-@Component({
-  selector: 'sp-data-import-dialog',
-  templateUrl: './data-import-dialog.component.html',
-  styleUrls: ['./data-import-dialog.component.scss'],
-})
-export class SpDataImportDialogComponent implements OnInit {
-
-  ngOnInit(): void {
-  }
-
-
-}
+@import 'src/scss/sp/sp-dialog';
diff --git a/ui/src/app/assets/dialog/create-asset/create-asset-dialog.component.ts b/ui/src/app/assets/dialog/create-asset/create-asset-dialog.component.ts
new file mode 100644
index 000000000..460a304c3
--- /dev/null
+++ b/ui/src/app/assets/dialog/create-asset/create-asset-dialog.component.ts
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ *
+ */
+
+
+import { Component, Input, OnInit } from '@angular/core';
+import { AssetManagementService, DataViewDataExplorerService, SpAssetModel } from '@streampipes/platform-services';
+import { DialogRef } from '@streampipes/shared-ui';
+
+@Component({
+  selector: 'sp-create-asset-dialog-component',
+  templateUrl: './create-asset-dialog.component.html',
+  styleUrls: ['./create-asset-dialog.component.scss']
+})
+export class SpCreateAssetDialogComponent implements OnInit {
+
+  @Input() createMode: boolean;
+  @Input() assetModel: SpAssetModel;
+
+  constructor(
+    private dialogRef: DialogRef<SpCreateAssetDialogComponent>,
+    private assetManagementService: AssetManagementService) {
+  }
+
+  ngOnInit() {
+
+  }
+
+  onCancel(): void {
+    this.dialogRef.close();
+  }
+
+  onSave(): void {
+    if (this.createMode) {
+      this.assetManagementService.createAsset(this.assetModel).subscribe(() => {
+        this.dialogRef.close();
+      });
+    } else {
+      this.assetManagementService.updateAsset(this.assetModel).subscribe(() => {
+        this.dialogRef.close();
+      });
+    }
+  }
+
+}
diff --git a/ui/src/app/configuration/export/data-export-import.component.html b/ui/src/app/configuration/export/data-export-import.component.html
index 82a86a161..44352f45c 100644
--- a/ui/src/app/configuration/export/data-export-import.component.html
+++ b/ui/src/app/configuration/export/data-export-import.component.html
@@ -30,7 +30,7 @@
                             color="accent"
                             [disabled]="selectedAssets.length === 0"
                             (click)="openExportDialog()">
-                        <i class="material-icons">file_download</i><span>&nbsp;Start export process</span>
+                        <i class="material-icons">cloud_download</i><span>&nbsp;Start export process</span>
                     </button>
                 </div>
             </sp-split-section>
@@ -41,8 +41,8 @@
                 <div class="subsection-title">Import from application package</div>
 
                 <div class="mt-10">
-                    <button mat-button mat-raised-button color="accent">
-                        <i class="material-icons">save</i><span>&nbsp;Save</span>
+                    <button mat-button mat-raised-button color="accent" (click)="openImportDialog()">
+                        <i class="material-icons">cloud_upload</i><span>&nbsp;Start import process</span>
                     </button>
                 </div>
             </sp-split-section>
diff --git a/ui/src/app/configuration/export/data-export-import.component.ts b/ui/src/app/configuration/export/data-export-import.component.ts
index 32b007e2b..f44891c4d 100644
--- a/ui/src/app/configuration/export/data-export-import.component.ts
+++ b/ui/src/app/configuration/export/data-export-import.component.ts
@@ -23,6 +23,7 @@ import { SpConfigurationTabs } from '../configuration-tabs';
 import { AssetManagementService, SpAsset } from '@streampipes/platform-services';
 import { MatCheckboxChange } from '@angular/material/checkbox';
 import { SpDataExportDialogComponent } from './export-dialog/data-export-dialog.component';
+import { SpDataImportDialogComponent } from './import-dialog/data-import-dialog.component';
 
 @Component({
   selector: 'sp-data-export-import',
@@ -63,7 +64,7 @@ export class SpDataExportImportComponent implements OnInit {
     }
   }
 
-  openExportDialog() {
+  openExportDialog(): void {
     const dialogRef = this.dialogService.open(SpDataExportDialogComponent, {
       panelType: PanelType.SLIDE_IN_PANEL,
       title: 'Export resources',
@@ -78,4 +79,18 @@ export class SpDataExportImportComponent implements OnInit {
     });
   }
 
+  openImportDialog(): void {
+    const dialogRef = this.dialogService.open(SpDataImportDialogComponent, {
+      panelType: PanelType.SLIDE_IN_PANEL,
+      title: 'Import resources',
+      width: '50vw',
+      data: {
+      }
+    });
+
+    dialogRef.afterClosed().subscribe(() => {
+
+    });
+  }
+
 }
diff --git a/ui/src/app/configuration/export/data-export.service.ts b/ui/src/app/configuration/export/data-export.service.ts
index 14994fdb4..5f3c1d9d3 100644
--- a/ui/src/app/configuration/export/data-export.service.ts
+++ b/ui/src/app/configuration/export/data-export.service.ts
@@ -17,10 +17,9 @@
  */
 
 import { Injectable } from '@angular/core';
-import { PlatformServicesCommons } from '@streampipes/platform-services';
-import { HttpClient } from '@angular/common/http';
+import { AssetExportConfiguration, PlatformServicesCommons, ExportConfiguration } from '@streampipes/platform-services';
+import { HttpClient, HttpEvent, HttpParams, HttpRequest } from '@angular/common/http';
 import { Observable } from 'rxjs';
-import { ExportConfiguration } from '../../../../projects/streampipes/platform-services/src/lib/model/gen/streampipes-model';
 import { map } from 'rxjs/operators';
 
 @Injectable({providedIn: 'root'})
@@ -31,16 +30,50 @@ export class DataExportService {
   }
 
   getExportPreview(assetIds: string[]): Observable<ExportConfiguration> {
-    return this.http.post(this.basePath + '/preview', assetIds)
+    return this.http.post(this.exportBasePath + '/preview', assetIds)
       .pipe(map(res => res as ExportConfiguration));
   }
 
   triggerExport(exportConfig: ExportConfiguration): Observable<Blob> {
-    return this.http.post(this.basePath + '/download', exportConfig, {responseType: 'blob'});
+    return this.http.post(this.exportBasePath + '/download', exportConfig, {responseType: 'blob'});
   }
 
-  private get basePath(): string {
+  triggerImport(file: File,
+                config: AssetExportConfiguration) {
+    const data: FormData = new FormData();
+    data.append('file_upload', file, file.name);
+    data.append('configuration', new Blob([JSON.stringify(config)], {type: 'application/json'}));
+
+    const params = new HttpParams();
+    const options = {
+      params,
+      reportProgress: true,
+    };
+
+    const req = new HttpRequest('POST', this.importBasePath, data, options);
+    return this.http.request(req);
+  }
+
+  getImportPreview(file: File): Observable<HttpEvent<any>> {
+    const data: FormData = new FormData();
+    data.append('file_upload', file, file.name);
+
+    const params = new HttpParams();
+    const options = {
+      params,
+      reportProgress: true,
+    };
+
+    const req = new HttpRequest('POST', this.importBasePath + '/preview', data, options);
+    return this.http.request(req);
+  }
+
+  private get exportBasePath(): string {
     return this.platformServicesCommons.apiBasePath + '/export';
   }
 
+  private get importBasePath(): string {
+    return this.platformServicesCommons.apiBasePath + '/import';
+  }
+
 }
diff --git a/ui/src/app/configuration/export/export-dialog/data-export-dialog.component.ts b/ui/src/app/configuration/export/export-dialog/data-export-dialog.component.ts
index 33af53433..d5ac02de0 100644
--- a/ui/src/app/configuration/export/export-dialog/data-export-dialog.component.ts
+++ b/ui/src/app/configuration/export/export-dialog/data-export-dialog.component.ts
@@ -58,6 +58,7 @@ export class SpDataExportDialogComponent implements OnInit {
     const blob = new Blob([data], { type: 'application/zip' });
     const url = window.URL.createObjectURL(blob);
     window.open(url);
+    this.dialogRef.close();
   }
 
 }
diff --git a/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.html b/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.html
index fb99b649e..206347f33 100644
--- a/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.html
+++ b/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.html
@@ -15,3 +15,79 @@
   ~ limitations under the License.
   ~
   -->
+
+<div class="sp-dialog-container">
+    <div class="sp-dialog-content">
+        <div fxFlex="100" fxLayout="column" class="p-15" *ngIf="currentImportStep === 0">
+            <h4>Upload application package file</h4>
+            <mat-form-field style="width: 95%" (click)="fileInput.click();" color="accent">
+                <input matInput placeholder="File" disabled (value)="fileName">
+                <input #fileInput type="file" style="display:none;"
+                       (change)="handleFileInput($event.target.files)">
+                <div>
+                    {{fileName}}
+                    <mat-progress-bar mode="determinate"
+                                      value="{{uploadStatus}}" *ngIf="uploadStatus > 0"
+                                      color="accent">
+                    </mat-progress-bar>
+                </div>
+                <button color="accent"
+                        matSuffix
+                        mat-button
+                        style="min-width: 0px">
+                    <mat-icon *ngIf="uploadStatus < 99">insert_drive_file</mat-icon>
+                    <mat-icon *ngIf="uploadStatus == 100" class="green-icon">check_circle</mat-icon>
+                </button>
+                <mat-error *ngIf="!hasInput">
+                    {{errorMessage}}
+                </mat-error>
+            </mat-form-field>
+        </div>
+        <div fxFlex="100" fxLayout="column" class="p-15" *ngIf="currentImportStep === 1">
+            <h4>Select resources to import</h4>
+            <sp-data-export-item [exportItems]="importConfiguration.assets" sectionTitle="Assets"></sp-data-export-item>
+            <sp-data-export-item [exportItems]="importConfiguration.adapters" sectionTitle="Adapters"></sp-data-export-item>
+            <sp-data-export-item [exportItems]="importConfiguration.dashboards" sectionTitle="Dashboards"></sp-data-export-item>
+            <sp-data-export-item [exportItems]="importConfiguration.dataViews" sectionTitle="Data Views"></sp-data-export-item>
+            <sp-data-export-item [exportItems]="importConfiguration.dataSources" sectionTitle="Data Sources"></sp-data-export-item>
+            <sp-data-export-item [exportItems]="importConfiguration.dataLakeMeasures" sectionTitle="Data Lake Storage"></sp-data-export-item>
+            <sp-data-export-item [exportItems]="importConfiguration.pipelines" sectionTitle="Pipelines"></sp-data-export-item>
+        </div>
+    </div>
+    <mat-divider></mat-divider>
+    <div class="sp-dialog-actions">
+        <div fxLayout="column">
+            <div fxLayout="row">
+                <button mat-button
+                        mat-raised-button
+                        color="accent"
+                        style="margin-right: 10px;" (click)="performPreview()"
+                        *ngIf="currentImportStep === 0"
+                        [disabled]="!hasInput">
+                    <span>&nbsp;Next</span>
+                </button>
+                <button mat-button
+                        mat-raised-button
+                        color="accent"
+                        style="margin-right: 10px;"
+                        *ngIf="currentImportStep === 1" (click)="performImport()">
+                    <i class="material-icons">file_download</i><span>&nbsp;Import data</span>
+                </button>
+                <button mat-button
+                        mat-raised-button
+                        class="mat-basic"
+                        style="margin-right: 10px;"
+                        (click)="back()" *ngIf="currentImportStep > 0">
+                    Back
+                </button>
+                <button mat-button
+                        mat-raised-button
+                        class="mat-basic"
+                        (click)="close()">
+                    Cancel
+                </button>
+            </div>
+        </div>
+    </div>
+</div>
+
diff --git a/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts b/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts
index 5ef2a7fb6..4a8b4cd65 100644
--- a/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts
+++ b/ui/src/app/configuration/export/import-dialog/data-import-dialog.component.ts
@@ -17,6 +17,10 @@
  */
 
 import { Component, OnInit } from '@angular/core';
+import { DialogRef } from '@streampipes/shared-ui';
+import { DataExportService } from '../data-export.service';
+import { HttpEventType, HttpResponse } from '@angular/common/http';
+import { AssetExportConfiguration } from '../../../../../dist/streampipes/platform-services';
 
 @Component({
   selector: 'sp-data-import-dialog',
@@ -25,8 +29,66 @@ import { Component, OnInit } from '@angular/core';
 })
 export class SpDataImportDialogComponent implements OnInit {
 
+  currentImportStep = 0;
+
+  inputValue: string;
+  fileName: string;
+
+  selectedUploadFile: File;
+  importConfiguration: AssetExportConfiguration;
+
+  hasInput = false;
+  errorMessage = 'Please enter a value';
+
+  uploadStatus = 0;
+
+  constructor(private dialogRef: DialogRef<SpDataImportDialogComponent>,
+              private dataExportService: DataExportService) {
+
+  }
+
   ngOnInit(): void {
   }
 
+  handleFileInput(files: any) {
+    this.hasInput = true;
+    this.selectedUploadFile = files[0];
+    this.fileName = this.selectedUploadFile.name;
+    this.uploadStatus = 0;
+    console.log(this.hasInput);
+  }
+
+  performPreview(): void {
+    this.uploadStatus = 0;
+    if (this.selectedUploadFile !== undefined) {
+      this.dataExportService.getImportPreview(this.selectedUploadFile).subscribe(
+        event => {
+          if (event.type === HttpEventType.UploadProgress) {
+            this.uploadStatus = Math.round(100 * event.loaded / event.total);
+          } else if (event instanceof HttpResponse) {
+           this.importConfiguration = event.body as AssetExportConfiguration;
+           this.currentImportStep++;
+          }
+        },
+        error => {
+        },
+      );
+    }
+  }
+
+  performImport(): void {
+    this.dataExportService.triggerImport(this.selectedUploadFile, this.importConfiguration).subscribe(result => {
+      this.dialogRef.close();
+    });
+  }
+
+  back(): void {
+    this.currentImportStep--;
+  }
+
+  close(): void {
+    this.dialogRef.close();
+  }
+
 
 }