You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by jf...@apache.org on 2020/05/22 14:53:39 UTC

[plc4x] branch feature/plc-simulator updated (b027126 -> 53d36ee)

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

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


    from b027126  - Explicitly set the _allow_illegal_access_reflection_in_tests profile to disabled to skip the detection on the site build node
     new 0e7b732  [SIMULATED PLC] Refactoring / new Handler Architecture.
     new 53d36ee  [SIMULATED PLC] Many improvements.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../plc4x/java/spi/optimizer/BaseOptimizer.java    |  3 +-
 .../org/apache/plc4x/simulator/PlcSimulator.java   | 24 ++++++-
 .../simulator/server/s7/FieldReadException.java    |  9 ++-
 .../server/s7/InvalidAddressException.java         | 12 ++--
 .../apache/plc4x/simulator/server/s7/S7Int.java    | 84 ++++++++++++++++++++++
 .../plc4x/simulator/server/s7/S7PlcHandler.java    | 19 +++--
 .../simulator/server/s7/S7PlcHandlerBase.java      | 37 +++++-----
 .../plc4x/simulator/server/s7/S7ServerModule.java  | 15 ++--
 .../apache/plc4x/simulator/server/s7/S7Value.java  | 14 ++--
 .../plc4x/simulator/server/s7/S7ValueFactory.java  | 51 +++++++------
 .../server/s7/protocol/S7Step7ServerAdapter.java   | 52 ++++++++++----
 11 files changed, 239 insertions(+), 81 deletions(-)
 copy plc4j/api/src/main/java/org/apache/plc4x/java/api/Experimental.java => sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/FieldReadException.java (78%)
 copy plc4j/api/src/main/java/org/apache/plc4x/java/api/messages/PlcRequest.java => sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/InvalidAddressException.java (72%)
 create mode 100644 sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7Int.java
 copy plc4j/tools/scraper/src/main/java/org/apache/plc4x/java/scraper/triggeredscraper/TriggeredScraperTaskMBean.java => sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandler.java (65%)
 copy plc4j/transports/serial/src/main/java/org/apache/plc4x/java/transport/serial/SerialSocketAddress.java => sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandlerBase.java (55%)
 copy plc4j/spi/src/main/java/org/apache/plc4x/java/spi/request/PlcRequest.java => sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7Value.java (80%)
 copy plc4j/transports/serial/src/main/java/org/apache/plc4x/java/transport/serial/SerialSelectorProvider.java => sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7ValueFactory.java (55%)


[plc4x] 01/02: [SIMULATED PLC] Refactoring / new Handler Architecture.

Posted by jf...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 0e7b7326bb0eefbc0e46a12f2b093054ed8b14a9
Author: julian <j....@pragmaticminds.de>
AuthorDate: Fri May 22 15:36:34 2020 +0200

    [SIMULATED PLC] Refactoring / new Handler Architecture.
---
 .../org/apache/plc4x/simulator/PlcSimulator.java   | 33 +++++++-
 .../plc4x/simulator/server/s7/S7PlcHandler.java    | 95 ++++++++++++++++++++++
 .../plc4x/simulator/server/s7/S7ServerModule.java  | 15 +++-
 .../server/s7/protocol/S7Step7ServerAdapter.java   | 32 +++++---
 4 files changed, 155 insertions(+), 20 deletions(-)

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 d9bdab5..360097f 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
@@ -20,6 +20,8 @@ package org.apache.plc4x.simulator;
 
 import org.apache.plc4x.simulator.model.Context;
 import org.apache.plc4x.simulator.server.ServerModule;
+import org.apache.plc4x.simulator.server.s7.S7PlcHandler;
+import org.apache.plc4x.simulator.server.s7.S7ServerModule;
 import org.apache.plc4x.simulator.simulation.SimulationModule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,7 +50,7 @@ public class PlcSimulator {
         SimulationModule foundSimulationModule = null;
         ServiceLoader<SimulationModule> simulationModuleLoader = ServiceLoader.load(SimulationModule.class, classLoader);
         for (SimulationModule curSimulationModule : simulationModuleLoader) {
-            if(curSimulationModule.getName().equals(simulationName)) {
+            if (curSimulationModule.getName().equals(simulationName)) {
                 LOGGER.info(String.format("Initializing simulation module: %s ...", simulationName));
                 foundSimulationModule = curSimulationModule;
                 context = curSimulationModule.getContext();
@@ -56,7 +58,7 @@ public class PlcSimulator {
             }
         }
         // If we couldn't find the simulation module provided, report an error and exit.
-        if(foundSimulationModule == null) {
+        if (foundSimulationModule == null) {
             LOGGER.info(String.format("Couldn't find simulation module %s", simulationName));
             System.exit(1);
         }
@@ -88,6 +90,31 @@ public class PlcSimulator {
         LOGGER.info("Starting Server Modules:");
         for (ServerModule serverModule : serverModules.values()) {
             LOGGER.info(String.format("Starting server module: %s ...", serverModule.getName()));
+            ((S7ServerModule) serverModule).setHandler(new S7PlcHandler() {
+                @Override
+                public void onConnectionInitiated() {
+
+                }
+
+                @Override
+                public void onConnectionEstablished() {
+
+                }
+
+                @Override
+                public void onConnectionClosed() {
+
+                }
+
+                @Override
+                public S7Int readIntFromDataBlock(int dbNumber, int byteAddress, byte bitAddress) {
+                    if (byteAddress == 0) {
+                        return S7Int._int((short) -42);
+                    } else {
+                        return S7Int._uint(42);
+                    }
+                }
+            });
             serverModule.start();
             LOGGER.info("Started");
         }
@@ -98,7 +125,7 @@ public class PlcSimulator {
             while (running) {
                 try {
                     simulationModule.loop();
-                } catch(Exception e) {
+                } catch (Exception e) {
                     LOGGER.error("Caught error while executing loop() method of " + simulationModule.getName() +
                         " simulation.", e);
                 }
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandler.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandler.java
new file mode 100644
index 0000000..d5b8efb
--- /dev/null
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandler.java
@@ -0,0 +1,95 @@
+/*
+ * 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.plc4x.simulator.server.s7;
+
+/**
+ * Handler for PLC Server.
+ *
+ * @author julian
+ * Created by julian on 22.05.20
+ */
+public interface S7PlcHandler {
+
+    void onConnectionInitiated();
+
+    void onConnectionEstablished();
+
+    void onConnectionClosed();
+
+    S7Int readIntFromDataBlock(int dbNumber, int byteAddress, byte bitAddress);
+
+    class S7Int {
+        private final Short signed;
+        private final Integer unsigned;
+
+        public S7Int(short signed) {
+            this.signed = signed;
+            this.unsigned = null;
+        }
+
+        public S7Int(int unsigned) {
+            if (unsigned < 0) {
+                throw new IllegalArgumentException("Signed value cannot be negative!");
+            }
+            this.unsigned = unsigned;
+            this.signed = null;
+        }
+
+        public static S7Int _int(short signed) {
+            return new S7Int(signed);
+        }
+
+        public static S7Int _uint(int unsigned) {
+            return new S7Int(unsigned);
+        }
+
+        public boolean isSigned() {
+            return signed != null;
+        }
+
+        public boolean isUnsigned() {
+            return unsigned != null;
+        }
+
+        public short getSigned() {
+            if (!isSigned()) {
+                throw new UnsupportedOperationException("Cannot get signed on unsigned!");
+            }
+            return signed;
+        }
+
+        public Integer getUnsigned() {
+            if (!isUnsigned()) {
+                throw new UnsupportedOperationException("Cannot get unsigned on signed!");
+            }
+            return unsigned;
+        }
+
+        @Override
+        public String toString() {
+            if (isSigned()) {
+                return "Signed(" + signed + ")";
+            } else {
+                return "Unsigned(" + unsigned + ")";
+            }
+        }
+    }
+
+}
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 5be811c..62d44fa 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
@@ -45,7 +45,8 @@ public class S7ServerModule implements ServerModule {
 
     private EventLoopGroup loopGroup;
     private EventLoopGroup workerGroup;
-    private Context context;
+    // private Context context;
+    private S7PlcHandler handler;
 
     @Override
     public String getName() {
@@ -54,12 +55,18 @@ public class S7ServerModule implements ServerModule {
 
     @Override
     public void setContext(Context context) {
-        this.context = context;
+        // Do Nothing...
+    }
+
+    public void setHandler(S7PlcHandler handler) {
+        this.handler = handler;
     }
 
     @Override
     public void start() throws SimulatorExcepiton {
-        S7Driver driver = new S7Driver();
+        if (this.handler == null) {
+            throw new IllegalStateException("A handler has to be set to start the PLC Server");
+        }
         if(loopGroup != null) {
             return;
         }
@@ -78,7 +85,7 @@ public class S7ServerModule implements ServerModule {
                         pipeline.addLast(new GeneratedProtocolMessageCodec<>(TPKTPacket.class, new TPKTPacketIO(), true, null,
                             new S7Driver.ByteLengthEstimator(),
                             new S7Driver.CorruptPackageCleaner()));
-                        pipeline.addLast(new S7Step7ServerAdapter(context));
+                        pipeline.addLast(new S7Step7ServerAdapter(handler));
                     }
                 }).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 9c60514..74bd580 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
@@ -21,11 +21,11 @@ package org.apache.plc4x.simulator.server.s7.protocol;
 import io.netty.channel.*;
 import org.apache.plc4x.java.s7.readwrite.*;
 import org.apache.plc4x.java.s7.readwrite.types.*;
-import org.apache.plc4x.simulator.model.Context;
+import org.apache.plc4x.java.spi.generation.WriteBuffer;
+import org.apache.plc4x.simulator.server.s7.S7PlcHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.nio.ByteOrder;
 import java.util.Arrays;
 import java.util.BitSet;
 import java.util.List;
@@ -34,7 +34,7 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
 
     private static final Logger LOGGER = LoggerFactory.getLogger(S7Step7ServerAdapter.class);
 
-    private Context context;
+    private S7PlcHandler handler;
 
     private State state;
 
@@ -56,8 +56,8 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
     private static final int maxPduLength = 240;
     private int pduLength;
 
-    public S7Step7ServerAdapter(Context context) {
-        this.context = context;
+    public S7Step7ServerAdapter(S7PlcHandler handler) {
+        this.handler = handler;
         state = State.INITIAL;
     }
 
@@ -257,14 +257,19 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
                                                     }
                                                     final byte bitAddress = addressAny.getBitAddress();
                                                     switch (addressAny.getTransportSize()) {
-                                                        case INT:
+                                                        case INT: // These case should never happen. UINT will always be picked
                                                         case UINT: {
-                                                            String firstKey = context.getMemory().keySet().iterator().next();
-                                                            Object value = context.getMemory().get(firstKey);
-                                                            short shortValue = 42; // ((Number) value).shortValue();
-                                                            byte[] data = new byte[2];
-                                                            data[0] = (byte) (shortValue & 0xff);
-                                                            data[1] = (byte) ((shortValue >> 8) & 0xff);
+                                                            // The value should be represented as Short
+                                                            S7PlcHandler.S7Int s7Int = handler.readIntFromDataBlock(addressAny.getDbNumber(), addressAny.getByteAddress(), addressAny.getBitAddress());
+
+                                                            WriteBuffer writeBuffer = new WriteBuffer(2, false);
+                                                            if (s7Int.isSigned()) {
+                                                                writeBuffer.writeShort(16, s7Int.getSigned());
+                                                            } else {
+                                                                writeBuffer.writeUnsignedInt(16, s7Int.getUnsigned());
+                                                            }
+                                                            byte[] data = writeBuffer.getData();
+
                                                             payloadItems[i] = new S7VarPayloadDataItem(DataTransportErrorCode.OK, DataTransportSize.BYTE_WORD_DWORD, 8 * data.length, data);
                                                             break;
                                                         }
@@ -279,7 +284,8 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
                                                     final int ioNumber = (addressAny.getByteAddress() * 8) + addressAny.getBitAddress();
                                                     final int numElements = (addressAny.getTransportSize() == TransportSize.BOOL) ?
                                                         addressAny.getNumberOfElements() : addressAny.getTransportSize().getSizeInBytes() * 8;
-                                                    final BitSet bitSet = toBitSet(context.getDigitalInputs(), ioNumber, numElements);
+                                                    // TODO
+                                                    final BitSet bitSet = new BitSet(); //toBitSet(handler.getDigitalInputs(), ioNumber, numElements);
                                                     final byte[] data = Arrays.copyOf(bitSet.toByteArray(), (numElements + 7) / 8);
                                                     payloadItems[i] = new S7VarPayloadDataItem(DataTransportErrorCode.OK, DataTransportSize.BYTE_WORD_DWORD, 8 * data.length, data);
                                                     break;


[plc4x] 02/02: [SIMULATED PLC] Many improvements.

Posted by jf...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 53d36ee4bf6d3badea248fbb02cc2cbd86455e62
Author: julian <j....@pragmaticminds.de>
AuthorDate: Fri May 22 16:53:32 2020 +0200

    [SIMULATED PLC] Many improvements.
---
 .../plc4x/java/spi/optimizer/BaseOptimizer.java    |  3 +-
 .../org/apache/plc4x/simulator/PlcSimulator.java   | 31 +++-----
 .../simulator/server/s7/FieldReadException.java    | 29 ++++++++
 .../server/s7/InvalidAddressException.java         | 30 ++++++++
 .../apache/plc4x/simulator/server/s7/S7Int.java    | 84 ++++++++++++++++++++++
 .../plc4x/simulator/server/s7/S7PlcHandler.java    | 63 ++--------------
 .../simulator/server/s7/S7PlcHandlerBase.java      | 51 +++++++++++++
 .../apache/plc4x/simulator/server/s7/S7Value.java  | 34 +++++++++
 .../plc4x/simulator/server/s7/S7ValueFactory.java  | 73 +++++++++++++++++++
 .../server/s7/protocol/S7Step7ServerAdapter.java   | 22 +++++-
 10 files changed, 339 insertions(+), 81 deletions(-)

diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/optimizer/BaseOptimizer.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/optimizer/BaseOptimizer.java
index 3c252f4..d352e4e 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/optimizer/BaseOptimizer.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/optimizer/BaseOptimizer.java
@@ -51,7 +51,8 @@ public abstract class BaseOptimizer {
                     PlcReadResponse subReadResponse = (PlcReadResponse) readResponse.getLeft();
                     fields.put(fieldName,
                         new ResponseItem<>(subReadResponse.getResponseCode(fieldName),
-                            subReadResponse.getAsPlcValue().getValue(fieldName)));
+                            // We cannot safely access the response value if response code != OK
+                            subReadResponse.getResponseCode(fieldName) == PlcResponseCode.OK ? subReadResponse.getAsPlcValue().getValue(fieldName) : null));
                 } else {
                     fields.put(fieldName, new ResponseItem<>(PlcResponseCode.INTERNAL_ERROR, null));
                 }
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 360097f..bc7865d 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
@@ -20,8 +20,7 @@ package org.apache.plc4x.simulator;
 
 import org.apache.plc4x.simulator.model.Context;
 import org.apache.plc4x.simulator.server.ServerModule;
-import org.apache.plc4x.simulator.server.s7.S7PlcHandler;
-import org.apache.plc4x.simulator.server.s7.S7ServerModule;
+import org.apache.plc4x.simulator.server.s7.*;
 import org.apache.plc4x.simulator.simulation.SimulationModule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,6 +30,9 @@ import java.util.ServiceLoader;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.plc4x.simulator.server.s7.S7ValueFactory.INT;
+import static org.apache.plc4x.simulator.server.s7.S7ValueFactory.UINT;
+
 public class PlcSimulator {
 
     private static final Logger LOGGER = LoggerFactory.getLogger(PlcSimulator.class);
@@ -90,30 +92,19 @@ public class PlcSimulator {
         LOGGER.info("Starting Server Modules:");
         for (ServerModule serverModule : serverModules.values()) {
             LOGGER.info(String.format("Starting server module: %s ...", serverModule.getName()));
-            ((S7ServerModule) serverModule).setHandler(new S7PlcHandler() {
-                @Override
-                public void onConnectionInitiated() {
-
-                }
+            ((S7ServerModule) serverModule).setHandler(new S7PlcHandlerBase() {
 
                 @Override
-                public void onConnectionEstablished() {
-
-                }
-
-                @Override
-                public void onConnectionClosed() {
-
-                }
-
-                @Override
-                public S7Int readIntFromDataBlock(int dbNumber, int byteAddress, byte bitAddress) {
+                public S7Int readIntFromDataBlock(int dbNumber, int byteAddress, byte bitAddress) throws FieldReadException {
                     if (byteAddress == 0) {
-                        return S7Int._int((short) -42);
+                        return INT((short) -42);
+                    } else if (byteAddress == 2) {
+                        return UINT(42);
                     } else {
-                        return S7Int._uint(42);
+                        throw new InvalidAddressException();
                     }
                 }
+
             });
             serverModule.start();
             LOGGER.info("Started");
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/FieldReadException.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/FieldReadException.java
new file mode 100644
index 0000000..cb860b6
--- /dev/null
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/FieldReadException.java
@@ -0,0 +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.
+ */
+
+package org.apache.plc4x.simulator.server.s7;
+
+/**
+ * Base class for Exceptions that can occur when a Filed is read.
+ *
+ * @author julian
+ * Created by julian on 22.05.20
+ */
+public abstract class FieldReadException extends Exception {
+}
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/InvalidAddressException.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/InvalidAddressException.java
new file mode 100644
index 0000000..078f57b
--- /dev/null
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/InvalidAddressException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.plc4x.simulator.server.s7;
+
+/**
+ * Exception that indicates that the Field is not known.
+ * This results in {@link org.apache.plc4x.java.api.types.PlcResponseCode}s INVALID_ADDRESS.
+ *
+ * @author julian
+ * Created by julian on 22.05.20
+ */
+public class InvalidAddressException extends FieldReadException {
+}
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7Int.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7Int.java
new file mode 100644
index 0000000..f5303c9
--- /dev/null
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7Int.java
@@ -0,0 +1,84 @@
+/*
+ * 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.plc4x.simulator.server.s7;
+
+/**
+ * TODO write comment
+ *
+ * @author julian
+ * Created by julian on 22.05.20
+ */
+public class S7Int {
+
+    private final Short signed;
+    private final Integer unsigned;
+
+    public S7Int(short signed) {
+        this.signed = signed;
+        this.unsigned = null;
+    }
+
+    public S7Int(int unsigned) {
+        if (unsigned < 0) {
+            throw new IllegalArgumentException("Signed value cannot be negative!");
+        }
+        this.unsigned = unsigned;
+        this.signed = null;
+    }
+
+    public static S7Int INT(short signed) {
+        return new S7Int(signed);
+    }
+
+    public static S7Int UINT(int unsigned) {
+        return new S7Int(unsigned);
+    }
+
+    public boolean isSigned() {
+        return signed != null;
+    }
+
+    public boolean isUnsigned() {
+        return unsigned != null;
+    }
+
+    public short getSigned() {
+        if (!isSigned()) {
+            throw new UnsupportedOperationException("Cannot get signed on unsigned!");
+        }
+        return signed;
+    }
+
+    public Integer getUnsigned() {
+        if (!isUnsigned()) {
+            throw new UnsupportedOperationException("Cannot get unsigned on signed!");
+        }
+        return unsigned;
+    }
+
+    @Override
+    public String toString() {
+        if (isSigned()) {
+            return "Signed(" + signed + ")";
+        } else {
+            return "Unsigned(" + unsigned + ")";
+        }
+    }
+}
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandler.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandler.java
index d5b8efb..a672998 100644
--- a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandler.java
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandler.java
@@ -19,6 +19,8 @@
 
 package org.apache.plc4x.simulator.server.s7;
 
+import java.net.InetSocketAddress;
+
 /**
  * Handler for PLC Server.
  *
@@ -27,69 +29,12 @@ package org.apache.plc4x.simulator.server.s7;
  */
 public interface S7PlcHandler {
 
-    void onConnectionInitiated();
+    void onConnectionInitiated(InetSocketAddress remoteAddress);
 
     void onConnectionEstablished();
 
     void onConnectionClosed();
 
-    S7Int readIntFromDataBlock(int dbNumber, int byteAddress, byte bitAddress);
-
-    class S7Int {
-        private final Short signed;
-        private final Integer unsigned;
-
-        public S7Int(short signed) {
-            this.signed = signed;
-            this.unsigned = null;
-        }
-
-        public S7Int(int unsigned) {
-            if (unsigned < 0) {
-                throw new IllegalArgumentException("Signed value cannot be negative!");
-            }
-            this.unsigned = unsigned;
-            this.signed = null;
-        }
-
-        public static S7Int _int(short signed) {
-            return new S7Int(signed);
-        }
-
-        public static S7Int _uint(int unsigned) {
-            return new S7Int(unsigned);
-        }
-
-        public boolean isSigned() {
-            return signed != null;
-        }
-
-        public boolean isUnsigned() {
-            return unsigned != null;
-        }
-
-        public short getSigned() {
-            if (!isSigned()) {
-                throw new UnsupportedOperationException("Cannot get signed on unsigned!");
-            }
-            return signed;
-        }
-
-        public Integer getUnsigned() {
-            if (!isUnsigned()) {
-                throw new UnsupportedOperationException("Cannot get unsigned on signed!");
-            }
-            return unsigned;
-        }
-
-        @Override
-        public String toString() {
-            if (isSigned()) {
-                return "Signed(" + signed + ")";
-            } else {
-                return "Unsigned(" + unsigned + ")";
-            }
-        }
-    }
+    S7Int readIntFromDataBlock(int dbNumber, int byteAddress, byte bitAddress) throws FieldReadException;
 
 }
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandlerBase.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandlerBase.java
new file mode 100644
index 0000000..ada966b
--- /dev/null
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7PlcHandlerBase.java
@@ -0,0 +1,51 @@
+/*
+ * 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.plc4x.simulator.server.s7;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Base Implementation.
+ */
+public abstract class S7PlcHandlerBase implements S7PlcHandler {
+
+    @Override
+    public void onConnectionInitiated(InetSocketAddress remoteAddress) {
+        // Intentionally do nothing
+    }
+
+    @Override
+    public void onConnectionEstablished() {
+        // Intentionally do nothing
+    }
+
+    @Override
+    public void onConnectionClosed() {
+        // Intentionally do nothing
+    }
+
+    /**
+     * Will always return invalid address.
+     */
+    @Override
+    public S7Int readIntFromDataBlock(int dbNumber, int byteAddress, byte bitAddress) throws FieldReadException {
+        throw new InvalidAddressException();
+    }
+}
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7Value.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7Value.java
new file mode 100644
index 0000000..28a4ca9
--- /dev/null
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7Value.java
@@ -0,0 +1,34 @@
+/*
+ * 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.plc4x.simulator.server.s7;
+
+/**
+ * TODO write comment
+ *
+ * @author julian
+ * Created by julian on 22.05.20
+ */
+public interface S7Value {
+
+    void _int();
+    void _uint();
+    void _real();
+
+}
diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7ValueFactory.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7ValueFactory.java
new file mode 100644
index 0000000..1f88e5c
--- /dev/null
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/s7/S7ValueFactory.java
@@ -0,0 +1,73 @@
+/*
+ * 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.plc4x.simulator.server.s7;
+
+import org.apache.commons.lang3.NotImplementedException;
+
+import java.math.BigInteger;
+
+/**
+ * TODO write comment
+ *
+ * @author julian
+ * Created by julian on 22.05.20
+ */
+public class S7ValueFactory {
+
+    // Signed 2 Byte INT
+    public static S7Int INT(short s) {
+        return S7Int.INT(s);
+    }
+
+    // Unsigned 2 Byte INT
+    public static S7Int UINT(int i) {
+        return S7Int.UINT(i);
+    }
+
+    // Signed 4 Byte INT
+    public static S7Int DINT(int l) {
+        throw new NotImplementedException("");
+    }
+
+    // Signed 8 Byte Int
+    public static S7Int LINT(long l) {
+        throw new NotImplementedException("");
+    }
+
+    // Unsigned 4 Byte INT
+    public static S7Int UDINT(long l) {
+        throw new NotImplementedException("");
+    }
+
+    // Unsigned 8 Byte INT
+    public static S7Int ULINT(BigInteger bi) {
+        throw new NotImplementedException("");
+    }
+
+    // 4 Byte floating point
+    public static Object REAL(float f) {
+        throw new NotImplementedException("");
+    }
+
+    // 8 Byte floating point
+    public static Object LREAL(double d) {
+        throw new NotImplementedException("");
+    }
+}
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 74bd580..8391d61 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
@@ -22,10 +22,13 @@ import io.netty.channel.*;
 import org.apache.plc4x.java.s7.readwrite.*;
 import org.apache.plc4x.java.s7.readwrite.types.*;
 import org.apache.plc4x.java.spi.generation.WriteBuffer;
+import org.apache.plc4x.simulator.server.s7.InvalidAddressException;
+import org.apache.plc4x.simulator.server.s7.S7Int;
 import org.apache.plc4x.simulator.server.s7.S7PlcHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.BitSet;
 import java.util.List;
@@ -73,6 +76,9 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
                         return;
                     }
 
+                    // Callback
+                    handler.onConnectionInitiated((InetSocketAddress) ctx.pipeline().channel().remoteAddress());
+
                     COTPTpduSize proposedTpduSize = null;
                     COTPPacketConnectionRequest cotpConnectionRequest = (COTPPacketConnectionRequest) cotpPacket;
                     for (COTPParameter parameter : cotpConnectionRequest.getParameters()) {
@@ -146,6 +152,9 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
                         s7TpduReference, s7ParameterSetupCommunicationResponse, null, (short) 0, (short) 0);
                     ctx.writeAndFlush(new TPKTPacket(new COTPPacketData(null, s7MessageResponse, true, cotpTpduRef)));
 
+                    // Now we should be connected
+                    handler.onConnectionEstablished();
+
                     state = State.S7_CONNECTED;
                     break;
                 }
@@ -260,7 +269,18 @@ public class S7Step7ServerAdapter extends ChannelInboundHandlerAdapter {
                                                         case INT: // These case should never happen. UINT will always be picked
                                                         case UINT: {
                                                             // The value should be represented as Short
-                                                            S7PlcHandler.S7Int s7Int = handler.readIntFromDataBlock(addressAny.getDbNumber(), addressAny.getByteAddress(), addressAny.getBitAddress());
+                                                            S7Int s7Int;
+                                                            try {
+                                                                s7Int = handler.readIntFromDataBlock(addressAny.getDbNumber(), addressAny.getByteAddress(), addressAny.getBitAddress());
+                                                            } catch (InvalidAddressException e) {
+                                                                // Send a INVALID_ADDRESS response
+                                                                payloadItems[i] = new S7VarPayloadDataItem(DataTransportErrorCode.INVALID_ADDRESS, DataTransportSize.NULL, 0, new byte[0]);
+                                                                break;
+                                                            } catch (Exception e) {
+                                                                // We have no idea, so just send INVALID_ADDRESS ?
+                                                                payloadItems[i] = new S7VarPayloadDataItem(DataTransportErrorCode.INVALID_ADDRESS, DataTransportSize.NULL, 0, new byte[0]);
+                                                                break;
+                                                            }
 
                                                             WriteBuffer writeBuffer = new WriteBuffer(2, false);
                                                             if (s7Int.isSigned()) {