You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by hu...@apache.org on 2022/05/29 05:56:41 UTC

[plc4x] branch logix_develop updated: major change, able to read values

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

hutcheb pushed a commit to branch logix_develop
in repository https://gitbox.apache.org/repos/asf/plc4x.git


The following commit(s) were added to refs/heads/logix_develop by this push:
     new a9e5e7685a major change, able to read values
a9e5e7685a is described below

commit a9e5e7685ae8a60fff64cfde12eac28f14c05d2f
Author: Ben Hutcheson <be...@gmail.com>
AuthorDate: Sun May 29 15:54:44 2022 +1000

    major change, able to read values
---
 .../plc4x/java/logix/readwrite/LogixDriver.java    |   6 +-
 .../java/logix/readwrite/field/LogixField.java     |   4 +-
 .../readwrite/protocol/LogixProtocolLogic.java     | 415 +++++++++++----------
 .../org/apache/plc4x/java/logix/LogixDriverIT.java |  90 +++++
 .../LogixParserSerializerTest.java                 |   2 +-
 pom.xml                                            |   4 +
 .../src/main/resources/protocols/logix/logix.mspec | 107 +++++-
 .../protocols/logix/ParserSerializerTestsuite.xml  |  11 +-
 8 files changed, 416 insertions(+), 223 deletions(-)

diff --git a/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/LogixDriver.java b/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/LogixDriver.java
index 56fc13f014..1c931c3512 100644
--- a/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/LogixDriver.java
+++ b/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/LogixDriver.java
@@ -61,13 +61,9 @@ public class LogixDriver extends GeneratedDriverBase<EipPacket> {
         return new IEC61131ValueHandler();
     }
 
-    /**
-     * This protocol doesn't have a disconnect procedure, so there is no need to wait for a login to finish.
-     * @return false
-     */
     @Override
     protected boolean awaitDisconnectComplete() {
-        return false;
+        return true;
     }
 
     @Override
diff --git a/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/field/LogixField.java b/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/field/LogixField.java
index 39d9d194b5..0ab13c5841 100644
--- a/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/field/LogixField.java
+++ b/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/field/LogixField.java
@@ -31,7 +31,7 @@ import java.util.regex.Pattern;
 public class LogixField implements PlcField, Serializable {
 
     private static final Pattern ADDRESS_PATTERN =
-        Pattern.compile("^%(?<tag>[a-zA-Z_.0-9]+\\[?[0-9]*\\]?):?(?<dataType>[A-Z]*):?(?<elementNb>[0-9]*)");
+        Pattern.compile("^(?<tag>[a-zA-Z_.0-9]+\\[?[0-9]*\\]?):?(?<dataType>[A-Z]*):?(?<elementNb>[0-9]*)");
 
     private static final String TAG = "tag";
     private static final String ELEMENTS = "elementNb";
@@ -97,6 +97,8 @@ public class LogixField implements PlcField, Serializable {
             }
             if (!matcher.group(TYPE).isEmpty()) {
                 type = CIPDataTypeCode.valueOf(matcher.group(TYPE));
+            } else {
+                type = CIPDataTypeCode.DINT;
             }
             if (nb != 0) {
                 if (type != null) {
diff --git a/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/protocol/LogixProtocolLogic.java b/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/protocol/LogixProtocolLogic.java
index 1ba27b8de2..8c088b8913 100644
--- a/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/protocol/LogixProtocolLogic.java
+++ b/plc4j/drivers/logix/src/main/java/org/apache/plc4x/java/logix/readwrite/protocol/LogixProtocolLogic.java
@@ -42,19 +42,27 @@ import org.slf4j.LoggerFactory;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.nio.charset.StandardCharsets;
+import java.security.Provider;
 import java.time.Duration;
 import java.util.*;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements HasConfiguration<LogixConfiguration> {
 
     private static final Logger logger = LoggerFactory.getLogger(LogixProtocolLogic.class);
     public static final Duration REQUEST_TIMEOUT = Duration.ofMillis(10000);
 
-    private static final List<Short> emptySenderContext = Arrays.asList((short) 0x00, (short) 0x00, (short) 0x00,
-        (short) 0x00, (short) 0x00, (short) 0x00, (short) 0x00, (short) 0x00);
-    private List<Short> senderContext;
+    private static final byte[] DEFAULT_SENDER_CONTEXT = "PLC4X   ".getBytes(StandardCharsets.US_ASCII);
+    private static final long EMPTY_SESSION_HANDLE = 0L;
+    private static final long EMPTY_INTERFACE_HANDLE = 0L;
+
+    private byte[] senderContext;
+
+    private long connectionId = 0L;
+    private int sequenceCount = 1;
     private LogixConfiguration configuration;
 
     private final AtomicInteger transactionCounterGenerator = new AtomicInteger(10);
@@ -70,24 +78,26 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
 
     @Override
     public void onConnect(ConversationContext<EipPacket> context) {
-        logger.debug("Sending RegisterSession EIP Package");
+        logger.debug("Sending Register Session EIP Package");
         EipConnectionRequest connectionRequest =
-            new EipConnectionRequest(0L, 0L, emptySenderContext, 0L);
+            new EipConnectionRequest(
+                EMPTY_SESSION_HANDLE,
+                CIPStatus.Success.getValue(),
+                DEFAULT_SENDER_CONTEXT,
+                0L);
         context.sendRequest(connectionRequest)
             .expectResponse(EipPacket.class, REQUEST_TIMEOUT).unwrap(p -> p)
             .check(p -> p instanceof EipConnectionRequest)
             .handle(p -> {
-                if (p.getStatus() == 0L) {
+                if (p.getStatus() == CIPStatus.Success.getValue()) {
                     sessionHandle = p.getSessionHandle();
                     senderContext = p.getSenderContext();
                     logger.debug("Got assigned with Session {}", sessionHandle);
-                    // Send an event that connection setup is complete.
                     onConnectOpenConnectionManager(context, p);
 
                 } else {
                     logger.warn("Got status code [{}]", p.getStatus());
                 }
-
             });
     }
 
@@ -99,21 +109,20 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
 
         CipExchange exchange = new CipExchange(
             new CipConnectionManagerRequest(
-                (byte) 2,
                 classSegment,
                 instanceSegment,
                 (byte) 0,
                 (byte) 10,
                 (short) 14,
-                (long) 536870914,
-                (long) 33944,
-                (int) 8592,
-                (int) 4919,
-                (long) 42,
+                536870914L,
+                33944L,
+                8592,
+                4919,
+                42L,
                 (short) 3,
-                (long) 2101812,
+                2101812L,
                 new NetworkConnectionParameters(
-                    (int) 4002,
+                    4002,
                     false,
                     (byte) 2,
                     (byte) 0,
@@ -121,7 +130,7 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
                 ),
                 (long) 2113537,
                 new NetworkConnectionParameters(
-                    (int) 4002,
+                    4002,
                     false,
                     (byte) 2,
                     (byte) 0,
@@ -142,6 +151,7 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
             response.getStatus(),
             senderContext,
             0L,
+            EMPTY_INTERFACE_HANDLE,
             0,
             2,
             exchange,
@@ -154,8 +164,13 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
             .check(p -> p instanceof CipRRData)
             .handle(p -> {
                 if (p.getStatus() == 0L) {
-                    sessionHandle = p.getSessionHandle();
-                    senderContext = p.getSenderContext();
+                    CipRRData rrData = (CipRRData) p;
+                    sessionHandle = rrData.getSessionHandle();
+                    senderContext = rrData.getSenderContext();
+                    CipExchange connectionManagerExchange = rrData.getExchange();
+                    CipConnectionManagerResponse connectionManagerResponse = (CipConnectionManagerResponse) connectionManagerExchange.getService();
+                    this.connectionId = connectionManagerResponse.getOtConnectionId();
+
                     logger.debug("Got assigned with Session {}", sessionHandle);
                     // Send an event that connection setup is complete.
                     context.fireConnected();
@@ -168,11 +183,80 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
     }
 
 
+    @Override
+    public void onDisconnect(ConversationContext<EipPacket> context) {
+        if (this.connectionId != 0L) {
+            logger.debug("Sending Connection Manager Close Event");
+            PathSegment classSegment = new LogicalSegment(new ClassID((byte) 0, (short) 6));
+            PathSegment instanceSegment = new LogicalSegment(new InstanceID((byte) 0, (short) 1));
+
+            CipExchange exchange = new CipExchange(
+                new CipConnectionManagerCloseRequest(
+                    (byte) 2,
+                    classSegment,
+                    instanceSegment,
+                    (byte) 0,
+                    (byte) 10,
+                    (short) 14,
+                    8592,
+                    4919,
+                    42L,
+                    (short) 3,
+                    new PortSegment(false, (byte) 1, (short) 0),
+                    new LogicalSegment(new ClassID((byte) 0, (short) 2)),
+                    new LogicalSegment(new InstanceID((byte) 0, (short) 1)),
+                    0
+                ),
+                -1
+            );
+
+            CipRRData eipWrapper = new CipRRData(
+                sessionHandle,
+                0L,
+                senderContext,
+                0L,
+                EMPTY_INTERFACE_HANDLE,
+                0,
+                2,
+                exchange,
+                -1
+            );
+
+
+            context.sendRequest(eipWrapper)
+                .expectResponse(EipPacket.class, REQUEST_TIMEOUT).unwrap(p -> p)
+                .check(p -> p instanceof CipRRData)
+                .handle(p -> {
+                    logger.debug("Un-Registering Session");
+                    onDisconnectUnregisterSession(context);
+                });
+        } else {
+            onDisconnectUnregisterSession(context);
+        }
+    }
+
+
+    public void onDisconnectUnregisterSession(ConversationContext<EipPacket> context) {
+        logger.debug("Sending Un RegisterSession EIP Package");
+        EipDisconnect connectionRequest =
+            new EipDisconnect(
+                0L,
+                sessionHandle,
+                DEFAULT_SENDER_CONTEXT,
+                0L);
+        context.sendRequest(connectionRequest)
+            .expectResponse(EipPacket.class, Duration.ofMillis(1))
+            .onTimeout(p -> context.fireDisconnected())
+            .handle(p -> context.fireDisconnected());
+    }
 
     @Override
     public CompletableFuture<PlcReadResponse> read(PlcReadRequest readRequest) {
+        CompletableFuture<PlcReadResponse> future = new CompletableFuture<>();
+        RequestTransactionManager.RequestTransaction transaction = tm.startRequest();
+
         DefaultPlcReadRequest request = (DefaultPlcReadRequest) readRequest;
-        List<CipReadRequest> requests = new ArrayList<>(request.getNumberOfFields());
+        List<CipService> requests = new ArrayList<>(request.getNumberOfFields());
         for (PlcField field : request.getFields()) {
             LogixField plcField = (LogixField) field;
             String tag = plcField.getTag();
@@ -180,13 +264,67 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
             if (plcField.getElementNb() > 1) {
                 elements = plcField.getElementNb();
             }
-            CipReadRequest req = new CipReadRequest(getRequestSize(tag), toAnsi(tag), elements, -1);
-            requests.add(req);
+            try {
+                CipReadRequest req = new CipReadRequest(
+                    toAnsi(tag),
+                    1,
+                    new byte[6],
+                    -1);
+                requests.add(req);
+            } catch (SerializationException e) {
+                e.printStackTrace();
+            }
+        }
+
+        List<TypeId> typeIds =new ArrayList<>(2);
+        typeIds.add(new ConnectedAddressItem(this.connectionId));
+        if (requests.size() == 1) {
+            typeIds.add(new ConnectedDataItem(this.sequenceCount, requests.get(0)));
+        } else {
+            List<Integer> offsets = new ArrayList<>(requests.size());
+            offsets.add(6);
+            for (CipService cipRequest : requests) {
+                if (requests.indexOf(cipRequest) != (requests.size() - 1)) {
+                    offsets.add(offsets.get(requests.indexOf(cipRequest)) + cipRequest.getLengthInBytes());
+                }
+
+            }
+            MultipleServiceRequest serviceRequest = new MultipleServiceRequest(new Services(requests.size(), offsets, requests, 0), 0);
+            typeIds.add(new ConnectedDataItem(this.sequenceCount, serviceRequest));
         }
-        return toPlcReadResponse(readRequest, readInternal(requests));
+
+
+        SendUnitData pkt = new SendUnitData(
+            sessionHandle,
+            0L,
+            DEFAULT_SENDER_CONTEXT,
+            0L,
+            0,
+            2,
+            typeIds
+        );
+
+        transaction.submit(() -> context.sendRequest(pkt)
+            .expectResponse(EipPacket.class, REQUEST_TIMEOUT)
+            .onTimeout(future::completeExceptionally)
+            .onError((p, e) -> future.completeExceptionally(e))
+            .check(p -> p instanceof SendUnitData)
+            .check(p -> p.getSessionHandle() == sessionHandle)
+            //.check(p -> p.getSenderContext() == senderContext)
+            .unwrap(p -> (SendUnitData) p)
+            .handle(p -> {
+                SendUnitData unitData = (SendUnitData) p;
+                List<TypeId> responseTypeIds = unitData.getTypeId();
+                ConnectedDataItem dataItem = (ConnectedDataItem) responseTypeIds.get(1);
+                future.complete(decodeReadResponse(dataItem.getService(), request));
+                // Finish the request-transaction.
+                transaction.endRequest();
+            }));
+
+        return future;
     }
 
-    private byte getRequestSize(String tag) {
+    private short getRequestSize(String tag) {
         //We need the size of the request in words (0x91, tagLength, ... tag + possible pad)
         // Taking half to get word size
         boolean isArray = false;
@@ -205,155 +343,51 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
             + (tagIsolated.length() % 2)
             + (isArray ? 2 : 0)
             + (isStruct ? 2 : 0);
-        byte requestPathSize = (byte) (dataLength / 2);
-        return requestPathSize;
+        return (short) (dataLength / 2);
     }
 
-    private byte[] toAnsi(String tag) {
-        int arrayIndex = 0;
-        boolean isArray = false;
-        boolean isStruct = false;
-        String tagFinal = tag;
-        if (tag.contains("[")) {
-            isArray = true;
-            String index = tag.substring(tag.indexOf("[") + 1, tag.indexOf("]"));
-            arrayIndex = Integer.parseInt(index);
-            tagFinal = tag.substring(0, tag.indexOf("["));
-        }
-        if (tag.contains(".")) {
-            tagFinal = tag.substring(0, tag.indexOf("."));
-            isStruct = true;
-        }
-        boolean isPadded = tagFinal.length() % 2 != 0;
-        int dataSegLength = 2 + tagFinal.length()
-            + (isPadded ? 1 : 0)
-            + (isArray ? 2 : 0);
-
-        if (isStruct) {
-            for (String subStr : tag.substring(tag.indexOf(".") + 1).split("\\.", -1)) {
-                dataSegLength += 2 + subStr.length() + subStr.length() % 2;
+    /*
+        Takes a Tag name e.g. ZZZ_ZZZ.XXX and returns a buffer containing an array of ANSI Extended Symbol Seqments
+     */
+    public static byte[] toAnsi(String tag) throws SerializationException {
+        final Pattern RESOURCE_ADDRESS_PATTERN = Pattern.compile("([.\\[\\]])*([A-Za-z_0-9]+){1}");
+        Matcher matcher = RESOURCE_ADDRESS_PATTERN.matcher(tag);
+        List<PathSegment> segments = new LinkedList<>();
+        String tagWithoutQualifiers = "";
+        int lengthBytes = 0;
+        while (matcher.find()) {
+            String identifier = matcher.group(2);
+            String qualifier = matcher.group(1);
+
+            PathSegment newSegment;
+            if (qualifier != null) {
+                switch (qualifier) {
+                    case "[":
+                        newSegment = new LogicalSegment(new MemberID((byte) 0x00, (short) Short.parseShort(identifier)));
+                        segments.add(newSegment);
+                        break;
+                    default:
+                        newSegment = new DataSegment(new AnsiExtendedSymbolSegment(identifier, (short) 0));
+                        segments.add(newSegment);
+                        tagWithoutQualifiers += identifier;
+                }
+            } else {
+                newSegment = new DataSegment(new AnsiExtendedSymbolSegment(identifier, (short) 0));
+                segments.add(newSegment);
+                tagWithoutQualifiers += identifier;
             }
-        }
-
-        ByteBuffer buffer = ByteBuffer.allocate(dataSegLength).order(ByteOrder.LITTLE_ENDIAN);
-
-        buffer.put((byte) 0x91);
-        buffer.put((byte) tagFinal.length());
-        byte[] tagBytes = null;
-        tagBytes = tagFinal.getBytes(StandardCharsets.US_ASCII);
-
-        buffer.put(tagBytes);
-        buffer.position(2 + tagBytes.length);
 
-
-        if (isPadded) {
-            buffer.put((byte) 0x00);
-        }
-
-        if (isArray) {
-            buffer.put((byte) 0x28);
-            buffer.put((byte) arrayIndex);
-        }
-        if (isStruct) {
-            buffer.put(toAnsi(tag.substring(tag.indexOf(".") + 1, tag.length())));
+            lengthBytes += newSegment.getLengthInBytes();
         }
-        return buffer.array();
-    }
-
-    private CompletableFuture<PlcReadResponse> toPlcReadResponse(PlcReadRequest readRequest, CompletableFuture<CipService> response) {
-        return response
-            .thenApply(p -> {
-                return ((PlcReadResponse) decodeReadResponse(p, readRequest));
-            });
-    }
-
-    private CompletableFuture<CipService> readInternal(List<CipReadRequest> request) {
-        CompletableFuture<CipService> future = new CompletableFuture<>();
-        RequestTransactionManager.RequestTransaction transaction = tm.startRequest();
-        if (request.size() > 1) {
-
-            short nb = (short) request.size();
-            List<Integer> offsets = new ArrayList<>(nb);
-            int offset = 2 + nb * 2;
-            for (int i = 0; i < nb; i++) {
-                offsets.add(offset);
-                offset += request.get(i).getLengthInBytes();
-            }
-
-            List<CipService> serviceArr = new ArrayList<>(nb);
-            for (int i = 0; i < nb; i++) {
-                serviceArr.add(request.get(i));
-            }
-            Services data = new Services(nb, offsets, serviceArr, -1);
-            //Encapsulate the data
-
-            PathSegment pathSegment0 = new AnsiExtendedSymbolSegment((short) 4,"test");
-
-            CipRRData pkt = new CipRRData(sessionHandle, 0L, emptySenderContext, 0L, 0, 2,
-                new CipExchange(
-                    new CipUnconnectedRequest(
-                        (short) pathSegment0.getLengthInBytes(),
-                        pathSegment0,
-                        pathSegment0,
-                        (Integer) 0
-                    ),
-                    -1
-                ),
-                -1
-            );
-
-
-            transaction.submit(() -> context.sendRequest(pkt)
-                .expectResponse(EipPacket.class, REQUEST_TIMEOUT)
-                .onTimeout(future::completeExceptionally)
-                .onError((p, e) -> future.completeExceptionally(e))
-                .check(p -> p instanceof CipRRData)
-                .check(p -> p.getSessionHandle() == sessionHandle)
-                //.check(p -> p.getSenderContext() == senderContext)
-                .unwrap(p -> (CipRRData) p)
-                .unwrap(p -> p.getExchange().getService()).check(p -> p instanceof MultipleServiceResponse)
-                .unwrap(p -> (MultipleServiceResponse) p)
-                .check(p -> p.getServiceNb() == nb)
-                .handle(p -> {
-                    future.complete(p);
-                    // Finish the request-transaction.
-                    transaction.endRequest();
-                }));
-        } else if (request.size() == 1) {
-
-            PathSegment pathSegment0 = new AnsiExtendedSymbolSegment((short) 4,"test");
+        WriteBufferByteBased buffer = new WriteBufferByteBased(lengthBytes, org.apache.plc4x.java.spi.generation.ByteOrder.LITTLE_ENDIAN);
 
-            CipExchange exchange = new CipExchange(
-                new CipUnconnectedRequest(
-                    (short) pathSegment0.getLengthInBytes(),
-                    pathSegment0,
-                    pathSegment0,
-                    (Integer) 0
-                ),
-                -1
-            );
-
-            CipRRData pkt = new CipRRData(sessionHandle, 0L, emptySenderContext, 0L, 0, 2, exchange, -1);
-            transaction.submit(() -> context.sendRequest(pkt)
-                .expectResponse(EipPacket.class, REQUEST_TIMEOUT)
-                .onTimeout(future::completeExceptionally)
-                .onError((p, e) -> future.completeExceptionally(e))
-                .check(p -> p instanceof CipRRData)
-                .check(p -> p.getSessionHandle() == sessionHandle)
-                //.check(p -> p.getSenderContext() == senderContext)
-                .unwrap(p -> (CipRRData) p)
-                .unwrap(p -> p.getExchange().getService()).check(p -> p instanceof CipReadResponse)
-                .unwrap(p -> (CipReadResponse) p)
-                .handle(p -> {
-                    future.complete(p);
-                    // Finish the request-transaction.
-                    transaction.endRequest();
-                }));
+        for (PathSegment segment : segments) {
+            segment.serialize(buffer);
         }
-        return future;
+        return buffer.getData();
     }
 
-    private PlcResponse decodeReadResponse(CipService p, PlcReadRequest readRequest) {
+    private PlcReadResponse decodeReadResponse(CipService p, PlcReadRequest readRequest) {
         Map<String, ResponseItem<PlcValue>> values = new HashMap<>();
         // only 1 field
         if (p instanceof CipReadResponse) {
@@ -493,31 +527,31 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
                 elements = field.getElementNb();
             }
 
-            //We need the size of the request in words (0x91, tagLength, ... tag + possible pad)
-            // Taking half to get word size
-            boolean isArray = false;
-            String tagIsolated = tag;
-            if (tag.contains("[")) {
-                isArray = true;
-                tagIsolated = tag.substring(0, tag.indexOf("["));
-            }
-            int dataLength = (tagIsolated.length() + 2 + (tagIsolated.length() % 2) + (isArray ? 2 : 0));
-            byte requestPathSize = (byte) (dataLength / 2);
             byte[] data = encodeValue(value, field.getType(), (short) elements);
-            CipWriteRequest writeReq = new CipWriteRequest(requestPathSize, toAnsi(tag), field.getType(), elements, data, -1);
-            items.add(writeReq);
+            try {
+                CipWriteRequest writeReq = new CipWriteRequest(toAnsi(tag), field.getType(), elements, data, -1);
+                items.add(writeReq);
+            } catch (SerializationException e) {
+                e.printStackTrace();
+            }
+
         }
 
         RequestTransactionManager.RequestTransaction transaction = tm.startRequest();
         if (items.size() == 1) {
             tm.startRequest();
-            PathSegment pathSegment0 = new AnsiExtendedSymbolSegment((short) 4,"test");
-            CipRRData rrdata = new CipRRData(sessionHandle, 0L, senderContext, 0L, 0, 2,
+
+            CipRRData rrdata = new CipRRData(
+                sessionHandle,
+                0L,
+                senderContext,
+                0L,
+                EMPTY_INTERFACE_HANDLE,
+                0,
+                2,
                 new CipExchange(
                     new CipUnconnectedRequest(
-                        (short) pathSegment0.getLengthInBytes(),
-                        pathSegment0,
-                        pathSegment0,
+                        new byte[10],
                         (Integer) 0
                     ),
                     -1
@@ -555,14 +589,19 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
             Services data = new Services(nb, offsets, serviceArr, -1);
             //Encapsulate the data
 
-            PathSegment pathSegment0 = new AnsiExtendedSymbolSegment((short) 4,"test");
 
-            CipRRData pkt = new CipRRData(sessionHandle, 0L, emptySenderContext, 0L, 0, 2,
+
+            CipRRData pkt = new CipRRData(
+                sessionHandle,
+                0L,
+                DEFAULT_SENDER_CONTEXT,
+                0L,
+                EMPTY_INTERFACE_HANDLE,
+                0,
+                2,
                 new CipExchange(
                     new CipUnconnectedRequest(
-                        (short) pathSegment0.getLengthInBytes(),
-                        pathSegment0,
-                        pathSegment0,
+                        new byte[10],
                         (Integer) 0
                     ),
                     -1
@@ -675,8 +714,6 @@ public class LogixProtocolLogic extends Plc4xProtocolBase<EipPacket> implements
 
     @Override
     public void close(ConversationContext<EipPacket> context) {
-        logger.debug("Sending UnregisterSession EIP Pakcet");
-        context.sendRequest(new EipDisconnectRequest(sessionHandle, 0L, emptySenderContext, 0L)); //Unregister gets no response
-        logger.debug("Unregistred Session {}", sessionHandle);
+        onDisconnect(context);
     }
 }
diff --git a/plc4j/drivers/logix/src/test/java/org/apache/plc4x/java/logix/LogixDriverIT.java b/plc4j/drivers/logix/src/test/java/org/apache/plc4x/java/logix/LogixDriverIT.java
new file mode 100644
index 0000000000..b794fafaa8
--- /dev/null
+++ b/plc4j/drivers/logix/src/test/java/org/apache/plc4x/java/logix/LogixDriverIT.java
@@ -0,0 +1,90 @@
+/*
+ * 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.logix;
+
+import org.apache.plc4x.java.PlcDriverManager;
+import org.apache.plc4x.java.api.PlcConnection;
+import org.apache.plc4x.java.api.exceptions.PlcConnectionException;
+import org.apache.plc4x.java.logix.readwrite.protocol.LogixProtocolLogic;
+import org.apache.plc4x.java.spi.generation.SerializationException;
+import org.apache.plc4x.test.driver.DriverTestsuiteRunner;
+import org.assertj.core.api.Condition;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.fail;
+
+public class LogixDriverIT {
+
+    @Test
+    public void simpleStructToANSI() throws SerializationException, IOException {
+        Charset charset = StandardCharsets.US_ASCII;
+        byte[] name0 = charset.encode("ZZZ_ZZZ_ZZZ").array();
+        byte[] name1 = charset.encode("XXX").array();
+        byte[] bytes = LogixProtocolLogic.toAnsi("ZZZ_ZZZ_ZZZ.XXX");
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        output.write(145);
+        output.write(11);
+        output.write(name0);
+        output.write(0);
+        output.write(145);
+        output.write(3);
+        output.write(name1);
+        output.write(0);
+
+        byte[] out = output.toByteArray();
+
+        try {
+            assert(Arrays.equals(out, bytes));
+        } catch (AssertionError e) {
+            fail("Structure to ANSI: " + Arrays.toString(bytes) + " != " + Arrays.toString(out));
+        }
+
+    }
+
+    @Test
+    public void simpleToANSIPadding() throws SerializationException, IOException {
+        Charset charset = StandardCharsets.US_ASCII;
+        byte[] name0 = charset.encode("ZZZ_ZZZ_ZZZ").array();
+        byte[] bytes = LogixProtocolLogic.toAnsi("ZZZ_ZZZ_ZZZ");
+
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        output.write(145);
+        output.write(11);
+        output.write(name0);
+        // Add Padding
+        output.write(0);
+
+        byte[] out = output.toByteArray();
+
+        try {
+            assert(Arrays.equals(out, bytes));
+        } catch (AssertionError e) {
+            fail("Structure to ANSI: " + Arrays.toString(bytes) + " != " + Arrays.toString(out));
+        }
+    }
+
+}
diff --git a/plc4j/drivers/logix/src/test/java/org/apache/plc4x/java/eip/readwrite/LogixParserSerializerTest.java b/plc4j/drivers/logix/src/test/java/org/apache/plc4x/java/logix/LogixParserSerializerTest.java
similarity index 95%
rename from plc4j/drivers/logix/src/test/java/org/apache/plc4x/java/eip/readwrite/LogixParserSerializerTest.java
rename to plc4j/drivers/logix/src/test/java/org/apache/plc4x/java/logix/LogixParserSerializerTest.java
index aa09eb238b..a666c16fee 100644
--- a/plc4j/drivers/logix/src/test/java/org/apache/plc4x/java/eip/readwrite/LogixParserSerializerTest.java
+++ b/plc4j/drivers/logix/src/test/java/org/apache/plc4x/java/logix/LogixParserSerializerTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.plc4x.java.logix.readwrite;
+package org.apache.plc4x.java.logix;
 
 import org.apache.plc4x.test.parserserializer.ParserSerializerTestsuiteRunner;
 
diff --git a/pom.xml b/pom.xml
index 676c64d11d..5f6dc4b298 100644
--- a/pom.xml
+++ b/pom.xml
@@ -92,6 +92,10 @@
 
     <java.version>11</java.version>
 
+    <maven.compiler.source>11</maven.compiler.source>
+    <maven.compiler.target>11</maven.compiler.target>
+
+
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputencoding>UTF-8</project.reporting.outputencoding>
 
diff --git a/protocols/logix/src/main/resources/protocols/logix/logix.mspec b/protocols/logix/src/main/resources/protocols/logix/logix.mspec
index a9712cfa15..141e6d3b17 100644
--- a/protocols/logix/src/main/resources/protocols/logix/logix.mspec
+++ b/protocols/logix/src/main/resources/protocols/logix/logix.mspec
@@ -26,17 +26,17 @@
     [implicit      uint 16 len 'lengthInBytes - 24']
     [simple        uint 32 sessionHandle]
     [simple        uint 32 status]
-    [array         uint 8  senderContext count '8']
+    [array         byte    senderContext count '8']
     [simple        uint 32 options]
     [typeSwitch command
             ['0x0065' EipConnectionRequest
                 [const      uint    16   protocolVersion   0x01]
                 [const      uint    16   flags             0x00]
             ]
-            ['0x0066' EipDisconnectRequest
+            ['0x0066' EipDisconnect
             ]
             ['0x006F' CipRRData(uint 16 len)
-                [reserved   uint    32    '0x00000000']
+                [simple     uint    32    interfaceHandle]
                 [simple     uint    16    timeout]
                 [simple     uint    16    itemCount]
                 [simple    CipExchange('len - 6')   exchange]
@@ -54,11 +54,11 @@
     [discriminator  uint    16  id]
     [typeSwitch id
         ['0x00A1'   ConnectedAddressItem
-            [simple         uint    16  size]
+            [reserved       uint    16  '0x04']
             [simple         uint    32  connectionId]
         ]
-        ['0x00B1'   ConnectedAddressItem
-            [simple         uint    16  size]
+        ['0x00B1'   ConnectedDataItem
+            [implicit       uint    16  size 'service.lengthInBytes + 2']
             [simple         uint    16  sequenceCount]
             [simple         CipService('size - 2')    service]
         ]
@@ -77,9 +77,10 @@
     [discriminator  uint    7   service]
     [typeSwitch service,response
         ['0x4C','false' CipReadRequest
-            [simple     int     8   requestPathSize]
-            [array      byte   tag   length  '(requestPathSize * 2)']
+            [implicit   int     8   requestPathSize 'COUNT(tag) / 2']
+            [array      byte   tag   count  '(requestPathSize * 2)']
             [simple     uint    16  elementNb]
+            [array      byte   data  count  '6']
         ]
         ['0x4C','true' CipReadResponse
               [reserved   uint            8   '0x00']
@@ -89,7 +90,7 @@
               [array      byte   data  count  'serviceLen - 6']
         ]
         ['0x4D','false' CipWriteRequest
-            [simple     int     8           requestPathSize]
+            [implicit   int     8   requestPathSize 'COUNT(tag) / 2']
             [array      byte           tag   length  'requestPathSize * 2']
             [simple     CIPDataTypeCode     dataType]
             [simple     uint    16          elementNb]
@@ -100,6 +101,31 @@
             [simple     uint        8   status]
             [simple     uint        8   extStatus]
         ]
+        ['0x4E','false' CipConnectionManagerCloseRequest
+              [simple      int     8           requestPathSize]
+              [simple      PathSegment         classSegment]
+              [simple      PathSegment         instanceSegment]
+              [simple      uint    4           priority]
+              [simple      uint    4           tickTime]
+              [simple      uint    8           timeoutTicks]
+              [simple      uint    16          connectionSerialNumber]
+              [simple      uint    16          originatorVendorId]
+              [simple      uint    32          originatorSerialNumber]
+              [simple      uint    8           connectionPathSize]
+              [reserved    byte                '0x00']
+              [simple      PathSegment         connectionPathPortSegment]
+              [simple      PathSegment         connectionPathClassSegment]
+              [simple      PathSegment         connectionPathInstanceSegment]
+        ]
+        ['0x4E','true' CipConnectionManagerCloseResponse
+              [simple   uint    8   status]
+              [simple   uint    8   additionalStatusWords]
+              [simple      uint    16          connectionSerialNumber]
+              [simple      uint    16          originatorVendorId]
+              [simple      uint    32          originatorSerialNumber]
+              [simple      uint     8          applicationReplySize]
+              [reserved    uint     8          '0x00']
+        ]
         ['0x0A','false' MultipleServiceRequest
                [const  int     8   requestPathSize   0x02]
                [const  uint    32  requestPath       0x01240220]   //Logical Segment: Class(0x20) 0x02, Instance(0x24) 01 (Message Router)
@@ -114,12 +140,11 @@
                [array      byte   servicesData count 'serviceLen - 6 - (2 * serviceNb)']
         ]
         ['0x52','false'   CipUnconnectedRequest
-               [simple     uint    8    requestPathSize ]
-               [simple     PathSegment  pathSegment0]
-               [simple     PathSegment  pathSegment1]
+               [implicit   uint    8    requestPathSize 'COUNT(pathSegments)']
+               [array      byte         pathSegments    count 'requestPathSize*2']
         ]
         ['0x5B','false'     CipConnectionManagerRequest
-               [simple      int     8           requestPathSize]
+               [implicit      int     8         requestPathSize '(classSegment.lengthInBytes + instanceSegment.lengthInBytes)/2']
                [simple      PathSegment         classSegment]
                [simple      PathSegment         instanceSegment]
                [simple      uint    4           priority]
@@ -169,10 +194,8 @@
         ['0x01'      LogicalSegment
             [simple LogicalSegmentType  segmentType]
         ]
-        ['0x04'      AnsiExtendedSymbolSegment
-            [simple  uint    5   dataSegment]
-            [implicit   uint    8   dataSize    'symbol.length']
-            [simple     vstring 'dataSize'  symbol]
+        ['0x04'      DataSegment
+            [simple DataSegmentType segmentType]
         ]
     ]
 ]
@@ -188,6 +211,21 @@
             [simple uint    2   format]
             [simple uint    8   instance]
         ]
+        ['0x02' MemberID
+            [simple uint    2   format]
+            [simple uint    8   instance]
+        ]
+    ]
+]
+
+[discriminatedType DataSegmentType
+    [discriminator  uint    5   dataSegmentType]
+    [typeSwitch dataSegmentType
+        ['0x11'      AnsiExtendedSymbolSegment
+            [implicit   uint    8   dataSize        'symbol.length']
+            [simple     vstring     'dataSize * 8'      symbol]
+            [optional   uint    8   pad         'symbol.length % 2 != 0']
+        ]
     ]
 ]
 
@@ -248,4 +286,39 @@
     ['0x0065'   RegisterSession ]
     ['0x0066'   UnregisterSession ]
     ['0x006F'   SendRRData ]
+]
+
+[enum   uint    32  CIPStatus
+    ['0x00000000'   Success]
+    ['0x00000001'   ConnectionFailure]
+    ['0x00000002'   ResourceUnAvailable]
+    ['0x00000003'   InvalidParameterValue]
+    ['0x00000004'   PathSegmentError]
+    ['0x00000005'   PathDestinationUnknown]
+    ['0x00000006'   PartialTransfer]
+    ['0x00000007'   ConnectionIDNotValid]
+    ['0x00000008'   ServiceNotSupported]
+    ['0x00000009'   InvalidAttributeValue]
+    ['0x0000000A'   AttributeListError]
+    ['0x0000000B'   AlreadyInRequestedState]
+    ['0x0000000C'   ObjectStateConflict]
+    ['0x0000000D'   ObjectAlreadyExists]
+    ['0x0000000E'   AttributeNotSettable]
+    ['0x0000000F'   PriviligeViolation]
+    ['0x00000010'   DeviceStateConflict]
+    ['0x00000011'   ReplyDataTooLarge]
+    ['0x00000012'   FragmentationOfPrimitiveValue]
+    ['0x00000013'   NotEnoughData]
+    ['0x00000014'   AttributeNotSupported]
+    ['0x00000015'   TooMuchData]
+    ['0x00000016'   ObjectDoesNotExist]
+    ['0x00000017'   ServiceFragmentation]
+    ['0x00000018'   NoStoredAttributeData]
+    ['0x00000019'   StoreOperationFailure]
+    ['0x0000001A'   RequestPacketTooLarge]
+    ['0x0000001B'   ResponsePacketTooLarge]
+    ['0x0000001C'   MissingAttributeListEntryData]
+    ['0x0000001D'   InvalidAttributeValueList]
+    ['0x0000001E'   EmbeddedServiceError]
+    ['0x0000001F'   VendorSpecificError]
 ]
\ No newline at end of file
diff --git a/protocols/logix/src/test/resources/protocols/logix/ParserSerializerTestsuite.xml b/protocols/logix/src/test/resources/protocols/logix/ParserSerializerTestsuite.xml
index 084307c44c..c4760268e9 100644
--- a/protocols/logix/src/test/resources/protocols/logix/ParserSerializerTestsuite.xml
+++ b/protocols/logix/src/test/resources/protocols/logix/ParserSerializerTestsuite.xml
@@ -35,16 +35,7 @@
         <len dataType="uint" bitLength="16">4</len>
         <sessionHandle dataType="uint" bitLength="32">0</sessionHandle>
         <status dataType="uint" bitLength="32">0</status>
-        <senderContext isList="true">
-          <value dataType="uint" bitLength="8">48</value>
-          <value dataType="uint" bitLength="8">46</value>
-          <value dataType="uint" bitLength="8">56</value>
-          <value dataType="uint" bitLength="8">46</value>
-          <value dataType="uint" bitLength="8">51</value>
-          <value dataType="uint" bitLength="8">32</value>
-          <value dataType="uint" bitLength="8">32</value>
-          <value dataType="uint" bitLength="8">32</value>
-        </senderContext>
+        <senderContext dataType="byte" bitLength="64">0x302e382e33202020</senderContext>
         <options dataType="uint" bitLength="32">0</options>
         <EipConnectionRequest>
           <protocolVersion dataType="uint" bitLength="16">1</protocolVersion>