You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by hu...@apache.org on 2022/10/06 15:23:58 UTC

[plc4x] 13/13: fix(plc4j/profinet): Continued to split device logic out.

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

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

commit e99d97c6b474c50ac653ca942a2b7219f0ebfcc2
Author: Ben Hutcheson <be...@gmail.com>
AuthorDate: Thu Oct 6 07:52:35 2022 -0600

    fix(plc4j/profinet): Continued to split device logic out.
---
 .../java/profinet/device/ProfinetCallable.java     |  25 +-
 .../plc4x/java/profinet/device/ProfinetDevice.java | 136 +++++++++-
 .../profinet/device/ProfinetMessageWrapper.java    |  27 +-
 .../profinet/protocol/ProfinetProtocolLogic.java   | 294 +--------------------
 .../HelloPlc4xDiscoverAndBrowse.java               |  35 +--
 .../src/main/resources/logback.xml                 |  12 +-
 6 files changed, 199 insertions(+), 330 deletions(-)

diff --git a/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetCallable.java b/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetCallable.java
index fd117c9f4..1cde0c154 100644
--- a/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetCallable.java
+++ b/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetCallable.java
@@ -1,10 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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.device;
 
 import org.apache.plc4x.java.api.exceptions.PlcException;
 import org.apache.plc4x.java.profinet.readwrite.DceRpc_Packet;
 
-public interface ProfinetCallable {
-    void handle(DceRpc_Packet packet) throws PlcException;
+public interface ProfinetCallable<T> {
+    void handle(T packet) throws PlcException;
 
-    DceRpc_Packet create() throws PlcException;
+    T create() throws PlcException;
 }
diff --git a/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetDevice.java b/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetDevice.java
index 80f10d961..d2bd0887e 100644
--- a/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetDevice.java
+++ b/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetDevice.java
@@ -61,6 +61,15 @@ public class ProfinetDevice {
 
     private AtomicInteger sessionKeyGenerator = new AtomicInteger(1);
 
+    private static final Uuid ARUUID;
+    static {
+        try {
+            ARUUID = new Uuid(Hex.decodeHex("654519352df3b6428f874371217c2b51"));
+        } catch (DecoderException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
 
     private void closeUDPSocket() {
         // Handle the closing of the connection, might need to send some messages beforehand.
@@ -70,7 +79,7 @@ public class ProfinetDevice {
         }
     }
 
-    private boolean createUDPSocket() {
+    private boolean createUdpSocket() {
         if (state != ProfinetDeviceState.IDLE) {
             closeUDPSocket();
         }
@@ -82,6 +91,7 @@ public class ProfinetDevice {
 
         rawSocketChannel = (RawSocketChannel) channel;
 
+
         // Create an udp socket
         try {
             udpSocket = new DatagramSocket();
@@ -94,16 +104,28 @@ public class ProfinetDevice {
     }
 
     public boolean onConnect() {
-        if (!createUDPSocket()) {
+        if (!createUdpSocket()) {
             // Unable to create UDP connection
             return false;
         }
 
-        ProfinetMessageWrapper.sendMessage(
+        ProfinetMessageWrapper.sendUdpMessage(
             new CreateConnection(),
             this
         );
 
+        ProfinetMessageWrapper.sendUdpMessage(
+            new WriteParameters(),
+            this
+        );
+
+        ProfinetMessageWrapper.sendUdpMessage(
+            new WriteParametersEnd(),
+            this
+        );
+
+
+
         return false;
     }
 
@@ -171,6 +193,10 @@ public class ProfinetDevice {
         return this.udpSocket;
     }
 
+    public RawSocketChannel getRawSocket() {
+        return this.rawSocketChannel;
+    }
+
     public InetAddress getIpAddress() throws UnknownHostException {
         return InetAddress.getByName(this.ipAddress);
     }
@@ -179,7 +205,7 @@ public class ProfinetDevice {
         return DEFAULT_UDP_PORT;
     }
 
-    public class CreateConnection implements ProfinetCallable {
+    public class CreateConnection implements ProfinetCallable<DceRpc_Packet> {
 
         public DceRpc_Packet create() throws PlcException {
             try {
@@ -298,4 +324,106 @@ public class ProfinetDevice {
             }
         }
     }
+
+    public class WriteParameters implements ProfinetCallable<DceRpc_Packet> {
+        public DceRpc_Packet create() {
+            return new DceRpc_Packet(
+                DceRpc_PacketType.REQUEST, true, false, false,
+                IntegerEncoding.BIG_ENDIAN, CharacterEncoding.ASCII, FloatingPointEncoding.IEEE,
+                new DceRpc_ObjectUuid((byte) 0x00, 0x0001, 0x0904, 0x002A),
+                new DceRpc_InterfaceUuid_DeviceInterface(),
+                uuid,
+                0, 1, DceRpc_Operation.WRITE,
+                new PnIoCm_Packet_Req(16696, 16696, 0, 244,
+                    Arrays.asList(
+                        new IODWriteRequestHeader(
+                            (short) 1,
+                            (short) 0,
+                            0,
+                            ARUUID,
+                            0x00000000,
+                            0x0000,
+                            0x0000,
+                            0xe040,
+                            180
+                        ),
+                        new IODWriteRequestHeader(
+                            (short) 1,
+                            (short) 0,
+                            1,
+                            ARUUID,
+                            0x00000000,
+                            0x0000,
+                            0x8000,
+                            0x8071,
+                            12
+                        ),
+                        new PDInterfaceAdjust(
+                            (short) 1,
+                            (short) 0,
+                            MultipleInterfaceModeNameOfDevice.NAME_PROVIDED_BY_LLDP
+                        )
+                    ))
+            );
+        }
+
+        @Override
+        public void handle(DceRpc_Packet packet) throws PlcException {
+            logger.debug("Received a Write Parameter Response");
+        }
+    }
+
+    public class WriteParametersEnd implements ProfinetCallable<DceRpc_Packet> {
+        public DceRpc_Packet create() {
+            return new DceRpc_Packet(
+                DceRpc_PacketType.REQUEST, true, false, false,
+                IntegerEncoding.BIG_ENDIAN, CharacterEncoding.ASCII, FloatingPointEncoding.IEEE,
+                new DceRpc_ObjectUuid((byte) 0x00, 0x0001, 0x0904, 0x002A),
+                new DceRpc_InterfaceUuid_DeviceInterface(),
+                uuid,
+                0, 1, DceRpc_Operation.CONTROL,
+                new PnIoCm_Packet_Req(16696, 16696, 0, 244,
+                    Arrays.asList(
+                        new PnIoCm_Control_Request(
+                            (short) 1,
+                            (short) 0,
+                            ARUUID,
+                            0x0001,
+                            0x0001
+                        )
+                    ))
+            );
+        }
+
+        @Override
+        public void handle(DceRpc_Packet packet) throws PlcException {
+            logger.debug("Received a Write Parameter End Response");
+        }
+    }
+
+    public class CyclicData implements ProfinetCallable<Ethernet_Frame> {
+        public Ethernet_Frame create() {
+            return new Ethernet_Frame(
+                macAddress,
+                macAddress,
+                new Ethernet_FramePayload_PnDcp(
+                new PnDcp_Pdu_RealTimeCyclic(
+                    0x8000,
+                    new PnIo_CyclicServiceDataUnit((short) 0,(short) 0, (short) 0),
+                    16696,
+                    false,
+                    false,
+                    false,
+                    false,
+                    false,
+                    false)));
+        }
+
+        @Override
+        public void handle(Ethernet_Frame packet) throws PlcException {
+            logger.debug("Received a Write Parameter End Response");
+        }
+    }
+
+
 }
diff --git a/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetMessageWrapper.java b/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetMessageWrapper.java
index bee0ae8d3..e56b92d61 100644
--- a/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetMessageWrapper.java
+++ b/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/device/ProfinetMessageWrapper.java
@@ -1,5 +1,25 @@
+/*
+ * 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.device;
 
+import io.netty.channel.Channel;
 import org.apache.plc4x.java.api.exceptions.PlcException;
 import org.apache.plc4x.java.profinet.readwrite.*;
 import org.apache.plc4x.java.spi.generation.ParseException;
@@ -13,7 +33,7 @@ import java.net.DatagramPacket;
 
 public class ProfinetMessageWrapper {
 
-    public static void sendMessage(ProfinetCallable callable, ProfinetDevice context) throws RuntimeException {
+    public static void sendUdpMessage(ProfinetCallable<DceRpc_Packet> callable, ProfinetDevice context) throws RuntimeException {
         try {
             DceRpc_Packet packet = callable.create();
             // Serialize it to a byte-payload
@@ -43,6 +63,9 @@ public class ProfinetMessageWrapper {
         } catch (PlcException e) {
             throw new RuntimeException(e);
         }
-
     }
+
+
+
+
 }
diff --git a/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/protocol/ProfinetProtocolLogic.java b/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/protocol/ProfinetProtocolLogic.java
index fe931765c..8d31d198f 100644
--- a/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/protocol/ProfinetProtocolLogic.java
+++ b/plc4j/drivers/profinet/src/main/java/org/apache/plc4x/java/profinet/protocol/ProfinetProtocolLogic.java
@@ -70,15 +70,6 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
 
     private ProfinetConfiguration configuration;
 
-    private static final Uuid ARUUID;
-
-    static {
-        try {
-            ARUUID = new Uuid(Hex.decodeHex("654519352df3b6428f874371217c2b51"));
-        } catch (DecoderException e) {
-            throw new RuntimeException(e);
-        }
-    }
 
     @Override
     public void setConfiguration(ProfinetConfiguration configuration) {
@@ -153,110 +144,8 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
             device.getValue().onConnect();
         }
 
-        ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
-        // Create the connection request.
-        try {
-            // Create the packet
-            final DceRpc_Packet profinetConnectionRequest = createProfinetConnectionRequest();
-            // Serialize it to a byte-payload
-            WriteBufferByteBased writeBuffer = new WriteBufferByteBased(profinetConnectionRequest.getLengthInBytes());
-            profinetConnectionRequest.serialize(writeBuffer);
-            // Create a udp packet.
-            DatagramPacket connectRequestPacket = new DatagramPacket(writeBuffer.getData(), writeBuffer.getData().length);
-            connectRequestPacket.setAddress(remoteAddress.getAddress());
-            connectRequestPacket.setPort(remoteAddress.getPort());
-            // Send it.
-
-            udpSocket.send(connectRequestPacket);
-
-            // Receive the response.
-            byte[] resultBuffer = new byte[profinetConnectionRequest.getLengthInBytes()];
-            DatagramPacket connectResponsePacket = new DatagramPacket(resultBuffer, resultBuffer.length);
-            udpSocket.receive(connectResponsePacket);
-            ReadBufferByteBased readBuffer = new ReadBufferByteBased(resultBuffer);
-            final DceRpc_Packet dceRpc_packet = DceRpc_Packet.staticParse(readBuffer);
-            if ((dceRpc_packet.getOperation() == DceRpc_Operation.CONNECT) && (dceRpc_packet.getPacketType() == DceRpc_PacketType.RESPONSE)) {
-                if (dceRpc_packet.getPayload().getPacketType() == DceRpc_PacketType.RESPONSE) {
-                    // Get the remote MAC address and store it in the context.
-                    final PnIoCm_Packet_Res connectResponse = (PnIoCm_Packet_Res) dceRpc_packet.getPayload();
-                    if ((connectResponse.getBlocks().size() > 0) && (connectResponse.getBlocks().get(0) instanceof PnIoCm_Block_ArRes)) {
-                        final PnIoCm_Block_ArRes pnIoCm_block_arRes = (PnIoCm_Block_ArRes) connectResponse.getBlocks().get(0);
-                        profinetDriverContext.setRemoteMacAddress(pnIoCm_block_arRes.getCmResponderMacAddr());
-
-                        // Update the raw-socket transports filter expression.
-                        ((RawSocketChannel) channel).setRemoteMacAddress(org.pcap4j.util.MacAddress.getByAddress(profinetDriverContext.getRemoteMacAddress().getAddress()));
-                    } else {
-                        throw new PlcException("Unexpected type of first block.");
-                    }
-                } else {
-                    throw new PlcException("Unexpected response");
-                }
-            } else if (dceRpc_packet.getPacketType() == DceRpc_PacketType.REJECT) {
-                throw new PlcException("Device rejected connection request");
-            } else {
-                throw new PlcException("Unexpected response");
-            }
-
-            // Create the packet
-            final DceRpc_Packet profinetAdvancedConnectionWriteRequest = createProfinetAdvancedConnectionWriteRequest();
-            // Serialize it to a byte-payload
-            writeBuffer = new WriteBufferByteBased(profinetAdvancedConnectionWriteRequest.getLengthInBytes());
-            profinetAdvancedConnectionWriteRequest.serialize(writeBuffer);
-            // Create a udp packet.
-            connectRequestPacket = new DatagramPacket(writeBuffer.getData(), writeBuffer.getData().length);
-            connectRequestPacket.setAddress(remoteAddress.getAddress());
-            connectRequestPacket.setPort(remoteAddress.getPort());
-            // Send it.
-
-            udpSocket.send(connectRequestPacket);
-
-            // Receive the response.
-            resultBuffer = new byte[profinetAdvancedConnectionWriteRequest.getLengthInBytes()];
-            connectResponsePacket = new DatagramPacket(resultBuffer, resultBuffer.length);
-            udpSocket.receive(connectResponsePacket);
-
-
-            // Create the packet
-            final DceRpc_Packet profinetAdvancedConnectionParameterEnd = createProfinetAdvancedConnectionParameterEnd();
-            // Serialize it to a byte-payload
-            writeBuffer = new WriteBufferByteBased(profinetAdvancedConnectionParameterEnd.getLengthInBytes());
-            profinetAdvancedConnectionParameterEnd.serialize(writeBuffer);
-            // Create a udp packet.
-            connectRequestPacket = new DatagramPacket(writeBuffer.getData(), writeBuffer.getData().length);
-            connectRequestPacket.setAddress(remoteAddress.getAddress());
-            connectRequestPacket.setPort(remoteAddress.getPort());
-            // Send it.
-
-            udpSocket.send(connectRequestPacket);
-
-            // Receive the response.
-            resultBuffer = new byte[profinetAdvancedConnectionParameterEnd.getLengthInBytes()];
-            connectResponsePacket = new DatagramPacket(resultBuffer, resultBuffer.length);
-            udpSocket.receive(connectResponsePacket);
-
-            // Create the packet
-            final DceRpc_Packet profinetAdvancedConnectionApplicationReady = createProfinetAdvancedConnectionApplicationReady();
-            // Serialize it to a byte-payload
-            writeBuffer = new WriteBufferByteBased(profinetAdvancedConnectionApplicationReady.getLengthInBytes());
-            profinetAdvancedConnectionApplicationReady.serialize(writeBuffer);
-            // Create a udp packet.
-            connectRequestPacket = new DatagramPacket(writeBuffer.getData(), writeBuffer.getData().length);
-            connectRequestPacket.setAddress(remoteAddress.getAddress());
-            connectRequestPacket.setPort(remoteAddress.getPort());
-            // Send it.
-
-            udpSocket.send(connectRequestPacket);
-
-            // Receive the response.
-            resultBuffer = new byte[profinetAdvancedConnectionApplicationReady.getLengthInBytes()];
-            connectResponsePacket = new DatagramPacket(resultBuffer, resultBuffer.length);
-            udpSocket.receive(connectResponsePacket);
-            context.fireConnected();
-            connected = true;
-
-        } catch (SerializationException | IOException | PlcException | ParseException e) {
-            logger.error("Error", e);
-        }
+        context.fireConnected();
+        connected = true;
     }
 
     @Override
@@ -390,188 +279,9 @@ public class ProfinetProtocolLogic extends Plc4xProtocolBase<Ethernet_Frame> imp
         return Optional.empty();
     }
 
-    private DceRpc_Packet createProfinetConnectionRequest() throws PlcException {
-        try {
-            return new DceRpc_Packet(
-                DceRpc_PacketType.REQUEST, true, false, false,
-                IntegerEncoding.BIG_ENDIAN, CharacterEncoding.ASCII, FloatingPointEncoding.IEEE,
-                new DceRpc_ObjectUuid((byte) 0x00, 0x0001, 0x0904, 0x002A),
-                new DceRpc_InterfaceUuid_DeviceInterface(),
-                profinetDriverContext.getDceRpcActivityUuid(),
-                0, 0, DceRpc_Operation.CONNECT,
-                new PnIoCm_Packet_Req(16696, 16696, 0, 0,
-                    Arrays.asList(
-                        new PnIoCm_Block_ArReq((short) 1, (short) 0, PnIoCm_ArType.IO_CONTROLLER,
-                            new Uuid(Hex.decodeHex("654519352df3b6428f874371217c2b51")),
-                            profinetDriverContext.getSessionKey(),
-                            profinetDriverContext.getLocalMacAddress(),
-                            new Uuid(Hex.decodeHex("dea000006c9711d1827100640008002a")),
-                            false, true, false,
-                            false, PnIoCm_CompanionArType.SINGLE_AR, false,
-                            true, false, PnIoCm_State.ACTIVE,
-                            600,
-                            // This actually needs to be set to this value and not the real port number.
-                            0x8892,
-                            // It seems that it must be set to this value, or it won't work.
-                            "plc4x"),
-                        new PnIoCm_Block_IoCrReq((short) 1, (short) 0, PnIoCm_IoCrType.INPUT_CR,
-                            0x0001,
-                            0x8892,
-                            false, false,
-                            false, false, PnIoCm_RtClass.RT_CLASS_2, 40,
-                            0xBBF0, 128, 8, 1, 0, 0xffffffff,
-                            50, 50, 0xC000,
-                            new org.apache.plc4x.java.profinet.readwrite.MacAddress(Hex.decodeHex("000000000000")),
-                            Collections.singletonList(
-                                new PnIoCm_IoCrBlockReqApi(
-                                    Arrays.asList(
-                                        new PnIoCm_IoDataObject(0, 0x0001, 0),
-                                        new PnIoCm_IoDataObject(0, 0x8000, 1),
-                                        new PnIoCm_IoDataObject(0, 0x8001, 2)
-                                    ),
-                                    new ArrayList<PnIoCm_IoCs>(0))
-                            )),
-                        new PnIoCm_Block_IoCrReq((short) 1, (short) 0, PnIoCm_IoCrType.OUTPUT_CR,
-                            0x0002, 0x8892, false, false,
-                            false, false, PnIoCm_RtClass.RT_CLASS_2, 40,
-                            0xFFFF, 128, 8, 1, 0, 0xffffffff,
-                            50, 50, 0xC000,
-                            new MacAddress(Hex.decodeHex("000000000000")),
-                            Collections.singletonList(
-                                new PnIoCm_IoCrBlockReqApi(
-                                    new ArrayList<PnIoCm_IoDataObject>(0),
-                                    Arrays.asList(
-                                        new PnIoCm_IoCs(0, 0x0001, 0),
-                                        new PnIoCm_IoCs(0, 0x8000, 1),
-                                        new PnIoCm_IoCs(0, 0x8001, 2)
-                                    )
-                                )
-                            )
-                        ),
-                        new PnIoCm_Block_ExpectedSubmoduleReq((short) 1, (short) 0,
-                            Collections.singletonList(
-                                new PnIoCm_ExpectedSubmoduleBlockReqApi(0,
-                                    0x00000001, 0x00000000,
-                                    Arrays.asList(
-                                        new PnIoCm_Submodule_NoInputNoOutputData(0x0001,
-                                            0x00000001, false, false,
-                                            false, false),
-                                        new PnIoCm_Submodule_NoInputNoOutputData(0x8000,
-                                            0x00008000, false, false,
-                                            false, false),
-                                        new PnIoCm_Submodule_NoInputNoOutputData(0x8001,
-                                            0x00008001, false, false,
-                                            false, false)
-                                    )
-                                )
-                            )
-                        ),
-                        new PnIoCm_Block_AlarmCrReq((short) 1, (short) 0,
-                            PnIoCm_AlarmCrType.ALARM_CR, 0x8892, false, false, 1, 3,
-                            0x0000, 200, 0xC000, 0xA000)
-                    ))
-            );
-
-            /*// Build the UDP/IP/EthernetFrame to transport the package.
-            return new Ethernet_Frame(profinetDriverContext.getRemoteMacAddress(), profinetDriverContext.getLocalMacAddress(),
-                new Ethernet_FramePayload_IPv4(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE), (short) 64,
-                    profinetDriverContext.getLocalIpAddress(), profinetDriverContext.getRemoteIpAddress(),
-                    new Udp_Packet(profinetDriverContext.getLocalUdpPort(), profinetDriverContext.getRemoteUdpPort(),
-                        dceRpcConnectionRequest)));*/
-        } catch (DecoderException e) {
-            throw new PlcException("Error creating connection request", e);
-        }
-    }
-
-    private DceRpc_Packet createProfinetAdvancedConnectionWriteRequest() throws PlcException {
-
-        return new DceRpc_Packet(
-            DceRpc_PacketType.REQUEST, true, false, false,
-            IntegerEncoding.BIG_ENDIAN, CharacterEncoding.ASCII, FloatingPointEncoding.IEEE,
-            new DceRpc_ObjectUuid((byte) 0x00, 0x0001, 0x0904, 0x002A),
-            new DceRpc_InterfaceUuid_DeviceInterface(),
-            profinetDriverContext.getDceRpcActivityUuid(),
-            0, 1, DceRpc_Operation.WRITE,
-            new PnIoCm_Packet_Req(16696, 16696, 0, 244,
-                Arrays.asList(
-                    new IODWriteRequestHeader(
-                        (short) 1,
-                        (short) 0,
-                        0,
-                        ARUUID,
-                        0x00000000,
-                        0x0000,
-                        0x0000,
-                        0xe040,
-                        180
-                        ),
-                    new IODWriteRequestHeader(
-                        (short) 1,
-                        (short) 0,
-                        1,
-                        ARUUID,
-                        0x00000000,
-                        0x0000,
-                        0x8000,
-                        0x8071,
-                        12
-                    ),
-                    new PDInterfaceAdjust(
-                        (short) 1,
-                        (short) 0,
-                        MultipleInterfaceModeNameOfDevice.NAME_PROVIDED_BY_LLDP
-                    )
-                ))
-        );
-    }
-
-    private DceRpc_Packet createProfinetAdvancedConnectionParameterEnd() throws PlcException {
-
-        return new DceRpc_Packet(
-            DceRpc_PacketType.REQUEST, true, false, false,
-            IntegerEncoding.BIG_ENDIAN, CharacterEncoding.ASCII, FloatingPointEncoding.IEEE,
-            new DceRpc_ObjectUuid((byte) 0x00, 0x0001, 0x0904, 0x002A),
-            new DceRpc_InterfaceUuid_DeviceInterface(),
-            profinetDriverContext.getDceRpcActivityUuid(),
-            0, 1, DceRpc_Operation.CONTROL,
-            new PnIoCm_Packet_Req(16696, 16696, 0, 244,
-                Arrays.asList(
-                    new PnIoCm_Control_Request(
-                        (short) 1,
-                        (short) 0,
-                        ARUUID,
-                        0x0001,
-                        0x0001
-                    )
-                ))
-        );
-    }
 
 
 
-    private DceRpc_Packet createProfinetAdvancedConnectionApplicationReady() throws PlcException {
-
-        return new DceRpc_Packet(
-            DceRpc_PacketType.REQUEST, true, false, false,
-            IntegerEncoding.BIG_ENDIAN, CharacterEncoding.ASCII, FloatingPointEncoding.IEEE,
-            new DceRpc_ObjectUuid((byte) 0x00, 0x0001, 0x0904, 0x002A),
-            new DceRpc_InterfaceUuid_DeviceInterface(),
-            profinetDriverContext.getDceRpcActivityUuid(),
-            0, 1, DceRpc_Operation.CONTROL,
-            new PnIoCm_Packet_Req(16696, 16696, 0, 244,
-                Arrays.asList(
-                    new PnIoCM_Block_Request(
-                        (short) 1,
-                        (short) 0,
-                        ARUUID,
-                        0x0001,
-                        0x0002,
-                        0x0000
-                    )
-                ))
-        );
-    }
-
 
 
 }
diff --git a/plc4j/examples/hello-world-plc4x-discover-and-browse/src/main/java/org/apache/plc4x/java/examples/helloplc4x/discoverandbrowse/HelloPlc4xDiscoverAndBrowse.java b/plc4j/examples/hello-world-plc4x-discover-and-browse/src/main/java/org/apache/plc4x/java/examples/helloplc4x/discoverandbrowse/HelloPlc4xDiscoverAndBrowse.java
index 9b8418dfc..e1b863745 100644
--- a/plc4j/examples/hello-world-plc4x-discover-and-browse/src/main/java/org/apache/plc4x/java/examples/helloplc4x/discoverandbrowse/HelloPlc4xDiscoverAndBrowse.java
+++ b/plc4j/examples/hello-world-plc4x-discover-and-browse/src/main/java/org/apache/plc4x/java/examples/helloplc4x/discoverandbrowse/HelloPlc4xDiscoverAndBrowse.java
@@ -35,33 +35,22 @@ public class HelloPlc4xDiscoverAndBrowse {
     public static void main(String[] args) throws Exception {
         // Iterate over all installed drivers and execute their browse functionality (If they support it)
         PlcDriverManager driverManager = new PlcDriverManager();
-        for (String protocolCode : driverManager.listDrivers()) {
-            PlcDriver driver = driverManager.getDriver(protocolCode);
-            if (driver.getMetadata().canDiscover()) {
-                logger.info("Performing discovery for {} protocol", driver.getProtocolName());
-
-                PlcDiscoveryRequest discoveryRequest = driver.discoveryRequestBuilder().build();
-
-                discoveryRequest.executeWithHandler(discoveryItem -> {
-                    logger.info(" - Found device with connection-url {}", discoveryItem.getConnectionUrl());
-                    try (PlcConnection connection = driverManager.getConnection(discoveryItem.getConnectionUrl())) {
-                        if (connection.getMetadata().canBrowse()) {
-                            PlcBrowseRequest browseRequest = connection.browseRequestBuilder().build();
-                            browseRequest.execute().whenComplete((browseResponse, throwable) -> {
-                                if (throwable != null) {
-                                    throwable.printStackTrace();
-                                } else {
-                                    for (PlcBrowseItem value : browseResponse.getValues()) {
-                                        outputBrowseItem(value, 0);
-                                    }
-                                }
-                            });
+        //try (PlcConnection connection = driverManager.getConnection("opcua:tcp://missy-nuc:53530/plc4x")) {
+        try (PlcConnection connection = driverManager.getConnection("profinet:raw://192.168.90.128")) {
+            if (connection.getMetadata().canBrowse()) {
+                PlcBrowseRequest browseRequest = connection.browseRequestBuilder().build();
+                browseRequest.execute().whenComplete((browseResponse, throwable) -> {
+                    if (throwable != null) {
+                        throwable.printStackTrace();
+                    } else {
+                        for (PlcBrowseItem value : browseResponse.getValues()) {
+                            outputBrowseItem(value, 0);
                         }
-                    } catch (Exception e) {
-                        throw new RuntimeException(e);
                     }
                 });
             }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
         }
     }
 
diff --git a/plc4j/examples/hello-world-plc4x-discover-and-browse/src/main/resources/logback.xml b/plc4j/examples/hello-world-plc4x-discover-and-browse/src/main/resources/logback.xml
index 2641def3a..f64d08114 100644
--- a/plc4j/examples/hello-world-plc4x-discover-and-browse/src/main/resources/logback.xml
+++ b/plc4j/examples/hello-world-plc4x-discover-and-browse/src/main/resources/logback.xml
@@ -28,13 +28,13 @@
   </appender>
 
   <!-- Remove the messages we're not interested in -->
-  <logger name="io.netty.channel" level="ERROR"/>
-  <logger name="org.pcap4j" level="WARN"/>
-  <logger name="org.apache.plc4x.java.modbus.tcp.discovery" level="WARN"/>
-  <logger name="org.apache.plc4x.java.transport" level="WARN"/>
-  <logger name="org.apache.plc4x.java.spi" level="WARN"/>
+  <logger name="io.netty.channel" level="TRACE"/>
+  <logger name="org.pcap4j" level="TRACE"/>
+  <logger name="org.apache.plc4x.java.modbus.tcp.discovery" level="TRACE"/>
+  <logger name="org.apache.plc4x.java.transport" level="TRACE"/>
+  <logger name="org.apache.plc4x.java.spi" level="TRACE"/>
 
-  <root level="INFO">
+  <root level="TRACE">
     <appender-ref ref="STDOUT" />
   </root>