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 2021/10/21 15:43:18 UTC

[incubator-streampipes] 02/02: Adpaters are now logging their status

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

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

commit 7b963f656dbb91e46c67294ebc0a9ceea344df26
Author: Philipp Zehnder <ze...@fzi.de>
AuthorDate: Thu Oct 21 17:42:42 2021 +0200

    Adpaters are now logging their status
---
 pom.xml                                            |  1 +
 streampipes-connect-api/pom.xml                    |  5 ++
 .../apache/streampipes/connect/api/IAdapter.java   |  3 +
 .../init/ConnectWorkerRegistrationService.java     |  2 +
 streampipes-connect/pom.xml                        |  5 ++
 .../connect/RunningAdapterInstances.java           | 16 ++++-
 .../streampipes/connect/adapter/Adapter.java       | 18 ++++--
 .../adapter/model/pipeline/AdapterPipeline.java    | 14 ++++-
 .../adapter/monitoring/AdapterMonitoring.java      | 34 -----------
 streampipes-monitoring/pom.xml                     | 19 ++++++
 .../streampipes/monitoring/AdapterMonitoring.java  | 71 ++++++++++++++++++++++
 .../streampipes}/monitoring/AdapterStatus.java     | 15 ++++-
 ui/cypress/tests/adapter/fileStream.ts             |  5 +-
 13 files changed, 161 insertions(+), 47 deletions(-)

diff --git a/pom.xml b/pom.xml
index f571c6e..27739ca 100644
--- a/pom.xml
+++ b/pom.xml
@@ -856,6 +856,7 @@
         <module>streampipes-connect-api</module>
         <module>streampipes-service-extensions-base</module>
         <module>streampipes-rest-core-base</module>
+        <module>streampipes-monitoring</module>
     </modules>
 
     <profiles>
diff --git a/streampipes-connect-api/pom.xml b/streampipes-connect-api/pom.xml
index abc70d4..dd707b6 100644
--- a/streampipes-connect-api/pom.xml
+++ b/streampipes-connect-api/pom.xml
@@ -35,6 +35,11 @@
             <artifactId>streampipes-model</artifactId>
             <version>0.69.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.streampipes</groupId>
+            <artifactId>streampipes-monitoring</artifactId>
+            <version>0.69.0-SNAPSHOT</version>
+        </dependency>
     </dependencies>
     <properties>
         <maven.compiler.source>8</maven.compiler.source>
diff --git a/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IAdapter.java b/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IAdapter.java
index bce9b1d..436bba2 100644
--- a/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IAdapter.java
+++ b/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IAdapter.java
@@ -22,6 +22,7 @@ import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
 import org.apache.streampipes.model.grounding.TransportProtocol;
+import org.apache.streampipes.monitoring.AdapterStatus;
 
 public interface IAdapter<T extends AdapterDescription> extends Connector {
 
@@ -36,6 +37,8 @@ public interface IAdapter<T extends AdapterDescription> extends Connector {
 
   GuessSchema getSchema(T adapterDescription) throws AdapterException, ParseException;
 
+  void init(AdapterStatus adapterStatus, String timestampField);
+
   void changeEventGrounding(TransportProtocol transportProtocol);
 
 }
diff --git a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/init/ConnectWorkerRegistrationService.java b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/init/ConnectWorkerRegistrationService.java
index 6d8260f..1b7cb94 100644
--- a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/init/ConnectWorkerRegistrationService.java
+++ b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/init/ConnectWorkerRegistrationService.java
@@ -17,6 +17,7 @@
  */
 package org.apache.streampipes.connect.container.worker.init;
 
+import org.apache.streampipes.connect.RunningAdapterInstances;
 import org.apache.streampipes.connect.container.worker.management.MasterRestClient;
 import org.apache.streampipes.container.model.SpServiceDefinition;
 import org.apache.streampipes.svcdiscovery.SpServiceDiscovery;
@@ -57,6 +58,7 @@ public class ConnectWorkerRegistrationService {
       }
     }
 
+    RunningAdapterInstances.INSTANCE.init();
 
 
   }
diff --git a/streampipes-connect/pom.xml b/streampipes-connect/pom.xml
index 865a2cb..bb5075e 100755
--- a/streampipes-connect/pom.xml
+++ b/streampipes-connect/pom.xml
@@ -86,6 +86,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.streampipes</groupId>
+            <artifactId>streampipes-monitoring</artifactId>
+            <version>0.69.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.streampipes</groupId>
             <artifactId>streampipes-sdk</artifactId>
             <version>0.69.0-SNAPSHOT</version>
         </dependency>
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/RunningAdapterInstances.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/RunningAdapterInstances.java
index 374880d..811e746 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/RunningAdapterInstances.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/RunningAdapterInstances.java
@@ -18,9 +18,10 @@
 
 package org.apache.streampipes.connect;
 
-import org.apache.streampipes.connect.adapter.monitoring.AdapterMonitoring;
 import org.apache.streampipes.connect.api.IAdapter;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
+import org.apache.streampipes.monitoring.AdapterMonitoring;
+import org.apache.streampipes.monitoring.AdapterStatus;
 
 import java.util.Collection;
 import java.util.HashMap;
@@ -35,11 +36,20 @@ public enum RunningAdapterInstances {
     private AdapterMonitoring adapterMonitoring = new AdapterMonitoring();
 
     public void addAdapter(String elementId, IAdapter<?> adapter, AdapterDescription adapterDescription) {
+
+        AdapterStatus adapterStatus = new AdapterStatus();
+        // TODO read timestamp dynamically
+        adapter.init(adapterStatus, "timestamp");
+
+        adapterMonitoring.put(elementId, adapterStatus);
         runningAdapterInstances.put(elementId, adapter);
+
         runningAdapterDescriptionInstances.put(elementId, adapterDescription);
     }
 
     public IAdapter<?> removeAdapter(String elementId) {
+
+        adapterMonitoring.remove(elementId);
         IAdapter<?> result = runningAdapterInstances.get(elementId);
         runningAdapterInstances.remove(elementId);
         runningAdapterDescriptionInstances.remove(elementId);
@@ -50,5 +60,9 @@ public enum RunningAdapterInstances {
        return this.runningAdapterDescriptionInstances.values();
     }
 
+    public void init() {
+        this.adapterMonitoring.init();
+    }
+
 
 }
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/Adapter.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/Adapter.java
index 5fa4474..fd53a67 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/Adapter.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/Adapter.java
@@ -21,9 +21,9 @@ package org.apache.streampipes.connect.adapter;
 import org.apache.streampipes.config.backend.BackendConfig;
 import org.apache.streampipes.config.backend.SpProtocol;
 import org.apache.streampipes.connect.adapter.model.pipeline.AdapterPipeline;
-import org.apache.streampipes.connect.api.IAdapterPipelineElement;
 import org.apache.streampipes.connect.adapter.preprocessing.elements.*;
 import org.apache.streampipes.connect.api.IAdapter;
+import org.apache.streampipes.connect.api.IAdapterPipelineElement;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
 import org.apache.streampipes.model.connect.rules.TransformationRuleDescription;
 import org.apache.streampipes.model.connect.rules.stream.EventRateTransformationRuleDescription;
@@ -35,6 +35,7 @@ import org.apache.streampipes.model.grounding.JmsTransportProtocol;
 import org.apache.streampipes.model.grounding.KafkaTransportProtocol;
 import org.apache.streampipes.model.grounding.MqttTransportProtocol;
 import org.apache.streampipes.model.grounding.TransportProtocol;
+import org.apache.streampipes.monitoring.AdapterStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,11 +54,14 @@ public abstract class Adapter<T extends AdapterDescription> implements IAdapter<
 
     public Adapter(T adapterDescription) {
         this.adapterDescription = adapterDescription;
+    }
 
+    public void init(AdapterStatus adapterStatus,
+                            String timestampField) {
         if (adapterDescription.getEventGrounding() != null && adapterDescription.getEventGrounding().getTransportProtocol() != null
                 && adapterDescription.getEventGrounding().getTransportProtocol().getBrokerHostname() != null) {
 
-            this.adapterPipeline = getAdapterPipeline(adapterDescription);
+            this.adapterPipeline = getAdapterPipeline(adapterDescription, adapterStatus, timestampField);
         }
     }
 
@@ -90,7 +94,9 @@ public abstract class Adapter<T extends AdapterDescription> implements IAdapter<
         }
     }
 
-    private AdapterPipeline getAdapterPipeline(T adapterDescription) {
+    private AdapterPipeline getAdapterPipeline(T adapterDescription,
+            AdapterStatus adapterStatus,
+            String timestampField) {
 
         List<IAdapterPipelineElement> pipelineElements = new ArrayList<>();
 
@@ -127,7 +133,11 @@ public abstract class Adapter<T extends AdapterDescription> implements IAdapter<
         }
         pipelineElements.add(transformStreamAdapterElement);
 
-        return new AdapterPipeline(pipelineElements, getAdapterSink(adapterDescription));
+        return new AdapterPipeline(
+                pipelineElements,
+                getAdapterSink(adapterDescription),
+                adapterStatus,
+                timestampField);
     }
 
     private SendToBrokerAdapterSink<?> getAdapterSink(AdapterDescription adapterDescription) {
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/model/pipeline/AdapterPipeline.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/model/pipeline/AdapterPipeline.java
index 4bf7af4..aafd73a 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/model/pipeline/AdapterPipeline.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/model/pipeline/AdapterPipeline.java
@@ -18,9 +18,9 @@
 
 package org.apache.streampipes.connect.adapter.model.pipeline;
 
-import org.apache.streampipes.connect.adapter.monitoring.AdapterMonitoring;
 import org.apache.streampipes.connect.api.IAdapterPipeline;
 import org.apache.streampipes.connect.api.IAdapterPipelineElement;
+import org.apache.streampipes.monitoring.AdapterStatus;
 
 import java.util.List;
 import java.util.Map;
@@ -30,11 +30,18 @@ public class AdapterPipeline implements IAdapterPipeline {
     private List<IAdapterPipelineElement> pipelineElements;
     private IAdapterPipelineElement pipelineSink;
 
-    private AdapterMonitoring adapterMonitoring;
+    private AdapterStatus adapterStatus;
+    private String timestampField;
 
-    public AdapterPipeline(List<IAdapterPipelineElement> pipelineElements, IAdapterPipelineElement pipelineSink) {
+    public AdapterPipeline(
+            List<IAdapterPipelineElement> pipelineElements,
+            IAdapterPipelineElement pipelineSink,
+            AdapterStatus adapterStatus,
+            String timestampField) {
         this.pipelineElements = pipelineElements;
         this.pipelineSink = pipelineSink;
+        this.adapterStatus = adapterStatus;
+        this.timestampField = timestampField;
     }
 
     @Override
@@ -46,6 +53,7 @@ public class AdapterPipeline implements IAdapterPipeline {
 
         if (pipelineSink != null) {
             // Write to statistics here
+            this.adapterStatus.increaseCount((Long) event.get(timestampField));
 
             pipelineSink.process(event);
         }
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/monitoring/AdapterMonitoring.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/monitoring/AdapterMonitoring.java
deleted file mode 100644
index f732dcd..0000000
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/monitoring/AdapterMonitoring.java
+++ /dev/null
@@ -1,34 +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.adapter.monitoring;
-
-import java.util.concurrent.ConcurrentHashMap;
-
-public class AdapterMonitoring {
-    private ConcurrentHashMap<String, AdapterStatus> adapterStatus;
-
-    public AdapterMonitoring() {
-        // Should I start the thread here?
-        this.adapterStatus = new ConcurrentHashMap<>();
-    }
-
-    public ConcurrentHashMap<String, AdapterStatus> getAdapterStatus() {
-        return adapterStatus;
-    }
-}
diff --git a/streampipes-monitoring/pom.xml b/streampipes-monitoring/pom.xml
new file mode 100644
index 0000000..0566bc3
--- /dev/null
+++ b/streampipes-monitoring/pom.xml
@@ -0,0 +1,19 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<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">
+    <parent>
+        <artifactId>streampipes-parent</artifactId>
+        <groupId>org.apache.streampipes</groupId>
+        <version>0.69.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>streampipes-monitoring</artifactId>
+
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+</project>
\ No newline at end of file
diff --git a/streampipes-monitoring/src/main/java/org/apache/streampipes/monitoring/AdapterMonitoring.java b/streampipes-monitoring/src/main/java/org/apache/streampipes/monitoring/AdapterMonitoring.java
new file mode 100644
index 0000000..a24e125
--- /dev/null
+++ b/streampipes-monitoring/src/main/java/org/apache/streampipes/monitoring/AdapterMonitoring.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.streampipes.monitoring;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+public class AdapterMonitoring {
+
+    private ConcurrentHashMap<String, AdapterStatus> adapterStatus;
+    private Thread monitoringThread;
+
+    public AdapterMonitoring() {
+        // Should I start the thread here?
+        this.adapterStatus = new ConcurrentHashMap<>();
+    }
+
+    public ConcurrentHashMap<String, AdapterStatus> getAdapterStatus() {
+        return adapterStatus;
+    }
+
+    public void put(String id, AdapterStatus adapterStatus) {
+        this.adapterStatus.put(id, adapterStatus);
+    }
+
+    public void remove(String id) {
+        this.adapterStatus.remove(id);
+    }
+
+    public void init() {
+        Runnable runnable = () -> {
+            while(true) {
+                try {
+                    System.out.println("Currently: " + getAdapterStatus().size() + " adapters are running.");
+                    long currentTimestamp = System.currentTimeMillis();
+                    getAdapterStatus().forEach((s, adapterStatus) -> {
+                        adapterStatus.setTimestamp(currentTimestamp);
+                        System.out.println("Adapter: " + s + " produced " + adapterStatus.getCount() + " events");
+
+
+
+                        adapterStatus.reset();
+                    });
+                    // TODO Scheduled task instead of Thread, see StreamPipesBackendApplication
+                    Thread.sleep(5000);
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+            }
+        };
+
+        this.monitoringThread = new Thread(runnable);
+        this.monitoringThread.start();
+    }
+
+}
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/monitoring/AdapterStatus.java b/streampipes-monitoring/src/main/java/org/apache/streampipes/monitoring/AdapterStatus.java
similarity index 78%
rename from streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/monitoring/AdapterStatus.java
rename to streampipes-monitoring/src/main/java/org/apache/streampipes/monitoring/AdapterStatus.java
index a32dc9d..796c648 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/monitoring/AdapterStatus.java
+++ b/streampipes-monitoring/src/main/java/org/apache/streampipes/monitoring/AdapterStatus.java
@@ -16,10 +16,11 @@
  *
  */
 
-package org.apache.streampipes.connect.adapter.monitoring;
+package org.apache.streampipes.monitoring;
 
 public class AdapterStatus {
     private long timestamp;
+    private long lastEvent;
     private int count;
 
     public AdapterStatus() {
@@ -31,11 +32,15 @@ public class AdapterStatus {
         count = 0;
     }
 
-    public void increaseCount(long newTimestamp) {
-        this.timestamp = newTimestamp;
+    public void increaseCount(Long newTimestamp) {
+        this.lastEvent = newTimestamp;
         this.count++;
     }
 
+    public void setTimestamp(long timestamp) {
+        this.timestamp = timestamp;
+    }
+
     public long getTimestamp() {
         return timestamp;
     }
@@ -43,4 +48,8 @@ public class AdapterStatus {
     public int getCount() {
         return count;
     }
+
+    public long getLastEvent() {
+        return lastEvent;
+    }
 }
diff --git a/ui/cypress/tests/adapter/fileStream.ts b/ui/cypress/tests/adapter/fileStream.ts
index 2c84ae3..bc65b0b 100644
--- a/ui/cypress/tests/adapter/fileStream.ts
+++ b/ui/cypress/tests/adapter/fileStream.ts
@@ -22,7 +22,8 @@ import { GenericAdapterBuilder } from '../../support/builder/GenericAdapterBuild
 
 describe('Test File Stream Adapter', () => {
   before('Setup Test', () => {
-    cy.initStreamPipesTest();
+    // cy.initStreamPipesTest();
+    cy.login();
     FileManagementUtils.addFile('fileTest/random.csv');
   });
 
@@ -39,7 +40,7 @@ describe('Test File Stream Adapter', () => {
       .build();
 
     AdapterUtils.testGenericStreamAdapter(adapterInput);
-    AdapterUtils.deleteAdapter();
+    // AdapterUtils.deleteAdapter();
   });
 
 });