You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@streampipes.apache.org by ze...@apache.org on 2020/02/26 20:54:44 UTC

[incubator-streampipes] 03/03: Data explorer dashboards are stored

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

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

commit 700e0088ade3020400b1fe622867e8cbd0fd26f0
Author: Philipp Zehnder <ze...@fzi.de>
AuthorDate: Wed Feb 26 16:05:37 2020 +0100

    Data explorer dashboards are stored
---
 .../backend/StreamPipesResourceConfig.java         |  2 +
 .../{Dashboard.java => AbstractDashboard.java}     |  8 +-
 .../streampipes/rest/impl/dashboard/Dashboard.java | 90 +++++-----------------
 .../rest/impl/datalake/DataLakeDashboard.java      | 31 ++++++++
 .../streampipes/storage/api/INoSqlStorage.java     |  2 +
 .../storage/couchdb/CouchDbStorageManager.java     |  5 ++
 .../impl/DataExplorerDashboardStorageImpl.java     | 58 ++++++++++++++
 .../streampipes/storage/couchdb/utils/Utils.java   |  7 ++
 .../shared/shared-dashboard.service.ts             | 52 +++++++++++++
 .../grid/data-explorer-dashboard-grid.component.ts |  4 +-
 .../data-explorer-dashboard-overview.component.ts  |  4 +-
 .../data-explorer-dashboard-panel.component.ts     |  4 +-
 .../data-explorer-dashboard-widget.component.ts    |  4 +-
 .../old-explorer-widget/old-explorer.component.ts  |  2 +-
 .../widgets/table/table-widget.component.ts        |  2 +-
 .../data-explorer-v2/data-explorer-v2.component.ts |  4 +-
 .../data-explorer-v2/data-explorer-v2.module.ts    |  6 +-
 ...xplorer-add-visualization-dialog.component.html | 12 ---
 ...-explorer-add-visualization-dialog.component.ts | 39 ++++------
 ...ata-explorer-edit-data-view-dialog.component.ts |  4 +-
 .../data-explorer-v2/registry/widget-registry.ts   |  2 +-
 ...rvice.ts => data-view-data-explorer.service.ts} | 41 +++++-----
 22 files changed, 235 insertions(+), 148 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 09e324e..746982d 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
@@ -55,6 +55,7 @@ import org.apache.streampipes.rest.impl.Visualization;
 import org.apache.streampipes.rest.impl.dashboard.Dashboard;
 import org.apache.streampipes.rest.impl.dashboard.DashboardWidget;
 import org.apache.streampipes.rest.impl.dashboard.VisualizablePipeline;
+import org.apache.streampipes.rest.impl.datalake.DataLakeDashboard;
 import org.apache.streampipes.rest.impl.datalake.DataLakeNoUserResourceV3;
 import org.apache.streampipes.rest.impl.datalake.DataLakeResourceV3;
 import org.apache.streampipes.rest.impl.nouser.FileServingResource;
@@ -110,6 +111,7 @@ public class StreamPipesResourceConfig extends ResourceConfig {
     register(PipelineElementRuntimeInfo.class);
     register(Version.class);
     register(PipelineElementAsset.class);
+    register(DataLakeDashboard.class);
     register(DataLakeResourceV3.class);
     register(DataLakeNoUserResourceV3.class);
     register(PipelineElementFile.class);
diff --git a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/Dashboard.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/AbstractDashboard.java
similarity index 91%
copy from streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/Dashboard.java
copy to streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/AbstractDashboard.java
index 692effc..3c06f67 100644
--- a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/Dashboard.java
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/AbstractDashboard.java
@@ -33,8 +33,7 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 
-@Path("/v2/users/{username}/ld/dashboards")
-public class Dashboard extends AbstractRestInterface implements IDashboard {
+public abstract class AbstractDashboard extends AbstractRestInterface implements IDashboard {
 
   @GET
   @Produces(MediaType.APPLICATION_JSON)
@@ -77,7 +76,6 @@ public class Dashboard extends AbstractRestInterface implements IDashboard {
     return ok();
   }
 
-  private IDashboardStorage getDashboardStorage() {
-    return getNoSqlStorage().getDashboardStorage();
-  }
+  protected abstract IDashboardStorage getDashboardStorage();
+
 }
diff --git a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/Dashboard.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/Dashboard.java
index 692effc..99d8374 100644
--- a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/Dashboard.java
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/dashboard/Dashboard.java
@@ -1,83 +1,29 @@
 /*
- * 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.
- *
- */
+Copyright 2020 FZI Forschungszentrum Informatik
+
+Licensed 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.dashboard;
 
-import org.apache.streampipes.model.dashboard.DashboardModel;
-import org.apache.streampipes.rest.api.dashboard.IDashboard;
-import org.apache.streampipes.rest.impl.AbstractRestInterface;
 import org.apache.streampipes.storage.api.IDashboardStorage;
 
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
 
 @Path("/v2/users/{username}/ld/dashboards")
-public class Dashboard extends AbstractRestInterface implements IDashboard {
-
-  @GET
-  @Produces(MediaType.APPLICATION_JSON)
-  @Override
-  public Response getAllDashboards() {
-    return ok(getDashboardStorage().getAllDashboards());
-  }
-
-  @GET
-  @Produces(MediaType.APPLICATION_JSON)
-  @Path("/{dashboardId}")
-  @Override
-  public Response getDashboard(@PathParam("dashboardId") String dashboardId) {
-    return ok(getDashboardStorage().getDashboard(dashboardId));
-  }
-
-  @PUT
-  @Produces(MediaType.APPLICATION_JSON)
-  @Path("/{dashboardId}")
-  @Override
-  public Response modifyDashboard(DashboardModel dashboardModel) {
-    getDashboardStorage().updateDashboard(dashboardModel);
-    return ok(getDashboardStorage().getDashboard(dashboardModel.getCouchDbId()));
-  }
-
-  @DELETE
-  @Produces(MediaType.APPLICATION_JSON)
-  @Path("/{dashboardId}")
-  @Override
-  public Response deleteDashboard(@PathParam("dashboardId") String dashboardId) {
-    getDashboardStorage().deleteDashboard(dashboardId);
-    return ok();
-  }
-
-  @POST
-  @Produces(MediaType.APPLICATION_JSON)
-  @Override
-  public Response createDashboard(DashboardModel dashboardModel) {
-    getDashboardStorage().storeDashboard(dashboardModel);
-    return ok();
-  }
+public class Dashboard extends AbstractDashboard {
 
-  private IDashboardStorage getDashboardStorage() {
-    return getNoSqlStorage().getDashboardStorage();
-  }
+    protected IDashboardStorage getDashboardStorage() {
+        return getNoSqlStorage().getDashboardStorage();
+    }
 }
diff --git a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/datalake/DataLakeDashboard.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/datalake/DataLakeDashboard.java
new file mode 100644
index 0000000..458a6bc
--- /dev/null
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/datalake/DataLakeDashboard.java
@@ -0,0 +1,31 @@
+/*
+Copyright 2020 FZI Forschungszentrum Informatik
+
+Licensed 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.datalake;
+
+
+import org.apache.streampipes.rest.impl.dashboard.AbstractDashboard;
+import org.apache.streampipes.storage.api.IDashboardStorage;
+
+import javax.ws.rs.Path;
+
+@Path("/v3/users/{username}/datalake/dashboard")
+public class DataLakeDashboard extends AbstractDashboard {
+
+    protected IDashboardStorage getDashboardStorage() {
+        return getNoSqlStorage().getDataExplorerDashboardStorage();
+    }
+}
diff --git a/streampipes-storage-api/src/main/java/org/apache/streampipes/storage/api/INoSqlStorage.java b/streampipes-storage-api/src/main/java/org/apache/streampipes/storage/api/INoSqlStorage.java
index 19536f7..e83e7b4 100644
--- a/streampipes-storage-api/src/main/java/org/apache/streampipes/storage/api/INoSqlStorage.java
+++ b/streampipes-storage-api/src/main/java/org/apache/streampipes/storage/api/INoSqlStorage.java
@@ -42,6 +42,8 @@ public interface INoSqlStorage {
 
   IDashboardStorage getDashboardStorage();
 
+  IDashboardStorage getDataExplorerDashboardStorage();
+
   IDashboardWidgetStorage getDashboardWidgetStorage();
 
   IVisualizablePipelineStorage getVisualizablePipelineStorage();
diff --git a/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/CouchDbStorageManager.java b/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/CouchDbStorageManager.java
index 09339b8..865aba6 100644
--- a/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/CouchDbStorageManager.java
+++ b/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/CouchDbStorageManager.java
@@ -85,6 +85,11 @@ public enum CouchDbStorageManager implements INoSqlStorage {
   }
 
   @Override
+  public IDashboardStorage getDataExplorerDashboardStorage() {
+    return new DataExplorerDashboardStorageImpl();
+  }
+
+  @Override
   public IDashboardWidgetStorage getDashboardWidgetStorage() {
     return new DashboardWidgetStorageImpl();
   }
diff --git a/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/impl/DataExplorerDashboardStorageImpl.java b/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/impl/DataExplorerDashboardStorageImpl.java
new file mode 100644
index 0000000..d637ae0
--- /dev/null
+++ b/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/impl/DataExplorerDashboardStorageImpl.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.streampipes.storage.couchdb.impl;
+
+import org.apache.streampipes.model.dashboard.DashboardModel;
+import org.apache.streampipes.storage.api.IDashboardStorage;
+import org.apache.streampipes.storage.couchdb.dao.AbstractDao;
+import org.apache.streampipes.storage.couchdb.utils.Utils;
+
+import java.util.List;
+
+public class DataExplorerDashboardStorageImpl extends AbstractDao<DashboardModel> implements IDashboardStorage {
+
+  public DataExplorerDashboardStorageImpl() {
+    super(Utils::getCouchDbDataExplorerDashboardClient, DashboardModel.class);
+  }
+
+  @Override
+  public List<DashboardModel> getAllDashboards() {
+    return findAll();
+  }
+
+  @Override
+  public void storeDashboard(DashboardModel dashboardModel) {
+    persist(dashboardModel);
+  }
+
+  @Override
+  public void updateDashboard(DashboardModel dashboardModel) {
+    update(dashboardModel);
+  }
+
+  @Override
+  public DashboardModel getDashboard(String dashboardId) {
+    return find(dashboardId).orElse(new DashboardModel());
+  }
+
+  @Override
+  public void deleteDashboard(String dashboardId) {
+    delete(dashboardId);
+  }
+}
diff --git a/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/utils/Utils.java b/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/utils/Utils.java
index 4a729ba..1f58625 100644
--- a/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/utils/Utils.java
+++ b/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/utils/Utils.java
@@ -88,6 +88,13 @@ public class Utils {
     return dbClient;
   }
 
+  public static CouchDbClient getCouchDbDataExplorerDashboardClient() {
+    CouchDbClient dbClient = new CouchDbClient(props("dataexplorerdashboard"));
+    dbClient.setGsonBuilder(GsonSerializer.getGsonBuilder());
+    return dbClient;
+  }
+
+
   public static CouchDbClient getCouchDbDashboardClient() {
     CouchDbClient dbClient = new CouchDbClient(props("dashboard"));
     dbClient.setGsonBuilder(GsonSerializer.getGsonBuilder());
diff --git a/ui/src/app/core-services/shared/shared-dashboard.service.ts b/ui/src/app/core-services/shared/shared-dashboard.service.ts
new file mode 100644
index 0000000..47bcbde
--- /dev/null
+++ b/ui/src/app/core-services/shared/shared-dashboard.service.ts
@@ -0,0 +1,52 @@
+/*
+ * 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 { HttpClient } from '@angular/common/http';
+import { Injectable } from '@angular/core';
+import { Observable } from 'rxjs/Observable';
+import { Dashboard } from '../../dashboard-v2/models/dashboard.model';
+import { AuthStatusService } from '../../services/auth-status.service';
+
+@Injectable()
+export class SharedDatalakeRestService {
+
+    constructor(private http: HttpClient,
+                private authStatusService: AuthStatusService) {
+    }
+
+    getDashboards(dashboardUrl: string): Observable<Dashboard[]> {
+        return this.http.get(dashboardUrl).map(data => {
+           return data as Dashboard[];
+        });
+    }
+
+    updateDashboard(dashboardUrl: string, dashboard: Dashboard): Observable<Dashboard> {
+        return this.http.put(dashboardUrl + '/' + dashboard._id, dashboard).map(data => {
+            return data as Dashboard;
+        });
+    }
+
+    deleteDashboard(dashboardUrl: string, dashboard: Dashboard): Observable<any> {
+        return this.http.delete(dashboardUrl + '/' + dashboard._id);
+    }
+
+    saveDashboard(dashboardUrl: string, dashboard: Dashboard): Observable<any> {
+        return this.http.post(dashboardUrl, dashboard);
+    }
+
+}
diff --git a/ui/src/app/data-explorer-v2/components/grid/data-explorer-dashboard-grid.component.ts b/ui/src/app/data-explorer-v2/components/grid/data-explorer-dashboard-grid.component.ts
index dc273a4..c3ccffa 100644
--- a/ui/src/app/data-explorer-v2/components/grid/data-explorer-dashboard-grid.component.ts
+++ b/ui/src/app/data-explorer-v2/components/grid/data-explorer-dashboard-grid.component.ts
@@ -29,7 +29,7 @@ import { GridsterItemComponent, GridType } from 'angular-gridster2';
 import { DashboardWidget } from '../../../core-model/dashboard/DashboardWidget';
 import { GridsterInfo } from '../../../dashboard-v2/models/gridster-info.model';
 import { IDataViewDashboard, IDataViewDashboardConfig, IDataViewDashboardItem } from '../../models/dataview-dashboard.model';
-import { DataViewDashboardService } from '../../services/data-view-dashboard.service';
+import { DataViewDataExplorerService } from '../../services/data-view-data-explorer.service';
 import { RefreshDashboardService } from '../../services/refresh-dashboard.service';
 import { ResizeService } from '../../services/resize.service';
 
@@ -52,7 +52,7 @@ export class DataExplorerDashboardGridComponent implements OnInit, OnChanges {
     @ViewChildren(GridsterItemComponent) gridsterItemComponents: QueryList<GridsterItemComponent>;
 
     constructor(private resizeService: ResizeService,
-                private dashboardService: DataViewDashboardService,
+                private dashboardService: DataViewDataExplorerService,
                 private refreshDashboardService: RefreshDashboardService) {
 
     }
diff --git a/ui/src/app/data-explorer-v2/components/overview/data-explorer-dashboard-overview.component.ts b/ui/src/app/data-explorer-v2/components/overview/data-explorer-dashboard-overview.component.ts
index 1c4bf3b..d560197 100644
--- a/ui/src/app/data-explorer-v2/components/overview/data-explorer-dashboard-overview.component.ts
+++ b/ui/src/app/data-explorer-v2/components/overview/data-explorer-dashboard-overview.component.ts
@@ -21,7 +21,7 @@ import { MatDialog } from '@angular/material/dialog';
 import { MatTableDataSource } from '@angular/material/table';
 import { DataExplorerEditDataViewDialogComponent } from '../../dialogs/edit-dashboard/data-explorer-edit-data-view-dialog.component';
 import { IDataViewDashboard } from '../../models/dataview-dashboard.model';
-import { DataViewDashboardService } from '../../services/data-view-dashboard.service';
+import { DataViewDataExplorerService } from '../../services/data-view-data-explorer.service';
 
 @Component({
     selector: 'sp-data-explorer-dashboard-overview',
@@ -37,7 +37,7 @@ export class DataExplorerDashboardOverviewComponent implements OnInit {
     dataSource = new MatTableDataSource<IDataViewDashboard>();
     displayedColumns: string[] = ['name', 'open', 'openWindow', 'edit', 'delete'];
 
-    constructor(private dashboardService: DataViewDashboardService,
+    constructor(private dashboardService: DataViewDataExplorerService,
                 public dialog: MatDialog) {
 
     }
diff --git a/ui/src/app/data-explorer-v2/components/panel/data-explorer-dashboard-panel.component.ts b/ui/src/app/data-explorer-v2/components/panel/data-explorer-dashboard-panel.component.ts
index 92c0035..c657ac1 100644
--- a/ui/src/app/data-explorer-v2/components/panel/data-explorer-dashboard-panel.component.ts
+++ b/ui/src/app/data-explorer-v2/components/panel/data-explorer-dashboard-panel.component.ts
@@ -22,7 +22,7 @@ import { forkJoin, Observable, Subscription } from 'rxjs';
 import { DashboardWidget } from '../../../core-model/dashboard/DashboardWidget';
 import { DataExplorerAddVisualizationDialogComponent } from '../../dialogs/add-widget/data-explorer-add-visualization-dialog.component';
 import { IDataViewDashboard, IDataViewDashboardItem } from '../../models/dataview-dashboard.model';
-import { DataViewDashboardService } from '../../services/data-view-dashboard.service';
+import { DataViewDataExplorerService } from '../../services/data-view-data-explorer.service';
 import { RefreshDashboardService } from '../../services/refresh-dashboard.service';
 
 @Component({
@@ -43,7 +43,7 @@ export class DataExplorerDashboardPanelComponent implements OnInit {
   widgetIdsToRemove: string[] = [];
   widgetsToUpdate: Map<string, DashboardWidget> = new Map<string, DashboardWidget>();
 
-  constructor(private dashboardService: DataViewDashboardService,
+  constructor(private dashboardService: DataViewDataExplorerService,
               public dialog: MatDialog,
               private refreshDashboardService: RefreshDashboardService) {
   }
diff --git a/ui/src/app/data-explorer-v2/components/widget/data-explorer-dashboard-widget.component.ts b/ui/src/app/data-explorer-v2/components/widget/data-explorer-dashboard-widget.component.ts
index 2cf2ea1..6399087 100644
--- a/ui/src/app/data-explorer-v2/components/widget/data-explorer-dashboard-widget.component.ts
+++ b/ui/src/app/data-explorer-v2/components/widget/data-explorer-dashboard-widget.component.ts
@@ -22,7 +22,7 @@ import { GridsterItem, GridsterItemComponent } from 'angular-gridster2';
 import { DashboardWidget } from '../../../core-model/dashboard/DashboardWidget';
 import { DataExplorerAddVisualizationDialogComponent } from '../../dialogs/add-widget/data-explorer-add-visualization-dialog.component';
 import { IDataViewDashboardItem } from '../../models/dataview-dashboard.model';
-import { DataViewDashboardService } from '../../services/data-view-dashboard.service';
+import { DataViewDataExplorerService } from '../../services/data-view-data-explorer.service';
 
 @Component({
     selector: 'sp-data-explorer-dashboard-widget',
@@ -42,7 +42,7 @@ export class DataExplorerDashboardWidgetComponent implements OnInit {
     widgetLoaded = false;
     configuredWidget: DashboardWidget;
 
-    constructor(private dashboardService: DataViewDashboardService,
+    constructor(private dashboardService: DataViewDataExplorerService,
                 private dialog: MatDialog) {
     }
 
diff --git a/ui/src/app/data-explorer-v2/components/widgets/old-explorer-widget/old-explorer.component.ts b/ui/src/app/data-explorer-v2/components/widgets/old-explorer-widget/old-explorer.component.ts
index 61fddb2..a2bf9b7 100644
--- a/ui/src/app/data-explorer-v2/components/widgets/old-explorer-widget/old-explorer.component.ts
+++ b/ui/src/app/data-explorer-v2/components/widgets/old-explorer-widget/old-explorer.component.ts
@@ -26,8 +26,8 @@ import { map, startWith } from 'rxjs/operators';
 import { DataResult } from '../../../../core-model/datalake/DataResult';
 import { GroupedDataResult } from '../../../../core-model/datalake/GroupedDataResult';
 import { InfoResult } from '../../../../core-model/datalake/InfoResult';
+import { DatalakeRestService } from '../../../../core-services/datalake/datalake-rest.service';
 import { DataDownloadDialog } from './datadownloadDialog/dataDownload.dialog';
-import { DatalakeRestService } from "../../../../core-services/datalake/datalake-rest.service";
 
 @Component({
     selector: 'sp-old-explorer',
diff --git a/ui/src/app/data-explorer-v2/components/widgets/table/table-widget.component.ts b/ui/src/app/data-explorer-v2/components/widgets/table/table-widget.component.ts
index 8b699b4..86e3846 100644
--- a/ui/src/app/data-explorer-v2/components/widgets/table/table-widget.component.ts
+++ b/ui/src/app/data-explorer-v2/components/widgets/table/table-widget.component.ts
@@ -45,7 +45,7 @@ export class TableWidgetComponent extends BaseDataExplorerWidget implements OnIn
     }
 
     extractConfig(extractor: StaticPropertyExtractor) {
-        this.selectedProperty = extractor.mappingPropertyValue(TableConfig.NUMBER_MAPPING_KEY);
+        // this.selectedProperty = extractor.mappingPropertyValue(TableConfig.NUMBER_MAPPING_KEY);
     }
 
     isNumber(item: any): boolean {
diff --git a/ui/src/app/data-explorer-v2/data-explorer-v2.component.ts b/ui/src/app/data-explorer-v2/data-explorer-v2.component.ts
index 3efde57..45bd06d 100644
--- a/ui/src/app/data-explorer-v2/data-explorer-v2.component.ts
+++ b/ui/src/app/data-explorer-v2/data-explorer-v2.component.ts
@@ -18,7 +18,7 @@
 
 import { Component, OnInit } from '@angular/core';
 import { IDataViewDashboard } from './models/dataview-dashboard.model';
-import { DataViewDashboardService } from './services/data-view-dashboard.service';
+import { DataViewDataExplorerService } from './services/data-view-data-explorer.service';
 import { RefreshDashboardService } from './services/refresh-dashboard.service';
 
 @Component({
@@ -37,7 +37,7 @@ export class DataExplorerV2Component implements OnInit {
 
     dataViewDashboards: IDataViewDashboard[];
 
-    constructor(private dataViewService: DataViewDashboardService,
+    constructor(private dataViewService: DataViewDataExplorerService,
                 private refreshDashboardService: RefreshDashboardService) {}
 
     public ngOnInit() {
diff --git a/ui/src/app/data-explorer-v2/data-explorer-v2.module.ts b/ui/src/app/data-explorer-v2/data-explorer-v2.module.ts
index 95fbd2b..3c7ee1e 100644
--- a/ui/src/app/data-explorer-v2/data-explorer-v2.module.ts
+++ b/ui/src/app/data-explorer-v2/data-explorer-v2.module.ts
@@ -34,6 +34,7 @@ import { DynamicModule } from 'ng-dynamic-component';
 import { ColorPickerModule } from 'ngx-color-picker';
 import { ConnectModule } from '../connect/connect.module';
 import { SemanticTypeUtilsService } from '../core-services/semantic-type/semantic-type-utils.service';
+import { SharedDatalakeRestService } from '../core-services/shared/shared-dashboard.service';
 import { CoreUiModule } from '../core-ui/core-ui.module';
 import { CustomMaterialModule } from '../CustomMaterial/custom-material.module';
 import { ElementIconText } from '../services/get-element-icon-text.service';
@@ -47,7 +48,7 @@ import { TableWidgetComponent } from './components/widgets/table/table-widget.co
 import { DataExplorerV2Component } from './data-explorer-v2.component';
 import { DataExplorerAddVisualizationDialogComponent } from './dialogs/add-widget/data-explorer-add-visualization-dialog.component';
 import { DataExplorerEditDataViewDialogComponent } from './dialogs/edit-dashboard/data-explorer-edit-data-view-dialog.component';
-import { DataViewDashboardService } from './services/data-view-dashboard.service';
+import { DataViewDataExplorerService } from './services/data-view-data-explorer.service';
 import { RefreshDashboardService } from './services/refresh-dashboard.service';
 import { ResizeService } from './services/resize.service';
 
@@ -104,7 +105,8 @@ export const MY_NATIVE_FORMATS = {
     OldExplorerComponent
   ],
   providers: [
-    DataViewDashboardService,
+    SharedDatalakeRestService,
+    DataViewDataExplorerService,
     ResizeService,
     RefreshDashboardService,
     SemanticTypeUtilsService,
diff --git a/ui/src/app/data-explorer-v2/dialogs/add-widget/data-explorer-add-visualization-dialog.component.html b/ui/src/app/data-explorer-v2/dialogs/add-widget/data-explorer-add-visualization-dialog.component.html
index 32158f3..9c3a2ca 100644
--- a/ui/src/app/data-explorer-v2/dialogs/add-widget/data-explorer-add-visualization-dialog.component.html
+++ b/ui/src/app/data-explorer-v2/dialogs/add-widget/data-explorer-add-visualization-dialog.component.html
@@ -61,18 +61,6 @@
                         </mat-grid-tile>
                     </mat-grid-list>
                 </div>
-
-                <!-- Select Scheme -->
-                <div *ngIf="page == 'configure-widget'" fxLayout="column">
-                    <b><h4>{{pages[2].description}}</h4></b>
-                    <div fxFlex="100" fxLayout="column">
-                        <app-static-property *ngFor="let config of selectedWidget.config" [staticProperty]="config"
-                                             [staticProperties]="selectedWidget.config"
-                                             [eventSchema]="selectedPipeline.schema"></app-static-property>
-                    </div>
-
-                </div>
-
             </div>
         </div>
     </div>
diff --git a/ui/src/app/data-explorer-v2/dialogs/add-widget/data-explorer-add-visualization-dialog.component.ts b/ui/src/app/data-explorer-v2/dialogs/add-widget/data-explorer-add-visualization-dialog.component.ts
index d83a242..6893b31 100644
--- a/ui/src/app/data-explorer-v2/dialogs/add-widget/data-explorer-add-visualization-dialog.component.ts
+++ b/ui/src/app/data-explorer-v2/dialogs/add-widget/data-explorer-add-visualization-dialog.component.ts
@@ -29,7 +29,7 @@ import { ElementIconText } from '../../../services/get-element-icon-text.service
 import { IDataViewDashboard } from '../../models/dataview-dashboard.model';
 import { WidgetRegistry } from '../../registry/widget-registry';
 import { MappingPropertyGenerator } from '../../sdk/matching/mapping-property-generator';
-import { DataViewDashboardService } from '../../services/data-view-dashboard.service';
+import { DataViewDataExplorerService } from '../../services/data-view-data-explorer.service';
 
 @Component({
     selector: 'sp-data-explorer-add-visualization-dialog-component',
@@ -42,7 +42,7 @@ export class DataExplorerAddVisualizationDialogComponent implements OnInit {
     constructor(
         public dialogRef: MatDialogRef<DataExplorerAddVisualizationDialogComponent>,
         @Inject(MAT_DIALOG_DATA) public data: any,
-        private dashboardService: DataViewDashboardService,
+        private dashboardService: DataViewDataExplorerService,
         public elementIconText: ElementIconText) {
     }
 
@@ -54,10 +54,6 @@ export class DataExplorerAddVisualizationDialogComponent implements OnInit {
         type: 'select-widget',
         title: 'Select Widget',
         description: 'Select widget'
-    }, {
-        type: 'configure-widget',
-        title: 'Configure Widget',
-        description: 'Configure widget'
     }];
 
     visualizableData: InfoResult[] = [];
@@ -111,8 +107,8 @@ export class DataExplorerAddVisualizationDialogComponent implements OnInit {
         if (page === this.page) { return 'md-fab md-accent'; } else { return 'md-fab md-accent wizard-inactive'; }
     }
 
-    selectPipeline(vis) {
-        this.selectedDataSet = vis;
+    selectPipeline(ds) {
+        this.selectedDataSet = ds;
         this.next();
     }
 
@@ -136,29 +132,28 @@ export class DataExplorerAddVisualizationDialogComponent implements OnInit {
         if (this.page === 'select-pipeline') {
             this.page = 'select-widget';
         } else if (this.page === 'select-widget') {
-            this.page = 'configure-widget';
-        } else {
             const configuredWidget: DashboardWidget = new DashboardWidget();
             configuredWidget.dashboardWidgetSettings = this.selectedWidget;
+
+            this.dialogRef.close();
+
             // configuredWidget.dashboardWidgetDataConfig = this.selectedDataSet;
-            if (!this.data) {
-                this.dashboardService.saveWidget(configuredWidget).subscribe(response => {
-                    this.dialogRef.close(response);
-                });
-            } else {
-                configuredWidget._id = this.data.widget._id;
-                configuredWidget._ref = this.data.widget._ref;
-                configuredWidget.widgetId = this.data.widget.widgetId;
-                this.dialogRef.close(configuredWidget);
-            }
+            // if (!this.data) {
+            //     this.dashboardService.saveWidget(configuredWidget).subscribe(response => {
+            //         this.dialogRef.close(response);
+            //     });
+            // } else {
+            //     configuredWidget._id = this.data.widget._id;
+            //     configuredWidget._ref = this.data.widget._ref;
+            //     configuredWidget.widgetId = this.data.widget.widgetId;
+            //     this.dialogRef.close(configuredWidget);
+            // }
         }
     }
 
     back() {
         if (this.page === 'select-widget') {
             this.page = 'select-pipeline';
-        } else if (this.page === 'configure-widget') {
-            this.page = 'select-widget';
         }
     }
 
diff --git a/ui/src/app/data-explorer-v2/dialogs/edit-dashboard/data-explorer-edit-data-view-dialog.component.ts b/ui/src/app/data-explorer-v2/dialogs/edit-dashboard/data-explorer-edit-data-view-dialog.component.ts
index 6369a24..5586f99 100644
--- a/ui/src/app/data-explorer-v2/dialogs/edit-dashboard/data-explorer-edit-data-view-dialog.component.ts
+++ b/ui/src/app/data-explorer-v2/dialogs/edit-dashboard/data-explorer-edit-data-view-dialog.component.ts
@@ -19,7 +19,7 @@
 import { Component, OnInit } from '@angular/core';
 import { MatDialogRef } from '@angular/material/dialog';
 import { IDataViewDashboard } from '../../models/dataview-dashboard.model';
-import { DataViewDashboardService } from '../../services/data-view-dashboard.service';
+import { DataViewDataExplorerService } from '../../services/data-view-data-explorer.service';
 
 @Component({
     selector: 'sp-data-explorer-edit-data-view-dialog-component',
@@ -33,7 +33,7 @@ export class DataExplorerEditDataViewDialogComponent implements OnInit {
 
     constructor(
         public dialogRef: MatDialogRef<DataExplorerEditDataViewDialogComponent>,
-        private dashboardService: DataViewDashboardService) {
+        private dashboardService: DataViewDataExplorerService) {
     }
 
     ngOnInit() {
diff --git a/ui/src/app/data-explorer-v2/registry/widget-registry.ts b/ui/src/app/data-explorer-v2/registry/widget-registry.ts
index d13f9cc..6613eb1 100644
--- a/ui/src/app/data-explorer-v2/registry/widget-registry.ts
+++ b/ui/src/app/data-explorer-v2/registry/widget-registry.ts
@@ -17,7 +17,7 @@
  */
 
 import { DashboardWidgetSettings } from '../../core-model/dashboard/DashboardWidgetSettings';
-import { WidgetConfig } from '../../dashboard-v2/components/widgets/base/base-config';
+import { WidgetConfig } from '../components/widgets/base/base-config';
 import { TableConfig } from '../components/widgets/table/table-config';
 
 export class WidgetRegistry {
diff --git a/ui/src/app/data-explorer-v2/services/data-view-dashboard.service.ts b/ui/src/app/data-explorer-v2/services/data-view-data-explorer.service.ts
similarity index 73%
rename from ui/src/app/data-explorer-v2/services/data-view-dashboard.service.ts
rename to ui/src/app/data-explorer-v2/services/data-view-data-explorer.service.ts
index 994572d..878d018 100644
--- a/ui/src/app/data-explorer-v2/services/data-view-dashboard.service.ts
+++ b/ui/src/app/data-explorer-v2/services/data-view-data-explorer.service.ts
@@ -24,20 +24,22 @@ import { map } from 'rxjs/operators';
 import { DashboardWidget } from '../../core-model/dashboard/DashboardWidget';
 import { InfoResult } from '../../core-model/datalake/InfoResult';
 import { DatalakeRestService } from '../../core-services/datalake/datalake-rest.service';
+import { SharedDatalakeRestService } from '../../core-services/shared/shared-dashboard.service';
 import { TsonLdSerializerService } from '../../platform-services/tsonld-serializer.service';
 import { AuthStatusService } from '../../services/auth-status.service';
 import { IDataViewDashboard } from '../models/dataview-dashboard.model';
 
 
 @Injectable()
-export class DataViewDashboardService {
+export class DataViewDataExplorerService {
 
     localDashboards: IDataViewDashboard[] = [];
 
     constructor(private http: HttpClient,
                 private authStatusService: AuthStatusService,
                 private tsonLdSerializerService: TsonLdSerializerService,
-                private dataLakeRestService: DatalakeRestService) {
+                private dataLakeRestService: DatalakeRestService,
+                private sharedDatalakeRestService: SharedDatalakeRestService) {
     }
 
     getVisualizableData(): Observable<InfoResult[]> {
@@ -50,43 +52,42 @@ export class DataViewDashboardService {
     }
 
     getDataViews(): Observable<IDataViewDashboard[]> {
-       return Observable.of(this.localDashboards);
+      return this.sharedDatalakeRestService.getDashboards(this.getDashboardUrl());
     }
 
     updateDashboard(dashboard: IDataViewDashboard): Observable<IDataViewDashboard> {
-        return this.http.put(this.dashboardUrl + '/' + dashboard._id, dashboard).map(data => {
-            return data as IDataViewDashboard;
-        });
+      return this.sharedDatalakeRestService.updateDashboard(this.getDashboardUrl(), dashboard);
     }
 
     deleteDashboard(dashboard: IDataViewDashboard): Observable<any> {
-      return Observable.of(this.localDashboards);
+      return this.sharedDatalakeRestService.deleteDashboard(this.getDashboardUrl(), dashboard);
     }
 
     saveDataView(dataViewDashboard: IDataViewDashboard): Observable<any> {
-      this.localDashboards.push(dataViewDashboard);
-      return Observable.of(this.localDashboards);
+      // this.localDashboards.push(dataViewDashboard);
+      // return Observable.of(this.localDashboards);
+      return this.sharedDatalakeRestService.saveDashboard(this.getDashboardUrl(), dataViewDashboard);
     }
 
-    private get baseUrl() {
+    private getbaseUrl() {
         return '/streampipes-backend';
     }
 
-    private get dashboardUrl() {
-        return this.baseUrl + '/api/v2/users/' + this.authStatusService.email + '/ld/dashboards';
+    private getDashboardUrl() {
+        return this.getbaseUrl() + '/api/v3/users/' + this.authStatusService.email + '/datalake/dashboard';
     }
 
-    private get dashboardWidgetUrl() {
-        return this.baseUrl + '/api/v2/users/' + this.authStatusService.email + '/ld/widgets';
+    private getDashboardWidgetUrl() {
+        return this.getbaseUrl() + '/api/v2/users/' + this.authStatusService.email + '/ld/widgets';
     }
 
-    private get visualizablePipelineUrl() {
-        return this.baseUrl + '/api/v2/users/' + this.authStatusService.email + '/ld/pipelines';
+    private getvisualizablePipelineUrl() {
+        return this.getbaseUrl() + '/api/v2/users/' + this.authStatusService.email + '/ld/pipelines';
     }
 
     getWidget(widgetId: string): Observable<DashboardWidget> {
         const promise = new Promise<DashboardWidget>((resolve, reject) => {
-            this.http.get(this.dashboardWidgetUrl + '/' + widgetId).subscribe(response => {
+            this.http.get(this.getDashboardWidgetUrl() + '/' + widgetId).subscribe(response => {
                 const dashboardWidget: DashboardWidget = this.tsonLdSerializerService.fromJsonLd(response, 'sp:DashboardWidgetModel');
                 dashboardWidget.dashboardWidgetSettings.config.sort((a, b) => {
                     return a.index - b.index;
@@ -98,17 +99,17 @@ export class DataViewDashboardService {
     }
 
     saveWidget(widget: DashboardWidget): Observable<DashboardWidget> {
-        return this.serializeAndPost(this.dashboardWidgetUrl, widget);
+        return this.serializeAndPost(this.getDashboardWidgetUrl(), widget);
     }
 
     deleteWidget(widgetId: string): Observable<any> {
-        return this.http.delete(this.dashboardWidgetUrl + '/' + widgetId);
+        return this.http.delete(this.getDashboardWidgetUrl() + '/' + widgetId);
     }
 
     updateWidget(widget: DashboardWidget): Observable<any> {
         const promise = new Promise<DashboardWidget>((resolve, reject) => {
             this.tsonLdSerializerService.toJsonLd(widget).subscribe(serialized => {
-                this.http.put(this.dashboardWidgetUrl + '/' + widget._id, serialized, this.jsonLdHeaders()).subscribe(result => {
+                this.http.put(this.getDashboardWidgetUrl() + '/' + widget._id, serialized, this.jsonLdHeaders()).subscribe(result => {
                     resolve();
                 });
             });