You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by sr...@apache.org on 2018/03/15 11:55:22 UTC

[incubator-plc4x] 04/04: introduced new protocol layer ads2payload and removed dependency from serial to tcp

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

sruehl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-plc4x.git

commit b4c71ecd32ee711744cd4ee0a9197d5f0b1deddf
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Mar 15 12:55:08 2018 +0100

    introduced new protocol layer ads2payload and removed dependency from
    serial to tcp
---
 .../plc4x/java/ads/api/commands/types/Data.java    |   9 +
 .../plc4x/java/ads/api/generic/AmsPacket.java      |  11 --
 .../plc4x/java/ads/api/serial/AmsSerialFrame.java  |  20 +-
 .../plc4x/java/ads/api/serial/types/UserData.java  |   9 +
 .../plc4x/java/ads/api/tcp/AmsTCPPacket.java       |  45 ++---
 .../ads/api/{serial => tcp}/types/UserData.java    |  11 +-
 .../apache/plc4x/java/ads/api/util/ByteValue.java  |  12 +-
 .../ads/connection/AdsSerialPlcConnection.java     |   6 +-
 .../java/ads/connection/AdsTcpPlcConnection.java   |   6 +-
 ...s2TcpProtocol.java => Ads2PayloadProtocol.java} |  29 +--
 ...alProtocol.java => Payload2SerialProtocol.java} |  46 +----
 .../java/ads/protocol/Payload2TcpProtocol.java     |  58 ++++++
 ...ProtocolTest.java => AbstractProtocolTest.java} |  77 +-------
 .../java/ads/protocol/Ads2PayloadProtocolTest.java |  99 ++++++++++
 .../java/ads/protocol/Ads2TcpProtocolTest.java     | 203 ---------------------
 .../ads/protocol/Payload2SerialProtocolTest.java   | 105 +++++++++++
 .../java/ads/protocol/Payload2TcpProtocolTest.java | 104 +++++++++++
 ...rotocolTest.java => Plc4x2AdsProtocolTest.java} |   4 +-
 18 files changed, 453 insertions(+), 401 deletions(-)

diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/commands/types/Data.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/commands/types/Data.java
index 12872e5..98c75e3 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/commands/types/Data.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/commands/types/Data.java
@@ -18,6 +18,7 @@
  */
 package org.apache.plc4x.java.ads.api.commands.types;
 
+import io.netty.buffer.ByteBuf;
 import org.apache.plc4x.java.ads.api.util.ByteValue;
 
 import java.nio.charset.Charset;
@@ -30,10 +31,18 @@ public class Data extends ByteValue {
         super(values);
     }
 
+    public Data(ByteBuf byteBuf) {
+        super(byteBuf);
+    }
+
     public static Data of(byte... values) {
         return new Data(values);
     }
 
+    public static Data of(ByteBuf byteBuf) {
+        return new Data(byteBuf);
+    }
+
     public static Data of(String value) {
         requireNonNull(value);
         return new Data(value.getBytes());
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/generic/AmsPacket.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/generic/AmsPacket.java
index 1ff6537..51e10ca 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/generic/AmsPacket.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/generic/AmsPacket.java
@@ -21,9 +21,6 @@ package org.apache.plc4x.java.ads.api.generic;
 import io.netty.buffer.ByteBuf;
 import org.apache.plc4x.java.ads.api.commands.AdsCommandType;
 import org.apache.plc4x.java.ads.api.generic.types.*;
-import org.apache.plc4x.java.ads.api.serial.AmsSerialFrame;
-import org.apache.plc4x.java.ads.api.serial.types.FragmentNumber;
-import org.apache.plc4x.java.ads.api.tcp.AmsTCPPacket;
 import org.apache.plc4x.java.ads.api.util.ByteReadable;
 
 import static java.util.Objects.requireNonNull;
@@ -62,14 +59,6 @@ public abstract class AmsPacket implements ByteReadable {
         return () -> buildByteBuff(byteReadables);
     }
 
-    public AmsTCPPacket toAmstcpPacket() {
-        return AmsTCPPacket.of(this);
-    }
-
-    public AmsSerialFrame toAmsSerialFrame(byte fragmentNumber) {
-        return AmsSerialFrame.of(FragmentNumber.of(fragmentNumber), this);
-    }
-
     @Override
     public ByteBuf getByteBuf() {
         return buildByteBuff(amsHeader, getAdsData());
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/AmsSerialFrame.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/AmsSerialFrame.java
index c90b042..f121f01 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/AmsSerialFrame.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/AmsSerialFrame.java
@@ -19,7 +19,6 @@
 package org.apache.plc4x.java.ads.api.serial;
 
 import io.netty.buffer.ByteBuf;
-import org.apache.plc4x.java.ads.api.generic.AmsPacket;
 import org.apache.plc4x.java.ads.api.serial.types.*;
 import org.apache.plc4x.java.ads.api.util.ByteReadable;
 import org.apache.plc4x.java.ads.protocol.util.DigestUtil;
@@ -71,7 +70,6 @@ public class AmsSerialFrame implements ByteReadable {
     /**
      * The AMS packet to be sent.
      */
-    private AmsPacket amsPacket;
     private final UserData userData;
 
     private final CRC crc;
@@ -86,18 +84,17 @@ public class AmsSerialFrame implements ByteReadable {
         this.crc = crc;
     }
 
-    private AmsSerialFrame(FragmentNumber fragmentNumber, AmsPacket amsPacket) {
+    private AmsSerialFrame(FragmentNumber fragmentNumber, UserData userData) {
         this.magicCookie = MagicCookie.of(ID);
         this.transmitterAddress = TransmitterAddress.RS232_COMM_ADDRESS;
         this.receiverAddress = ReceiverAddress.RS232_COMM_ADDRESS;
         this.fragmentNumber = fragmentNumber;
-        long calculatedLength = amsPacket.getCalculatedLength();
+        long calculatedLength = userData.getCalculatedLength();
         if (calculatedLength > 255) {
             throw new IllegalArgumentException("Paket length must not exceed 255");
         }
         this.userDataLength = UserDataLength.of((byte) calculatedLength);
-        this.amsPacket = amsPacket;
-        byte[] amsPacketBytes = amsPacket.getBytes();
+        byte[] amsPacketBytes = userData.getBytes();
         this.userData = UserData.of(amsPacketBytes);
         this.crc = CRC.of(DigestUtil.calculateCrc16(() -> buildByteBuff(magicCookie, transmitterAddress, receiverAddress, fragmentNumber, userDataLength, userData)));
     }
@@ -106,12 +103,8 @@ public class AmsSerialFrame implements ByteReadable {
         return new AmsSerialFrame(magicCookie, transmitterAddress, receiverAddress, fragmentNumber, userDataLength, userData, crc);
     }
 
-    public static AmsSerialFrame of(FragmentNumber fragmentNumber, AmsPacket amsPacket) {
-        return new AmsSerialFrame(fragmentNumber, amsPacket);
-    }
-
-    public AmsPacket getAmsPacket() {
-        return amsPacket;
+    public static AmsSerialFrame of(FragmentNumber fragmentNumber, UserData userData) {
+        return new AmsSerialFrame(fragmentNumber, userData);
     }
 
     @Override
@@ -131,7 +124,6 @@ public class AmsSerialFrame implements ByteReadable {
         if (!receiverAddress.equals(that.receiverAddress)) return false;
         if (!fragmentNumber.equals(that.fragmentNumber)) return false;
         if (!userDataLength.equals(that.userDataLength)) return false;
-        if (!amsPacket.equals(that.amsPacket)) return false;
         if (!userData.equals(that.userData)) return false;
         return crc.equals(that.crc);
     }
@@ -143,7 +135,6 @@ public class AmsSerialFrame implements ByteReadable {
         result = 31 * result + receiverAddress.hashCode();
         result = 31 * result + fragmentNumber.hashCode();
         result = 31 * result + userDataLength.hashCode();
-        result = 31 * result + amsPacket.hashCode();
         result = 31 * result + userData.hashCode();
         result = 31 * result + crc.hashCode();
         return result;
@@ -157,7 +148,6 @@ public class AmsSerialFrame implements ByteReadable {
             ", receiverAddress=" + receiverAddress +
             ", fragmentNumber=" + fragmentNumber +
             ", userDataLength=" + userDataLength +
-            ", amsPacket=" + amsPacket +
             ", userData=" + userData +
             ", crc=" + crc +
             '}';
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/types/UserData.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/types/UserData.java
index 7b99cf6..5a54490 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/types/UserData.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/types/UserData.java
@@ -18,6 +18,7 @@
  */
 package org.apache.plc4x.java.ads.api.serial.types;
 
+import io.netty.buffer.ByteBuf;
 import org.apache.plc4x.java.ads.api.util.ByteValue;
 
 import java.nio.charset.Charset;
@@ -32,6 +33,10 @@ public class UserData extends ByteValue {
         super(values);
     }
 
+    public UserData(ByteBuf byteBuf) {
+        super(byteBuf);
+    }
+
     public static UserData of(byte... values) {
         return new UserData(values);
     }
@@ -46,6 +51,10 @@ public class UserData extends ByteValue {
         return new UserData(value.getBytes(charset));
     }
 
+    public static UserData of(ByteBuf byteBuf) {
+       return new UserData(byteBuf);
+    }
+
     @Override
     public String toString() {
         // TODO: maybe we could find a way to implement this to string
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/tcp/AmsTCPPacket.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/tcp/AmsTCPPacket.java
index 43512cf..2cfc60b 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/tcp/AmsTCPPacket.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/tcp/AmsTCPPacket.java
@@ -19,52 +19,45 @@
 package org.apache.plc4x.java.ads.api.tcp;
 
 import io.netty.buffer.ByteBuf;
-import org.apache.plc4x.java.ads.api.generic.AmsHeader;
-import org.apache.plc4x.java.ads.api.generic.AmsPacket;
+import org.apache.plc4x.java.ads.api.serial.types.UserData;
 import org.apache.plc4x.java.ads.api.util.ByteReadable;
 
 import static java.util.Objects.requireNonNull;
 
 public class AmsTCPPacket implements ByteReadable {
+    /**
+     * The ams - tcp to be sent.
+     */
     private final AmsTcpHeader amsTcpHeader;
 
-    private final AmsPacket amsPacket;
+    /**
+     * The AMS packet to be sent.
+     */
+    private final UserData userData;
 
-    private AmsTCPPacket(AmsTcpHeader amsTcpHeader, AmsPacket amsPacket) {
+    private AmsTCPPacket(AmsTcpHeader amsTcpHeader, UserData userData) {
         this.amsTcpHeader = requireNonNull(amsTcpHeader);
-        this.amsPacket = requireNonNull(amsPacket);
+        this.userData = requireNonNull(userData);
     }
 
-    private AmsTCPPacket(AmsPacket amsPacket) {
-        this.amsPacket = requireNonNull(amsPacket);
+    private AmsTCPPacket(UserData userData) {
+        this.userData = requireNonNull(userData);
         // It is important that we wrap the ads data call as this will initialized in the constructor
         // so this value will be null if we call adsData now.
-        this.amsTcpHeader = AmsTcpHeader.of(requireNonNull(amsPacket.getAmsHeader()), amsPacket);
+        this.amsTcpHeader = AmsTcpHeader.of(requireNonNull(userData));
     }
 
     public AmsTcpHeader getAmsTcpHeader() {
         return amsTcpHeader;
     }
 
-    public AmsPacket getAmsPacket() {
-        return amsPacket;
-    }
-
-    public AmsHeader getAmsHeader() {
-        return amsPacket.getAmsHeader();
-    }
-
     @Override
     public ByteBuf getByteBuf() {
-        return buildByteBuff(amsTcpHeader, amsPacket);
-    }
-
-    public static AmsTCPPacket of(AmsTcpHeader amsTcpHeader, AmsPacket amsPacket) {
-        return new AmsTCPPacket(amsTcpHeader, amsPacket);
+        return buildByteBuff(amsTcpHeader, userData);
     }
 
-    public static AmsTCPPacket of(AmsPacket amsPacket) {
-        return new AmsTCPPacket(amsPacket);
+    public static AmsTCPPacket of(UserData userData) {
+        return new AmsTCPPacket(userData);
     }
 
     @Override
@@ -75,13 +68,13 @@ public class AmsTCPPacket implements ByteReadable {
         AmsTCPPacket that = (AmsTCPPacket) o;
 
         if (!amsTcpHeader.equals(that.amsTcpHeader)) return false;
-        return amsPacket.equals(that.amsPacket);
+        return userData.equals(that.userData);
     }
 
     @Override
     public int hashCode() {
         int result = amsTcpHeader.hashCode();
-        result = 31 * result + amsPacket.hashCode();
+        result = 31 * result + userData.hashCode();
         return result;
     }
 
@@ -89,7 +82,7 @@ public class AmsTCPPacket implements ByteReadable {
     public String toString() {
         return "AmsTCPPacket{" +
             "amsTcpHeader=" + amsTcpHeader +
-            ", amsPacket=" + amsPacket +
+            ", amsPacket=" + userData +
             '}';
     }
 }
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/types/UserData.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/tcp/types/UserData.java
similarity index 86%
copy from plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/types/UserData.java
copy to plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/tcp/types/UserData.java
index 7b99cf6..a30bdb1 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/serial/types/UserData.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/tcp/types/UserData.java
@@ -16,8 +16,9 @@
  specific language governing permissions and limitations
  under the License.
  */
-package org.apache.plc4x.java.ads.api.serial.types;
+package org.apache.plc4x.java.ads.api.tcp.types;
 
+import io.netty.buffer.ByteBuf;
 import org.apache.plc4x.java.ads.api.util.ByteValue;
 
 import java.nio.charset.Charset;
@@ -32,6 +33,10 @@ public class UserData extends ByteValue {
         super(values);
     }
 
+    public UserData(ByteBuf byteBuf) {
+        super(byteBuf);
+    }
+
     public static UserData of(byte... values) {
         return new UserData(values);
     }
@@ -46,6 +51,10 @@ public class UserData extends ByteValue {
         return new UserData(value.getBytes(charset));
     }
 
+    public static UserData of(ByteBuf byteBuf) {
+        return new UserData(byteBuf);
+    }
+
     @Override
     public String toString() {
         // TODO: maybe we could find a way to implement this to string
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/util/ByteValue.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/util/ByteValue.java
index 6149e36..d4d5b73 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/util/ByteValue.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/api/util/ByteValue.java
@@ -23,15 +23,21 @@ import io.netty.buffer.Unpooled;
 
 import java.math.BigInteger;
 import java.util.Arrays;
-import java.util.Objects;
+
+import static java.util.Objects.requireNonNull;
 
 public class ByteValue implements ByteReadable {
 
     protected final byte[] value;
 
     protected ByteValue(byte... value) {
-        Objects.requireNonNull(value);
-        this.value = value;
+        this.value = requireNonNull(value);
+    }
+
+    protected ByteValue(ByteBuf byteBuf) {
+        requireNonNull(byteBuf);
+        value = new byte[byteBuf.readableBytes()];
+        byteBuf.readBytes(value);
     }
 
     protected void assertLength(int length) {
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/connection/AdsSerialPlcConnection.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/connection/AdsSerialPlcConnection.java
index 1786e1a..8b68e6e 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/connection/AdsSerialPlcConnection.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/connection/AdsSerialPlcConnection.java
@@ -24,7 +24,8 @@ import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelPipeline;
 import org.apache.plc4x.java.ads.api.generic.types.AmsNetId;
 import org.apache.plc4x.java.ads.api.generic.types.AmsPort;
-import org.apache.plc4x.java.ads.protocol.Ads2SerialProtocol;
+import org.apache.plc4x.java.ads.protocol.Ads2PayloadProtocol;
+import org.apache.plc4x.java.ads.protocol.Payload2SerialProtocol;
 import org.apache.plc4x.java.ads.protocol.Plc4x2AdsProtocol;
 import org.apache.plc4x.java.base.connection.SerialChannelFactory;
 
@@ -56,7 +57,8 @@ public class AdsSerialPlcConnection extends AdsAbstractPlcConnection {
                 // Build the protocol stack for communicating with the ads protocol.
                 ChannelPipeline pipeline = channel.pipeline();
                 pipeline.addLast(new Plc4x2AdsProtocol(targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort));
-                pipeline.addLast(new Ads2SerialProtocol());
+                pipeline.addLast(new Ads2PayloadProtocol());
+                pipeline.addLast(new Payload2SerialProtocol());
             }
         };
     }
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/connection/AdsTcpPlcConnection.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/connection/AdsTcpPlcConnection.java
index ad502c3..d25169d 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/connection/AdsTcpPlcConnection.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/connection/AdsTcpPlcConnection.java
@@ -24,7 +24,8 @@ import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelPipeline;
 import org.apache.plc4x.java.ads.api.generic.types.AmsNetId;
 import org.apache.plc4x.java.ads.api.generic.types.AmsPort;
-import org.apache.plc4x.java.ads.protocol.Ads2TcpProtocol;
+import org.apache.plc4x.java.ads.protocol.Ads2PayloadProtocol;
+import org.apache.plc4x.java.ads.protocol.Payload2TcpProtocol;
 import org.apache.plc4x.java.ads.protocol.Plc4x2AdsProtocol;
 import org.apache.plc4x.java.base.connection.TcpSocketChannelFactory;
 
@@ -75,7 +76,8 @@ public class AdsTcpPlcConnection extends AdsAbstractPlcConnection {
                 // Build the protocol stack for communicating with the ads protocol.
                 ChannelPipeline pipeline = channel.pipeline();
                 pipeline.addLast(new Plc4x2AdsProtocol(targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort));
-                pipeline.addLast(new Ads2TcpProtocol());
+                pipeline.addLast(new Ads2PayloadProtocol());
+                pipeline.addLast(new Payload2TcpProtocol());
             }
         };
     }
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Ads2TcpProtocol.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Ads2PayloadProtocol.java
similarity index 94%
rename from plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Ads2TcpProtocol.java
rename to plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Ads2PayloadProtocol.java
index 6e6ecf4..c2a3e94 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Ads2TcpProtocol.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Ads2PayloadProtocol.java
@@ -26,8 +26,6 @@ import org.apache.plc4x.java.ads.api.commands.types.*;
 import org.apache.plc4x.java.ads.api.generic.AmsHeader;
 import org.apache.plc4x.java.ads.api.generic.AmsPacket;
 import org.apache.plc4x.java.ads.api.generic.types.*;
-import org.apache.plc4x.java.ads.api.tcp.AmsTcpHeader;
-import org.apache.plc4x.java.ads.api.tcp.types.TcpLength;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,21 +35,14 @@ import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-public class Ads2TcpProtocol extends MessageToMessageCodec<ByteBuf, AmsPacket> {
+public class Ads2PayloadProtocol extends MessageToMessageCodec<ByteBuf, AmsPacket> {
 
-    private static final Logger LOGGER = LoggerFactory.getLogger(Ads2TcpProtocol.class);
+    private static final Logger LOGGER = LoggerFactory.getLogger(Ads2PayloadProtocol.class);
 
     private final ConcurrentMap<Invoke, AmsPacket> requests;
 
-    private final boolean ignoreBrokenPackages;
-
-    public Ads2TcpProtocol() {
-        this(false);
-    }
-
-    public Ads2TcpProtocol(boolean ignoreBrokenPackages) {
+    public Ads2PayloadProtocol() {
         this.requests = new ConcurrentHashMap<>();
-        this.ignoreBrokenPackages = ignoreBrokenPackages;
     }
 
     /**
@@ -67,20 +58,11 @@ public class Ads2TcpProtocol extends MessageToMessageCodec<ByteBuf, AmsPacket> {
         if (invokeId != Invoke.NONE) {
             requests.put(invokeId, amsPacket);
         }
-        out.add(amsPacket.toAmstcpPacket().getByteBuf());
+        out.add(amsPacket.getByteBuf());
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     protected void decode(ChannelHandlerContext channelHandlerContext, ByteBuf byteBuf, List<Object> out) throws Exception {
-        // Tcp decoding
-        // Reserved
-        byteBuf.skipBytes(AmsTcpHeader.Reserved.NUM_BYTES);
-        TcpLength packetLength = TcpLength.of(byteBuf);
-        AmsTcpHeader amsTcpHeader = AmsTcpHeader.of(packetLength);
-        LOGGER.debug("AMS TCP Header {}", amsTcpHeader);
-
-        // Ams decoding
         AmsNetId targetAmsNetId = AmsNetId.of(byteBuf);
         AmsPort targetAmsPort = AmsPort.of(byteBuf);
         AmsNetId sourceAmsNetId = AmsNetId.of(byteBuf);
@@ -138,7 +120,7 @@ public class Ads2TcpProtocol extends MessageToMessageCodec<ByteBuf, AmsPacket> {
         }
         out.add(amsPacket);
         LOGGER.trace("Set amsPacket {} to out", amsPacket);
-        if (!ignoreBrokenPackages && commandBuffer.readableBytes() > 0) {
+        if (commandBuffer.readableBytes() > 0) {
             commandBuffer.release();
             byteBuf.release();
             throw new IllegalStateException("Unread bytes left: " + commandBuffer.readableBytes());
@@ -147,6 +129,7 @@ public class Ads2TcpProtocol extends MessageToMessageCodec<ByteBuf, AmsPacket> {
         byteBuf.release();
     }
 
+
     private AmsPacket handleInvalidCommand(ByteBuf commandBuffer, AmsHeader amsHeader) {
         return UnknownCommand.of(amsHeader, commandBuffer);
     }
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Ads2SerialProtocol.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocol.java
similarity index 69%
rename from plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Ads2SerialProtocol.java
rename to plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocol.java
index eaeb39d..046a4bc 100644
--- a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Ads2SerialProtocol.java
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocol.java
@@ -19,54 +19,26 @@
 package org.apache.plc4x.java.ads.protocol;
 
 import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.MessageToMessageCodec;
-import org.apache.plc4x.java.ads.api.generic.AmsPacket;
-import org.apache.plc4x.java.ads.api.generic.types.Invoke;
 import org.apache.plc4x.java.ads.api.serial.AmsSerialAcknowledgeFrame;
 import org.apache.plc4x.java.ads.api.serial.AmsSerialFrame;
 import org.apache.plc4x.java.ads.api.serial.AmsSerialResetFrame;
 import org.apache.plc4x.java.ads.api.serial.types.*;
-import org.apache.plc4x.java.ads.api.tcp.AmsTcpHeader;
 import org.apache.plc4x.java.ads.protocol.util.DigestUtil;
 import org.apache.plc4x.java.api.exceptions.PlcProtocolException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 
-public class Ads2SerialProtocol extends MessageToMessageCodec<ByteBuf, AmsPacket> {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(Ads2TcpProtocol.class);
-
-    private final ConcurrentMap<Invoke, AmsPacket> requests;
-
-    private final Ads2TcpProtocol ads2TcpProtocol;
-
-    public Ads2SerialProtocol() {
-        this.requests = new ConcurrentHashMap<>();
-        this.ads2TcpProtocol = new Ads2TcpProtocol(true);
-    }
-
-    /**
-     * Resets this protocol and discard all send requests.
-     */
-    public void reset() {
-        requests.clear();
-    }
+public class Payload2SerialProtocol extends MessageToMessageCodec<ByteBuf, ByteBuf> {
 
+    private static final Logger LOGGER = LoggerFactory.getLogger(Payload2TcpProtocol.class);
 
     @Override
-    protected void encode(ChannelHandlerContext channelHandlerContext, AmsPacket amsPacket, List<Object> out) throws Exception {
-        Invoke invokeId = amsPacket.getAmsHeader().getInvokeId();
-        if (invokeId != Invoke.NONE) {
-            requests.put(invokeId, amsPacket);
-        }
-        byte asLong = (byte) (invokeId.getAsLong() % 255);
-        out.add(amsPacket.toAmsSerialFrame(asLong).getByteBuf());
+    protected void encode(ChannelHandlerContext channelHandlerContext, ByteBuf amsPacket, List<Object> out) throws Exception {
+        out.add(AmsSerialFrame.of(FragmentNumber.of((byte) 0), UserData.of(amsPacket)).getByteBuf());
         // TODO: we need to remember the fragment and maybe even need to spilt up the package
         // TODO: if we exceed 255 byte
     }
@@ -92,15 +64,9 @@ public class Ads2SerialProtocol extends MessageToMessageCodec<ByteBuf, AmsPacket
         switch (magicCookie.getAsInt()) {
             case AmsSerialFrame.ID:
                 // This is a lazy implementation. we just reuse the tcp implementation
-                ByteBuf fakeTcpHeader = AmsTcpHeader.of(0).getByteBuf();
-                ads2TcpProtocol.decode(channelHandlerContext, Unpooled.wrappedBuffer(fakeTcpHeader, userData.getByteBuf()), out);
-                AmsPacket amsPacket = (AmsPacket) out.get(0);
-                AmsPacket correlatedAmsPacket = requests.remove(amsPacket.getAmsHeader().getInvokeId());
-                if (correlatedAmsPacket != null) {
-                    LOGGER.debug("Correlated packet received {}", correlatedAmsPacket);
-                }
-                AmsSerialFrame amsSerialFrame = amsPacket.toAmsSerialFrame(fragmentNumber.getBytes()[0]);
+                AmsSerialFrame amsSerialFrame = AmsSerialFrame.of(magicCookie, transmitterAddress, receiverAddress, fragmentNumber, userDataLength, userData, crc);
                 LOGGER.debug("Ams Serial Frame received {}", amsSerialFrame);
+                out.add(userData.getByteBuf());
                 break;
             case AmsSerialAcknowledgeFrame.ID:
                 AmsSerialAcknowledgeFrame amsSerialAcknowledgeFrame = AmsSerialAcknowledgeFrame.of(magicCookie, transmitterAddress, receiverAddress, fragmentNumber, userDataLength, crc);
diff --git a/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Payload2TcpProtocol.java b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Payload2TcpProtocol.java
new file mode 100644
index 0000000..8b723e8
--- /dev/null
+++ b/plc4j/protocols/ads/src/main/java/org/apache/plc4x/java/ads/protocol/Payload2TcpProtocol.java
@@ -0,0 +1,58 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing,
+ software distributed under the License is distributed on an
+ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ KIND, either express or implied.  See the License for the
+ specific language governing permissions and limitations
+ under the License.
+ */
+package org.apache.plc4x.java.ads.protocol;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.MessageToMessageCodec;
+import org.apache.plc4x.java.ads.api.serial.types.UserData;
+import org.apache.plc4x.java.ads.api.tcp.AmsTCPPacket;
+import org.apache.plc4x.java.ads.api.tcp.AmsTcpHeader;
+import org.apache.plc4x.java.ads.api.tcp.types.TcpLength;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class Payload2TcpProtocol extends MessageToMessageCodec<ByteBuf, ByteBuf> {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(Payload2TcpProtocol.class);
+
+    @Override
+    protected void encode(ChannelHandlerContext channelHandlerContext, ByteBuf amsPacket, List<Object> out) throws Exception {
+        out.add(AmsTCPPacket.of(UserData.of(amsPacket)).getByteBuf());
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected void decode(ChannelHandlerContext channelHandlerContext, ByteBuf byteBuf, List<Object> out) throws Exception {
+        // Reserved
+        byteBuf.skipBytes(AmsTcpHeader.Reserved.NUM_BYTES);
+        TcpLength packetLength = TcpLength.of(byteBuf);
+        AmsTcpHeader amsTcpHeader = AmsTcpHeader.of(packetLength);
+        LOGGER.debug("AMS TCP Header {}", amsTcpHeader);
+
+        int readableBytes = byteBuf.readableBytes();
+        if (readableBytes != packetLength.getAsLong()) {
+            throw new IllegalStateException("To many bytes to read: " + readableBytes + " bytes. Expected " + packetLength.getAsLong() + " bytes.");
+        }
+        out.add(byteBuf.readBytes(readableBytes));
+    }
+
+}
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Ads2SerialProtocolTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/AbstractProtocolTest.java
similarity index 67%
rename from plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Ads2SerialProtocolTest.java
rename to plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/AbstractProtocolTest.java
index 4a62eea..fed68ab 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Ads2SerialProtocolTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/AbstractProtocolTest.java
@@ -18,50 +18,20 @@
  */
 package org.apache.plc4x.java.ads.protocol;
 
-import io.netty.buffer.ByteBuf;
 import org.apache.plc4x.java.ads.api.commands.*;
 import org.apache.plc4x.java.ads.api.commands.types.*;
 import org.apache.plc4x.java.ads.api.generic.AmsPacket;
 import org.apache.plc4x.java.ads.api.generic.types.AmsNetId;
 import org.apache.plc4x.java.ads.api.generic.types.AmsPort;
 import org.apache.plc4x.java.ads.api.generic.types.Invoke;
-import org.apache.plc4x.java.ads.api.serial.AmsSerialFrame;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
-import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import static org.hamcrest.Matchers.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
+public class AbstractProtocolTest {
 
-@RunWith(Parameterized.class)
-public class Ads2SerialProtocolTest {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(Ads2SerialProtocolTest.class);
-
-    private Ads2SerialProtocol SUT;
-
-    @Parameterized.Parameter
-    public AmsSerialFrame amsSerialFrame;
-
-    @Parameterized.Parameter(1)
-    public String clazzName;
-
-    @Parameterized.Parameter(2)
-    public AmsPacket amsPacket;
-
-    @Parameterized.Parameters(name = "{index} {1}")
-    public static Collection<Object[]> data() {
+    public static Stream<AmsPacket> amsPacketStream() {
         AmsNetId targetAmsNetId = AmsNetId.of("1.2.3.4.5.6");
         AmsPort targetAmsPort = AmsPort.of(7);
         AmsNetId sourceAmsNetId = AmsNetId.of("8.9.10.11.12.13");
@@ -162,45 +132,6 @@ public class Ads2SerialProtocolTest {
                 targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, State.DEFAULT, invokeId,
                 Unpooled.wrappedBuffer(new byte[]{42})
             )*/
-        ).map(amsPacket -> new Object[]{amsPacket.toAmsSerialFrame((byte) 0), amsPacket.getClass().getSimpleName(), amsPacket}).collect(Collectors.toList());
-    }
-
-    @Before
-    public void setUp() throws Exception {
-        SUT = new Ads2SerialProtocol();
-        byte[] bytes = amsSerialFrame.getBytes();
-        LOGGER.info("amsPacket:\n{} has \n{}bytes\nHexDump:\n{}", amsSerialFrame, bytes.length, amsSerialFrame.dump());
-    }
-
-    @Test
-    public void encode() throws Exception {
-        ArrayList<Object> out = new ArrayList<>();
-        SUT.encode(null, amsSerialFrame.getAmsPacket(), out);
-        assertEquals(1, out.size());
-        assertThat(out, hasSize(1));
-    }
-
-    @Test
-    public void decode() throws Exception {
-        ArrayList<Object> out = new ArrayList<>();
-        SUT.decode(null, amsSerialFrame.getByteBuf(), out);
-        assertThat(out, hasSize(1));
-    }
-
-    @Test
-    public void roundTrip() throws Exception {
-        ArrayList<Object> outbound = new ArrayList<>();
-        SUT.encode(null, amsSerialFrame.getAmsPacket(), outbound);
-        assertEquals(1, outbound.size());
-        assertThat(outbound, hasSize(1));
-        assertThat(outbound.get(0), instanceOf(ByteBuf.class));
-        ByteBuf byteBuf = (ByteBuf) outbound.get(0);
-        ArrayList<Object> inbound = new ArrayList<>();
-        SUT.decode(null, byteBuf, inbound);
-        assertEquals(1, inbound.size());
-        assertThat(inbound, hasSize(1));
-        assertThat(inbound.get(0), instanceOf(AmsPacket.class));
-        AmsPacket inboundAmsPacket = (AmsPacket) inbound.get(0);
-        assertThat("inbound divers from outbound", this.amsSerialFrame, equalTo(inboundAmsPacket.toAmsSerialFrame((byte) 0)));
+        );
     }
-}
\ No newline at end of file
+}
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Ads2PayloadProtocolTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Ads2PayloadProtocolTest.java
new file mode 100644
index 0000000..87de9ea
--- /dev/null
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Ads2PayloadProtocolTest.java
@@ -0,0 +1,99 @@
+/*
+ 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.java.ads.protocol;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.plc4x.java.ads.api.generic.AmsPacket;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.Matchers.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+@RunWith(Parameterized.class)
+public class Ads2PayloadProtocolTest extends AbstractProtocolTest {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(Ads2PayloadProtocolTest.class);
+
+    private Ads2PayloadProtocol SUT;
+
+    @Parameterized.Parameter
+    public AmsPacket amsPacket;
+
+    @Parameterized.Parameter(1)
+    public String clazzName;
+
+    @Parameterized.Parameters(name = "{index} {1}")
+    public static Collection<Object[]> data() {
+        return amsPacketStream()
+            .map(amsPacket -> new Object[]{
+                amsPacket,
+                amsPacket.getClass().getSimpleName()
+            })
+            .collect(Collectors.toList());
+    }
+
+    @Before
+    public void setUp() throws Exception {
+        SUT = new Ads2PayloadProtocol();
+        byte[] bytes = amsPacket.getBytes();
+        LOGGER.info("amsPacket:\n{} has \n{}bytes\nHexDump:\n{}", amsPacket, bytes.length, amsPacket.dump());
+    }
+
+    @Test
+    public void encode() throws Exception {
+        ArrayList<Object> out = new ArrayList<>();
+        SUT.encode(null, amsPacket, out);
+        assertEquals(1, out.size());
+        assertThat(out, hasSize(1));
+    }
+
+    @Test
+    public void decode() throws Exception {
+        ArrayList<Object> out = new ArrayList<>();
+        SUT.decode(null, amsPacket.getByteBuf(), out);
+        assertThat(out, hasSize(1));
+    }
+
+    @Test
+    public void roundTrip() throws Exception {
+        ArrayList<Object> outbound = new ArrayList<>();
+        SUT.encode(null, amsPacket, outbound);
+        assertEquals(1, outbound.size());
+        assertThat(outbound, hasSize(1));
+        assertThat(outbound.get(0), instanceOf(ByteBuf.class));
+        ByteBuf byteBuf = (ByteBuf) outbound.get(0);
+        ArrayList<Object> inbound = new ArrayList<>();
+        SUT.decode(null, byteBuf, inbound);
+        assertEquals(1, inbound.size());
+        assertThat(inbound, hasSize(1));
+        assertThat(inbound.get(0), instanceOf(AmsPacket.class));
+        AmsPacket inboundAmsPacket = (AmsPacket) inbound.get(0);
+        assertThat("inbound divers from outbound", this.amsPacket, equalTo(inboundAmsPacket));
+    }
+}
\ No newline at end of file
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Ads2TcpProtocolTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Ads2TcpProtocolTest.java
deleted file mode 100644
index 7a23aa4..0000000
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Ads2TcpProtocolTest.java
+++ /dev/null
@@ -1,203 +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.plc4x.java.ads.protocol;
-
-import io.netty.buffer.ByteBuf;
-import org.apache.plc4x.java.ads.api.commands.*;
-import org.apache.plc4x.java.ads.api.commands.types.*;
-import org.apache.plc4x.java.ads.api.generic.AmsPacket;
-import org.apache.plc4x.java.ads.api.generic.types.AmsNetId;
-import org.apache.plc4x.java.ads.api.generic.types.AmsPort;
-import org.apache.plc4x.java.ads.api.generic.types.Invoke;
-import org.apache.plc4x.java.ads.api.tcp.AmsTCPPacket;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.hamcrest.Matchers.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-@RunWith(Parameterized.class)
-public class Ads2TcpProtocolTest {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(Ads2TcpProtocolTest.class);
-
-    private Ads2TcpProtocol SUT;
-
-    @Parameterized.Parameter
-    public AmsTCPPacket amsTCPPacket;
-
-    @Parameterized.Parameter(1)
-    public String clazzName;
-
-    @Parameterized.Parameters(name = "{index} {1}")
-    public static Collection<Object[]> data() {
-        AmsNetId targetAmsNetId = AmsNetId.of("1.2.3.4.5.6");
-        AmsPort targetAmsPort = AmsPort.of(7);
-        AmsNetId sourceAmsNetId = AmsNetId.of("8.9.10.11.12.13");
-        AmsPort sourceAmsPort = AmsPort.of(14);
-        Invoke invokeId = Invoke.of(15);
-        Data data = Data.of("Hello World!".getBytes());
-        return Stream.of(
-            AdsAddDeviceNotificationRequest.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                IndexGroup.of(1), IndexOffset.of(1), Length.of(1), TransmissionMode.of(1), MaxDelay.of(1), CycleTime.of(1)).toAmstcpPacket(),
-            AdsAddDeviceNotificationResponse.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                Result.of(0),
-                NotificationHandle.of(0)
-            ).toAmstcpPacket(),
-            AdsDeleteDeviceNotificationRequest.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                NotificationHandle.of(0)
-            ).toAmstcpPacket(),
-            AdsDeleteDeviceNotificationResponse.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                Result.of(0)
-            ).toAmstcpPacket(),
-            AdsDeviceNotificationRequest.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                Stamps.of(1),
-                Collections.singletonList(
-                    // Nano times need to be offset by (1.1.1970 - 1.1.1601) years in nanos
-                    AdsStampHeader.of(TimeStamp.of(new Date()),
-                        Collections.singletonList(
-                            AdsNotificationSample.of(NotificationHandle.of(0), data))
-                    )
-                )
-            ).toAmstcpPacket(),
-            AdsReadDeviceInfoRequest.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId
-            ).toAmstcpPacket(),
-            AdsReadDeviceInfoResponse.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                Result.of(0),
-                MajorVersion.of((byte) 1),
-                MinorVersion.of((byte) 2),
-                Version.of(3),
-                Device.of("Random DeviceId")
-            ).toAmstcpPacket(),
-            AdsReadRequest.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                IndexGroup.of(0),
-                IndexOffset.of(0),
-                Length.of(1)
-            ).toAmstcpPacket(),
-            AdsReadResponse.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                Result.of(0),
-                data
-            ).toAmstcpPacket(),
-            AdsReadStateRequest.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId
-            ).toAmstcpPacket(),
-            AdsReadStateResponse.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                Result.of(0)
-            ).toAmstcpPacket(),
-            AdsReadWriteRequest.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                IndexGroup.of(0),
-                IndexOffset.of(0),
-                ReadLength.of(data.getCalculatedLength()),
-                data
-            ).toAmstcpPacket(),
-            AdsReadWriteResponse.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                Result.of(0),
-                data
-            ).toAmstcpPacket(),
-            AdsWriteControlRequest.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                AdsState.of(0xaffe),
-                DeviceState.of(0xaffe),
-                data
-            ).toAmstcpPacket(),
-            AdsWriteControlResponse.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                Result.of(0)
-            ).toAmstcpPacket(),
-            AdsWriteRequest.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                IndexGroup.of(0),
-                IndexOffset.of(0),
-                data
-            ).toAmstcpPacket(),
-            AdsWriteResponse.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId,
-                Result.of(0)
-            ).toAmstcpPacket()
-            /*,
-            UnknownCommand.of(
-                targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, State.DEFAULT, invokeId,
-                Unpooled.wrappedBuffer(new byte[]{42})
-            )*/
-        ).map(amsTCPPacket -> new Object[]{amsTCPPacket, amsTCPPacket.getClass().getSimpleName()}).collect(Collectors.toList());
-    }
-
-    @Before
-    public void setUp() throws Exception {
-        SUT = new Ads2TcpProtocol();
-        byte[] bytes = amsTCPPacket.getBytes();
-        LOGGER.info("amsPacket:\n{} has \n{}bytes\nHexDump:\n{}", amsTCPPacket, bytes.length, amsTCPPacket.dump());
-    }
-
-    @Test
-    public void encode() throws Exception {
-        ArrayList<Object> out = new ArrayList<>();
-        SUT.encode(null, amsTCPPacket.getAmsPacket(), out);
-        assertEquals(1, out.size());
-        assertThat(out, hasSize(1));
-    }
-
-    @Test
-    public void decode() throws Exception {
-        ArrayList<Object> out = new ArrayList<>();
-        SUT.decode(null, amsTCPPacket.getByteBuf(), out);
-        assertThat(out, hasSize(1));
-    }
-
-    @Test
-    public void roundTrip() throws Exception {
-        ArrayList<Object> outbound = new ArrayList<>();
-        SUT.encode(null, amsTCPPacket.getAmsPacket(), outbound);
-        assertEquals(1, outbound.size());
-        assertThat(outbound, hasSize(1));
-        assertThat(outbound.get(0), instanceOf(ByteBuf.class));
-        ByteBuf byteBuf = (ByteBuf) outbound.get(0);
-        ArrayList<Object> inbound = new ArrayList<>();
-        SUT.decode(null, byteBuf, inbound);
-        assertEquals(1, inbound.size());
-        assertThat(inbound, hasSize(1));
-        assertThat(inbound.get(0), instanceOf(AmsPacket.class));
-        AmsPacket inboundAmsPacket = (AmsPacket) inbound.get(0);
-        assertThat("inbound divers from outbound", this.amsTCPPacket, equalTo(inboundAmsPacket.toAmstcpPacket()));
-    }
-}
\ No newline at end of file
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocolTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocolTest.java
new file mode 100644
index 0000000..fe1a0fc
--- /dev/null
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocolTest.java
@@ -0,0 +1,105 @@
+/*
+ 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.java.ads.protocol;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.plc4x.java.ads.api.serial.AmsSerialFrame;
+import org.apache.plc4x.java.ads.api.serial.types.FragmentNumber;
+import org.apache.plc4x.java.ads.api.serial.types.UserData;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.Matchers.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+@RunWith(Parameterized.class)
+public class Payload2SerialProtocolTest extends AbstractProtocolTest {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(Payload2SerialProtocolTest.class);
+
+    private Payload2SerialProtocol SUT;
+
+    @Parameterized.Parameter
+    public AmsSerialFrame amsSerialFrame;
+
+    @Parameterized.Parameter(1)
+    public String clazzName;
+
+    @Parameterized.Parameter(2)
+    public ByteBuf amsPacketByteBuf;
+
+    @Parameterized.Parameters(name = "{index} {1}")
+    public static Collection<Object[]> data() {
+        return amsPacketStream()
+            .map(amsPacket -> new Object[]{
+                AmsSerialFrame.of(FragmentNumber.of((byte) 0), UserData.of(amsPacket.getByteBuf())),
+                amsPacket.getClass().getSimpleName(),
+                amsPacket.getByteBuf()
+            })
+            .collect(Collectors.toList());
+    }
+
+    @Before
+    public void setUp() throws Exception {
+        SUT = new Payload2SerialProtocol();
+        byte[] bytes = amsSerialFrame.getBytes();
+        LOGGER.info("amsPacket:\n{} has \n{}bytes\nHexDump:\n{}", amsSerialFrame, bytes.length, amsSerialFrame.dump());
+    }
+
+    @Test
+    public void encode() throws Exception {
+        ArrayList<Object> out = new ArrayList<>();
+        SUT.encode(null, amsPacketByteBuf, out);
+        assertEquals(1, out.size());
+        assertThat(out, hasSize(1));
+    }
+
+    @Test
+    public void decode() throws Exception {
+        ArrayList<Object> out = new ArrayList<>();
+        SUT.decode(null, amsSerialFrame.getByteBuf(), out);
+        assertThat(out, hasSize(1));
+    }
+
+    @Test
+    public void roundTrip() throws Exception {
+        ArrayList<Object> outbound = new ArrayList<>();
+        SUT.encode(null, amsPacketByteBuf, outbound);
+        assertEquals(1, outbound.size());
+        assertThat(outbound, hasSize(1));
+        assertThat(outbound.get(0), instanceOf(ByteBuf.class));
+        ByteBuf byteBuf = (ByteBuf) outbound.get(0);
+        ArrayList<Object> inbound = new ArrayList<>();
+        SUT.decode(null, byteBuf, inbound);
+        assertEquals(1, inbound.size());
+        assertThat(inbound, hasSize(1));
+        assertThat(inbound.get(0), instanceOf(ByteBuf.class));
+        ByteBuf inboundAmsPacketByteBuf = (ByteBuf) inbound.get(0);
+        assertThat("inbound divers from outbound", this.amsPacketByteBuf, equalTo(inboundAmsPacketByteBuf));
+    }
+}
\ No newline at end of file
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2TcpProtocolTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2TcpProtocolTest.java
new file mode 100644
index 0000000..32c8888
--- /dev/null
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2TcpProtocolTest.java
@@ -0,0 +1,104 @@
+/*
+ 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.java.ads.protocol;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.plc4x.java.ads.api.serial.types.UserData;
+import org.apache.plc4x.java.ads.api.tcp.AmsTCPPacket;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.Matchers.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+@RunWith(Parameterized.class)
+public class Payload2TcpProtocolTest extends AbstractProtocolTest {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(Payload2TcpProtocolTest.class);
+
+    private Payload2TcpProtocol SUT;
+
+    @Parameterized.Parameter
+    public AmsTCPPacket amsTCPPacket;
+
+    @Parameterized.Parameter(1)
+    public String clazzName;
+
+    @Parameterized.Parameter(2)
+    public ByteBuf amsPacketByteBuf;
+
+    @Parameterized.Parameters(name = "{index} {1}")
+    public static Collection<Object[]> data() {
+        return amsPacketStream()
+            .map(amsPacket -> new Object[]{
+                AmsTCPPacket.of(UserData.of(amsPacket.getByteBuf())),
+                amsPacket.getClass().getSimpleName(),
+                amsPacket.getByteBuf()
+            })
+            .collect(Collectors.toList());
+    }
+
+    @Before
+    public void setUp() throws Exception {
+        SUT = new Payload2TcpProtocol();
+        byte[] bytes = amsTCPPacket.getBytes();
+        LOGGER.info("amsPacket:\n{} has \n{}bytes\nHexDump:\n{}", amsTCPPacket, bytes.length, amsTCPPacket.dump());
+    }
+
+    @Test
+    public void encode() throws Exception {
+        ArrayList<Object> out = new ArrayList<>();
+        SUT.encode(null, amsPacketByteBuf, out);
+        assertEquals(1, out.size());
+        assertThat(out, hasSize(1));
+    }
+
+    @Test
+    public void decode() throws Exception {
+        ArrayList<Object> out = new ArrayList<>();
+        SUT.decode(null, amsTCPPacket.getByteBuf(), out);
+        assertThat(out, hasSize(1));
+    }
+
+    @Test
+    public void roundTrip() throws Exception {
+        ArrayList<Object> outbound = new ArrayList<>();
+        SUT.encode(null, amsPacketByteBuf, outbound);
+        assertEquals(1, outbound.size());
+        assertThat(outbound, hasSize(1));
+        assertThat(outbound.get(0), instanceOf(ByteBuf.class));
+        ByteBuf byteBuf = (ByteBuf) outbound.get(0);
+        ArrayList<Object> inbound = new ArrayList<>();
+        SUT.decode(null, byteBuf, inbound);
+        assertEquals(1, inbound.size());
+        assertThat(inbound, hasSize(1));
+        assertThat(inbound.get(0), instanceOf(ByteBuf.class));
+        ByteBuf inboundAmsPacket = (ByteBuf) inbound.get(0);
+        assertThat("inbound divers from outbound", this.amsPacketByteBuf, equalTo(inboundAmsPacket));
+    }
+}
\ No newline at end of file
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4X2Ads2TcpProtocolTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java
similarity index 99%
rename from plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4X2Ads2TcpProtocolTest.java
rename to plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java
index 0de9738..4af7a4a 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4X2Ads2TcpProtocolTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java
@@ -54,8 +54,8 @@ import static org.hamcrest.Matchers.*;
 import static org.junit.Assert.assertThat;
 
 @RunWith(Parameterized.class)
-public class Plc4X2Ads2TcpProtocolTest {
-    private static final Logger LOGGER = LoggerFactory.getLogger(Ads2TcpProtocolTest.class);
+public class Plc4x2AdsProtocolTest {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Plc4x2AdsProtocolTest.class);
 
     public static final Calendar calenderInstance = Calendar.getInstance();
 

-- 
To stop receiving notification emails like this one, please contact
sruehl@apache.org.