You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by ld...@apache.org on 2021/03/29 11:56:20 UTC

[plc4x] branch feature/profinet2 updated (d8ddeb6 -> 7d0fe2b)

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

ldywicki pushed a change to branch feature/profinet2
in repository https://gitbox.apache.org/repos/asf/plc4x.git.


    from d8ddeb6  Fix troubles with padding, yet not perfect.
     new b490038  Use latest SNAPSHOT version to support JMPS
     new a502f54  Re-organize tests to match implementation #PLC4X-286
     new efaff9e  Add more PN_DCP constants without implementation
     new 7d0fe2b  Support for tagged ethernet frames.

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


Summary of changes:
 .../protocols/profinet/profinet.dcp.mspec          |  92 +++++++--
 sandbox/test-java-profinet-driver/pom.xml          |   3 +
 .../java/profinet/dcp/ProfinetDCPPlcDriver.java    |   8 +-
 .../java/profinet/dcp/field/ProfinetDcpField.java  |   5 +-
 .../dcp/protocol/ProfinetDCPProtocolLogic.java     | 126 ++++++++++--
 .../protocol/ProfinetDCPSubscriptionHandle.java    |  54 +++++
 .../test/resources/testsuite/ProfinetDCPBlock.xml  | 112 ++++++-----
 .../test/resources/testsuite/ProfinetDCPFrame.xml  | 220 ++++++++++-----------
 8 files changed, 421 insertions(+), 199 deletions(-)
 copy plc4j/api/src/test/java/org/apache/plc4x/java/mock/DuplicateMockDriver.java => sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/field/ProfinetDcpField.java (84%)
 create mode 100644 sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPSubscriptionHandle.java

[plc4x] 03/04: Add more PN_DCP constants without implementation

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

ldywicki pushed a commit to branch feature/profinet2
in repository https://gitbox.apache.org/repos/asf/plc4x.git

commit efaff9e793c0d91982144441192ed5867ef99feb
Author: Adi <ad...@gmail.com>
AuthorDate: Fri Mar 26 17:51:22 2021 +0100

    Add more PN_DCP constants without implementation
    
    From https://github.com/dark-lbp/isf/blob/master/icssploit/protocols/pn_dcp.py
    and https://rt-labs.com/docs/p-net/profinet_details.html#dcp-protocol-via-ethernet-layer-2
---
 .../protocols/profinet/profinet.dcp.mspec          | 51 ++++++++++++++++++++--
 1 file changed, 48 insertions(+), 3 deletions(-)

diff --git a/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec b/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
index fb0595d..d202968 100644
--- a/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
+++ b/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
@@ -106,12 +106,44 @@
     [padding uint 8                 'pad' '0x00' 'length % 2']
 ]
 
-
 [enum uint 8 'DCPBlockOption'
     ['0x01' IP                          ]
     ['0x02' DEVICE_PROPERTIES           ]
+    ['0x03' DHCP                        ]
+    // TODO implement DCPBlockOption case
+    ['0x05' CONTROL                     ]
+    ['0x06' DEVICE_INACTIVE             ]
     ['0xFF' ALL_SELECTOR                ]
 ]
+// TODO implement DCPBlockOption case
+[enum uint 8 'ControlOption'
+    ['0x00' RESERVED                    ]
+    ['0x01' START_TRANSACTION           ]
+    ['0x02' END_TRANSACTION             ]
+    ['0x03' FLASH_LED                   ]
+    ['0x04' RESPONSE                    ]
+    //Reset Factory Settings (0x05)
+    ['0x05' FACTORY_RESET               ]
+    ['0x06' RESET_TO_FACTORY            ]
+]
+// TODO implement DCPBlockOption case
+[enum uint 8 'DeviceInactiveOption'
+    ['0x00' RESERVED                    ]
+    ['0x01' DEVICE_INITIATIVE           ]
+]
+
+// Not implemented yet! Vars from https://github.com/dark-lbp/isf/blob/master/icssploit/protocols/pn_dcp.py
+[enum uint 8 'DHCPOptions'
+    ['0x0c' HOST_NAME                   ]
+    ['0x2b' VENDOR_SPECIFIC             ]
+    ['0x36' SERVER_IDENTIFIER           ]
+    ['0x37' PARAMETER_REQUEST_LIST      ]
+    ['0x3c' CLASS_IDENTIFIER            ]
+    ['0x3d' DHCP_CLIENT_IDENTIFIER      ]
+    ['0x51' FQDN_FULLY_QUALIFIED_DOMAIN_NAME       ]
+    ['0x61' UUID_GUID_BASED_CLIENT      ]
+    ['0xFF' CONTROL_DHCP_FOR_ADDRESS_RESOLUTION    ]
+]
 
 [enum uint 8 'DevicePropertiesSubOption'
     ['0x01' STATION_TYPE                ]
@@ -119,15 +151,22 @@
     ['0x03' DEVICE_ID                   ]
     ['0x04' DEVICE_ROLE                 ]
     ['0x05' DEVICE_OPTIONS              ]
+    ['0x06' ALIAS_NAME                  ]
     ['0x07' DEVICE_INSTANCE             ]
+    ['0x08' OEM_DEVICE_ID               ]
 ]
 
 [enum uint 8 'IpSubOption'
+    ['0x00' RESERVED                    ]
+    ['0x01' MAC_ADDRESS                 ]
     ['0x02' IP_PARAMETER                ]
 ]
 
 [enum uint 8 'DCPServiceID'
-    ['0x05' IDENTIFY]
+    ['0x03' GET                         ]
+    ['0x04' SET                         ]
+    ['0x05' IDENTIFY                    ]
+    ['0x06' HELLO                       ]
     ['0xFEFE' IDENTIFY_RESPONSE         ]
 ]
 
@@ -164,4 +203,10 @@
         'STATIC_CALL("org.apache.plc4x.java.profinet.dcp.DCPUtil.writeString", io, _value, _type.encoding)'
         'STATIC_CALL("org.apache.plc4x.java.profinet.dcp.DCPUtil.length", text)'
     ]
-]
\ No newline at end of file
+]
+
+[enum uint 16 'TypeLAN'
+    ['0x8100' VLAN             ]
+    ['0x8892' PN_DCP           ]
+    ['0x88CC' LLDP             ]
+]

[plc4x] 01/04: Use latest SNAPSHOT version to support JMPS

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

ldywicki pushed a commit to branch feature/profinet2
in repository https://gitbox.apache.org/repos/asf/plc4x.git

commit b49003820f16df3f76191c009895abb384e8944b
Author: Adi <ad...@gmail.com>
AuthorDate: Fri Mar 19 14:06:33 2021 +0100

    Use latest SNAPSHOT version to support JMPS
---
 sandbox/test-java-profinet-driver/pom.xml | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/sandbox/test-java-profinet-driver/pom.xml b/sandbox/test-java-profinet-driver/pom.xml
index 0d26ff2..93405f3 100644
--- a/sandbox/test-java-profinet-driver/pom.xml
+++ b/sandbox/test-java-profinet-driver/pom.xml
@@ -61,16 +61,19 @@
     <dependency>
       <groupId>org.pcap4j</groupId>
       <artifactId>pcap4j-core</artifactId>
+      <version>1.8.3-SNAPSHOT</version>
       <scope>compile</scope>
     </dependency>
     <dependency>
       <groupId>org.pcap4j</groupId>
       <artifactId>pcap4j-packetfactory-static</artifactId>
+      <version>1.8.3-SNAPSHOT</version>
       <scope>compile</scope>
     </dependency>
     <dependency>
       <groupId>ch.qos.logback</groupId>
       <artifactId>logback-classic</artifactId>
+      <version>1.3.0-alpha5</version>
       <scope>compile</scope>
     </dependency>
 

[plc4x] 04/04: Support for tagged ethernet frames.

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

ldywicki pushed a commit to branch feature/profinet2
in repository https://gitbox.apache.org/repos/asf/plc4x.git

commit 7d0fe2b6f60ff8bc8c6b8e64e1ae552bb2a07c3f
Author: Ɓukasz Dywicki <lu...@code-house.org>
AuthorDate: Mon Mar 29 13:55:52 2021 +0200

    Support for tagged ethernet frames.
    
    Some sketch work around subscriptions for ident answers, without actual field definitions and concrete implementations.
---
 .../protocols/profinet/profinet.dcp.mspec          |  33 ++++--
 .../java/profinet/dcp/ProfinetDCPPlcDriver.java    |   8 +-
 .../java/profinet/dcp/field/ProfinetDcpField.java  |  24 ++++
 .../dcp/protocol/ProfinetDCPProtocolLogic.java     | 126 ++++++++++++++++++---
 .../protocol/ProfinetDCPSubscriptionHandle.java    |  54 +++++++++
 5 files changed, 217 insertions(+), 28 deletions(-)

diff --git a/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec b/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
index d202968..fb9bf40 100644
--- a/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
+++ b/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
@@ -17,11 +17,22 @@
 // under the License.
 //
 
-[type 'EthernetFrame'
-    [simple MacAddress 'destination' ]
-    [simple MacAddress 'source'      ]
-    [simple uint 16    'ethernetType']
-    [simple ProfinetFrame 'payload'  ]
+[discriminatedType 'BaseEthernetFrame'
+    [simple        MacAddress    'destination' ]
+    [simple        MacAddress    'source'      ]
+    [simple        uint 16       'etherType']
+    [typeSwitch 'etherType'
+        ['0x8100' TaggedFrame
+            [simple uint 3       'priority']
+            [simple bit          'droppable']
+            [simple uint 12      'vlan']
+            [simple uint 16      'ethernetType']
+        ]
+        [EthernetFrame
+            [implicit uint 16    'ethernetType' 'etherType']
+        ]
+    ]
+    [simple        ProfinetFrame 'payload'     ]
 ]
 
 [type 'ProfinetFrame'
@@ -40,11 +51,19 @@
             [simple uint 16 'dcpDataLength'                ]
             [array DCPBlock 'blocks' length 'dcpDataLength']
         ]
+        ['FrameType.GET_SET' DcpGetSetPDU
+            [enum DCPServiceID 'serviceId'                 ]
+            [enum DCPServiceType 'serviceType'             ]
+            [simple uint 32 'xid'                          ]
+            [reserved uint 16 '0x00'                       ]
+            [simple uint 16 'dcpDataLength'                ]
+            [array DCPBlock 'blocks' length 'dcpDataLength']
+        ]
         ['FrameType.IDENTIFY_RESPONSE' DcpIdentResponsePDU
             [enum DCPServiceID 'serviceId'                 ]
             [enum DCPServiceType 'serviceType'             ]
             [simple uint 32 'xid'                          ]
-            [simple uint 16 'responseDelay'                ]
+            [reserved uint 16 '0x00'                       ]
             [simple uint 16 'dcpDataLength'                ]
             [array DCPBlock 'blocks' length 'dcpDataLength']
         ]
@@ -167,7 +186,6 @@
     ['0x04' SET                         ]
     ['0x05' IDENTIFY                    ]
     ['0x06' HELLO                       ]
-    ['0xFEFE' IDENTIFY_RESPONSE         ]
 ]
 
 [enum uint 8 'DCPServiceType'
@@ -177,6 +195,7 @@
 
 [enum uint 16 'FrameType'
     ['0xFEFE' IDENTIFY_MULTICAST_REQUEST]
+    ['0xFEFD' GET_SET                   ]
     ['0xFEFF' IDENTIFY_RESPONSE         ]
 ]
 
diff --git a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/ProfinetDCPPlcDriver.java b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/ProfinetDCPPlcDriver.java
index 5baa18e..eba4937 100644
--- a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/ProfinetDCPPlcDriver.java
+++ b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/ProfinetDCPPlcDriver.java
@@ -24,7 +24,9 @@ import org.apache.plc4x.java.api.value.PlcValueHandler;
 import org.apache.plc4x.java.profinet.dcp.configuration.ProfinetConfiguration;
 import org.apache.plc4x.java.profinet.dcp.field.ProfinetFieldHandler;
 import org.apache.plc4x.java.profinet.dcp.protocol.ProfinetDCPProtocolLogic;
+import org.apache.plc4x.java.profinet.dcp.readwrite.BaseEthernetFrame;
 import org.apache.plc4x.java.profinet.dcp.readwrite.EthernetFrame;
+import org.apache.plc4x.java.profinet.dcp.readwrite.io.BaseEthernetFrameIO;
 import org.apache.plc4x.java.profinet.dcp.readwrite.io.EthernetFrameIO;
 import org.apache.plc4x.java.spi.configuration.Configuration;
 import org.apache.plc4x.java.spi.connection.GeneratedDriverBase;
@@ -37,7 +39,7 @@ import org.apache.plc4x.java.spi.values.IEC61131ValueHandler;
 /**
  * Proof of concept implementation of Profinet DCP driver.
  */
-public class ProfinetDCPPlcDriver extends GeneratedDriverBase<EthernetFrame> {
+public class ProfinetDCPPlcDriver extends GeneratedDriverBase<BaseEthernetFrame> {
 
     @Override
     public String getProtocolCode() {
@@ -70,8 +72,8 @@ public class ProfinetDCPPlcDriver extends GeneratedDriverBase<EthernetFrame> {
     }
 
     @Override
-    protected ProtocolStackConfigurer<EthernetFrame> getStackConfigurer() {
-        return SingleProtocolStackConfigurer.builder(EthernetFrame.class, EthernetFrameIO.class)
+    protected ProtocolStackConfigurer<BaseEthernetFrame> getStackConfigurer() {
+        return SingleProtocolStackConfigurer.builder(BaseEthernetFrame.class, BaseEthernetFrameIO.class)
             .withProtocol(ProfinetDCPProtocolLogic.class)
             .withPacketSizeEstimator(ProfinetPacketEstimator.class)
             .withCorruptPacketRemover(CorruptEthernetFrameRemover.class)
diff --git a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/field/ProfinetDcpField.java b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/field/ProfinetDcpField.java
new file mode 100644
index 0000000..98d92d5
--- /dev/null
+++ b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/field/ProfinetDcpField.java
@@ -0,0 +1,24 @@
+/*
+ * 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.profinet.dcp.field;
+
+import org.apache.plc4x.java.api.model.PlcField;
+
+public class ProfinetDcpField implements PlcField {
+}
diff --git a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPProtocolLogic.java b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPProtocolLogic.java
index 9348d2c..c8ba4a0 100644
--- a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPProtocolLogic.java
+++ b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPProtocolLogic.java
@@ -19,18 +19,28 @@
 package org.apache.plc4x.java.profinet.dcp.protocol;
 
 import java.time.Duration;
+import java.time.Instant;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+
+import ch.qos.logback.classic.util.LogbackMDCAdapter;
+import org.apache.plc4x.java.api.messages.PlcSubscriptionEvent;
+import org.apache.plc4x.java.api.messages.PlcSubscriptionRequest;
+import org.apache.plc4x.java.api.messages.PlcSubscriptionResponse;
+import org.apache.plc4x.java.api.model.PlcConsumerRegistration;
+import org.apache.plc4x.java.api.model.PlcSubscriptionHandle;
+import org.apache.plc4x.java.api.types.PlcResponseCode;
+import org.apache.plc4x.java.api.types.PlcSubscriptionType;
+import org.apache.plc4x.java.api.value.PlcValue;
 import org.apache.plc4x.java.profinet.dcp.configuration.ProfinetConfiguration;
-import org.apache.plc4x.java.profinet.dcp.readwrite.AllSelector;
-import org.apache.plc4x.java.profinet.dcp.readwrite.DCPBlock;
-import org.apache.plc4x.java.profinet.dcp.readwrite.DcpIdentRequestPDU;
-import org.apache.plc4x.java.profinet.dcp.readwrite.DcpIdentResponsePDU;
-import org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties;
-import org.apache.plc4x.java.profinet.dcp.readwrite.EthernetFrame;
-import org.apache.plc4x.java.profinet.dcp.readwrite.IP;
-import org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address;
-import org.apache.plc4x.java.profinet.dcp.readwrite.MacAddress;
-import org.apache.plc4x.java.profinet.dcp.readwrite.ProfinetFrame;
+import org.apache.plc4x.java.profinet.dcp.field.ProfinetDcpField;
+import org.apache.plc4x.java.profinet.dcp.readwrite.*;
 import org.apache.plc4x.java.profinet.dcp.readwrite.types.DCPBlockOption;
 import org.apache.plc4x.java.profinet.dcp.readwrite.types.DCPServiceID;
 import org.apache.plc4x.java.profinet.dcp.readwrite.types.DCPServiceType;
@@ -38,20 +48,33 @@ import org.apache.plc4x.java.profinet.dcp.readwrite.types.FrameType;
 import org.apache.plc4x.java.spi.ConversationContext;
 import org.apache.plc4x.java.spi.Plc4xProtocolBase;
 import org.apache.plc4x.java.spi.configuration.HasConfiguration;
+import org.apache.plc4x.java.spi.generation.ParseException;
+import org.apache.plc4x.java.spi.generation.ReadBuffer;
+import org.apache.plc4x.java.spi.messages.DefaultPlcSubscriptionEvent;
+import org.apache.plc4x.java.spi.messages.DefaultPlcSubscriptionRequest;
+import org.apache.plc4x.java.spi.messages.DefaultPlcSubscriptionResponse;
+import org.apache.plc4x.java.spi.messages.PlcSubscriber;
+import org.apache.plc4x.java.spi.messages.utils.ResponseItem;
+import org.apache.plc4x.java.spi.model.DefaultPlcConsumerRegistration;
+import org.apache.plc4x.java.spi.model.DefaultPlcSubscriptionField;
+import org.apache.plc4x.java.spi.model.DefaultPlcSubscriptionHandle;
+import org.apache.plc4x.java.spi.values.PlcNull;
+import org.apache.plc4x.java.spi.values.PlcValues;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * Driver logic for handling Profinet-DCP packets.
  */
-public class ProfinetDCPProtocolLogic extends Plc4xProtocolBase<EthernetFrame> implements
-    HasConfiguration<ProfinetConfiguration> {
+public class ProfinetDCPProtocolLogic extends Plc4xProtocolBase<BaseEthernetFrame> implements
+    HasConfiguration<ProfinetConfiguration>, PlcSubscriber {
 
     public static MacAddress PROFINET_BROADCAST = createAddress(0x01, 0x0E, 0xCF, 0x00, 0x00, 0x00);
     public static int PN_DCP = 0x8892;
 
     public static final Duration REQUEST_TIMEOUT = Duration.ofMillis(10000);
     private final Logger logger = LoggerFactory.getLogger(ProfinetDCPProtocolLogic.class);
+    private Map<DefaultPlcConsumerRegistration, Consumer<PlcSubscriptionEvent>> consumers = new ConcurrentHashMap<>();
 
     private AtomicInteger invokeId = new AtomicInteger(0);
     private ProfinetConfiguration configuration;
@@ -62,7 +85,7 @@ public class ProfinetDCPProtocolLogic extends Plc4xProtocolBase<EthernetFrame> i
     }
 
     @Override
-    public void onConnect(ConversationContext<EthernetFrame> context) {
+    public void onConnect(ConversationContext<BaseEthernetFrame> context) {
         DCPServiceID serviceId = DCPServiceID.IDENTIFY;
         DCPServiceType serviceType = DCPServiceType.REQUEST;
         long xid = invokeId.incrementAndGet();
@@ -80,17 +103,48 @@ public class ProfinetDCPProtocolLogic extends Plc4xProtocolBase<EthernetFrame> i
     }
 
     @Override
-    public void onDisconnect(ConversationContext<EthernetFrame> context) {
+    public void onDisconnect(ConversationContext<BaseEthernetFrame> context) {
         context.fireDisconnected();
     }
 
     @Override
-    protected void decode(ConversationContext<EthernetFrame> context, EthernetFrame msg) throws Exception {
-        if (msg.getEthernetType() != PN_DCP) {
-            logger.trace("Discarding unwanted frame type {}", msg.getEthernetType());
+    protected void decode(ConversationContext<BaseEthernetFrame> context, BaseEthernetFrame msg) throws Exception {
+        if (msg instanceof TaggedFrame) {
+            TaggedFrame frame = (TaggedFrame) msg;
+            if (frame.getEthernetType() != PN_DCP) {
+                logger.trace("Discarding unwanted frame type {}", frame.getEthernetType());
+            }
+        } else if (msg.getEtherType() != PN_DCP) {
+            logger.trace("Discarding unwanted frame type {}", msg.getEtherType());
         }
 
         ProfinetFrame profinetFrame = msg.getPayload();
+
+        for (Map.Entry<DefaultPlcConsumerRegistration, Consumer<PlcSubscriptionEvent>> entry : consumers.entrySet()) {
+            DefaultPlcConsumerRegistration registration = entry.getKey();
+            Consumer<PlcSubscriptionEvent> consumer = entry.getValue();
+
+            for (PlcSubscriptionHandle handler : registration.getSubscriptionHandles()) {
+                ProfinetDCPSubscriptionHandle handle = (ProfinetDCPSubscriptionHandle) handler;
+
+                if (handle.matches(profinetFrame)) {
+                    logger.trace("Dispatching frame {} to {}", profinetFrame, handle);
+
+                    ProfinetDcpField field = handle.getField();
+                    // todo map actual DCP fields to PlcValues ?
+                    PlcValue value = PlcValues.of(profinetFrame);
+                    DefaultPlcSubscriptionEvent event = new DefaultPlcSubscriptionEvent(
+                        Instant.now(),
+                        Collections.singletonMap(
+                            handle.getName(),
+                            new ResponseItem<>(PlcResponseCode.OK, value)
+                        )
+                    );
+                    consumer.accept(event);
+                }
+            }
+        }
+
         if (profinetFrame.getFrameType() == FrameType.IDENTIFY_RESPONSE) {
             logger.info("Ident response from Profinet device:");
             if (profinetFrame.getFrame() instanceof DcpIdentResponsePDU) {
@@ -113,12 +167,48 @@ public class ProfinetDCPProtocolLogic extends Plc4xProtocolBase<EthernetFrame> i
         }
     }
 
+    @Override
+    public CompletableFuture<PlcSubscriptionResponse> subscribe(PlcSubscriptionRequest request) {
+        DefaultPlcSubscriptionRequest rq = (DefaultPlcSubscriptionRequest) request;
+
+        Map<String, ResponseItem<PlcSubscriptionHandle>> answers = new LinkedHashMap<>();
+        DefaultPlcSubscriptionResponse response = new DefaultPlcSubscriptionResponse(rq, answers);
+
+        for (String key : rq.getFieldNames()) {
+            DefaultPlcSubscriptionField subscription = (DefaultPlcSubscriptionField) rq.getField(key);
+            if (subscription.getPlcSubscriptionType() != PlcSubscriptionType.EVENT) {
+                answers.put(key, new ResponseItem<>(PlcResponseCode.UNSUPPORTED, null));
+            } else if ((subscription.getPlcField() instanceof ProfinetDcpField)) {
+                answers.put(key, new ResponseItem<>(PlcResponseCode.OK,
+                    new ProfinetDCPSubscriptionHandle(this, key, (ProfinetDcpField) subscription.getPlcField())
+                ));
+            } else {
+                answers.put(key, new ResponseItem<>(PlcResponseCode.INVALID_ADDRESS, null));
+            }
+        }
+
+        return CompletableFuture.completedFuture(response);
+    }
+
+    @Override
+    public PlcConsumerRegistration register(Consumer<PlcSubscriptionEvent> consumer, Collection<PlcSubscriptionHandle> handles) {
+        final DefaultPlcConsumerRegistration consumerRegistration = new DefaultPlcConsumerRegistration(this, consumer, handles.toArray(new DefaultPlcSubscriptionHandle[0]));
+        consumers.put(consumerRegistration, consumer);
+        return consumerRegistration;
+    }
+
+    @Override
+    public void unregister(PlcConsumerRegistration registration) {
+        consumers.remove(registration);
+    }
+
+
     private String addressString(IPv4Address address) {
         return address.getOctet1() + "." + address.getOctet2() + "." + address.getOctet3() + "." + address.getOctet4();
     }
 
     @Override
-    public void close(ConversationContext<EthernetFrame> context) {
+    public void close(ConversationContext<BaseEthernetFrame> context) {
 
     }
 
diff --git a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPSubscriptionHandle.java b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPSubscriptionHandle.java
new file mode 100644
index 0000000..2c3b06e
--- /dev/null
+++ b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPSubscriptionHandle.java
@@ -0,0 +1,54 @@
+/*
+ * 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.profinet.dcp.protocol;
+
+import org.apache.plc4x.java.profinet.dcp.field.ProfinetDcpField;
+import org.apache.plc4x.java.profinet.dcp.readwrite.ProfinetFrame;
+import org.apache.plc4x.java.spi.messages.PlcSubscriber;
+import org.apache.plc4x.java.spi.model.DefaultPlcSubscriptionHandle;
+
+public class ProfinetDCPSubscriptionHandle extends DefaultPlcSubscriptionHandle {
+
+    private final String name;
+    private final ProfinetDcpField field;
+
+    public ProfinetDCPSubscriptionHandle(PlcSubscriber subscriber, String name, ProfinetDcpField field) {
+        super(subscriber);
+        this.name = name;
+        this.field = field;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    boolean matches(ProfinetFrame pdu) {
+        // TODO implement matching logic
+        return false;
+    }
+
+    public ProfinetDcpField getField() {
+        return field;
+    }
+
+    public String toString() {
+        return "ProfinetDCPSubscriptionHandle [service=" + field + "]";
+    }
+
+}

[plc4x] 02/04: Re-organize tests to match implementation #PLC4X-286

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

ldywicki pushed a commit to branch feature/profinet2
in repository https://gitbox.apache.org/repos/asf/plc4x.git

commit a502f542c1f0cdf3006ab00b78a582d2076d4f2e
Author: Adi <ad...@gmail.com>
AuthorDate: Fri Mar 26 15:39:11 2021 +0100

    Re-organize tests to match implementation #PLC4X-286
    
    Also added padding to reduce default frame size from 1024 to 60
---
 .../protocols/profinet/profinet.dcp.mspec          |   8 +-
 .../test/resources/testsuite/ProfinetDCPBlock.xml  | 112 ++++++-----
 .../test/resources/testsuite/ProfinetDCPFrame.xml  | 220 ++++++++++-----------
 3 files changed, 174 insertions(+), 166 deletions(-)

diff --git a/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec b/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
index c32e582..fb0595d 100644
--- a/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
+++ b/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
@@ -27,6 +27,7 @@
 [type 'ProfinetFrame'
     [enum FrameType 'frameType'                ]
     [simple ProfinetData 'frame'  ['frameType']]
+    [padding uint 8 'alignment' '0x00' '40 - frame.lengthInBytes']
 ]
 
 [discriminatedType 'ProfinetData' [FrameType 'frameType']
@@ -95,11 +96,14 @@
             [enum DevicePropertiesSubOption 'subOption'         ]
         ]
         ['DevicePropertiesSubOption.DEVICE_INSTANCE' DeviceInstance
-            [simple uint 8 'instanceLow'         ]
             [simple uint 8 'instanceHigh'        ]
+            [simple uint 8 'instanceLow'         ]
         ]
     ]
-    [padding uint 8                 'pad' '0x00' '(length % 2 == 0 ? 0 : 1)']
+    // Padding using 0..1 old format
+    //[padding uint 8                 'pad' '0x00' '(length % 2 == 0 ? 0 : 1)']
+    // Padding using 0..N new format
+    [padding uint 8                 'pad' '0x00' 'length % 2']
 ]
 
 
diff --git a/sandbox/test-java-profinet-driver/src/test/resources/testsuite/ProfinetDCPBlock.xml b/sandbox/test-java-profinet-driver/src/test/resources/testsuite/ProfinetDCPBlock.xml
index faf3f95..a0f9e22 100644
--- a/sandbox/test-java-profinet-driver/src/test/resources/testsuite/ProfinetDCPBlock.xml
+++ b/sandbox/test-java-profinet-driver/src/test/resources/testsuite/ProfinetDCPBlock.xml
@@ -22,26 +22,23 @@
   <name>Profinet DCP Block</name>
 
   <testcase>
-    <name>Profinet: AllSelector</name>
-    <raw>ff0000</raw>
+    <name>Wireshark Block: All/All</name>
+    <raw>ffff0000</raw>
     <root-type>DCPBlock</root-type>
-    <parser-arguments>
-      <blockType>ALL_SELECTOR</blockType>
-    </parser-arguments>
     <xml>
-      <AllSelector className="org.apache.plc4x.java.profinet.dcp.readwrite.AllSelector"/>
+      <AllSelector className="org.apache.plc4x.java.profinet.dcp.readwrite.AllSelector">
+        <blockType>ALL_SELECTOR</blockType>
+      </AllSelector>
     </xml>
   </testcase>
 
   <testcase>
-    <name>Profinet: Device Options</name>
-    <raw>05000400000207</raw>
+    <name>Wireshark Block: Device/Device Options</name>
+    <raw>0205000400000207</raw>
     <root-type>DCPBlock</root-type>
-    <parser-arguments>
-      <blockType>DEVICE_PROPERTIES</blockType>
-    </parser-arguments>
     <xml>
       <DeviceProperties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
+        <blockType>DEVICE_PROPERTIES</blockType>
         <subOption>DEVICE_OPTIONS</subOption>
         <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceOptions">
           <length>4</length>
@@ -54,20 +51,18 @@
   </testcase>
 
   <testcase>
-    <name>Profinet: Station Type</name>
-    <raw>010009000053372d31323030</raw>
+    <name>Wireshark Block: Device/Manufacturer specific (Station Type)</name>
+    <raw>0201000c000041584320462032313532</raw>
     <root-type>DCPBlock</root-type>
-    <parser-arguments>
-      <blockType>DEVICE_PROPERTIES</blockType>
-    </parser-arguments>
     <xml>
       <DeviceProperties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
+        <blockType>DEVICE_PROPERTIES</blockType>
         <subOption>STATION_TYPE</subOption>
         <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.StationType">
-          <length>9</length>
+          <length>12</length>
           <info>0</info>
           <vendorNameForDevice className="org.apache.plc4x.java.profinet.dcp.readwrite.Text">
-            <text>S7-1200</text>
+            <text>AXC F 2152</text>
           </vendorNameForDevice>
         </properties>
       </DeviceProperties>
@@ -75,20 +70,18 @@
   </testcase>
 
   <testcase>
-    <name>Profinet: Station Name</name>
-    <raw>02000c0000706c6378623164306564</raw>
+    <name>Wireshark Block: Device/NameOfStation</name>
+    <raw>0202000e00006178632d662d323135322d31</raw>
     <root-type>DCPBlock</root-type>
-    <parser-arguments>
-      <blockType>DEVICE_PROPERTIES</blockType>
-    </parser-arguments>
     <xml>
       <DeviceProperties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
+        <blockType>DEVICE_PROPERTIES</blockType>
         <subOption>STATION_NAME</subOption>
         <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.StationName">
-          <length>12</length>
+          <length>14</length>
           <info>0</info>
           <name className="org.apache.plc4x.java.profinet.dcp.readwrite.Text">
-            <text>plcxb1d0ed</text>
+            <text>axc-f-2152-1</text>
           </name>
         </properties>
       </DeviceProperties>
@@ -96,81 +89,93 @@
   </testcase>
 
   <testcase>
-    <name>Profinet: Device ID</name>
-    <raw>0300060000002a010d</raw>
+    <name>Wireshark Block: Device/NameOfStation (with padding)</name>
+    <raw>0202002300006d313835302e786430312d34353575312d65787868617573742e66616e6435343500</raw>
     <root-type>DCPBlock</root-type>
-    <parser-arguments>
-      <blockType>DEVICE_PROPERTIES</blockType>
-    </parser-arguments>
     <xml>
       <DeviceProperties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
+        <blockType>DEVICE_PROPERTIES</blockType>
+        <subOption>STATION_NAME</subOption>
+        <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.StationName">
+          <length>35</length>
+          <info>0</info>
+          <name className="org.apache.plc4x.java.profinet.dcp.readwrite.Text">
+            <text>m1850.xd01-455u1-exxhaust.fand545</text>
+          </name>
+        </properties>
+      </DeviceProperties>
+    </xml>
+  </testcase>
+
+  <testcase>
+    <name>Wireshark Block: Device/Device ID</name>
+    <raw>02030006000000b00142</raw>
+    <root-type>DCPBlock</root-type>
+    <xml>
+      <DeviceProperties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
+        <blockType>DEVICE_PROPERTIES</blockType>
         <subOption>DEVICE_ID</subOption>
         <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceId">
           <length>6</length>
           <info>0</info>
-          <vendorId>42</vendorId>
-          <deviceId>269</deviceId>
+          <vendorId>176</vendorId>
+          <deviceId>322</deviceId>
         </properties>
       </DeviceProperties>
     </xml>
   </testcase>
 
   <testcase>
-    <name>Profinet: Device Role</name>
-    <raw>04000400000200</raw>
+    <name>Wireshark Block: Device/Device Role</name>
+    <raw>0204000400000300</raw>
     <root-type>DCPBlock</root-type>
-    <parser-arguments>
-      <blockType>DEVICE_PROPERTIES</blockType>
-    </parser-arguments>
     <xml>
       <DeviceProperties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
+        <blockType>DEVICE_PROPERTIES</blockType>
         <subOption>DEVICE_ROLE</subOption>
         <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceRole">
           <length>4</length>
           <info>0</info>
-          <role>2</role>
+          <role>3</role>
         </properties>
       </DeviceProperties>
     </xml>
   </testcase>
 
   <testcase>
-    <name>Profinet: Device Instance</name>
-    <raw>07000400000064</raw>
+    <name>Wireshark Block: Device/Device Instance</name>
+    <raw>0207000400000064</raw>
     <root-type>DCPBlock</root-type>
-    <parser-arguments>
-      <blockType>DEVICE_PROPERTIES</blockType>
-    </parser-arguments>
     <xml>
       <DeviceProperties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
+        <blockType>DEVICE_PROPERTIES</blockType>
         <subOption>DEVICE_INSTANCE</subOption>
         <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceInstance">
           <length>4</length>
           <info>0</info>
-          <instanceLow>0</instanceLow>
-          <instanceHigh>100</instanceHigh>
+          <instanceHigh>0</instanceHigh>
+          <instanceLow>100</instanceLow>
         </properties>
       </DeviceProperties>
     </xml>
   </testcase>
 
   <testcase>
-    <name>Profinet: IP/IP</name>
-    <raw>02000e0001c0a802f1ffffff00c0a80201</raw>
+    <name>Wireshark Block: IP/IP</name>
+<!--    <raw>02000e0001c0a802f1ffffff00c0a80201</raw>-->
+    <raw>0102000e0001c0a80065ffffff00c0a80001</raw>
     <root-type>DCPBlock</root-type>
-    <parser-arguments>
-      <blockType>IP</blockType>
-    </parser-arguments>
     <xml>
       <IP className="org.apache.plc4x.java.profinet.dcp.readwrite.IP">
+        <blockType>IP</blockType>
         <subOption>IP_PARAMETER</subOption>
         <length>14</length>
         <info>1</info>
         <ipAddress className="org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address">
           <octet1>192</octet1>
           <octet2>168</octet2>
-          <octet3>2</octet3>
-          <octet4>241</octet4>
+          <octet3>0</octet3>
+          <octet4>101</octet4>
         </ipAddress>
         <subnetMask className="org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address">
           <octet1>255</octet1>
@@ -181,12 +186,11 @@
         <standardGateway className="org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address">
           <octet1>192</octet1>
           <octet2>168</octet2>
-          <octet3>2</octet3>
+          <octet3>0</octet3>
           <octet4>1</octet4>
         </standardGateway>
       </IP>
     </xml>
   </testcase>
 
-
 </test:testsuite>
\ No newline at end of file
diff --git a/sandbox/test-java-profinet-driver/src/test/resources/testsuite/ProfinetDCPFrame.xml b/sandbox/test-java-profinet-driver/src/test/resources/testsuite/ProfinetDCPFrame.xml
index 71c9c4c..9b78989 100644
--- a/sandbox/test-java-profinet-driver/src/test/resources/testsuite/ProfinetDCPFrame.xml
+++ b/sandbox/test-java-profinet-driver/src/test/resources/testsuite/ProfinetDCPFrame.xml
@@ -22,8 +22,10 @@
   <name>Profinet DCP Frame</name>
 
   <testcase>
-    <name>Profinet: DCP Identify Multicast</name>
-    <raw>fefe05000501000f00800004ffff0000</raw>
+    <name>Profinet: DCP Identify Multicast Request (with padding=len 40)</name>
+    <raw>fefe05000000000100010004ffff00000000000000000000000000000000000000000000000000000000</raw>
+<!--    Whithout padding:-->
+<!--    <raw>fefe05000000000100010004ffff0000</raw>-->
     <root-type>ProfinetFrame</root-type>
     <xml>
       <ProfinetFrame className="org.apache.plc4x.java.profinet.dcp.readwrite.ProfinetFrame">
@@ -31,8 +33,8 @@
         <frame className="org.apache.plc4x.java.profinet.dcp.readwrite.DcpIdentRequestPDU">
           <serviceId>IDENTIFY</serviceId>
           <serviceType>REQUEST</serviceType>
-          <xid>83951631</xid>
-          <responseDelay>128</responseDelay>
+          <xid>1</xid>
+          <responseDelay>1</responseDelay>
           <dcpDataLength>4</dcpDataLength>
           <blocks>
             <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.AllSelector">
@@ -44,111 +46,109 @@
     </xml>
   </testcase>
 
-  <testcase>
-    <name>Profinet: DCP Identify Response</name>
-    <raw>feff05010501000f00000052020500040000020702010009000053372d31323030000202000c0000706c6378623164306564020300060000002a010d020400040000020002070004000000640102000e0001c0a802f1ffffff00c0a80201</raw>
-    <root-type>ProfinetFrame</root-type>
-    <xml>
-      <ProfinetFrame className="org.apache.plc4x.java.profinet.dcp.readwrite.ProfinetFrame">
-        <frameType>IDENTIFY_RESPONSE</frameType>
-        <frame className="org.apache.plc4x.java.profinet.dcp.readwrite.DcpIdentResponsePDU">
-          <serviceId>IDENTIFY</serviceId>
-          <serviceType>RESPONSE_SUCCESS</serviceType>
-          <xid>83951631</xid>
-          <responseDelay>0</responseDelay>
-          <dcpDataLength>82</dcpDataLength>
-          <blocks>
-            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
-              <blockType>DEVICE_PROPERTIES</blockType>
-              <subOption>DEVICE_OPTIONS</subOption>
-              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceOptions">
-                <length>4</length>
-                <info>0</info>
-                <blockType>DEVICE_PROPERTIES</blockType>
-                <subOption>DEVICE_INSTANCE</subOption>
-              </properties>
-            </blocks>
-            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
-              <blockType>DEVICE_PROPERTIES</blockType>
-              <subOption>STATION_TYPE</subOption>
-              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.StationType">
-                <length>9</length>
-                <info>0</info>
-                <vendorNameForDevice className="org.apache.plc4x.java.profinet.dcp.readwrite.Text">
-                  <text>S7-1200</text>
-                </vendorNameForDevice>
-              </properties>
-            </blocks>
-            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
-              <blockType>DEVICE_PROPERTIES</blockType>
-              <subOption>STATION_NAME</subOption>
-              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.StationName">
-                <length>12</length>
-                <info>0</info>
-                <name className="org.apache.plc4x.java.profinet.dcp.readwrite.Text">
-                  <text>plcxb1d0ed</text>
-                </name>
-              </properties>
-            </blocks>
-            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
-              <blockType>DEVICE_PROPERTIES</blockType>
-              <subOption>DEVICE_ID</subOption>
-              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceId">
-                <length>6</length>
-                <info>0</info>
-                <vendorId>42</vendorId>
-                <deviceId>269</deviceId>
-              </properties>
-            </blocks>
-            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
-              <blockType>DEVICE_PROPERTIES</blockType>
-              <subOption>DEVICE_ROLE</subOption>
-              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceRole">
-                <length>4</length>
-                <info>0</info>
-                <role>2</role>
-              </properties>
-            </blocks>
-            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">
-              <blockType>DEVICE_PROPERTIES</blockType>
-              <subOption>DEVICE_INSTANCE</subOption>
-              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceInstance">
-                <length>4</length>
-                <info>0</info>
-                <instanceLow>0</instanceLow>
-                <instanceHigh>100</instanceHigh>
-              </properties>
-            </blocks>
-            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.IP">
-              <blockType>IP</blockType>
-              <subOption>IP_PARAMETER</subOption>
-              <length>14</length>
-              <info>1</info>
-              <ipAddress className="org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address">
-                <octet1>192</octet1>
-                <octet2>168</octet2>
-                <octet3>2</octet3>
-                <octet4>241</octet4>
-              </ipAddress>
-              <subnetMask className="org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address">
-                <octet1>255</octet1>
-                <octet2>255</octet2>
-                <octet3>255</octet3>
-                <octet4>0</octet4>
-              </subnetMask>
-              <standardGateway className="org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address">
-                <octet1>192</octet1>
-                <octet2>168</octet2>
-                <octet3>2</octet3>
-                <octet4>1</octet4>
-              </standardGateway>
-            </blocks>
-          </blocks>
-        </frame>
-      </ProfinetFrame>
-    </xml>
-  </testcase>
-
-
+<!--  <testcase>-->
+<!--    <name>Profinet: DCP Identify Response</name>-->
+<!--    <raw>feff05010501000f00000052020500040000020702010009000053372d31323030000202000c0000706c6378623164306564020300060000002a010d020400040000020002070004000000640102000e0001c0a802f1ffffff00c0a80201</raw>-->
+<!--    <root-type>ProfinetFrame</root-type>-->
+<!--    <xml>-->
+<!--      <ProfinetFrame className="org.apache.plc4x.java.profinet.dcp.readwrite.ProfinetFrame">-->
+<!--        <frameType>IDENTIFY_RESPONSE</frameType>-->
+<!--        <frame className="org.apache.plc4x.java.profinet.dcp.readwrite.DcpIdentResponsePDU">-->
+<!--          <serviceId>IDENTIFY</serviceId>-->
+<!--          <serviceType>RESPONSE_SUCCESS</serviceType>-->
+<!--          <xid>83951631</xid>-->
+<!--          <responseDelay>0</responseDelay>-->
+<!--          <dcpDataLength>82</dcpDataLength>-->
+<!--          <blocks>-->
+<!--            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">-->
+<!--              <blockType>DEVICE_PROPERTIES</blockType>-->
+<!--              <subOption>DEVICE_OPTIONS</subOption>-->
+<!--              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceOptions">-->
+<!--                <length>4</length>-->
+<!--                <info>0</info>-->
+<!--                <blockType>DEVICE_PROPERTIES</blockType>-->
+<!--                <subOption>DEVICE_INSTANCE</subOption>-->
+<!--              </properties>-->
+<!--            </blocks>-->
+<!--            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">-->
+<!--              <blockType>DEVICE_PROPERTIES</blockType>-->
+<!--              <subOption>STATION_TYPE</subOption>-->
+<!--              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.StationType">-->
+<!--                <length>9</length>-->
+<!--                <info>0</info>-->
+<!--                <vendorNameForDevice className="org.apache.plc4x.java.profinet.dcp.readwrite.Text">-->
+<!--                  <text>S7-1200</text>-->
+<!--                </vendorNameForDevice>-->
+<!--              </properties>-->
+<!--            </blocks>-->
+<!--            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">-->
+<!--              <blockType>DEVICE_PROPERTIES</blockType>-->
+<!--              <subOption>STATION_NAME</subOption>-->
+<!--              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.StationName">-->
+<!--                <length>12</length>-->
+<!--                <info>0</info>-->
+<!--                <name className="org.apache.plc4x.java.profinet.dcp.readwrite.Text">-->
+<!--                  <text>plcxb1d0ed</text>-->
+<!--                </name>-->
+<!--              </properties>-->
+<!--            </blocks>-->
+<!--            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">-->
+<!--              <blockType>DEVICE_PROPERTIES</blockType>-->
+<!--              <subOption>DEVICE_ID</subOption>-->
+<!--              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceId">-->
+<!--                <length>6</length>-->
+<!--                <info>0</info>-->
+<!--                <vendorId>42</vendorId>-->
+<!--                <deviceId>269</deviceId>-->
+<!--              </properties>-->
+<!--            </blocks>-->
+<!--            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">-->
+<!--              <blockType>DEVICE_PROPERTIES</blockType>-->
+<!--              <subOption>DEVICE_ROLE</subOption>-->
+<!--              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceRole">-->
+<!--                <length>4</length>-->
+<!--                <info>0</info>-->
+<!--                <role>2</role>-->
+<!--              </properties>-->
+<!--            </blocks>-->
+<!--            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties">-->
+<!--              <blockType>DEVICE_PROPERTIES</blockType>-->
+<!--              <subOption>DEVICE_INSTANCE</subOption>-->
+<!--              <properties className="org.apache.plc4x.java.profinet.dcp.readwrite.DeviceInstance">-->
+<!--                <length>4</length>-->
+<!--                <info>0</info>-->
+<!--                <instanceLow>0</instanceLow>-->
+<!--                <instanceHigh>100</instanceHigh>-->
+<!--              </properties>-->
+<!--            </blocks>-->
+<!--            <blocks className="org.apache.plc4x.java.profinet.dcp.readwrite.IP">-->
+<!--              <blockType>IP</blockType>-->
+<!--              <subOption>IP_PARAMETER</subOption>-->
+<!--              <length>14</length>-->
+<!--              <info>1</info>-->
+<!--              <ipAddress className="org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address">-->
+<!--                <octet1>192</octet1>-->
+<!--                <octet2>168</octet2>-->
+<!--                <octet3>2</octet3>-->
+<!--                <octet4>241</octet4>-->
+<!--              </ipAddress>-->
+<!--              <subnetMask className="org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address">-->
+<!--                <octet1>255</octet1>-->
+<!--                <octet2>255</octet2>-->
+<!--                <octet3>255</octet3>-->
+<!--                <octet4>0</octet4>-->
+<!--              </subnetMask>-->
+<!--              <standardGateway className="org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address">-->
+<!--                <octet1>192</octet1>-->
+<!--                <octet2>168</octet2>-->
+<!--                <octet3>2</octet3>-->
+<!--                <octet4>1</octet4>-->
+<!--              </standardGateway>-->
+<!--            </blocks>-->
+<!--          </blocks>-->
+<!--        </frame>-->
+<!--      </ProfinetFrame>-->
+<!--    </xml>-->
+<!--  </testcase>-->
 
 </test:testsuite>
\ No newline at end of file