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 2020/12/30 12:45:26 UTC

[incubator-streampipes] 03/04: [STREAMPIPES-269] Remove JSON-LD serializer from container modules

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

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

commit f020e2a02f1b7a52cdc8797684545b8065b80e99
Author: Dominik Riemer <ri...@fzi.de>
AuthorDate: Tue Dec 29 22:29:29 2020 +0100

    [STREAMPIPES-269] Remove JSON-LD serializer from container modules
---
 .../backend/StreamPipesResourceConfig.java         |   1 +
 streampipes-commons/pom.xml                        |   8 -
 .../java/org/apache/streampipes/commons/Utils.java |  36 -----
 .../master/management/AdapterMasterManagement.java |  15 +-
 .../connect/init/AdapterContainerConfig.java       |   4 +-
 .../connect/management/AdapterDeserializer.java    |  41 -----
 .../utils/ConnectContainerResourceTest.java        | 175 ---------------------
 .../extensions/ExtensionsResourceConfig.java       |   1 -
 .../PipelineElementContainerResourceConfig.java    |   2 -
 .../api/DataSourcePipelineElementResource.java     |   3 +-
 streampipes-rest-shared/pom.xml                    |   5 -
 .../rest/shared/annotation/JsonLdSerialized.java   |  30 ----
 .../rest/shared/annotation/RdfRootElement.java     |  32 ----
 .../rest/impl/AbstractRestInterface.java           |   6 +-
 .../apache/streampipes/rest/impl/Deployment.java   |  32 ++--
 .../rest}/serializer/JsonLdProvider.java           |   4 +-
 .../apache/streampipes/rest}/util/JsonLdUtils.java |   7 +-
 .../streampipes/rest}/util/JsonLdUtilsTest.java    |   2 +-
 streampipes-serializers-json/pom.xml               |  17 ++
 streampipes-serializers-jsonld/pom.xml             |  25 +++
 .../serializers/jsonld/JsonLdUtils.java            |  33 +---
 21 files changed, 72 insertions(+), 407 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 6b921a9..0a62a4d 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
@@ -30,6 +30,7 @@ import org.apache.streampipes.rest.impl.datalake.DataLakeWidgetResource;
 import org.apache.streampipes.rest.impl.nouser.FileServingResource;
 import org.apache.streampipes.rest.impl.nouser.PipelineElementImportNoUser;
 import org.apache.streampipes.rest.impl.nouser.PipelineNoUserResource;
+import org.apache.streampipes.rest.serializer.JsonLdProvider;
 import org.apache.streampipes.rest.shared.serializer.*;
 import org.glassfish.jersey.media.multipart.MultiPartFeature;
 import org.glassfish.jersey.server.ResourceConfig;
diff --git a/streampipes-commons/pom.xml b/streampipes-commons/pom.xml
index a7bae77..1c700f6 100644
--- a/streampipes-commons/pom.xml
+++ b/streampipes-commons/pom.xml
@@ -43,13 +43,5 @@
             <groupId>ch.qos.logback</groupId>
             <artifactId>logback-classic</artifactId>
         </dependency>
-        <dependency>
-            <groupId>org.eclipse.rdf4j</groupId>
-            <artifactId>rdf4j-rio-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.eclipse.rdf4j</groupId>
-            <artifactId>rdf4j-model</artifactId>
-        </dependency>
     </dependencies>
 </project>
\ No newline at end of file
diff --git a/streampipes-commons/src/main/java/org/apache/streampipes/commons/Utils.java b/streampipes-commons/src/main/java/org/apache/streampipes/commons/Utils.java
index f245b1e..a94e612 100644
--- a/streampipes-commons/src/main/java/org/apache/streampipes/commons/Utils.java
+++ b/streampipes-commons/src/main/java/org/apache/streampipes/commons/Utils.java
@@ -19,17 +19,7 @@
 package org.apache.streampipes.commons;
 
 import org.apache.commons.lang.RandomStringUtils;
-import org.eclipse.rdf4j.model.Model;
-import org.eclipse.rdf4j.rio.RDFFormat;
-import org.eclipse.rdf4j.rio.RDFHandlerException;
-import org.eclipse.rdf4j.rio.RDFWriter;
-import org.eclipse.rdf4j.rio.Rio;
-import org.eclipse.rdf4j.rio.helpers.BasicWriterSettings;
-import org.eclipse.rdf4j.rio.helpers.JSONLDMode;
-import org.eclipse.rdf4j.rio.helpers.JSONLDSettings;
 
-import java.io.ByteArrayOutputStream;
-import java.io.OutputStream;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -56,30 +46,4 @@ public class Utils {
     return RandomStringUtils.randomAlphabetic(10);
   }
 
-  public static String asString(Model model) throws RDFHandlerException {
-    OutputStream stream = new ByteArrayOutputStream();
-
-    RDFWriter writer = Utils.getRioWriter(stream);
-
-    Rio.write(model, writer);
-    return stream.toString();
-  }
-
-
-  private static RDFWriter getRioWriter(OutputStream stream) throws RDFHandlerException {
-    RDFWriter writer = Rio.createWriter(RDFFormat.JSONLD, stream);
-
-    writer.handleNamespace("sp", "https://streampipes.org/vocabulary/v1/");
-    writer.handleNamespace("ssn", "http://purl.oclc.org/NET/ssnx/ssn#");
-    writer.handleNamespace("xsd", "http://www.w3.org/2001/XMLSchema#");
-    writer.handleNamespace("empire", "urn:clarkparsia.com:empire:");
-    writer.handleNamespace("spi", "urn:streampipes.org:spi:");
-
-    writer.getWriterConfig().set(JSONLDSettings.JSONLD_MODE, JSONLDMode.COMPACT);
-    writer.getWriterConfig().set(JSONLDSettings.OPTIMIZE, true);
-    writer.getWriterConfig().set(BasicWriterSettings.PRETTY_PRINT, true);
-
-    return writer;
-  }
-
 }
diff --git a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/AdapterMasterManagement.java b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/AdapterMasterManagement.java
index c99f6b0..37e1686 100644
--- a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/AdapterMasterManagement.java
+++ b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/AdapterMasterManagement.java
@@ -20,8 +20,6 @@ package org.apache.streampipes.connect.container.master.management;
 
 import org.apache.http.client.fluent.Form;
 import org.apache.http.client.fluent.Request;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.streampipes.connect.adapter.GroundingService;
 import org.apache.streampipes.connect.adapter.exception.AdapterException;
 import org.apache.streampipes.connect.config.ConnectContainerConfig;
@@ -32,8 +30,9 @@ import org.apache.streampipes.model.connect.adapter.AdapterStreamDescription;
 import org.apache.streampipes.model.connect.worker.ConnectWorkerContainer;
 import org.apache.streampipes.model.grounding.EventGrounding;
 import org.apache.streampipes.model.util.Cloner;
-import org.apache.streampipes.rest.shared.util.JsonLdUtils;
 import org.apache.streampipes.storage.couchdb.impl.AdapterStorageImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.List;
@@ -205,14 +204,4 @@ public class AdapterMasterManagement {
     return ad instanceof AdapterStreamDescription;
   }
 
-  private static <T> String toJsonLd(T object) {
-    JsonLdUtils.toJsonLD(object);
-    String s = JsonLdUtils.toJsonLD(object);
-
-    if (s == null) {
-      LOG.error("Could not serialize Object " + object + " into json ld");
-    }
-
-    return s;
-  }
 }
diff --git a/streampipes-connect-container/src/main/java/org/apache/streampipes/connect/init/AdapterContainerConfig.java b/streampipes-connect-container/src/main/java/org/apache/streampipes/connect/init/AdapterContainerConfig.java
index 0af1f5f..02a7f33 100644
--- a/streampipes-connect-container/src/main/java/org/apache/streampipes/connect/init/AdapterContainerConfig.java
+++ b/streampipes-connect-container/src/main/java/org/apache/streampipes/connect/init/AdapterContainerConfig.java
@@ -18,11 +18,10 @@
 
 package org.apache.streampipes.connect.init;
 
-import org.glassfish.jersey.server.ResourceConfig;
 import org.apache.streampipes.rest.shared.serializer.GsonClientModelProvider;
 import org.apache.streampipes.rest.shared.serializer.GsonWithIdProvider;
 import org.apache.streampipes.rest.shared.serializer.GsonWithoutIdProvider;
-import org.apache.streampipes.rest.shared.serializer.JsonLdProvider;
+import org.glassfish.jersey.server.ResourceConfig;
 
 public abstract class AdapterContainerConfig extends ResourceConfig {
 
@@ -30,6 +29,5 @@ public abstract class AdapterContainerConfig extends ResourceConfig {
     register(GsonWithIdProvider.class);
     register(GsonWithoutIdProvider.class);
     register(GsonClientModelProvider.class);
-    register(JsonLdProvider.class);
   }
 }
diff --git a/streampipes-connect-container/src/main/java/org/apache/streampipes/connect/management/AdapterDeserializer.java b/streampipes-connect-container/src/main/java/org/apache/streampipes/connect/management/AdapterDeserializer.java
deleted file mode 100644
index 37ed6c4..0000000
--- a/streampipes-connect-container/src/main/java/org/apache/streampipes/connect/management/AdapterDeserializer.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.connect.management;
-
-import org.apache.streampipes.connect.adapter.exception.AdapterException;
-import org.apache.streampipes.model.connect.adapter.*;
-import org.apache.streampipes.rest.shared.util.JsonLdUtils;
-
-public class AdapterDeserializer {
-
-    public static AdapterDescription getAdapterDescription(String jsonld) throws AdapterException {
-        if (jsonld.contains("SpecificAdapterSetDescription")) {
-            return JsonLdUtils.fromJsonLd(jsonld, SpecificAdapterSetDescription.class);
-        } else if (jsonld.contains("SpecificAdapterStreamDescription")) {
-            return JsonLdUtils.fromJsonLd(jsonld, SpecificAdapterStreamDescription.class);
-        } else if (jsonld.contains("GenericAdapterSetDescription")) {
-            return JsonLdUtils.fromJsonLd(jsonld, GenericAdapterSetDescription.class);
-        } else if (jsonld.contains("GenericAdapterStreamDescription")) {
-            return JsonLdUtils.fromJsonLd(jsonld, GenericAdapterStreamDescription.class);
-        } else {
-            throw new AdapterException("Adapter type not registerd");
-        }
-
-    }
-}
diff --git a/streampipes-connect-container/src/test/java/org/streampipes/connect/utils/ConnectContainerResourceTest.java b/streampipes-connect-container/src/test/java/org/streampipes/connect/utils/ConnectContainerResourceTest.java
deleted file mode 100644
index 1a9ac65..0000000
--- a/streampipes-connect-container/src/test/java/org/streampipes/connect/utils/ConnectContainerResourceTest.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * 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.connect.utils;
-
-import static com.jayway.restassured.RestAssured.given;
-import static org.hamcrest.core.IsEqual.equalTo;
-
-import com.jayway.restassured.response.Response;
-import com.jayway.restassured.response.ValidatableResponseOptions;
-import org.eclipse.jetty.server.Server;
-import org.glassfish.jersey.server.ResourceConfig;
-import org.apache.streampipes.connect.rest.AbstractContainerResource;
-import org.apache.streampipes.rest.shared.serializer.GsonClientModelProvider;
-import org.apache.streampipes.rest.shared.serializer.GsonWithIdProvider;
-import org.apache.streampipes.rest.shared.serializer.GsonWithoutIdProvider;
-import org.apache.streampipes.rest.shared.serializer.JsonLdProvider;
-import org.apache.streampipes.rest.shared.util.JsonLdUtils;
-
-import java.net.URI;
-
-import javax.ws.rs.core.UriBuilder;
-
-public abstract class ConnectContainerResourceTest {
-
-    protected static final String ERROR_MESSAGE = "error";
-
-    protected Server getMasterServer(AbstractContainerResource resource) {
-        return getServer(resource, "http://localhost:8099");
-    }
-
-    protected Server getWorkerServer(AbstractContainerResource resource) {
-        return getServer(resource, "http://localhost:8099");
-
-    }
-
-    private Server getServer(AbstractContainerResource resource, String url) {
-
-         ResourceConfig config = new ResourceConfig()
-                .register(GsonWithIdProvider.class)
-                .register(GsonWithoutIdProvider.class)
-                .register(GsonClientModelProvider.class)
-                .register(JsonLdProvider.class)
-                .register(resource);
-
-        URI baseUri = UriBuilder
-                .fromUri(url)
-                .build();
-
-        //TODO after ref
-        // return JettyHttpContainerFactory.createServer(baseUri, config);
-        return null;
-    }
-
-
-    protected abstract String getApi();
-
-    protected <T> T getJsonLdSucessRequest(String route, Class<T> clazz) {
-        return getJsonLdSucessRequest(route, clazz, "");
-
-    }
-
-    protected <T> T getJsonLdSucessRequest(String route, Class<T> clazz, String rootElement) {
-        Response response = given().contentType("application/ld+json")
-                .when()
-                .get(getApi() + route);
-
-        response.then()
-                .statusCode(200);
-
-        String resultString = response.body().print();
-
-        T resultObject;
-        if ("".equals(rootElement)) {
-            resultObject = JsonLdUtils.fromJsonLd(resultString, clazz);
-        } else {
-            resultObject = JsonLdUtils.fromJsonLd(resultString, clazz, rootElement);
-        }
-
-        return resultObject;
-
-    }
-
-    protected void getJsonLdFailRequest(String route) {
-        given().contentType("application/ld+json")
-                .when()
-                .get(getApi() + route)
-                .then()
-                .statusCode(500);
-
-    }
-
-    protected  void deleteJsonLdSucessRequest(String route) {
-        Response response = given().contentType("application/ld+json")
-                .when()
-                .delete(getApi() + route);
-
-        response.then()
-                .statusCode(200);
-    }
-
-    protected  void deleteJsonLdFailRequest(String route) {
-        Response response = given().contentType("application/ld+json")
-                .when()
-                .delete(getApi() + route);
-
-        response.then()
-                .statusCode(500);
-    }
-
-    protected ValidatableResponseOptions postJsonLdSuccessRequest(String data, String route, String responseMessage) {
-        return  postJsonLdRequest(data, route)
-                .body("success", equalTo(true))
-                .body("notifications[0].title", equalTo(responseMessage));
-    }
-
-    protected ValidatableResponseOptions postJsonLdFailRequest(String data, String route) {
-        return  postJsonLdFailRequest(data, route, ERROR_MESSAGE);
-    }
-
-    protected ValidatableResponseOptions postJsonLdFailRequest(String data, String route, String errorMessage) {
-        return  postJsonLdRequest(data, route)
-                .body("success", equalTo(false))
-                .body("notifications[0].title", equalTo(errorMessage));
-    }
-    protected ValidatableResponseOptions postJsonLdRequest(String data, String route) {
-        return given().contentType("application/ld+json")
-                .body(data)
-                .when()
-                .post(getApi() + route)
-                .then()
-                .assertThat()
-                .statusCode(200);
-    }
-
-    protected ValidatableResponseOptions postJsonSuccessRequest(String data, String route, String responseMessage) {
-        return  postJsonRequest(data, route)
-                .body("success", equalTo(true))
-                .body("notifications[0].title", equalTo(responseMessage));
-    }
-
-    protected ValidatableResponseOptions postJsonFailRequest(String data, String route) {
-        return  postJsonFailRequest(data, route, ERROR_MESSAGE);
-    }
-
-    protected ValidatableResponseOptions postJsonFailRequest(String data, String route, String errorMessage) {
-        return  postJsonRequest(data, route)
-                .body("success", equalTo(false))
-                .body("notifications[0].title", equalTo(errorMessage));
-    }
-    protected ValidatableResponseOptions postJsonRequest(String data, String route) {
-        return given().contentType("application/json")
-                .body(data)
-                .when()
-                .post(getApi() + route)
-                .then()
-                .assertThat()
-                .statusCode(200);
-    }
-}
diff --git a/streampipes-container-extensions/src/main/java/org/apache/streampipes/container/extensions/ExtensionsResourceConfig.java b/streampipes-container-extensions/src/main/java/org/apache/streampipes/container/extensions/ExtensionsResourceConfig.java
index 0c48ef4..f595dd1 100644
--- a/streampipes-container-extensions/src/main/java/org/apache/streampipes/container/extensions/ExtensionsResourceConfig.java
+++ b/streampipes-container-extensions/src/main/java/org/apache/streampipes/container/extensions/ExtensionsResourceConfig.java
@@ -43,7 +43,6 @@ public class ExtensionsResourceConfig extends ResourceConfig {
         register(GsonWithIdProvider.class);
         register(GsonWithoutIdProvider.class);
         register(GsonClientModelProvider.class);
-        register(JsonLdProvider.class);
         register(JacksonSerializationProvider.class);
     }
 }
diff --git a/streampipes-container-standalone/src/main/java/org/apache/streampipes/container/standalone/init/PipelineElementContainerResourceConfig.java b/streampipes-container-standalone/src/main/java/org/apache/streampipes/container/standalone/init/PipelineElementContainerResourceConfig.java
index 022a2c0..69ef3a9 100644
--- a/streampipes-container-standalone/src/main/java/org/apache/streampipes/container/standalone/init/PipelineElementContainerResourceConfig.java
+++ b/streampipes-container-standalone/src/main/java/org/apache/streampipes/container/standalone/init/PipelineElementContainerResourceConfig.java
@@ -19,7 +19,6 @@ package org.apache.streampipes.container.standalone.init;
 
 import org.apache.streampipes.container.api.*;
 import org.apache.streampipes.rest.shared.serializer.JacksonSerializationProvider;
-import org.apache.streampipes.rest.shared.serializer.JsonLdProvider;
 import org.glassfish.jersey.server.ResourceConfig;
 import org.springframework.stereotype.Component;
 
@@ -33,7 +32,6 @@ public class PipelineElementContainerResourceConfig extends ResourceConfig {
     register(WelcomePage.class);
     register(PipelineElementTemplateResource.class);
 
-    register(JsonLdProvider.class);
     register(JacksonSerializationProvider.class);
   }
 }
diff --git a/streampipes-container/src/main/java/org/apache/streampipes/container/api/DataSourcePipelineElementResource.java b/streampipes-container/src/main/java/org/apache/streampipes/container/api/DataSourcePipelineElementResource.java
index 549e205..c6afdc1 100644
--- a/streampipes-container/src/main/java/org/apache/streampipes/container/api/DataSourcePipelineElementResource.java
+++ b/streampipes-container/src/main/java/org/apache/streampipes/container/api/DataSourcePipelineElementResource.java
@@ -28,7 +28,6 @@ import org.apache.streampipes.model.Response;
 import org.apache.streampipes.model.SpDataSet;
 import org.apache.streampipes.model.SpDataStream;
 import org.apache.streampipes.model.graph.DataSourceDescription;
-import org.apache.streampipes.rest.shared.util.SpMediaType;
 import org.apache.streampipes.serializers.json.JacksonSerializer;
 import org.eclipse.rdf4j.repository.RepositoryException;
 import org.eclipse.rdf4j.rio.RDFParseException;
@@ -49,7 +48,7 @@ public class DataSourcePipelineElementResource extends AbstractPipelineElementRe
 
   @GET
   @Path("{sourceId}/{streamId}")
-  @Produces({MediaType.APPLICATION_JSON, SpMediaType.JSONLD})
+  @Produces(MediaType.APPLICATION_JSON)
   public javax.ws.rs.core.Response getDescription(@PathParam("sourceId") String sourceId, @PathParam("streamId") String streamId) {
     Optional<SpDataStream> stream = getStreamBySourceId(sourceId, streamId);
     if (stream.isPresent()) {
diff --git a/streampipes-rest-shared/pom.xml b/streampipes-rest-shared/pom.xml
index 42d11e7..b478757 100644
--- a/streampipes-rest-shared/pom.xml
+++ b/streampipes-rest-shared/pom.xml
@@ -45,11 +45,6 @@
 			<artifactId>streampipes-serializers-json</artifactId>
 			<version>0.68.0-SNAPSHOT</version>
 		</dependency>
-		<dependency>
-			<groupId>org.apache.streampipes</groupId>
-			<artifactId>streampipes-serializers-jsonld</artifactId>
-			<version>0.68.0-SNAPSHOT</version>
-		</dependency>
 
 		<!-- External dependencies -->
 		<dependency>
diff --git a/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/annotation/JsonLdSerialized.java b/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/annotation/JsonLdSerialized.java
deleted file mode 100644
index 0e851d6..0000000
--- a/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/annotation/JsonLdSerialized.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.shared.annotation;
-
-import javax.ws.rs.NameBinding;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-@NameBinding
-@Target({ElementType.TYPE, ElementType.METHOD})
-@Retention(RetentionPolicy.RUNTIME)
-public @interface JsonLdSerialized {
-}
diff --git a/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/annotation/RdfRootElement.java b/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/annotation/RdfRootElement.java
deleted file mode 100644
index 573d13d..0000000
--- a/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/annotation/RdfRootElement.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.shared.annotation;
-
-import javax.ws.rs.NameBinding;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-@NameBinding
-@Target({ElementType.TYPE, ElementType.METHOD})
-@Retention(RetentionPolicy.RUNTIME)
-public @interface RdfRootElement {
-    String value();
-}
diff --git a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/AbstractRestInterface.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/AbstractRestInterface.java
index cbab8c3..f44dd57 100644
--- a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/AbstractRestInterface.java
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/AbstractRestInterface.java
@@ -22,7 +22,6 @@ import io.fogsy.empire.core.empire.annotation.InvalidRdfException;
 import org.apache.http.client.ClientProtocolException;
 import org.apache.shiro.SecurityUtils;
 import org.apache.shiro.authc.AuthenticationException;
-import org.apache.streampipes.commons.Utils;
 import org.apache.streampipes.manager.endpoint.HttpJsonParser;
 import org.apache.streampipes.manager.storage.UserManagementService;
 import org.apache.streampipes.manager.storage.UserService;
@@ -32,6 +31,7 @@ import org.apache.streampipes.model.message.Notification;
 import org.apache.streampipes.model.message.*;
 import org.apache.streampipes.serializers.json.GsonSerializer;
 import org.apache.streampipes.serializers.jsonld.JsonLdTransformer;
+import org.apache.streampipes.serializers.jsonld.JsonLdUtils;
 import org.apache.streampipes.storage.api.*;
 import org.apache.streampipes.storage.management.StorageDispatcher;
 import org.apache.streampipes.storage.management.StorageManager;
@@ -49,7 +49,7 @@ public abstract class AbstractRestInterface {
 
   protected <T> String toJsonLd(T object) {
     try {
-      return Utils.asString(new JsonLdTransformer().toJsonLd(object));
+      return JsonLdUtils.asString(new JsonLdTransformer().toJsonLd(object));
     } catch (RDFHandlerException | IllegalArgumentException
             | IllegalAccessException | SecurityException | InvocationTargetException
             | ClassNotFoundException | InvalidRdfException e) {
@@ -61,7 +61,7 @@ public abstract class AbstractRestInterface {
 
   protected <T> String toJsonLd(String rootElementUri, T object) {
     try {
-      return Utils.asString(new JsonLdTransformer(rootElementUri).toJsonLd(object));
+      return JsonLdUtils.asString(new JsonLdTransformer(rootElementUri).toJsonLd(object));
     } catch (IllegalAccessException | InvocationTargetException | InvalidRdfException | ClassNotFoundException e) {
       return toJson(constructErrorMessage(new Notification(NotificationType.UNKNOWN_ERROR.title(),
               NotificationType.UNKNOWN_ERROR.description(),
diff --git a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/Deployment.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/Deployment.java
index 3d9e968..952faf8 100644
--- a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/Deployment.java
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/Deployment.java
@@ -18,36 +18,30 @@
 
 package org.apache.streampipes.rest.impl;
 
-import org.eclipse.rdf4j.rio.RDFHandlerException;
-import org.glassfish.jersey.media.multipart.FormDataParam;
+import io.fogsy.empire.core.empire.annotation.InvalidRdfException;
 import org.apache.streampipes.codegeneration.api.CodeGenerator;
-import org.apache.streampipes.commons.Utils;
 import org.apache.streampipes.commons.exceptions.SepaParseException;
-import io.fogsy.empire.core.empire.annotation.InvalidRdfException;
 import org.apache.streampipes.manager.operations.Operations;
 import org.apache.streampipes.model.base.NamedStreamPipesEntity;
 import org.apache.streampipes.model.client.deployment.DeploymentConfiguration;
 import org.apache.streampipes.model.client.deployment.ElementType;
-import org.apache.streampipes.model.message.Message;
-import org.apache.streampipes.model.message.Notifications;
+import org.apache.streampipes.model.graph.DataProcessorDescription;
 import org.apache.streampipes.model.graph.DataSinkDescription;
 import org.apache.streampipes.model.graph.DataSourceDescription;
-import org.apache.streampipes.model.graph.DataProcessorDescription;
-import org.apache.streampipes.serializers.jsonld.JsonLdTransformer;
+import org.apache.streampipes.model.message.Message;
+import org.apache.streampipes.model.message.Notifications;
 import org.apache.streampipes.serializers.json.GsonSerializer;
+import org.apache.streampipes.serializers.jsonld.JsonLdTransformer;
+import org.apache.streampipes.serializers.jsonld.JsonLdUtils;
 import org.apache.streampipes.storage.management.StorageManager;
+import org.eclipse.rdf4j.rio.RDFHandlerException;
+import org.glassfish.jersey.media.multipart.FormDataParam;
 
-import java.io.File;
-import java.lang.reflect.InvocationTargetException;
-
-import javax.ws.rs.Consumes;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.*;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import java.io.File;
+import java.lang.reflect.InvocationTargetException;
 
 
 @Path("/v2/users/{username}/deploy")
@@ -99,7 +93,7 @@ public class Deployment extends AbstractRestInterface {
         DataSourceDescription sep = new DataSourceDescription(GsonSerializer.getGsonWithIds().fromJson(model, DataSourceDescription.class));
         try {
             Message message =
-                    Operations.verifyAndAddElement(Utils.asString(new JsonLdTransformer().toJsonLd(sep)), username, true, true);
+                    Operations.verifyAndAddElement(JsonLdUtils.asString(new JsonLdTransformer().toJsonLd(sep)), username, true, true);
             return ok(message);
         } catch (RDFHandlerException | IllegalAccessException
                 | IllegalArgumentException | InvocationTargetException
@@ -166,7 +160,7 @@ public class Deployment extends AbstractRestInterface {
         NamedStreamPipesEntity element = getElement(deploymentConfig, model);
 
         try {
-            return Response.ok(Utils.asString(new JsonLdTransformer().toJsonLd(element))).build();
+            return Response.ok(JsonLdUtils.asString(new JsonLdTransformer().toJsonLd(element))).build();
         } catch (RDFHandlerException | IllegalAccessException
                 | IllegalArgumentException | InvocationTargetException
                 | SecurityException | ClassNotFoundException
diff --git a/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/serializer/JsonLdProvider.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/serializer/JsonLdProvider.java
similarity index 96%
rename from streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/serializer/JsonLdProvider.java
rename to streampipes-rest/src/main/java/org/apache/streampipes/rest/serializer/JsonLdProvider.java
index 6c5ab5f..eac1243 100644
--- a/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/serializer/JsonLdProvider.java
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/serializer/JsonLdProvider.java
@@ -15,10 +15,10 @@
  * limitations under the License.
  *
  */
-package org.apache.streampipes.rest.shared.serializer;
+package org.apache.streampipes.rest.serializer;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.streampipes.rest.shared.util.JsonLdUtils;
+import org.apache.streampipes.rest.util.JsonLdUtils;
 import org.apache.streampipes.rest.shared.util.SpMediaType;
 import org.apache.streampipes.serializers.jsonld.JsonLdTransformer;
 
diff --git a/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/util/JsonLdUtils.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/util/JsonLdUtils.java
similarity index 91%
rename from streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/util/JsonLdUtils.java
rename to streampipes-rest/src/main/java/org/apache/streampipes/rest/util/JsonLdUtils.java
index f26cf9d..6097504 100644
--- a/streampipes-rest-shared/src/main/java/org/apache/streampipes/rest/shared/util/JsonLdUtils.java
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/util/JsonLdUtils.java
@@ -16,10 +16,9 @@
  *
  */
 
-package org.apache.streampipes.rest.shared.util;
+package org.apache.streampipes.rest.util;
 
 import io.fogsy.empire.core.empire.annotation.InvalidRdfException;
-import org.apache.streampipes.commons.Utils;
 import org.apache.streampipes.model.base.AbstractStreamPipesEntity;
 import org.apache.streampipes.model.base.StreamPipesJsonLdContainer;
 import org.apache.streampipes.serializers.jsonld.JsonLdTransformer;
@@ -39,10 +38,10 @@ public class JsonLdUtils {
         JsonLdTransformer jsonLdTransformer = new JsonLdTransformer();
         try {
             if (o instanceof List) {
-                return Utils.asString(createJsonLdContainer(jsonLdTransformer,
+                return org.apache.streampipes.serializers.jsonld.JsonLdUtils.asString(createJsonLdContainer(jsonLdTransformer,
                         (List<? extends AbstractStreamPipesEntity>) o));
             } else {
-                return Utils.asString(jsonLdTransformer.toJsonLd(o));
+                return org.apache.streampipes.serializers.jsonld.JsonLdUtils.asString(jsonLdTransformer.toJsonLd(o));
             }
         } catch (IllegalAccessException | InvocationTargetException | InvalidRdfException | ClassNotFoundException e) {
             logger.error("Could not serialize JsonLd", e);
diff --git a/streampipes-rest-shared/src/test/java/org/apache/streampipes/rest/shared/util/JsonLdUtilsTest.java b/streampipes-rest/src/test/java/org/apache/streampipes/rest/util/JsonLdUtilsTest.java
similarity index 98%
rename from streampipes-rest-shared/src/test/java/org/apache/streampipes/rest/shared/util/JsonLdUtilsTest.java
rename to streampipes-rest/src/test/java/org/apache/streampipes/rest/util/JsonLdUtilsTest.java
index 892282c..7ca7cd2 100644
--- a/streampipes-rest-shared/src/test/java/org/apache/streampipes/rest/shared/util/JsonLdUtilsTest.java
+++ b/streampipes-rest/src/test/java/org/apache/streampipes/rest/util/JsonLdUtilsTest.java
@@ -16,7 +16,7 @@
  *
  */
 
-package org.apache.streampipes.rest.shared.util;
+package org.apache.streampipes.rest.util;
 
 import org.apache.streampipes.model.staticproperty.FreeTextStaticProperty;
 import org.apache.streampipes.model.staticproperty.StaticProperty;
diff --git a/streampipes-serializers-json/pom.xml b/streampipes-serializers-json/pom.xml
index ad6f919..ca6c6da 100644
--- a/streampipes-serializers-json/pom.xml
+++ b/streampipes-serializers-json/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  ~
+  -->
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
diff --git a/streampipes-serializers-jsonld/pom.xml b/streampipes-serializers-jsonld/pom.xml
index 9d4d495..8b0430f 100644
--- a/streampipes-serializers-jsonld/pom.xml
+++ b/streampipes-serializers-jsonld/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  ~
+  -->
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
@@ -29,6 +46,14 @@
             <groupId>io.fogsy</groupId>
             <artifactId>empire-pinto</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.eclipse.rdf4j</groupId>
+            <artifactId>rdf4j-rio-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.rdf4j</groupId>
+            <artifactId>rdf4j-model</artifactId>
+        </dependency>
     </dependencies>
 
 
diff --git a/streampipes-commons/src/main/java/org/apache/streampipes/commons/Utils.java b/streampipes-serializers-jsonld/src/main/java/org/apache/streampipes/serializers/jsonld/JsonLdUtils.java
similarity index 74%
copy from streampipes-commons/src/main/java/org/apache/streampipes/commons/Utils.java
copy to streampipes-serializers-jsonld/src/main/java/org/apache/streampipes/serializers/jsonld/JsonLdUtils.java
index f245b1e..b117ca7 100644
--- a/streampipes-commons/src/main/java/org/apache/streampipes/commons/Utils.java
+++ b/streampipes-serializers-jsonld/src/main/java/org/apache/streampipes/serializers/jsonld/JsonLdUtils.java
@@ -15,10 +15,8 @@
  * limitations under the License.
  *
  */
+package org.apache.streampipes.serializers.jsonld;
 
-package org.apache.streampipes.commons;
-
-import org.apache.commons.lang.RandomStringUtils;
 import org.eclipse.rdf4j.model.Model;
 import org.eclipse.rdf4j.rio.RDFFormat;
 import org.eclipse.rdf4j.rio.RDFHandlerException;
@@ -30,42 +28,18 @@ import org.eclipse.rdf4j.rio.helpers.JSONLDSettings;
 
 import java.io.ByteArrayOutputStream;
 import java.io.OutputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class Utils {
 
-  public static List<URI> createURI(String... uris) {
-    List<URI> result = new ArrayList<>();
-    for (String uri : uris) {
-      result.add(URI.create(uri));
-    }
-    return result;
-  }
-
-  @SafeVarargs
-  public static <T> List<T> createList(T... objects) {
-    List<T> result = new ArrayList<>();
-    Collections.addAll(result, objects);
-    return result;
-  }
-
-  public static String getRandomString() {
-    return RandomStringUtils.randomAlphabetic(10);
-  }
+public class JsonLdUtils {
 
   public static String asString(Model model) throws RDFHandlerException {
     OutputStream stream = new ByteArrayOutputStream();
 
-    RDFWriter writer = Utils.getRioWriter(stream);
+    RDFWriter writer = getRioWriter(stream);
 
     Rio.write(model, writer);
     return stream.toString();
   }
 
-
   private static RDFWriter getRioWriter(OutputStream stream) throws RDFHandlerException {
     RDFWriter writer = Rio.createWriter(RDFFormat.JSONLD, stream);
 
@@ -81,5 +55,4 @@ public class Utils {
 
     return writer;
   }
-
 }