You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by cd...@apache.org on 2019/10/17 07:57:20 UTC

[plc4x] branch feature/plc-simulator updated: - Made data be transported in "byte" instead of "short" arrays. - Changed the PLCSimulator to only run with one selected simulation (Due to the problem of how to map I/O to the different simulations)

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

cdutz pushed a commit to branch feature/plc-simulator
in repository https://gitbox.apache.org/repos/asf/plc4x.git


The following commit(s) were added to refs/heads/feature/plc-simulator by this push:
     new 62ffc28  - Made data be transported in "byte" instead of "short" arrays. - Changed the PLCSimulator to only run with one selected simulation (Due to the problem of how to map I/O to the different simulations)
62ffc28 is described below

commit 62ffc288f051b4a21416fb4fab5cb390e17c57ea
Author: Christofer Dutz <ch...@c-ware.de>
AuthorDate: Thu Oct 17 09:57:08 2019 +0200

    - Made data be transported in "byte" instead of "short" arrays.
    - Changed the PLCSimulator to only run with one selected simulation (Due to the problem of how to map I/O to the different simulations)
---
 .../s7/src/main/resources/protocols/s7/s7.mspec    |  2 +-
 .../org/apache/plc4x/simulator/PlcSimulator.java   | 48 ++++++++++++----------
 .../plc4x/simulator/server/ServerModule.java       |  4 +-
 .../plc4x/simulator/server/s7/S7ServerModule.java  | 12 ++----
 .../server/s7/protocol/S7Step7ServerAdapter.java   | 46 +++++++++++++++++----
 .../watertank/WaterTankSimulationModule.java       |  2 +-
 6 files changed, 72 insertions(+), 42 deletions(-)

diff --git a/protocols/s7/src/main/resources/protocols/s7/s7.mspec b/protocols/s7/src/main/resources/protocols/s7/s7.mspec
index 21f2db8..40b5061 100644
--- a/protocols/s7/src/main/resources/protocols/s7/s7.mspec
+++ b/protocols/s7/src/main/resources/protocols/s7/s7.mspec
@@ -231,7 +231,7 @@
     [simple  uint 8       'returnCode']
     [enum    PayloadSize  'transportSize']
     [simple  uint 16      'dataLength']
-    [array   uint 8       'data' count 'dataLength / 8']
+    [array   int 8        'data' count 'dataLength / 8']
     [padding uint 8       'pad' '0x00' '(dataLength / 8) % 2 == 1']
 ]
 
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/PlcSimulator.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/PlcSimulator.java
index 26aab3a..d9bdab5 100644
--- a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/PlcSimulator.java
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/PlcSimulator.java
@@ -35,25 +35,32 @@ public class PlcSimulator {
 
     private boolean running;
     private final Map<String, ServerModule> serverModules;
-    private final Map<String, SimulationModule> simulationModules;
+    private final SimulationModule simulationModule;
 
-    private PlcSimulator() {
-        this(Thread.currentThread().getContextClassLoader());
+    private PlcSimulator(String simulationName) {
+        this(simulationName, Thread.currentThread().getContextClassLoader());
     }
 
-    private PlcSimulator(ClassLoader classLoader) {
-        Map<String, Context> contexts = new TreeMap<>();
-
+    private PlcSimulator(String simulationName, ClassLoader classLoader) {
+        Context context = null;
         // Initialize all the simulation modules.
         LOGGER.info("Initializing Simulation Modules:");
-        simulationModules = new TreeMap<>();
+        SimulationModule foundSimulationModule = null;
         ServiceLoader<SimulationModule> simulationModuleLoader = ServiceLoader.load(SimulationModule.class, classLoader);
-        for (SimulationModule simulationModule : simulationModuleLoader) {
-            LOGGER.info(String.format("Initializing simulation module: %s ...", simulationModule.getName()));
-            simulationModules.put(simulationModule.getName(), simulationModule);
-            contexts.put(simulationModule.getName(), simulationModule.getContext());
-            LOGGER.info("Initialized");
+        for (SimulationModule curSimulationModule : simulationModuleLoader) {
+            if(curSimulationModule.getName().equals(simulationName)) {
+                LOGGER.info(String.format("Initializing simulation module: %s ...", simulationName));
+                foundSimulationModule = curSimulationModule;
+                context = curSimulationModule.getContext();
+                LOGGER.info("Initialized");
+            }
+        }
+        // If we couldn't find the simulation module provided, report an error and exit.
+        if(foundSimulationModule == null) {
+            LOGGER.info(String.format("Couldn't find simulation module %s", simulationName));
+            System.exit(1);
         }
+        simulationModule = foundSimulationModule;
         LOGGER.info("Finished Initializing Simulation Modules\n");
 
         // Initialize all the server modules.
@@ -64,7 +71,7 @@ public class PlcSimulator {
             LOGGER.info(String.format("Initializing server module: %s ...", serverModule.getName()));
             serverModules.put(serverModule.getName(), serverModule);
             // Inject the contexts.
-            serverModule.setContexts(contexts);
+            serverModule.setContext(context);
             LOGGER.info("Initialized");
         }
         LOGGER.info("Finished Initializing Server Modules\n");
@@ -89,14 +96,11 @@ public class PlcSimulator {
         try {
             LOGGER.info("Starting simulations ...");
             while (running) {
-                // Give all the simulation modules the chance to do something.
-                for (SimulationModule simulationModule : simulationModules.values()) {
-                    try {
-                        simulationModule.loop();
-                    } catch(Exception e) {
-                        LOGGER.error("Caught error while executing loop() method of " + simulationModule.getName() +
-                            " simulation.", e);
-                    }
+                try {
+                    simulationModule.loop();
+                } catch(Exception e) {
+                    LOGGER.error("Caught error while executing loop() method of " + simulationModule.getName() +
+                        " simulation.", e);
                 }
                 // Sleep 100 ms to not run the simulation too eagerly.
                 TimeUnit.MILLISECONDS.sleep(100);
@@ -113,7 +117,7 @@ public class PlcSimulator {
     }
 
     public static void main(String[] args) throws Exception {
-        final PlcSimulator simulator = new PlcSimulator();
+        final PlcSimulator simulator = new PlcSimulator("Water Tank");
         // Make sure we stop everything correctly.
         Runtime.getRuntime().addShutdownHook(new Thread(simulator::stop));
         // Start the simulator.
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/ServerModule.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/ServerModule.java
index 97bf1c0..61a1ac9 100644
--- a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/ServerModule.java
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/ServerModule.java
@@ -21,8 +21,6 @@ package org.apache.plc4x.simulator.server;
 import org.apache.plc4x.simulator.exceptions.SimulatorExcepiton;
 import org.apache.plc4x.simulator.model.Context;
 
-import java.util.Map;
-
 public interface ServerModule {
 
     /**
@@ -30,7 +28,7 @@ public interface ServerModule {
      */
     String getName();
 
-    void setContexts(Map<String, Context> contexts);
+    void setContext(Context contexts);
 
     void start() throws SimulatorExcepiton;
 
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7ServerModule.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7ServerModule.java
index 483acc6..5a63fc9 100644
--- a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7ServerModule.java
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7ServerModule.java
@@ -32,17 +32,13 @@ import org.apache.plc4x.simulator.model.Context;
 import org.apache.plc4x.simulator.server.ServerModule;
 import org.apache.plc4x.simulator.server.s7.protocol.S7Step7ServerAdapter;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
 public class S7ServerModule implements ServerModule {
 
     private static final int ISO_ON_TCP_PORT = 102;
 
     private EventLoopGroup loopGroup;
     private EventLoopGroup workerGroup;
-    private List<Context> contexts;
+    private Context context;
 
     @Override
     public String getName() {
@@ -50,8 +46,8 @@ public class S7ServerModule implements ServerModule {
     }
 
     @Override
-    public void setContexts(Map<String, Context> contexts) {
-        this.contexts = new ArrayList<>(contexts.values());
+    public void setContext(Context context) {
+        this.context = context;
     }
 
     @Override
@@ -72,7 +68,7 @@ public class S7ServerModule implements ServerModule {
                     public void initChannel(SocketChannel channel) {
                         ChannelPipeline pipeline = channel.pipeline();
                         pipeline.addLast(new S7Step7Protocol());
-                        pipeline.addLast(new S7Step7ServerAdapter(contexts));
+                        pipeline.addLast(new S7Step7ServerAdapter(context));
                     }
                 }).option(ChannelOption.SO_BACKLOG, 128)
                 .childOption(ChannelOption.SO_KEEPALIVE, true);
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/protocol/S7Step7ServerAdapter.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/protocol/S7Step7ServerAdapter.java
index 1ab07b5..c6ae342 100644
--- a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/protocol/S7Step7ServerAdapter.java
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/protocol/S7Step7ServerAdapter.java
@@ -25,14 +25,15 @@ import org.apache.plc4x.simulator.model.Context;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Arrays;
+import java.util.BitSet;
 import java.util.List;
-import java.util.Map;
 
 public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
 
     private static final Logger LOGGER = LoggerFactory.getLogger(S7Step7ServerAdapter.class);
 
-    private List<Context> contexts;
+    private Context context;
 
     private State state;
 
@@ -54,8 +55,8 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
     private static final int maxPduLength = 240;
     private int pduLength;
 
-    public S7Step7ServerAdapter(List<Context> contexts) {
-        this.contexts = contexts;
+    public S7Step7ServerAdapter(Context context) {
+        this.context = context;
         state = State.INITIAL;
     }
 
@@ -181,6 +182,7 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
                                                         function.getCpuSubfunction(), (short) 1,
                                                         (short) 0, (short) 0, 0);
 
+                                                // This is the product number of a S7-1200
                                                 SzlDataTreeItem[] items = new SzlDataTreeItem[1];
                                                 items[0] = new SzlDataTreeItem((short) 0x0001,
                                                     "6ES7 212-1BD30-0XB0 ".getBytes(), 0x2020, 0x0001, 0x2020);
@@ -207,7 +209,7 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
                                                 ctx.writeAndFlush(new TPKTPacket(new COTPPacketData(null, s7ResponseMessage, true, cotpTpduRef)));
                                             } else {
                                                 LOGGER.error("Not able to respond to the given request Read SZL with SZL type class " +
-                                                    szlId.getTypeClass().name() + " and SZL sublise " + szlId.getSublistList().name());
+                                                    szlId.getTypeClass().name() + " and SZL sublist " + szlId.getSublistList().name());
                                             }
 
                                         }
@@ -225,17 +227,39 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
                                 S7ParameterReadVarRequest readVarRequestParameter =
                                     (S7ParameterReadVarRequest) request.getParameter();
                                 final S7VarRequestParameterItem[] items = readVarRequestParameter.getItems();
-                                for (S7VarRequestParameterItem item : items) {
+                                S7VarPayloadDataItem[] payloadItems = new S7VarPayloadDataItem[items.length];
+                                for (int i = 0; i < items.length; i++) {
+                                    S7VarRequestParameterItem item = items[i];
                                     if(item instanceof S7VarRequestParameterItemAddress) {
                                         S7VarRequestParameterItemAddress address =
                                             (S7VarRequestParameterItemAddress) item;
                                         final S7Address address1 = address.getAddress();
                                         if(address1 instanceof S7AddressAny) {
                                             S7AddressAny addressAny = (S7AddressAny) address1;
-
+                                            switch (addressAny.getArea()) {
+                                                case DATA_BLOCKS: {
+
+                                                    break;
+                                                }
+                                                case INPUTS:
+                                                case OUTPUTS: {
+                                                    int ioNumber = (addressAny.getByteAddress() * 8) + addressAny.getBitAddress();
+                                                    int numElements = (addressAny.getTransportSize() == ParameterSize.BOOL) ?
+                                                        addressAny.getNumberOfElements() : addressAny.getTransportSize().getSizeInBytes() * 8;
+                                                    final BitSet bitSet = toBitSet(context.getDigitalInputs(), ioNumber, numElements);
+                                                    final byte[] data = Arrays.copyOf(bitSet.toByteArray(), (numElements + 7) / 8);
+                                                    payloadItems[i] = new S7VarPayloadDataItem((short) 0xFF, PayloadSize.BYTE_WORD_DWORD, data.length, data);
+                                                    break;
+                                                }
+                                            }
                                         }
                                     }
                                 }
+                                S7ParameterReadVarResponse readVarResponseParameter = new S7ParameterReadVarResponse((short) items.length);
+                                S7PayloadReadVarResponse readVarResponsePayload = new S7PayloadReadVarResponse(payloadItems);
+                                S7MessageResponse response = new S7MessageResponse(request.getTpduReference(),
+                                    readVarResponseParameter, readVarResponsePayload, (short) 0x00, (short) 0x00);
+                                ctx.writeAndFlush(new TPKTPacket(new COTPPacketData(null, response, true, cotpTpduRef)));
                             }
                             else if(request.getParameter() instanceof S7ParameterWriteVarRequest) {
                                 S7ParameterWriteVarRequest writeVarRequestParameter =
@@ -263,4 +287,12 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
         S7_CONNECTED
     }
 
+    private BitSet toBitSet(List<Boolean> booleans, int startIndex, int numElements) {
+        BitSet bitSet = new BitSet(booleans.size());
+        for(int i = 0; i < Math.min(booleans.size() - startIndex, numElements); i++) {
+            bitSet.set(i, booleans.get(i + startIndex));
+        }
+        return bitSet;
+    }
+
 }
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/simulation/watertank/WaterTankSimulationModule.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/simulation/watertank/WaterTankSimulationModule.java
index e49b949..9824676 100644
--- a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/simulation/watertank/WaterTankSimulationModule.java
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/simulation/watertank/WaterTankSimulationModule.java
@@ -54,7 +54,7 @@ public class WaterTankSimulationModule implements SimulationModule {
             // The input valve
             .addDigitalInput(false)
             // The output valve
-            .addDigitalInput(false)
+            .addDigitalInput(true)
             // The emergency valve
             .addDigitalOutput(false)
             // The water level