You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by cd...@apache.org on 2023/06/11 10:29:02 UTC

[plc4x] 02/02: chore(plc4j/profinet-ng): - Changed the ProfinetDriverContext to no longer contain the raw device profile data, but pre-processed structures based on slot and subslot. - Simplified the browse logic - Started implementing the subscription logic - Added an empty dummy for PnIoCm_Block_ReqPluginAlarmApplicationReady as it seems that this is sent from the remote devices

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

cdutz pushed a commit to branch chore/profinet-phase-3
in repository https://gitbox.apache.org/repos/asf/plc4x.git

commit 67ed07a98381d5cba823a6d7afe9234e0e7a7f8e
Author: Christofer Dutz <cd...@apache.org>
AuthorDate: Sun Jun 11 12:07:56 2023 +0200

    chore(plc4j/profinet-ng):
     - Changed the ProfinetDriverContext to no longer contain the raw device profile data, but pre-processed structures based on slot and subslot.
     - Simplified the browse logic
     - Started implementing the subscription logic
     - Added an empty dummy for PnIoCm_Block_ReqPluginAlarmApplicationReady as it seems that this is sent from the remote devices
---
 .../java/profinet/readwrite/PnIoCm_Block.java      |   4 +
 ...nIoCm_Block_ReqPluginAlarmApplicationReady.java | 123 ++++++++++
 .../profinet/context/ProfinetDriverContext.java    |  40 ++--
 .../java/profinet/gsdml/ProfinetDataItem.java      |   4 +
 .../profinet/protocol/ProfinetProtocolLogic.java   | 259 ++++++++++++---------
 .../profinet/utils/ProfinetDataTypeMapper.java     |   6 +-
 .../java/profinet/readwrite/PnIoCm_Block.java      |   4 +
 ...nIoCm_Block_ReqPluginAlarmApplicationReady.java | 123 ++++++++++
 .../main/resources/protocols/profinet/pnio.mspec   |   3 +
 9 files changed, 432 insertions(+), 134 deletions(-)

diff --git a/plc4j/drivers/profinet-ng/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block.java b/plc4j/drivers/profinet-ng/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block.java
index e4bf5c8538..4065b85537 100644
--- a/plc4j/drivers/profinet-ng/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block.java
+++ b/plc4j/drivers/profinet-ng/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block.java
@@ -149,6 +149,10 @@ public abstract class PnIoCm_Block implements Message {
       builder = PnIoCm_Block_ArServer.staticParsePnIoCm_BlockBuilder(readBuffer);
     } else if (EvaluationHelper.equals(blockType, PnIoCm_BlockType.REAL_IDENTIFICATION_DATA)) {
       builder = PnIoCm_Block_RealIdentificationData.staticParsePnIoCm_BlockBuilder(readBuffer);
+    } else if (EvaluationHelper.equals(
+        blockType, PnIoCm_BlockType.IOD_BLOCK_REQ_PLUGIN_ALARM_APPLICATION_READY)) {
+      builder =
+          PnIoCm_Block_ReqPluginAlarmApplicationReady.staticParsePnIoCm_BlockBuilder(readBuffer);
     } else if (EvaluationHelper.equals(blockType, PnIoCm_BlockType.I_AND_M_0)) {
       builder = PnIoCm_Block_IAndM0.staticParsePnIoCm_BlockBuilder(readBuffer);
     } else if (EvaluationHelper.equals(blockType, PnIoCm_BlockType.I_AND_M_1)) {
diff --git a/plc4j/drivers/profinet-ng/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block_ReqPluginAlarmApplicationReady.java b/plc4j/drivers/profinet-ng/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block_ReqPluginAlarmApplicationReady.java
new file mode 100644
index 0000000000..73b9d230c4
--- /dev/null
+++ b/plc4j/drivers/profinet-ng/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block_ReqPluginAlarmApplicationReady.java
@@ -0,0 +1,123 @@
+/*
+ * 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
+ *
+ *   https://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.readwrite;
+
+import static org.apache.plc4x.java.spi.codegen.fields.FieldReaderFactory.*;
+import static org.apache.plc4x.java.spi.codegen.fields.FieldWriterFactory.*;
+import static org.apache.plc4x.java.spi.codegen.io.DataReaderFactory.*;
+import static org.apache.plc4x.java.spi.codegen.io.DataWriterFactory.*;
+import static org.apache.plc4x.java.spi.generation.StaticHelper.*;
+
+import java.time.*;
+import java.util.*;
+import org.apache.plc4x.java.api.exceptions.*;
+import org.apache.plc4x.java.api.value.*;
+import org.apache.plc4x.java.spi.codegen.*;
+import org.apache.plc4x.java.spi.codegen.fields.*;
+import org.apache.plc4x.java.spi.codegen.io.*;
+import org.apache.plc4x.java.spi.generation.*;
+
+// Code generated by code-generation. DO NOT EDIT.
+
+public class PnIoCm_Block_ReqPluginAlarmApplicationReady extends PnIoCm_Block implements Message {
+
+  // Accessors for discriminator values.
+  public PnIoCm_BlockType getBlockType() {
+    return PnIoCm_BlockType.IOD_BLOCK_REQ_PLUGIN_ALARM_APPLICATION_READY;
+  }
+
+  public PnIoCm_Block_ReqPluginAlarmApplicationReady() {
+    super();
+  }
+
+  @Override
+  protected void serializePnIoCm_BlockChild(WriteBuffer writeBuffer) throws SerializationException {
+    PositionAware positionAware = writeBuffer;
+    boolean _lastItem = ThreadLocalHelper.lastItemThreadLocal.get();
+    writeBuffer.pushContext("PnIoCm_Block_ReqPluginAlarmApplicationReady");
+
+    writeBuffer.popContext("PnIoCm_Block_ReqPluginAlarmApplicationReady");
+  }
+
+  @Override
+  public int getLengthInBytes() {
+    return (int) Math.ceil((float) getLengthInBits() / 8.0);
+  }
+
+  @Override
+  public int getLengthInBits() {
+    int lengthInBits = super.getLengthInBits();
+    PnIoCm_Block_ReqPluginAlarmApplicationReady _value = this;
+    boolean _lastItem = ThreadLocalHelper.lastItemThreadLocal.get();
+
+    return lengthInBits;
+  }
+
+  public static PnIoCm_BlockBuilder staticParsePnIoCm_BlockBuilder(ReadBuffer readBuffer)
+      throws ParseException {
+    readBuffer.pullContext("PnIoCm_Block_ReqPluginAlarmApplicationReady");
+    PositionAware positionAware = readBuffer;
+    boolean _lastItem = ThreadLocalHelper.lastItemThreadLocal.get();
+
+    readBuffer.closeContext("PnIoCm_Block_ReqPluginAlarmApplicationReady");
+    // Create the instance
+    return new PnIoCm_Block_ReqPluginAlarmApplicationReadyBuilderImpl();
+  }
+
+  public static class PnIoCm_Block_ReqPluginAlarmApplicationReadyBuilderImpl
+      implements PnIoCm_Block.PnIoCm_BlockBuilder {
+
+    public PnIoCm_Block_ReqPluginAlarmApplicationReadyBuilderImpl() {}
+
+    public PnIoCm_Block_ReqPluginAlarmApplicationReady build() {
+      PnIoCm_Block_ReqPluginAlarmApplicationReady pnIoCm_Block_ReqPluginAlarmApplicationReady =
+          new PnIoCm_Block_ReqPluginAlarmApplicationReady();
+      return pnIoCm_Block_ReqPluginAlarmApplicationReady;
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof PnIoCm_Block_ReqPluginAlarmApplicationReady)) {
+      return false;
+    }
+    PnIoCm_Block_ReqPluginAlarmApplicationReady that =
+        (PnIoCm_Block_ReqPluginAlarmApplicationReady) o;
+    return super.equals(that) && true;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode());
+  }
+
+  @Override
+  public String toString() {
+    WriteBufferBoxBased writeBufferBoxBased = new WriteBufferBoxBased(true, true);
+    try {
+      writeBufferBoxBased.writeSerializable(this);
+    } catch (SerializationException e) {
+      throw new RuntimeException(e);
+    }
+    return "\n" + writeBufferBoxBased.getBox().toString() + "\n";
+  }
+}
diff --git a/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/context/ProfinetDriverContext.java b/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/context/ProfinetDriverContext.java
index 684dd86f72..3893fac0b6 100644
--- a/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/context/ProfinetDriverContext.java
+++ b/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/context/ProfinetDriverContext.java
@@ -21,18 +21,16 @@ package org.apache.plc4x.java.profinet.context;
 
 import org.apache.commons.codec.DecoderException;
 import org.apache.commons.codec.binary.Hex;
-import org.apache.plc4x.java.profinet.gsdml.ProfinetISO15745Profile;
+import org.apache.plc4x.java.profinet.gsdml.ProfinetDeviceAccessPointItem;
+import org.apache.plc4x.java.profinet.gsdml.ProfinetModuleItem;
+import org.apache.plc4x.java.profinet.gsdml.ProfinetVirtualSubmoduleItem;
 import org.apache.plc4x.java.profinet.readwrite.DceRpc_ActivityUuid;
-import org.apache.plc4x.java.profinet.readwrite.DceRpc_Packet;
-import org.apache.plc4x.java.profinet.readwrite.PnIoCm_Block_RealIdentificationData;
 import org.apache.plc4x.java.profinet.readwrite.Uuid;
-import org.apache.plc4x.java.spi.ConversationContext;
 import org.apache.plc4x.java.spi.context.DriverContext;
 import org.apache.plc4x.java.spi.generation.*;
 
-import java.net.DatagramSocket;
-import java.net.SocketException;
 import java.util.List;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -47,13 +45,13 @@ public class ProfinetDriverContext implements DriverContext {
     private int deviceId;
 
     private String dapId;
+    private ProfinetDeviceAccessPointItem dap;
+    private Map<Integer, ProfinetModuleItem> moduleIndex;
+    private Map<Integer, Map<Integer, ProfinetVirtualSubmoduleItem>> submoduleIndex;
 
     private int localPort;
     private int remotePortImplicitCommunication;
 
-    private ProfinetISO15745Profile deviceProfile;
-    private PnIoCm_Block_RealIdentificationData identificationData;
-
     // PN-CM Related:
     private final DceRpc_ActivityUuid activityUuid;
     private final Uuid applicationRelationUuid;
@@ -142,20 +140,28 @@ public class ProfinetDriverContext implements DriverContext {
         this.dapId = dapId;
     }
 
-    public ProfinetISO15745Profile getDeviceProfile() {
-        return deviceProfile;
+    public ProfinetDeviceAccessPointItem getDap() {
+        return dap;
+    }
+
+    public void setDap(ProfinetDeviceAccessPointItem dap) {
+        this.dap = dap;
+    }
+
+    public Map<Integer, ProfinetModuleItem> getModuleIndex() {
+        return moduleIndex;
     }
 
-    public void setDeviceProfile(ProfinetISO15745Profile deviceProfile) {
-        this.deviceProfile = deviceProfile;
+    public void setModuleIndex(Map<Integer, ProfinetModuleItem> moduleIndex) {
+        this.moduleIndex = moduleIndex;
     }
 
-    public PnIoCm_Block_RealIdentificationData getIdentificationData() {
-        return identificationData;
+    public Map<Integer, Map<Integer, ProfinetVirtualSubmoduleItem>> getSubmoduleIndex() {
+        return submoduleIndex;
     }
 
-    public void setIdentificationData(PnIoCm_Block_RealIdentificationData identificationData) {
-        this.identificationData = identificationData;
+    public void setSubmoduleIndex(Map<Integer, Map<Integer, ProfinetVirtualSubmoduleItem>> submoduleIndex) {
+        this.submoduleIndex = submoduleIndex;
     }
 
     public DceRpc_ActivityUuid getActivityUuid() {
diff --git a/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/gsdml/ProfinetDataItem.java b/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/gsdml/ProfinetDataItem.java
index 2ceee0ce51..33d4ed857d 100644
--- a/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/gsdml/ProfinetDataItem.java
+++ b/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/gsdml/ProfinetDataItem.java
@@ -54,6 +54,10 @@ public class ProfinetDataItem {
         return textId;
     }
 
+    public void setTextId(String textId) {
+        this.textId = textId;
+    }
+
     public int getLength() {
         return length;
     }
diff --git a/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/protocol/ProfinetProtocolLogic.java b/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/protocol/ProfinetProtocolLogic.java
index d95029f517..b1de93aa68 100644
--- a/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/protocol/ProfinetProtocolLogic.java
+++ b/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/protocol/ProfinetProtocolLogic.java
@@ -21,9 +21,8 @@ package org.apache.plc4x.java.profinet.protocol;
 
 import org.apache.plc4x.java.api.exceptions.PlcConnectionException;
 import org.apache.plc4x.java.api.exceptions.PlcRuntimeException;
-import org.apache.plc4x.java.api.messages.PlcBrowseItem;
-import org.apache.plc4x.java.api.messages.PlcBrowseRequest;
-import org.apache.plc4x.java.api.messages.PlcBrowseResponse;
+import org.apache.plc4x.java.api.messages.*;
+import org.apache.plc4x.java.api.model.PlcSubscriptionTag;
 import org.apache.plc4x.java.api.types.PlcResponseCode;
 import org.apache.plc4x.java.profinet.config.ProfinetConfiguration;
 import org.apache.plc4x.java.profinet.context.ProfinetDriverContext;
@@ -108,7 +107,6 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
                 context.getChannel().close();
                 return;
             }
-            driverContext.setDeviceProfile(deviceProfile);
 
             // If the user provided a DAP id in the connection string, use that (after checking that it exists)
             if (configuration.dapId != null) {
@@ -121,7 +119,6 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
                 if(driverContext.getDapId() == null) {
                     logger.error("Couldn't find requested device access points (DAP): {}", configuration.dapId);
                     context.getChannel().close();
-                    return;
                 }
             }
 
@@ -136,11 +133,16 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
             // correct port, and we can calculate the object id based on the vendor id and device id, which we
             // already have from the discovery.
 
-            // If there's more than one DAP, read the I&M0 block to get the order number,
-            // which allows us to find out which DAP to use.
-            else if(deviceProfile.getProfileBody().getApplicationProcess().getDeviceAccessPointList().size() > 1) {
-                RawSocketChannel pnChannel = ((RawSocketChannel) context.getChannel());
+            // If there's more at least one DAP, read the real identification data and prepare all the data-structures.
+            else if(deviceProfile.getProfileBody().getApplicationProcess().getDeviceAccessPointList().size() > 0) {
+                // Build an index of the String names.
+                Map<String, String> textMapping = new HashMap<>();
+                for (ProfinetTextIdValue profinetTextIdValue : deviceProfile.getProfileBody().getApplicationProcess().getExternalTextList().getPrimaryLanguage().getText()) {
+                    textMapping.put(profinetTextIdValue.getTextId(), profinetTextIdValue.getValue());
+                }
+
                 // Try to read the RealIdentificationData ...
+                RawSocketChannel pnChannel = ((RawSocketChannel) context.getChannel());
                 CompletableFuture<PnIoCm_Block_RealIdentificationData> future1 =
                     PnDcpPacketFactory.sendRealIdentificationDataRequest(context, pnChannel, driverContext);
                 future1.whenComplete((realIdentificationData, throwable1) -> {
@@ -149,21 +151,26 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
                         context.getChannel().close();
                         return;
                     }
-                    driverContext.setIdentificationData(realIdentificationData);
 
-                    // Get the module identification number of slot 0 (Which is always the DAP)
-                    long dapModuleIdentificationNumber = 0;
-                    outerLoop:
+                    // Build an index of all identification numbers for each slot and subslot the device supports.
+                    Map<Integer, Long> slotModuleIdentificationNumbers = new HashMap<>();
+                    Map<Integer, Map<Integer, Long>> subslotModuleIdentificationNumbers = new HashMap<>();
                     for (PnIoCm_RealIdentificationApi api : realIdentificationData.getApis()) {
-                        for (PnIoCm_RealIdentificationApi_Slot slot : api.getSlots()) {
-                            if(slot.getSlotNumber() == 0) {
-                                dapModuleIdentificationNumber = slot.getModuleIdentNumber();
-                                break outerLoop;
+                        for (PnIoCm_RealIdentificationApi_Slot curSlot : api.getSlots()) {
+                            slotModuleIdentificationNumbers.put(curSlot.getSlotNumber(), curSlot.getModuleIdentNumber());
+                            if(!subslotModuleIdentificationNumbers.containsKey(curSlot.getSlotNumber())) {
+                                subslotModuleIdentificationNumbers.put(curSlot.getSlotNumber(), new HashMap<>());
+                            }
+                            for (PnIoCm_RealIdentificationApi_Subslot curSubslot : curSlot.getSubslots()) {
+                                subslotModuleIdentificationNumbers.get(curSlot.getSlotNumber()).put(curSubslot.getSubslotNumber(), curSubslot.getSubmoduleIdentNumber());
                             }
                         }
                     }
+
+                    // Get the module identification number of slot 0 (Which is always the DAP)
+                    long dapModuleIdentificationNumber = slotModuleIdentificationNumbers.get(0);
                     if(dapModuleIdentificationNumber == 0){
-                        logger.error("Unable to detect device access point, closing channel...", throwable1);
+                        logger.error("Unable to detect device access point, closing channel...");
                         context.getChannel().close();
                         return;
                     }
@@ -176,17 +183,78 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
                         }
                         long moduleIdentNumber = Long.parseLong(moduleIdentNumberStr, 16);
                         if(moduleIdentNumber == dapModuleIdentificationNumber) {
-                            driverContext.setDapId(curDap.getId());
+                            driverContext.setDap(curDap);
                             break;
                         }
                     }
                     // Abort, if we weren't able to detect a DAP.
-                    if(driverContext.getDapId() == null) {
+                    if(driverContext.getDap() == null) {
                         logger.error("Unable to auto-detect the device access point, closing channel...");
                         context.getChannel().close();
                         return;
                     }
 
+                    // Iterate through all available modules and find the ones we're using and build an index of them.
+                    Map<Integer, ProfinetModuleItem> moduleIndex = new HashMap<>();
+                    Map<Integer, Map<Integer, ProfinetVirtualSubmoduleItem>> submoduleIndex = new HashMap<>();
+                    for (Map.Entry<Integer, Long> moduleEntry : slotModuleIdentificationNumbers.entrySet()) {
+                        int curSlot = moduleEntry.getKey();
+                        // Slot 0 is the DAP, so we'll continue with the next one.
+                        if(curSlot == 0) {
+                            continue;
+                        }
+                        long curModuleIdentifier = moduleEntry.getValue();
+                        // Find the module that has the given module ident number.
+                        for (ProfinetModuleItem curModule : deviceProfile.getProfileBody().getApplicationProcess().getModuleList()) {
+                            String moduleIdentNumberStr = curModule.getModuleIdentNumber();
+                            if(moduleIdentNumberStr.startsWith("0x") || moduleIdentNumberStr.startsWith("0X")) {
+                                moduleIdentNumberStr = moduleIdentNumberStr.substring(2);
+                            }
+                            long moduleIdentNumber = Long.parseLong(moduleIdentNumberStr, 16);
+                            if(curModuleIdentifier == moduleIdentNumber) {
+                                moduleIndex.put(curSlot, curModule);
+
+                                // Now get all submodules of this module.
+                                Map<Integer, Long> curSubmoduleIndex = subslotModuleIdentificationNumbers.get(curSlot);
+                                for (Map.Entry<Integer, Long> submoduleEntry : curSubmoduleIndex.entrySet()) {
+                                    int curSubslot = submoduleEntry.getKey();
+                                    long curSubmoduleIdentNumber = submoduleEntry.getValue();
+                                    for (ProfinetVirtualSubmoduleItem curSubmodule : curModule.getVirtualSubmoduleList()) {
+                                        String submoduleIdentNumberStr = curSubmodule.getSubmoduleIdentNumber();
+                                        if(submoduleIdentNumberStr.startsWith("0x") || submoduleIdentNumberStr.startsWith("0X")) {
+                                            submoduleIdentNumberStr = submoduleIdentNumberStr.substring(2);
+                                        }
+                                        long submoduleIdentNumber = Long.parseLong(submoduleIdentNumberStr, 16);
+                                        if(curSubmoduleIdentNumber == submoduleIdentNumber) {
+                                            if(!submoduleIndex.containsKey(curSlot)) {
+                                                submoduleIndex.put(curSlot, new HashMap<>());
+                                            }
+                                            submoduleIndex.get(curSlot).put(curSubslot, curSubmodule);
+
+                                            // Replace the text-ids with readable values
+                                            for (ProfinetIoDataInput profinetIoDataInput : curSubmodule.getIoData().getInput()) {
+                                                for (ProfinetDataItem profinetDataItem : profinetIoDataInput.getDataItemList()) {
+                                                    if(textMapping.containsKey(profinetDataItem.getTextId())) {
+                                                        profinetDataItem.setTextId(textMapping.get(profinetDataItem.getTextId()));
+                                                    }
+                                                }
+                                            }
+                                            for (ProfinetIoDataOutput profinetIoDataOutput : curSubmodule.getIoData().getOutput()) {
+                                                for (ProfinetDataItem profinetDataItem : profinetIoDataOutput.getDataItemList()) {
+                                                    if(textMapping.containsKey(profinetDataItem.getTextId())) {
+                                                        profinetDataItem.setTextId(textMapping.get(profinetDataItem.getTextId()));
+                                                    }
+                                                }
+                                            }
+                                        }
+                                    }
+                                }
+                            }
+                        }
+                    }
+                    driverContext.setModuleIndex(moduleIndex);
+                    driverContext.setSubmoduleIndex(submoduleIndex);
+
                     context.fireConnected();
                 });
                 // Try to read the I&M0 block
@@ -227,12 +295,6 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
                 });*/
             }
 
-            // If the current device only has one DAP (like most devices), simply use that.
-            else if (deviceProfile.getProfileBody().getApplicationProcess().getDeviceAccessPointList().size() == 1) {
-                driverContext.setDapId(deviceProfile.getProfileBody().getApplicationProcess().getDeviceAccessPointList().get(0).getId());
-                context.fireConnected();
-            }
-
             else {
                 logger.error("GSD descriptor doesn't contain any device access points");
                 context.getChannel().close();
@@ -247,102 +309,45 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
 
     @Override
     public CompletableFuture<PlcBrowseResponse> browse(PlcBrowseRequest browseRequest) {
-        if (driverContext.getDeviceProfile() == null) {
-            return CompletableFuture.failedFuture(new PlcConnectionException("Unable to find GSD file for given device"));
-        }
-        if (driverContext.getDapId() == null) {
-            return CompletableFuture.failedFuture(new PlcConnectionException("DAP not set"));
-        }
-
-        ProfinetISO15745Profile deviceProfile = driverContext.getDeviceProfile();
-
-        // Build an index of all modules.
-        Map<Long, ProfinetModuleItem> moduleMap = new HashMap<>();
-        for (ProfinetModuleItem profinetModuleItem : deviceProfile.getProfileBody().getApplicationProcess().getModuleList()) {
-            String moduleIdentNumberString = profinetModuleItem.getModuleIdentNumber();
-            if(moduleIdentNumberString.startsWith("0x") || moduleIdentNumberString.startsWith("0X")) {
-                moduleIdentNumberString = moduleIdentNumberString.substring(2);
-            }
-            long moduleIdentNumber = Long.parseLong(moduleIdentNumberString,16);
-            moduleMap.put(moduleIdentNumber, profinetModuleItem);
-        }
-
-        // Build an index of the String names.
-        Map<String, String> textMapping = new HashMap<>();
-        for (ProfinetTextIdValue profinetTextIdValue : deviceProfile.getProfileBody().getApplicationProcess().getExternalTextList().getPrimaryLanguage().getText()) {
-            textMapping.put(profinetTextIdValue.getTextId(), profinetTextIdValue.getValue());
-        }
-
         Map<String, PlcResponseCode> responseCodes = new HashMap<>();
         Map<String, List<PlcBrowseItem>> values = new HashMap<>();
         for (String queryName : browseRequest.getQueryNames()) {
             List<PlcBrowseItem> items = new ArrayList<>();
-            for (PnIoCm_RealIdentificationApi api : driverContext.getIdentificationData().getApis()) {
-                for (PnIoCm_RealIdentificationApi_Slot slot : api.getSlots()) {
-                    // Slot 0 is always the DAP module, I haven't come across any DataItems here ...
-                    if(slot.getSlotNumber() == 0) {
-                        continue;
-                    }
-
-                    // Find the matching module.
-                    long moduleIdentNumber = slot.getModuleIdentNumber();
-                    ProfinetModuleItem slotModule = moduleMap.get(moduleIdentNumber);
-                    if(slotModule == null) {
-                        return CompletableFuture.failedFuture(new PlcRuntimeException(
-                            "Module with ident number " + moduleIdentNumber + " not found in GSD."));
-                    }
-
-                    for (PnIoCm_RealIdentificationApi_Subslot subslot : slot.getSubslots()) {
-                        // Find the submodule
-                        ProfinetVirtualSubmoduleItem subSlotSubModule = null;
-                        String moduleIdentNumberHex = String.format("0x%08X", subslot.getSubmoduleIdentNumber());
-                        for (ProfinetVirtualSubmoduleItem virtualSubmoduleItem : slotModule.getVirtualSubmoduleList()) {
-                            if (virtualSubmoduleItem.getSubmoduleIdentNumber().equalsIgnoreCase(moduleIdentNumberHex)) {
-                                subSlotSubModule = virtualSubmoduleItem;
-                                break;
-                            }
-                        }
-                        if (subSlotSubModule == null) {
-                            return CompletableFuture.failedFuture(new PlcRuntimeException(
-                                "SubModule with ident number " + subslot.getSubmoduleIdentNumber() + " not found in GSD."));
-                       }
-
-                        // Add all the input tags.
-                        for (ProfinetIoDataInput profinetIoDataInput : subSlotSubModule.getIoData().getInput()) {
-                            for (int i = 0; i < profinetIoDataInput.getDataItemList().size(); i++) {
-                                ProfinetDataItem profinetDataItem = profinetIoDataInput.getDataItemList().get(i);
-                                ProfinetDataTypeMapper.DataTypeInformation dataTypeInformation =
-                                    ProfinetDataTypeMapper.getPlcValueType(profinetDataItem);
-                                String name = profinetDataItem.getTextId();
-                                // Try to replace the text id with a meaningful value.
-                                if (textMapping.containsKey(name)) {
-                                    name = textMapping.get(name);
-                                }
-                                items.add(new DefaultPlcBrowseItem(new ProfinetTag(
-                                    slot.getSlotNumber(), subslot.getSubslotNumber(), ProfinetTag.Direction.INPUT,
-                                    i, dataTypeInformation.getPlcValueType(), dataTypeInformation.getNumElements()),
-                                    name, false, true, true,
-                                    Collections.emptyMap(), Collections.emptyMap()));
-                            }
+            for(Map.Entry<Integer, Map<Integer, ProfinetVirtualSubmoduleItem>> slotEntry : driverContext.getSubmoduleIndex().entrySet()) {
+                int slot = slotEntry.getKey();
+                for(Map.Entry<Integer, ProfinetVirtualSubmoduleItem> subslotEntry: slotEntry.getValue().entrySet()) {
+                    int subslot = subslotEntry.getKey();
+                    ProfinetVirtualSubmoduleItem subslotModule = subslotEntry.getValue();
+
+                    // Add all the input tags.
+                    for (ProfinetIoDataInput profinetIoDataInput : subslotModule.getIoData().getInput()) {
+                        for (int i = 0; i < profinetIoDataInput.getDataItemList().size(); i++) {
+                            ProfinetDataItem profinetDataItem = profinetIoDataInput.getDataItemList().get(i);
+                            ProfinetDataTypeMapper.DataTypeInformation dataTypeInformation =
+                                ProfinetDataTypeMapper.getPlcValueType(profinetDataItem);
+                            // The ids have been replaced by real textual values in the connection phase.
+                            String name = profinetDataItem.getTextId();
+                            items.add(new DefaultPlcBrowseItem(new ProfinetTag(
+                                slot, subslot, ProfinetTag.Direction.INPUT,
+                                i, dataTypeInformation.getPlcValueType(), dataTypeInformation.getNumElements()),
+                                name, false, true, true,
+                                Collections.emptyMap(), Collections.emptyMap()));
                         }
+                    }
 
-                        // Add all the output tags.
-                        for (ProfinetIoDataOutput profinetIoDataOutput : subSlotSubModule.getIoData().getOutput()) {
-                            for (int i = 0; i < profinetIoDataOutput.getDataItemList().size(); i++) {
-                                ProfinetDataItem profinetDataItem = profinetIoDataOutput.getDataItemList().get(i);
-                                ProfinetDataTypeMapper.DataTypeInformation dataTypeInformation =
-                                    ProfinetDataTypeMapper.getPlcValueType(profinetDataItem);
-                                String name = profinetDataItem.getTextId();
-                                // Try to replace the text id with a meaningful value.
-                                if (textMapping.containsKey(name)) {
-                                    name = textMapping.get(name);
-                                }
-                                items.add(new DefaultPlcBrowseItem(new ProfinetTag(
-                                    slot.getSlotNumber(), subslot.getSubslotNumber(), ProfinetTag.Direction.OUTPUT,
-                                    i, dataTypeInformation.getPlcValueType(), dataTypeInformation.getNumElements()),
-                                    name, false, true, true,
-                                    Collections.emptyMap(), Collections.emptyMap()));
-                            }
+                    // Add all the output tags.
+                    for (ProfinetIoDataOutput profinetIoDataOutput : subslotModule.getIoData().getOutput()) {
+                        for (int i = 0; i < profinetIoDataOutput.getDataItemList().size(); i++) {
+                            ProfinetDataItem profinetDataItem = profinetIoDataOutput.getDataItemList().get(i);
+                            ProfinetDataTypeMapper.DataTypeInformation dataTypeInformation =
+                                ProfinetDataTypeMapper.getPlcValueType(profinetDataItem);
+                            // The ids have been replaced by real textual values in the connection phase.
+                            String name = profinetDataItem.getTextId();
+                            items.add(new DefaultPlcBrowseItem(new ProfinetTag(
+                                slot, subslot, ProfinetTag.Direction.OUTPUT,
+                                i, dataTypeInformation.getPlcValueType(), dataTypeInformation.getNumElements()),
+                                name, false, true, true,
+                                Collections.emptyMap(), Collections.emptyMap()));
                         }
                     }
                 }
@@ -354,6 +359,28 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
         return CompletableFuture.completedFuture(response);
     }
 
+    @Override
+    public CompletableFuture<PlcSubscriptionResponse> subscribe(PlcSubscriptionRequest subscriptionRequest) {
+        // When subscribing, we actually set up the PN IO Application Relation and make the remote device start sending data.
+        if (driverContext.getDap() == null) {
+            return CompletableFuture.failedFuture(new PlcConnectionException("DAP not set"));
+        }
+
+        // Find the matching data in the device descriptor.
+        for (String tagName : subscriptionRequest.getTagNames()) {
+            PlcSubscriptionTag tag = subscriptionRequest.getTag(tagName);
+            if(!(tag instanceof ProfinetTag)) {
+                // TODO: Add an error code for this field.
+                continue;
+            }
+            ProfinetTag profinetTag = (ProfinetTag) tag;
+            int slot = profinetTag.getSlot();
+            int subSlot = profinetTag.getSubSlot();
+
+        }
+        return null;
+    }
+
     protected void extractBlockInfo(List<PnDcp_Block> blocks) {
         // Index the blocks of the response
         Map<String, PnDcp_Block> blockMap = new HashMap<>();
diff --git a/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/utils/ProfinetDataTypeMapper.java b/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/utils/ProfinetDataTypeMapper.java
index bb420b3760..70e8f96585 100644
--- a/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/utils/ProfinetDataTypeMapper.java
+++ b/plc4j/drivers/profinet-ng/src/main/java/org/apache/plc4x/java/profinet/utils/ProfinetDataTypeMapper.java
@@ -70,7 +70,11 @@ public class ProfinetDataTypeMapper {
                 case "VisibleString":
                     break;
                 case "OctetString":
-                    return new DataTypeInformation(PlcValueType.BOOL, dataItem.getLength());
+                    if(dataItem.isUseAsBits()) {
+                        return new DataTypeInformation(PlcValueType.List, dataItem.getLength() * 8);
+                    } else {
+                        return new DataTypeInformation(PlcValueType.List, dataItem.getLength());
+                    }
                 case "Unsigned8+Unsigned8":
                 case "Float32+Unsigned8":
                 case "Float32+Status8":
diff --git a/plc4j/drivers/profinet/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block.java b/plc4j/drivers/profinet/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block.java
index e4bf5c8538..4065b85537 100644
--- a/plc4j/drivers/profinet/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block.java
+++ b/plc4j/drivers/profinet/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block.java
@@ -149,6 +149,10 @@ public abstract class PnIoCm_Block implements Message {
       builder = PnIoCm_Block_ArServer.staticParsePnIoCm_BlockBuilder(readBuffer);
     } else if (EvaluationHelper.equals(blockType, PnIoCm_BlockType.REAL_IDENTIFICATION_DATA)) {
       builder = PnIoCm_Block_RealIdentificationData.staticParsePnIoCm_BlockBuilder(readBuffer);
+    } else if (EvaluationHelper.equals(
+        blockType, PnIoCm_BlockType.IOD_BLOCK_REQ_PLUGIN_ALARM_APPLICATION_READY)) {
+      builder =
+          PnIoCm_Block_ReqPluginAlarmApplicationReady.staticParsePnIoCm_BlockBuilder(readBuffer);
     } else if (EvaluationHelper.equals(blockType, PnIoCm_BlockType.I_AND_M_0)) {
       builder = PnIoCm_Block_IAndM0.staticParsePnIoCm_BlockBuilder(readBuffer);
     } else if (EvaluationHelper.equals(blockType, PnIoCm_BlockType.I_AND_M_1)) {
diff --git a/plc4j/drivers/profinet/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block_ReqPluginAlarmApplicationReady.java b/plc4j/drivers/profinet/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block_ReqPluginAlarmApplicationReady.java
new file mode 100644
index 0000000000..73b9d230c4
--- /dev/null
+++ b/plc4j/drivers/profinet/src/main/generated/org/apache/plc4x/java/profinet/readwrite/PnIoCm_Block_ReqPluginAlarmApplicationReady.java
@@ -0,0 +1,123 @@
+/*
+ * 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
+ *
+ *   https://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.readwrite;
+
+import static org.apache.plc4x.java.spi.codegen.fields.FieldReaderFactory.*;
+import static org.apache.plc4x.java.spi.codegen.fields.FieldWriterFactory.*;
+import static org.apache.plc4x.java.spi.codegen.io.DataReaderFactory.*;
+import static org.apache.plc4x.java.spi.codegen.io.DataWriterFactory.*;
+import static org.apache.plc4x.java.spi.generation.StaticHelper.*;
+
+import java.time.*;
+import java.util.*;
+import org.apache.plc4x.java.api.exceptions.*;
+import org.apache.plc4x.java.api.value.*;
+import org.apache.plc4x.java.spi.codegen.*;
+import org.apache.plc4x.java.spi.codegen.fields.*;
+import org.apache.plc4x.java.spi.codegen.io.*;
+import org.apache.plc4x.java.spi.generation.*;
+
+// Code generated by code-generation. DO NOT EDIT.
+
+public class PnIoCm_Block_ReqPluginAlarmApplicationReady extends PnIoCm_Block implements Message {
+
+  // Accessors for discriminator values.
+  public PnIoCm_BlockType getBlockType() {
+    return PnIoCm_BlockType.IOD_BLOCK_REQ_PLUGIN_ALARM_APPLICATION_READY;
+  }
+
+  public PnIoCm_Block_ReqPluginAlarmApplicationReady() {
+    super();
+  }
+
+  @Override
+  protected void serializePnIoCm_BlockChild(WriteBuffer writeBuffer) throws SerializationException {
+    PositionAware positionAware = writeBuffer;
+    boolean _lastItem = ThreadLocalHelper.lastItemThreadLocal.get();
+    writeBuffer.pushContext("PnIoCm_Block_ReqPluginAlarmApplicationReady");
+
+    writeBuffer.popContext("PnIoCm_Block_ReqPluginAlarmApplicationReady");
+  }
+
+  @Override
+  public int getLengthInBytes() {
+    return (int) Math.ceil((float) getLengthInBits() / 8.0);
+  }
+
+  @Override
+  public int getLengthInBits() {
+    int lengthInBits = super.getLengthInBits();
+    PnIoCm_Block_ReqPluginAlarmApplicationReady _value = this;
+    boolean _lastItem = ThreadLocalHelper.lastItemThreadLocal.get();
+
+    return lengthInBits;
+  }
+
+  public static PnIoCm_BlockBuilder staticParsePnIoCm_BlockBuilder(ReadBuffer readBuffer)
+      throws ParseException {
+    readBuffer.pullContext("PnIoCm_Block_ReqPluginAlarmApplicationReady");
+    PositionAware positionAware = readBuffer;
+    boolean _lastItem = ThreadLocalHelper.lastItemThreadLocal.get();
+
+    readBuffer.closeContext("PnIoCm_Block_ReqPluginAlarmApplicationReady");
+    // Create the instance
+    return new PnIoCm_Block_ReqPluginAlarmApplicationReadyBuilderImpl();
+  }
+
+  public static class PnIoCm_Block_ReqPluginAlarmApplicationReadyBuilderImpl
+      implements PnIoCm_Block.PnIoCm_BlockBuilder {
+
+    public PnIoCm_Block_ReqPluginAlarmApplicationReadyBuilderImpl() {}
+
+    public PnIoCm_Block_ReqPluginAlarmApplicationReady build() {
+      PnIoCm_Block_ReqPluginAlarmApplicationReady pnIoCm_Block_ReqPluginAlarmApplicationReady =
+          new PnIoCm_Block_ReqPluginAlarmApplicationReady();
+      return pnIoCm_Block_ReqPluginAlarmApplicationReady;
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof PnIoCm_Block_ReqPluginAlarmApplicationReady)) {
+      return false;
+    }
+    PnIoCm_Block_ReqPluginAlarmApplicationReady that =
+        (PnIoCm_Block_ReqPluginAlarmApplicationReady) o;
+    return super.equals(that) && true;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode());
+  }
+
+  @Override
+  public String toString() {
+    WriteBufferBoxBased writeBufferBoxBased = new WriteBufferBoxBased(true, true);
+    try {
+      writeBufferBoxBased.writeSerializable(this);
+    } catch (SerializationException e) {
+      throw new RuntimeException(e);
+    }
+    return "\n" + writeBufferBoxBased.getBox().toString() + "\n";
+  }
+}
diff --git a/protocols/profinet/src/main/resources/protocols/profinet/pnio.mspec b/protocols/profinet/src/main/resources/protocols/profinet/pnio.mspec
index 079d456ea2..4518b0eec5 100644
--- a/protocols/profinet/src/main/resources/protocols/profinet/pnio.mspec
+++ b/protocols/profinet/src/main/resources/protocols/profinet/pnio.mspec
@@ -335,6 +335,9 @@
             [implicit uint 16                      numApis          'COUNT(apis)'                ]
             [array    PnIoCm_RealIdentificationApi apis             count               'numApis']
         ]
+        ['IOD_BLOCK_REQ_PLUGIN_ALARM_APPLICATION_READY' PnIoCm_Block_ReqPluginAlarmApplicationReady
+            // TODO: Implement ...
+        ]
 
         // https://cache.industry.siemens.com/dl/files/491/26435491/att_859456/v1/PGH_IO-Base_0.pdf (page 231)
         ['I_AND_M_0' PnIoCm_Block_IAndM0